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 2018/10/09 15:50:03 UTC

[01/45] carbondata git commit: [CARBONDATA-2969]local dictioanry query fix for spark-2.3 [Forced Update!]

Repository: carbondata
Updated Branches:
  refs/heads/branch-1.5 ef1068cad -> 3c7b33992 (forced update)


[CARBONDATA-2969]local dictioanry query fix for spark-2.3

This closes #2761


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

Branch: refs/heads/branch-1.5
Commit: 2ab2254be84f82fd2f4b99a6b73353f4c7a55d10
Parents: f239894
Author: akashrn5 <ak...@gmail.com>
Authored: Tue Sep 25 20:43:06 2018 +0530
Committer: Jacky Li <ja...@qq.com>
Committed: Wed Sep 26 15:01:38 2018 +0800

----------------------------------------------------------------------
 .../LocalDictionarySupportLoadTableTest.scala   | 14 +++++
 .../vectorreader/CarbonDictionaryWrapper.java   | 44 ---------------
 .../vectorreader/ColumnarVectorWrapper.java     | 11 +---
 .../spark/sql/CarbonDictionaryWrapper.java      | 44 +++++++++++++++
 .../org/apache/spark/sql/CarbonVectorProxy.java | 10 ++--
 .../spark/sql/CarbonDictionaryWrapper.java      | 56 ++++++++++++++++++++
 .../org/apache/spark/sql/CarbonVectorProxy.java |  8 +--
 7 files changed, 127 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/2ab2254b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/localdictionary/LocalDictionarySupportLoadTableTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/localdictionary/LocalDictionarySupportLoadTableTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/localdictionary/LocalDictionarySupportLoadTableTest.scala
index e88d8a9..d23c844 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/localdictionary/LocalDictionarySupportLoadTableTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/localdictionary/LocalDictionarySupportLoadTableTest.scala
@@ -136,6 +136,20 @@ class LocalDictionarySupportLoadTableTest extends QueryTest with BeforeAndAfterA
     assert(checkForLocalDictionary(getDimRawChunk(2)))
   }
 
+  test("test local dictionary data validation") {
+    sql("drop table if exists local_query_enable")
+    sql("drop table if exists local_query_disable")
+    sql(
+      "CREATE TABLE local_query_enable(name string) STORED BY 'carbondata' tblproperties" +
+      "('local_dictionary_enable'='false','local_dictionary_include'='name')")
+    sql("load data inpath '" + file1 + "' into table local_query_enable OPTIONS('header'='false')")
+    sql(
+      "CREATE TABLE local_query_disable(name string) STORED BY 'carbondata' tblproperties" +
+      "('local_dictionary_enable'='true','local_dictionary_include'='name')")
+    sql("load data inpath '" + file1 + "' into table local_query_disable OPTIONS('header'='false')")
+    checkAnswer(sql("select name from local_query_enable"), sql("select name from local_query_disable"))
+  }
+
   test("test to validate local dictionary values"){
     sql("drop table if exists local2")
     sql("CREATE TABLE local2(name string) STORED BY 'carbondata' tblproperties('local_dictionary_enable'='true')")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2ab2254b/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/CarbonDictionaryWrapper.java
----------------------------------------------------------------------
diff --git a/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/CarbonDictionaryWrapper.java b/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/CarbonDictionaryWrapper.java
deleted file mode 100644
index 7f1e577..0000000
--- a/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/CarbonDictionaryWrapper.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.spark.vectorreader;
-
-import org.apache.carbondata.core.scan.result.vector.CarbonDictionary;
-
-import org.apache.parquet.column.Dictionary;
-import org.apache.parquet.column.Encoding;
-import org.apache.parquet.io.api.Binary;
-
-public class CarbonDictionaryWrapper extends Dictionary {
-
-  private Binary[] binaries;
-
-  CarbonDictionaryWrapper(Encoding encoding, CarbonDictionary dictionary) {
-    super(encoding);
-    binaries = new Binary[dictionary.getDictionarySize()];
-    for (int i = 0; i < binaries.length; i++) {
-      binaries[i] = Binary.fromReusedByteArray(dictionary.getDictionaryValue(i));
-    }
-  }
-
-  @Override public int getMaxId() {
-    return binaries.length - 1;
-  }
-
-  @Override public Binary decodeToBinary(int id) {
-    return binaries[id];
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2ab2254b/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java
----------------------------------------------------------------------
diff --git a/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java b/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java
index a0938da..6acf31f 100644
--- a/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java
+++ b/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java
@@ -269,16 +269,7 @@ class ColumnarVectorWrapper implements CarbonColumnVector {
   }
 
   @Override public void setDictionary(CarbonDictionary dictionary) {
-    if (dictionary == null) {
-      sparkColumnVectorProxy.setDictionary(null, ordinal);
-    } else {
-      sparkColumnVectorProxy
-          .setDictionary(new CarbonDictionaryWrapper(Encoding.PLAIN, dictionary),ordinal);
-    }
-  }
-
-  private void  setDictionaryType(boolean type) {
-    this.isDictionary = type;
+      sparkColumnVectorProxy.setDictionary(dictionary, ordinal);
   }
 
   @Override public boolean hasDictionary() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2ab2254b/integration/spark-datasource/src/main/spark2.1andspark2.2/org/apache/spark/sql/CarbonDictionaryWrapper.java
----------------------------------------------------------------------
diff --git a/integration/spark-datasource/src/main/spark2.1andspark2.2/org/apache/spark/sql/CarbonDictionaryWrapper.java b/integration/spark-datasource/src/main/spark2.1andspark2.2/org/apache/spark/sql/CarbonDictionaryWrapper.java
new file mode 100644
index 0000000..b7c6741
--- /dev/null
+++ b/integration/spark-datasource/src/main/spark2.1andspark2.2/org/apache/spark/sql/CarbonDictionaryWrapper.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql;
+
+import org.apache.carbondata.core.scan.result.vector.CarbonDictionary;
+
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.io.api.Binary;
+
+public class CarbonDictionaryWrapper extends Dictionary {
+
+  private Binary[] binaries;
+
+  CarbonDictionaryWrapper(Encoding encoding, CarbonDictionary dictionary) {
+    super(encoding);
+    binaries = new Binary[dictionary.getDictionarySize()];
+    for (int i = 0; i < binaries.length; i++) {
+      binaries[i] = Binary.fromReusedByteArray(dictionary.getDictionaryValue(i));
+    }
+  }
+
+  @Override public int getMaxId() {
+    return binaries.length - 1;
+  }
+
+  @Override public Binary decodeToBinary(int id) {
+    return binaries[id];
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2ab2254b/integration/spark-datasource/src/main/spark2.1andspark2.2/org/apache/spark/sql/CarbonVectorProxy.java
----------------------------------------------------------------------
diff --git a/integration/spark-datasource/src/main/spark2.1andspark2.2/org/apache/spark/sql/CarbonVectorProxy.java b/integration/spark-datasource/src/main/spark2.1andspark2.2/org/apache/spark/sql/CarbonVectorProxy.java
index f39bc93..80e6dbd 100644
--- a/integration/spark-datasource/src/main/spark2.1andspark2.2/org/apache/spark/sql/CarbonVectorProxy.java
+++ b/integration/spark-datasource/src/main/spark2.1andspark2.2/org/apache/spark/sql/CarbonVectorProxy.java
@@ -18,7 +18,10 @@ package org.apache.spark.sql;
 
 import java.math.BigInteger;
 
+import org.apache.carbondata.core.scan.result.vector.CarbonDictionary;
+
 import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.Encoding;
 import org.apache.spark.memory.MemoryMode;
 import org.apache.spark.sql.catalyst.InternalRow;
 import org.apache.spark.sql.execution.vectorized.ColumnarBatch;
@@ -82,9 +85,10 @@ public class CarbonVectorProxy {
         return columnarBatch.capacity();
     }
 
-    public void setDictionary(Object dictionary, int ordinal) {
-        if (dictionary instanceof Dictionary) {
-            columnarBatch.column(ordinal).setDictionary((Dictionary) dictionary);
+    public void setDictionary(CarbonDictionary dictionary, int ordinal) {
+        if (null != dictionary) {
+            columnarBatch.column(ordinal)
+                .setDictionary(new CarbonDictionaryWrapper(Encoding.PLAIN, dictionary));
         } else {
             columnarBatch.column(ordinal).setDictionary(null);
         }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2ab2254b/integration/spark-datasource/src/main/spark2.3plus/org/apache/spark/sql/CarbonDictionaryWrapper.java
----------------------------------------------------------------------
diff --git a/integration/spark-datasource/src/main/spark2.3plus/org/apache/spark/sql/CarbonDictionaryWrapper.java b/integration/spark-datasource/src/main/spark2.3plus/org/apache/spark/sql/CarbonDictionaryWrapper.java
new file mode 100644
index 0000000..5a99c68
--- /dev/null
+++ b/integration/spark-datasource/src/main/spark2.3plus/org/apache/spark/sql/CarbonDictionaryWrapper.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql;
+
+import org.apache.carbondata.core.scan.result.vector.CarbonDictionary;
+
+import org.apache.spark.sql.execution.vectorized.Dictionary;
+
+public class CarbonDictionaryWrapper implements Dictionary {
+
+  /**
+   * dictionary values
+   */
+  private byte[][] binaries;
+
+  CarbonDictionaryWrapper(CarbonDictionary dictionary) {
+    binaries = new byte[dictionary.getDictionarySize()][];
+    for (int i = 0; i < binaries.length; i++) {
+      binaries[i] = dictionary.getDictionaryValue(i);
+    }
+  }
+
+  @Override public int decodeToInt(int id) {
+    throw new UnsupportedOperationException("Dictionary encoding does not support int");
+  }
+
+  @Override public long decodeToLong(int id) {
+    throw new UnsupportedOperationException("Dictionary encoding does not support long");
+  }
+
+  @Override public float decodeToFloat(int id) {
+    throw new UnsupportedOperationException("Dictionary encoding does not support float");
+  }
+
+  @Override public double decodeToDouble(int id) {
+    throw new UnsupportedOperationException("Dictionary encoding does not support double");
+  }
+
+  @Override public byte[] decodeToBinary(int id) {
+    return binaries[id];
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2ab2254b/integration/spark-datasource/src/main/spark2.3plus/org/apache/spark/sql/CarbonVectorProxy.java
----------------------------------------------------------------------
diff --git a/integration/spark-datasource/src/main/spark2.3plus/org/apache/spark/sql/CarbonVectorProxy.java b/integration/spark-datasource/src/main/spark2.3plus/org/apache/spark/sql/CarbonVectorProxy.java
index 0f23294..4a0fb9e 100644
--- a/integration/spark-datasource/src/main/spark2.3plus/org/apache/spark/sql/CarbonVectorProxy.java
+++ b/integration/spark-datasource/src/main/spark2.3plus/org/apache/spark/sql/CarbonVectorProxy.java
@@ -18,6 +18,8 @@ package org.apache.spark.sql;
 
 import java.math.BigInteger;
 
+import org.apache.carbondata.core.scan.result.vector.CarbonDictionary;
+
 import org.apache.spark.memory.MemoryMode;
 import org.apache.spark.sql.catalyst.InternalRow;
 import org.apache.spark.sql.execution.vectorized.Dictionary;
@@ -262,9 +264,9 @@ public class CarbonVectorProxy {
         return columnVectors[ordinal].hasDictionary();
     }
 
-    public void setDictionary(Object dictionary, int ordinal) {
-        if (dictionary instanceof Dictionary) {
-            columnVectors[ordinal].setDictionary((Dictionary) dictionary);
+    public void setDictionary(CarbonDictionary dictionary, int ordinal) {
+        if (null != dictionary) {
+            columnVectors[ordinal].setDictionary(new CarbonDictionaryWrapper(dictionary));
         } else {
             columnVectors[ordinal].setDictionary(null);
         }


[45/45] carbondata git commit: [CARBONDATA-2990] Queries slow down after some time due to broadcast issue

Posted by ra...@apache.org.
[CARBONDATA-2990] Queries slow down after some time due to broadcast issue

Problem
It is observed that during consecutive run of queries after some time queries are slowing down. This is causing the degrade in query performance.
No exception is thrown in driver and executor logs but as observed from the logs the time to broadcast hadoop conf is increasing after every query run.

Analysis

This is happening because in carbon SerializableConfiguration class is overriden from spark. Spark registers this class with Kryo serializer and hence the computation using the kryo is fast. The same benefit is not observed in carbondata becuase of overriding the class.
Internal Spark sizeEstimator calculates the size of object and there are few extra objects in carbondata overriden class because of which the computation time is increasing.
Solution
Use the spark class instead of overriding the class in carbondata

This closes #2803


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

Branch: refs/heads/branch-1.5
Commit: 3c7b33992e06d81fb47d81bf8ccf7884f845b3ff
Parents: 19097f2
Author: manishgupta88 <to...@gmail.com>
Authored: Mon Oct 8 19:38:54 2018 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Tue Oct 9 13:38:51 2018 +0530

----------------------------------------------------------------------
 .../carbondata/spark/load/CsvRDDHelper.scala    |  4 +--
 .../load/DataLoadProcessBuilderOnSpark.scala    |  6 ++--
 .../load/DataLoadProcessorStepOnSpark.scala     |  6 ++--
 .../apache/carbondata/spark/rdd/CarbonRDD.scala |  4 +--
 .../spark/rdd/NewCarbonDataLoadRDD.scala        | 33 --------------------
 .../apache/spark/sql/util/SparkSQLUtil.scala    | 21 ++++++++++++-
 .../spark/rdd/CarbonDataRDDFactory.scala        |  4 +--
 .../spark/sql/CarbonDictionaryDecoder.scala     |  8 ++---
 .../management/CarbonLoadDataCommand.scala      |  7 +++--
 .../command/mutation/DeleteExecution.scala      |  7 ++---
 .../command/mutation/HorizontalCompaction.scala |  8 ++---
 11 files changed, 46 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/3c7b3399/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/CsvRDDHelper.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/CsvRDDHelper.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/CsvRDDHelper.scala
index 8d6dd32..5511645 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/CsvRDDHelper.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/CsvRDDHelper.scala
@@ -34,6 +34,7 @@ import org.apache.spark.sql.SparkSession
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions.GenericInternalRow
 import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, PartitionedFile}
+import org.apache.spark.sql.util.SparkSQLUtil
 import org.apache.spark.sql.util.SparkSQLUtil.sessionState
 
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
@@ -41,7 +42,6 @@ import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.util.ThreadLocalSessionInfo
 import org.apache.carbondata.processing.loading.csvinput.CSVInputFormat
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
-import org.apache.carbondata.spark.rdd.SerializableConfiguration
 import org.apache.carbondata.spark.util.CommonUtil
 
 object CsvRDDHelper {
@@ -110,7 +110,7 @@ object CsvRDDHelper {
     closePartition()
 
     // 2. read function
-    val serializableConfiguration = new SerializableConfiguration(hadoopConf)
+    val serializableConfiguration = SparkSQLUtil.getSerializableConfigurableInstance(hadoopConf)
     val readFunction = new (PartitionedFile => Iterator[InternalRow]) with Serializable {
       override def apply(file: PartitionedFile): Iterator[InternalRow] = {
         new Iterator[InternalRow] {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/3c7b3399/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessBuilderOnSpark.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessBuilderOnSpark.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessBuilderOnSpark.scala
index 2e74a94..923676c 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessBuilderOnSpark.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessBuilderOnSpark.scala
@@ -23,6 +23,7 @@ import org.apache.hadoop.conf.Configuration
 import org.apache.spark.TaskContext
 import org.apache.spark.sql.{DataFrame, SparkSession}
 import org.apache.spark.sql.execution.command.ExecutionErrors
+import org.apache.spark.sql.util.SparkSQLUtil
 import org.apache.spark.storage.StorageLevel
 
 import org.apache.carbondata.common.logging.LogServiceFactory
@@ -35,7 +36,6 @@ import org.apache.carbondata.processing.loading.{DataLoadProcessBuilder, Failure
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.processing.sort.sortdata.{NewRowComparator, NewRowComparatorForNormalDims, SortParameters}
 import org.apache.carbondata.processing.util.CarbonDataProcessorUtil
-import org.apache.carbondata.spark.rdd.SerializableConfiguration
 
 /**
  * Use sortBy operator in spark to load the data
@@ -66,7 +66,7 @@ object DataLoadProcessBuilderOnSpark {
     val sortStepRowCounter = sc.accumulator(0, "Sort Processor Accumulator")
     val writeStepRowCounter = sc.accumulator(0, "Write Processor Accumulator")
 
-    val conf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
+    val conf = SparkSQLUtil.broadCastHadoopConf(sc, hadoopConf)
     // 1. Input
     val inputRDD = originRDD
       .mapPartitions(rows => DataLoadProcessorStepOnSpark.toRDDIterator(rows, modelBroadcast))
@@ -121,7 +121,7 @@ object DataLoadProcessBuilderOnSpark {
     // 4. Write
     sc.runJob(sortRDD, (context: TaskContext, rows: Iterator[CarbonRow]) =>
       DataLoadProcessorStepOnSpark.writeFunc(rows, context.partitionId, modelBroadcast,
-        writeStepRowCounter, conf))
+        writeStepRowCounter, conf.value.value))
 
     // clean cache only if persisted and keeping unpersist non-blocking as non-blocking call will
     // not have any functional impact as spark automatically monitors the cache usage on each node

http://git-wip-us.apache.org/repos/asf/carbondata/blob/3c7b3399/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
index f17bd91..f1a12bf 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
@@ -42,7 +42,7 @@ import org.apache.carbondata.processing.loading.steps.DataWriterProcessorStepImp
 import org.apache.carbondata.processing.sort.sortdata.SortParameters
 import org.apache.carbondata.processing.store.{CarbonFactHandler, CarbonFactHandlerFactory}
 import org.apache.carbondata.processing.util.{CarbonBadRecordUtil, CarbonDataProcessorUtil}
-import org.apache.carbondata.spark.rdd.{NewRddIterator, SerializableConfiguration, StringArrayRow}
+import org.apache.carbondata.spark.rdd.{NewRddIterator, StringArrayRow}
 import org.apache.carbondata.spark.util.{CarbonScalaUtil, Util}
 
 object DataLoadProcessorStepOnSpark {
@@ -230,8 +230,8 @@ object DataLoadProcessorStepOnSpark {
       index: Int,
       modelBroadcast: Broadcast[CarbonLoadModel],
       rowCounter: Accumulator[Int],
-      conf: Broadcast[SerializableConfiguration]) {
-    ThreadLocalSessionInfo.setConfigurationToCurrentThread(conf.value.value)
+      conf: Configuration) {
+    ThreadLocalSessionInfo.setConfigurationToCurrentThread(conf)
     var model: CarbonLoadModel = null
     var tableName: String = null
     var rowConverter: RowConverterImpl = null

http://git-wip-us.apache.org/repos/asf/carbondata/blob/3c7b3399/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
index 3a02f85..83cd59c 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
@@ -22,7 +22,6 @@ import scala.reflect.ClassTag
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext}
-import org.apache.spark.broadcast.Broadcast
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.SparkSession
 import org.apache.spark.sql.util.SparkSQLUtil
@@ -49,8 +48,7 @@ abstract class CarbonRDD[T: ClassTag](
 
   @transient val hadoopConf = SparkSQLUtil.sessionState(ss).newHadoopConf()
 
-  val config: Broadcast[SerializableConfiguration] = sparkContext
-    .broadcast(new SerializableConfiguration(hadoopConf))
+  val config = SparkSQLUtil.broadCastHadoopConf(sparkContext, hadoopConf)
 
   /** Construct an RDD with just a one-to-one dependency on one parent */
   def this(@transient sparkSession: SparkSession, @transient oneParent: RDD[_]) =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/3c7b3399/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
index 87c8e4c..6076e4a 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
@@ -54,39 +54,6 @@ import org.apache.carbondata.processing.util.CarbonQueryUtil
 import org.apache.carbondata.spark.DataLoadResult
 import org.apache.carbondata.spark.util.{CarbonScalaUtil, CommonUtil, Util}
 
-class SerializableConfiguration(@transient var value: Configuration) extends Serializable {
-
-  @transient
-  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-
-  private def writeObject(out: ObjectOutputStream): Unit =
-    try {
-      out.defaultWriteObject()
-      value.write(out)
-    } catch {
-      case e: IOException =>
-        LOGGER.error(e, "Exception encountered")
-        throw e
-      case NonFatal(e) =>
-        LOGGER.error(e, "Exception encountered")
-        throw new IOException(e)
-    }
-
-
-  private def readObject(in: ObjectInputStream): Unit =
-    try {
-      value = new Configuration(false)
-      value.readFields(in)
-    } catch {
-      case e: IOException =>
-        LOGGER.error(e, "Exception encountered")
-        throw e
-      case NonFatal(e) =>
-        LOGGER.error(e, "Exception encountered")
-        throw new IOException(e)
-    }
-}
-
 /**
  * This partition class use to split by Host
  *

http://git-wip-us.apache.org/repos/asf/carbondata/blob/3c7b3399/integration/spark-common/src/main/scala/org/apache/spark/sql/util/SparkSQLUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/util/SparkSQLUtil.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/util/SparkSQLUtil.scala
index b7d47a0..9ffe6e1 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/util/SparkSQLUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/util/SparkSQLUtil.scala
@@ -19,11 +19,14 @@ package org.apache.spark.sql.util
 
 import java.lang.reflect.Method
 
+import org.apache.hadoop.conf.Configuration
+import org.apache.spark.SparkContext
+import org.apache.spark.broadcast.Broadcast
 import org.apache.spark.sql.{DataFrame, Dataset, SparkSession}
 import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.internal.{SessionState, SQLConf}
-import org.apache.spark.util.{CarbonReflectionUtils, SparkUtil}
+import org.apache.spark.util.{CarbonReflectionUtils, SerializableConfiguration, SparkUtil}
 
 object SparkSQLUtil {
   def sessionState(sparkSession: SparkSession): SessionState = sparkSession.sessionState
@@ -99,4 +102,20 @@ object SparkSQLUtil {
       throw new UnsupportedOperationException("Spark version not supported")
     }
   }
+
+  /**
+   * Method to broadcast a variable using spark SerializableConfiguration class
+   *
+   * @param sparkContext
+   * @param hadoopConf
+   * @return
+   */
+  def broadCastHadoopConf(sparkContext: SparkContext,
+      hadoopConf: Configuration): Broadcast[SerializableConfiguration] = {
+    sparkContext.broadcast(getSerializableConfigurableInstance(hadoopConf))
+  }
+
+  def getSerializableConfigurableInstance(hadoopConf: Configuration): SerializableConfiguration = {
+    new SerializableConfiguration(hadoopConf)
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/3c7b3399/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 6350b50..0ec3bc6 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -43,7 +43,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.execution.command.{CompactionModel, ExecutionErrors, UpdateTableModel}
 import org.apache.spark.sql.hive.DistributionUtil
 import org.apache.spark.sql.optimizer.CarbonFilters
-import org.apache.spark.sql.util.CarbonException
+import org.apache.spark.sql.util.{CarbonException, SparkSQLUtil}
 
 import org.apache.carbondata.common.constants.LoggerAction
 import org.apache.carbondata.common.logging.LogServiceFactory
@@ -728,7 +728,7 @@ object CarbonDataRDDFactory {
 
       // because partitionId=segmentIdIndex*parallelism+RandomPart and RandomPart<parallelism,
       // so segmentIdIndex=partitionId/parallelism, this has been verified.
-      val conf = sqlContext.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
+      val conf = SparkSQLUtil.broadCastHadoopConf(sqlContext.sparkSession.sparkContext, hadoopConf)
       partitionByRdd.map(_._2).mapPartitions { partition =>
         ThreadLocalSessionInfo.setConfigurationToCurrentThread(conf.value.value)
         val partitionId = TaskContext.getPartitionId()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/3c7b3399/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
index d0ed56e..ff7ac60 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
@@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.plans.physical.Partitioning
 import org.apache.spark.sql.execution.{CodegenSupport, SparkPlan, UnaryExecNode}
 import org.apache.spark.sql.optimizer.CarbonDecoderRelation
 import org.apache.spark.sql.types._
-import org.apache.spark.sql.util.SparkTypeConverter
+import org.apache.spark.sql.util.{SparkSQLUtil, SparkTypeConverter}
 
 import org.apache.carbondata.core.cache.{Cache, CacheProvider, CacheType}
 import org.apache.carbondata.core.cache.dictionary.{Dictionary, DictionaryColumnUniqueIdentifier}
@@ -44,7 +44,7 @@ import org.apache.carbondata.core.scan.executor.util.QueryUtil
 import org.apache.carbondata.core.util.{DataTypeUtil, ThreadLocalSessionInfo}
 import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil
 import org.apache.carbondata.spark.CarbonAliasDecoderRelation
-import org.apache.carbondata.spark.rdd.{CarbonRDDWithTableInfo, SerializableConfiguration}
+import org.apache.carbondata.spark.rdd.CarbonRDDWithTableInfo
 
 /**
  * It decodes the data.
@@ -76,8 +76,8 @@ case class CarbonDictionaryDecoder(
         (carbonTable.getTableName, carbonTable)
       }.toMap
 
-      val conf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(sparkSession
-        .sessionState.newHadoopConf()))
+      val conf = SparkSQLUtil
+        .broadCastHadoopConf(sparkSession.sparkContext, sparkSession.sessionState.newHadoopConf())
       if (CarbonDictionaryDecoder.isRequiredToDecode(getDictionaryColumnIds)) {
         val dataTypes = child.output.map { attr => attr.dataType }
         child.execute().mapPartitions { iter =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/3c7b3399/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
index f7a5f42..43c8b86 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
@@ -42,6 +42,7 @@ import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, FindDataSou
 import org.apache.spark.sql.hive.CarbonRelation
 import org.apache.spark.sql.optimizer.CarbonFilters
 import org.apache.spark.sql.types._
+import org.apache.spark.sql.util.SparkSQLUtil
 import org.apache.spark.storage.StorageLevel
 import org.apache.spark.unsafe.types.UTF8String
 import org.apache.spark.util.{CarbonReflectionUtils, CausedBy, FileUtils}
@@ -78,7 +79,7 @@ import org.apache.carbondata.processing.util.{CarbonBadRecordUtil, CarbonDataPro
 import org.apache.carbondata.spark.dictionary.provider.SecureDictionaryServiceProvider
 import org.apache.carbondata.spark.dictionary.server.SecureDictionaryServer
 import org.apache.carbondata.spark.load.{CsvRDDHelper, DataLoadProcessorStepOnSpark}
-import org.apache.carbondata.spark.rdd.{CarbonDataRDDFactory, SerializableConfiguration}
+import org.apache.carbondata.spark.rdd.CarbonDataRDDFactory
 import org.apache.carbondata.spark.util.{CarbonScalaUtil, GlobalDictionaryUtil}
 
 case class CarbonLoadDataCommand(
@@ -986,8 +987,8 @@ case class CarbonLoadDataCommand(
           array
         }
       }
-    val conf = sparkSession.sparkContext
-      .broadcast(new SerializableConfiguration(sparkSession.sessionState.newHadoopConf()))
+    val conf = SparkSQLUtil
+      .broadCastHadoopConf(sparkSession.sparkContext, sparkSession.sessionState.newHadoopConf())
     val finalRDD = convertRDD.mapPartitionsWithIndex { case(index, rows) =>
         DataTypeUtil.setDataTypeConverter(new SparkDataTypeConverterImpl)
       ThreadLocalSessionInfo.setConfigurationToCurrentThread(conf.value.value)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/3c7b3399/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala
index 4921b33..7e7f671 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala
@@ -32,6 +32,7 @@ import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.execution.command.ExecutionErrors
 import org.apache.spark.sql.optimizer.CarbonFilters
+import org.apache.spark.sql.util.SparkSQLUtil
 
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
@@ -49,7 +50,6 @@ import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil
 import org.apache.carbondata.processing.exception.MultipleMatchingException
 import org.apache.carbondata.processing.loading.FailureCauses
 import org.apache.carbondata.spark.DeleteDelataResultImpl
-import org.apache.carbondata.spark.rdd.SerializableConfiguration
 
 object DeleteExecution {
   val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getName)
@@ -120,9 +120,8 @@ object DeleteExecution {
         blockMappingVO.getCompleteBlockRowDetailVO.asScala.toSeq,
         keyRdd.partitions.length)
 
-    val conf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(sparkSession
-      .sessionState.newHadoopConf()))
-
+    val conf = SparkSQLUtil
+      .broadCastHadoopConf(sparkSession.sparkContext, sparkSession.sessionState.newHadoopConf())
     val rdd = rowContRdd.join(keyRdd)
     res = rdd.mapPartitionsWithIndex(
       (index: Int, records: Iterator[((String), (RowCountDetailsVO, Iterable[Row]))]) =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/3c7b3399/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/HorizontalCompaction.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/HorizontalCompaction.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/HorizontalCompaction.scala
index 66066ed..35fc3c3 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/HorizontalCompaction.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/HorizontalCompaction.scala
@@ -26,16 +26,16 @@ import org.apache.spark.sql._
 import org.apache.spark.sql.execution.command.AlterTableModel
 import org.apache.spark.sql.execution.command.management.CarbonAlterTableCompactionCommand
 import org.apache.spark.sql.hive.CarbonRelation
+import org.apache.spark.sql.util.SparkSQLUtil
 
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.datamap.Segment
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager
-import org.apache.carbondata.core.util.{ThreadLocalSessionInfo}
+import org.apache.carbondata.core.util.ThreadLocalSessionInfo
 import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil
 import org.apache.carbondata.processing.merger.{CarbonDataMergerUtil, CarbonDataMergerUtilResult, CompactionType}
-import org.apache.carbondata.spark.rdd.SerializableConfiguration
 
 object HorizontalCompaction {
 
@@ -191,8 +191,8 @@ object HorizontalCompaction {
 
       val timestamp = factTimeStamp
       val updateStatusDetails = segmentUpdateStatusManager.getUpdateStatusDetails
-      val conf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(sparkSession
-        .sessionState.newHadoopConf()))
+      val conf = SparkSQLUtil
+        .broadCastHadoopConf(sparkSession.sparkContext, sparkSession.sessionState.newHadoopConf())
       val result = rdd1.mapPartitions(iter =>
         new Iterator[Seq[CarbonDataMergerUtilResult]] {
           ThreadLocalSessionInfo.setConfigurationToCurrentThread(conf.value.value)


[07/45] carbondata git commit: [HOTFIX] fix compile error after merging PR-2749

Posted by ra...@apache.org.
[HOTFIX] fix compile error after merging PR-2749

fix compile error after merging PR-2749

This closes #2763


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

Branch: refs/heads/branch-1.5
Commit: a9ddfbd7bdc9dc2e4c68b9bc53878ba257c80cd3
Parents: b3a5e3a
Author: Zhang Zhichao <44...@qq.com>
Authored: Wed Sep 26 01:00:11 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Wed Sep 26 18:48:50 2018 +0800

----------------------------------------------------------------------
 .../org/apache/carbondata/tool/CarbonCliTest.java   | 16 ++++------------
 1 file changed, 4 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/a9ddfbd7/tools/cli/src/test/java/org/apache/carbondata/tool/CarbonCliTest.java
----------------------------------------------------------------------
diff --git a/tools/cli/src/test/java/org/apache/carbondata/tool/CarbonCliTest.java b/tools/cli/src/test/java/org/apache/carbondata/tool/CarbonCliTest.java
index 4dc34c4..fcd46c8 100644
--- a/tools/cli/src/test/java/org/apache/carbondata/tool/CarbonCliTest.java
+++ b/tools/cli/src/test/java/org/apache/carbondata/tool/CarbonCliTest.java
@@ -45,10 +45,8 @@ public class CarbonCliTest {
     fields[0] = new Field("name", DataTypes.STRING);
     fields[1] = new Field("age", DataTypes.INT);
 
-    TestUtil.writeFilesAndVerify(5000000, new Schema(fields), path, new String[]{"name"},
-        true, 3, 8, true);
-    TestUtil.writeFilesAndVerify(5000000, new Schema(fields), path, new String[]{"name"},
-        true, 3, 8, true);
+    TestUtil.writeFilesAndVerify(5000000, new Schema(fields), path, new String[]{"name"}, 3, 8);
+    TestUtil.writeFilesAndVerify(5000000, new Schema(fields), path, new String[]{"name"}, 3, 8);
   }
 
   @Test
@@ -102,10 +100,7 @@ public class CarbonCliTest {
     Assert.assertTrue(
         output.contains(
             "## Table Properties\n"
-          + "Property Name              Property Value  \n"
-          + "'table_blocksize'          '8'             \n"
-          + "'table_blocklet_size'      '3'             \n"
-          + "'local_dictionary_enable'  'false'    "));
+          + "schema file not found"));
 
     String[] args4 = {"-cmd", "summary", "-p", path, "-b"};
     out = new ByteArrayOutputStream();
@@ -163,10 +158,7 @@ public class CarbonCliTest {
     Assert.assertTrue(
         output.contains(
             "## Table Properties\n"
-          + "Property Name              Property Value  \n"
-          + "'table_blocksize'          '8'             \n"
-          + "'table_blocklet_size'      '3'             \n"
-          + "'local_dictionary_enable'  'false'    "));
+          + "schema file not found"));
 
     Assert.assertTrue(
         output.contains(


[44/45] carbondata git commit: [CARBONDATA-2994] unify badrecordpath property name for create and load

Posted by ra...@apache.org.
[CARBONDATA-2994] unify badrecordpath property name for create and load

Problem:
Currently bad records path can be specified in create and load. In create the property name is bad_records_path and load is bad_record_path. This can cause confusion for the user.

Solution: Use bad_record_path as the property for create so that both load and create use the same name.

This closes #2799


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

Branch: refs/heads/branch-1.5
Commit: 19097f272fe3227c71c86338bb8bf788e87cd4aa
Parents: fa08825
Author: kunal642 <ku...@gmail.com>
Authored: Fri Oct 5 14:57:26 2018 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Mon Oct 8 14:29:59 2018 +0530

----------------------------------------------------------------------
 docs/ddl-of-carbondata.md                                | 11 +++++++++++
 docs/dml-of-carbondata.md                                |  8 --------
 .../carbondata/hadoop/api/CarbonTableOutputFormat.java   |  2 +-
 .../StandardPartitionBadRecordLoggerTest.scala           |  2 +-
 .../org/apache/carbondata/spark/StreamingOption.scala    |  2 +-
 .../sql/execution/command/carbonTableSchemaCommon.scala  |  4 ++--
 .../command/table/CarbonDescribeFormattedCommand.scala   |  7 +++++++
 .../spark/carbondata/BadRecordPathLoadOptionTest.scala   |  4 ++--
 .../spark/carbondata/TestStreamingTableOperation.scala   |  8 ++++----
 .../carbondata/processing/util/CarbonBadRecordUtil.java  |  2 +-
 10 files changed, 30 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/19097f27/docs/ddl-of-carbondata.md
----------------------------------------------------------------------
diff --git a/docs/ddl-of-carbondata.md b/docs/ddl-of-carbondata.md
index 22d754a..c1a891d 100644
--- a/docs/ddl-of-carbondata.md
+++ b/docs/ddl-of-carbondata.md
@@ -33,6 +33,7 @@ CarbonData DDL statements are documented here,which includes:
   * [Hive/Parquet folder Structure](#support-flat-folder-same-as-hiveparquet)
   * [Extra Long String columns](#string-longer-than-32000-characters)
   * [Compression for Table](#compression-for-table)
+  * [Bad Records Path](#bad-records-path)
 * [CREATE TABLE AS SELECT](#create-table-as-select)
 * [CREATE EXTERNAL TABLE](#create-external-table)
   * [External Table on Transactional table location](#create-external-table-on-managed-table-data-location)
@@ -454,6 +455,16 @@ CarbonData DDL statements are documented here,which includes:
      ```
      carbon.column.compressor=zstd
      ```
+     
+   - ##### Bad Records Path
+     This property is used to specify the location where bad records would be written.
+     As the table path remains the same after rename therefore the user can use this property to
+     specify bad records path for the table at the time of creation, so that the same path can 
+     be later viewed in table description for reference.
+     
+     ```
+       TBLPROPERTIES('BAD_RECORD_PATH'='/opt/badrecords'')
+     ```
 
 ## CREATE TABLE AS SELECT
   This function allows user to create a Carbon table from any of the Parquet/Hive/Carbon table. This is beneficial when the user wants to create Carbon table from any other Parquet/Hive table and use the Carbon query engine to query and achieve better query results for cases where Carbon is faster than other file formats. Also this feature can be used for backing up the data.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/19097f27/docs/dml-of-carbondata.md
----------------------------------------------------------------------
diff --git a/docs/dml-of-carbondata.md b/docs/dml-of-carbondata.md
index db7c118..393ebd3 100644
--- a/docs/dml-of-carbondata.md
+++ b/docs/dml-of-carbondata.md
@@ -240,14 +240,6 @@ CarbonData DML statements are documented here,which includes:
   * Since Bad Records Path can be specified in create, load and carbon properties. 
     Therefore, value specified in load will have the highest priority, and value specified in carbon properties will have the least priority.
 
-   **Bad Records Path:**
-         This property is used to specify the location where bad records would be written.
-        
-
-   ```
-   TBLPROPERTIES('BAD_RECORDS_PATH'='/opt/badrecords'')
-   ```
-
   Example:
 
   ```

http://git-wip-us.apache.org/repos/asf/carbondata/blob/19097f27/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableOutputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableOutputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableOutputFormat.java
index 762983b..f0ad94d 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableOutputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableOutputFormat.java
@@ -374,7 +374,7 @@ public class CarbonTableOutputFormat extends FileOutputFormat<NullWritable, Obje
     String badRecordsPath = conf.get(BAD_RECORD_PATH);
     if (StringUtils.isEmpty(badRecordsPath)) {
       badRecordsPath =
-          carbonTable.getTableInfo().getFactTable().getTableProperties().get("bad_records_path");
+          carbonTable.getTableInfo().getFactTable().getTableProperties().get("bad_record_path");
       if (StringUtils.isEmpty(badRecordsPath)) {
         badRecordsPath = carbonProperty
             .getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORD_PATH, carbonProperty

http://git-wip-us.apache.org/repos/asf/carbondata/blob/19097f27/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionBadRecordLoggerTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionBadRecordLoggerTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionBadRecordLoggerTest.scala
index d9e5d3c..9689f3d 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionBadRecordLoggerTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionBadRecordLoggerTest.scala
@@ -44,7 +44,7 @@ class StandardPartitionBadRecordLoggerTest extends QueryTest with BeforeAndAfter
   test("test partition redirect") {
     sql(
       s"""CREATE TABLE IF NOT EXISTS sales(ID BigInt, date Timestamp,
-          actual_price Double, Quantity int, sold_price Decimal(19,2)) partitioned by (country String) STORED BY 'carbondata' TBLPROPERTIES('BAD_RECORDS_PATH'='$warehouse')""")
+          actual_price Double, Quantity int, sold_price Decimal(19,2)) partitioned by (country String) STORED BY 'carbondata' TBLPROPERTIES('BAD_RECORD_PATH'='$warehouse')""")
 
     val csvFilePath = s"$resourcesPath/badrecords/datasample.csv"
     sql("LOAD DATA local inpath '" + csvFilePath + "' INTO TABLE sales OPTIONS"

http://git-wip-us.apache.org/repos/asf/carbondata/blob/19097f27/integration/spark-common/src/main/scala/org/apache/carbondata/spark/StreamingOption.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/StreamingOption.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/StreamingOption.scala
index 2402d83..087bef2 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/StreamingOption.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/StreamingOption.scala
@@ -56,7 +56,7 @@ class StreamingOption(val userInputMap: Map[String, String]) {
 
   lazy val badRecordsPath: String =
     userInputMap
-      .getOrElse("bad_records_path", CarbonProperties.getInstance()
+      .getOrElse("bad_record_path", CarbonProperties.getInstance()
         .getProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC,
           CarbonCommonConstants.CARBON_BADRECORDS_LOC_DEFAULT_VAL))
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/19097f27/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
index da22658..5e0fe8b 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
@@ -844,7 +844,7 @@ class TableNewProcessor(cm: TableModel) {
       cm.tableName,
       tableSchema.getTableId,
       cm.databaseNameOp.getOrElse("default"))
-    tablePropertiesMap.put("bad_records_path", badRecordsPath)
+    tablePropertiesMap.put("bad_record_path", badRecordsPath)
     tableSchema.setTableProperties(tablePropertiesMap)
     if (cm.bucketFields.isDefined) {
       val bucketCols = cm.bucketFields.get.bucketColumns.map { b =>
@@ -898,7 +898,7 @@ class TableNewProcessor(cm: TableModel) {
       tableId: String,
       databaseName: String): String = {
     val badRecordsPath = tablePropertiesMap.asScala
-      .getOrElse("bad_records_path", CarbonCommonConstants.CARBON_BADRECORDS_LOC_DEFAULT_VAL)
+      .getOrElse("bad_record_path", CarbonCommonConstants.CARBON_BADRECORDS_LOC_DEFAULT_VAL)
     if (badRecordsPath == null || badRecordsPath.isEmpty) {
       CarbonCommonConstants.CARBON_BADRECORDS_LOC_DEFAULT_VAL
     } else {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/19097f27/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala
index 9b9e8bd..029c0e3 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.command.table
 
 import scala.collection.JavaConverters._
 
+import org.apache.commons.lang.StringUtils
 import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
@@ -217,6 +218,12 @@ private[sql] case class CarbonDescribeFormattedCommand(
     results ++= Seq(("SORT_COLUMNS", relation.metaData.carbonTable.getSortColumns(
       relation.carbonTable.getTableName).asScala
       .map(column => column).mkString(","), ""))
+
+    val bad_record_path = relation.carbonTable.getTableInfo.getFactTable
+      .getTableProperties.get("bad_record_path")
+    if (!StringUtils.isEmpty(bad_record_path)) {
+      results ++= Seq(("BAD_RECORD_PATH", bad_record_path, ""))
+    }
     // add columns configured in column meta cache
     if (null != tblProps.get(CarbonCommonConstants.COLUMN_META_CACHE)) {
       results ++=

http://git-wip-us.apache.org/repos/asf/carbondata/blob/19097f27/integration/spark2/src/test/scala/org/apache/spark/carbondata/BadRecordPathLoadOptionTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/BadRecordPathLoadOptionTest.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/BadRecordPathLoadOptionTest.scala
index dfefa9b..e3e261f 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/BadRecordPathLoadOptionTest.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/BadRecordPathLoadOptionTest.scala
@@ -44,7 +44,7 @@ class BadRecordPathLoadOptionTest extends Spark2QueryTest with BeforeAndAfterAll
   test("data load log file and csv file written at the configured location") {
     sql(
       s"""CREATE TABLE IF NOT EXISTS salestest(ID BigInt, date Timestamp, country String,
-          actual_price Double, Quantity int, sold_price Decimal(19,2)) STORED BY 'carbondata' TBLPROPERTIES('BAD_RECORDS_PATH'='$warehouse')""")
+          actual_price Double, Quantity int, sold_price Decimal(19,2)) STORED BY 'carbondata' TBLPROPERTIES('BAD_RECORD_PATH'='$warehouse')""")
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
     val csvFilePath = s"$resourcesPath/badrecords/datasample.csv"
@@ -64,7 +64,7 @@ class BadRecordPathLoadOptionTest extends Spark2QueryTest with BeforeAndAfterAll
   def isFilesWrittenAtBadStoreLocation: Boolean = {
     val badStorePath =
       CarbonEnv.getCarbonTable(Some("default"), "salestest")(sqlContext.sparkSession).getTableInfo
-        .getFactTable.getTableProperties.get("bad_records_path") + "/0/0"
+        .getFactTable.getTableProperties.get("bad_record_path") + "/0/0"
     val carbonFile: CarbonFile = FileFactory
       .getCarbonFile(badStorePath, FileFactory.getFileType(badStorePath))
     var exists: Boolean = carbonFile.exists()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/19097f27/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
index 607c429..62c0221 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
@@ -1880,7 +1880,7 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
         |  'interval'='1 seconds',
         |  'BAD_RECORDS_LOGGER_ENABLE' = 'FALSE',
         |  'BAD_RECORDS_ACTION' = 'FORCE',
-        |  'BAD_RECORDS_PATH'='$warehouse')
+        |  'BAD_RECORD_PATH'='$warehouse')
         |AS
         |  SELECT *
         |  FROM source
@@ -1894,7 +1894,7 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
         |  'interval'='1 seconds',
         |  'BAD_RECORDS_LOGGER_ENABLE' = 'FALSE',
         |  'BAD_RECORDS_ACTION' = 'FORCE',
-        |  'BAD_RECORDS_PATH'='$warehouse')
+        |  'BAD_RECORD_PATH'='$warehouse')
         |AS
         |  SELECT *
         |  FROM source
@@ -2554,7 +2554,7 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
          | )
          | STORED BY 'carbondata'
          | TBLPROPERTIES(${if (streaming) "'streaming'='true', " else "" }
-         | 'sort_columns'='name', 'dictionary_include'='city,register', 'BAD_RECORDS_PATH'='$badRecordFilePath')
+         | 'sort_columns'='name', 'dictionary_include'='city,register', 'BAD_RECORD_PATH'='$badRecordFilePath')
          | """.stripMargin)
 
     if (withBatchLoad) {
@@ -2583,7 +2583,7 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
          | )
          | STORED BY 'carbondata'
          | TBLPROPERTIES(${if (streaming) "'streaming'='true', " else "" }
-         | 'sort_columns'='name', 'dictionary_include'='id,name,salary,tax,percent,updated', 'BAD_RECORDS_PATH'='$badRecordFilePath')
+         | 'sort_columns'='name', 'dictionary_include'='id,name,salary,tax,percent,updated', 'BAD_RECORD_PATH'='$badRecordFilePath')
          | """.stripMargin)
 
     if (withBatchLoad) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/19097f27/processing/src/main/java/org/apache/carbondata/processing/util/CarbonBadRecordUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonBadRecordUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonBadRecordUtil.java
index 55bc580..ecc6afb 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonBadRecordUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonBadRecordUtil.java
@@ -130,7 +130,7 @@ public class CarbonBadRecordUtil {
   public static String getBadRecordsPath(Map<String, String> loadOptions, CarbonTable table) {
     String badRecordsFromLoad = loadOptions.get("bad_record_path");
     String badRecordsFromCreate =
-        table.getTableInfo().getFactTable().getTableProperties().get("bad_records_path");
+        table.getTableInfo().getFactTable().getTableProperties().get("bad_record_path");
     String badRecordsPath;
     if (StringUtils.isNotEmpty(badRecordsFromLoad)) {
       badRecordsPath =


[06/45] carbondata git commit: [CARBONDATA-2966]Update Documentation For Avro DataType conversion

Posted by ra...@apache.org.
[CARBONDATA-2966]Update Documentation For Avro DataType conversion

Updated document for the following features:
1. Avro DataType conversion to carbon
2. Remove min, max for varchar columns
3. LRU enhancements for driver cache

This closes #2756


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

Branch: refs/heads/branch-1.5
Commit: b3a5e3a8bb4b051779f91bca071336703742296c
Parents: d84cd81
Author: Indhumathi27 <in...@gmail.com>
Authored: Mon Sep 24 23:34:04 2018 +0530
Committer: kunal642 <ku...@gmail.com>
Committed: Wed Sep 26 16:16:02 2018 +0530

----------------------------------------------------------------------
 docs/configuration-parameters.md           |  6 ++-
 docs/faq.md                                | 16 +++++++
 docs/sdk-guide.md                          | 55 +++++++++++++++++--------
 docs/supported-data-types-in-carbondata.md |  1 +
 4 files changed, 58 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/b3a5e3a8/docs/configuration-parameters.md
----------------------------------------------------------------------
diff --git a/docs/configuration-parameters.md b/docs/configuration-parameters.md
index c6b0fcb..7edae47 100644
--- a/docs/configuration-parameters.md
+++ b/docs/configuration-parameters.md
@@ -42,6 +42,7 @@ This section provides the details of all the configurations required for the Car
 | carbon.lock.type | LOCALLOCK | This configuration specifies the type of lock to be acquired during concurrent operations on table. There are following types of lock implementation: - LOCALLOCK: Lock is created on local file system as file. This lock is useful when only one spark driver (thrift server) runs on a machine and no other CarbonData spark application is launched concurrently. - HDFSLOCK: Lock is created on HDFS file system as file. This lock is useful when multiple CarbonData spark applications are launched and no ZooKeeper is running on cluster and HDFS supports file based locking. |
 | carbon.lock.path | TABLEPATH | This configuration specifies the path where lock files have to be created. Recommended to configure zookeeper lock type or configure HDFS lock path(to this property) in case of S3 file system as locking is not feasible on S3. |
 | carbon.unsafe.working.memory.in.mb | 512 | CarbonData supports storing data in off-heap memory for certain operations during data loading and query.This helps to avoid the Java GC and thereby improve the overall performance.The Minimum value recommeded is 512MB.Any value below this is reset to default value of 512MB.**NOTE:** The below formulas explain how to arrive at the off-heap size required.<u>Memory Required For Data Loading:</u>(*carbon.number.of.cores.while.loading*) * (Number of tables to load in parallel) * (*offheap.sort.chunk.size.inmb* + *carbon.blockletgroup.size.in.mb* + *carbon.blockletgroup.size.in.mb*/3.5 ). <u>Memory required for Query:</u>SPARK_EXECUTOR_INSTANCES * (*carbon.blockletgroup.size.in.mb* + *carbon.blockletgroup.size.in.mb* * 3.5) * spark.executor.cores |
+| carbon.unsafe.driver.working.memory.in.mb | 60% of JVM Heap Memory | CarbonData supports storing data in unsafe on-heap memory in driver for certain operations like insert into, query for loading datamap cache. The Minimum value recommended is 512MB. |
 | carbon.update.sync.folder | /tmp/carbondata | CarbonData maintains last modification time entries in modifiedTime.mdt to determine the schema changes and reload only when necessary.This configuration specifies the path where the file needs to be written. |
 | carbon.invisible.segments.preserve.count | 200 | CarbonData maintains each data load entry in tablestatus file. The entries from this file are not deleted for those segments that are compacted or dropped, but are made invisible.If the number of data loads are very high, the size and number of entries in tablestatus file can become too many causing unnecessary reading of all data.This configuration specifies the number of segment entries to be maintained afte they are compacted or dropped.Beyond this, the entries are moved to a separate history tablestatus file.**NOTE:** The entries in tablestatus file help to identify the operations performed on CarbonData table and is also used for checkpointing during various data manupulation operations.This is similar to AUDIT file maintaining all the operations and its status.Hence the entries are never deleted but moved to a separate history file. |
 | carbon.lock.retries | 3 | CarbonData ensures consistency of operations by blocking certain operations from running in parallel.In order to block the operations from running in parallel, lock is obtained on the table.This configuration specifies the maximum number of retries to obtain the lock for any operations other than load.**NOTE:** Data manupulation operations like Compaction,UPDATE,DELETE  or LOADING,UPDATE,DELETE are not allowed to run in parallel.How ever data loading can happen in parallel to compaction. |
@@ -92,7 +93,8 @@ This section provides the details of all the configurations required for the Car
 | carbon.load.directWriteHdfs.enabled | false | During data load all the carbondata files are written to local disk and finally copied to the target location in HDFS.Enabling this parameter will make carrbondata files to be written directly onto target HDFS location bypassing the local disk.**NOTE:** Writing directly to HDFS saves local disk IO(once for writing the files and again for copying to HDFS) there by improving the performance.But the drawback is when data loading fails or the application crashes, unwanted carbondata files will remain in the target HDFS location until it is cleared during next data load or by running *CLEAN FILES* DDL command |
 | carbon.options.serialization.null.format | \N | Based on the business scenarios, some columns might need to be loaded with null values.As null value cannot be written in csv files, some special characters might be adopted to specify null values.This configuration can be used to specify the null values format in the data being loaded. |
 | carbon.sort.storage.inmemory.size.inmb | 512 | CarbonData writes every ***carbon.sort.size*** number of records to intermediate temp files during data loading to ensure memory footprint is within limits.When ***enable.unsafe.sort*** configuration is enabled, instead of using ***carbon.sort.size*** which is based on rows count, size occupied in memory is used to determine when to flush data pages to intermediate temp files.This configuration determines the memory to be used for storing data pages in memory.**NOTE:** Configuring a higher values ensures more data is maintained in memory and hence increases data loading performance due to reduced or no IO.Based on the memory availability in the nodes of the cluster, configure the values accordingly. |
-| carbon.column.compressor | snappy | CarbonData will compress the column values using the compressor specified by this configuration. Currently CarbonData supports 'snappy' and 'zstd' compressors. | |
+| carbon.column.compressor | snappy | CarbonData will compress the column values using the compressor specified by this configuration. Currently CarbonData supports 'snappy' and 'zstd' compressors. |
+| carbon.minmax.allowed.byte.count | 200 | CarbonData will write the min max values for string/varchar types column using the byte count specified by this configuration. Max value is 1000 bytes(500 characters) and Min value is 10 bytes(5 characters). **NOTE:** This property is useful for reducing the store size thereby improving the query performance but can lead to query degradation if value is not configured properly. | |
 
 ## Compaction Configuration
 
@@ -117,7 +119,7 @@ This section provides the details of all the configurations required for the Car
 
 | Parameter | Default Value | Description |
 |--------------------------------------|---------------|---------------------------------------------------|
-| carbon.max.driver.lru.cache.size | -1 | Maximum memory **(in MB)** upto which the driver process can cache the data (BTree and dictionary values). Beyond this, least recently used data will be removed from cache before loading new set of values.Default value of -1 means there is no memory limit for caching. Only integer values greater than 0 are accepted.**NOTE:** Minimum number of entries that needs to be removed from cache in order to load the new set of data is determined and unloaded.ie.,for example if 3 cache entries qualify for pre-emption, out of these, those entries that free up more cache memory is removed prior to others. |
+| carbon.max.driver.lru.cache.size | -1 | Maximum memory **(in MB)** upto which the driver process can cache the data (BTree and dictionary values). Beyond this, least recently used data will be removed from cache before loading new set of values.Default value of -1 means there is no memory limit for caching. Only integer values greater than 0 are accepted.**NOTE:** Minimum number of entries that needs to be removed from cache in order to load the new set of data is determined and unloaded.ie.,for example if 3 cache entries qualify for pre-emption, out of these, those entries that free up more cache memory is removed prior to others. Please refer [FAQs](./faq.md#how-to-check-LRU-cache-memory-footprint) for checking LRU cache memory footprint. |
 | carbon.max.executor.lru.cache.size | -1 | Maximum memory **(in MB)** upto which the executor process can cache the data (BTree and reverse dictionary values).Default value of -1 means there is no memory limit for caching. Only integer values greater than 0 are accepted.**NOTE:** If this parameter is not configured, then the value of ***carbon.max.driver.lru.cache.size*** will be used. |
 | max.query.execution.time | 60 | Maximum time allowed for one query to be executed. The value is in minutes. |
 | carbon.enableMinMax | true | CarbonData maintains the metadata which enables to prune unnecessary files from being scanned as per the query conditions.To achieve pruning, Min,Max of each column is maintined.Based on the filter condition in the query, certain data can be skipped from scanning by matching the filter value against the min,max values of the column(s) present in that carbondata file.This pruing enhances query performance significantly. |

http://git-wip-us.apache.org/repos/asf/carbondata/blob/b3a5e3a8/docs/faq.md
----------------------------------------------------------------------
diff --git a/docs/faq.md b/docs/faq.md
index 8ec7290..dbf9155 100644
--- a/docs/faq.md
+++ b/docs/faq.md
@@ -28,6 +28,7 @@
 * [Why aggregate query is not fetching data from aggregate table?](#why-aggregate-query-is-not-fetching-data-from-aggregate-table)
 * [Why all executors are showing success in Spark UI even after Dataload command failed at Driver side?](#why-all-executors-are-showing-success-in-spark-ui-even-after-dataload-command-failed-at-driver-side)
 * [Why different time zone result for select query output when query SDK writer output?](#why-different-time-zone-result-for-select-query-output-when-query-sdk-writer-output)
+* [How to check LRU cache memory footprint?](#how-to-check-LRU-cache-memory-footprint)
 
 # TroubleShooting
 
@@ -212,7 +213,22 @@ cluster timezone is Asia/Shanghai
 TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
 ```
 
+## How to check LRU cache memory footprint?
+To observe the LRU cache memory footprint in the logs, configure the below properties in log4j.properties file.
+```
+log4j.logger.org.apache.carbondata.core.memory.UnsafeMemoryManager = DEBUG
+log4j.logger.org.apache.carbondata.core.cache.CarbonLRUCache = DEBUG
+```
+These properties will enable the DEBUG log for the CarbonLRUCache and UnsafeMemoryManager which will print the information of memory consumed using which the LRU cache size can be decided. **Note:** Enabling the DEBUG log will degrade the query performance.
 
+**Example:**
+```
+18/09/26 15:05:28 DEBUG UnsafeMemoryManager: pool-44-thread-1 Memory block (org.apache.carbondata.core.memory.MemoryBlock@21312095) is created with size 10. Total memory used 413Bytes, left 536870499Bytes
+18/09/26 15:05:29 DEBUG CarbonLRUCache: main Required size for entry /home/target/store/default/stored_as_carbondata_table/Fact/Part0/Segment_0/0_1537954529044.carbonindexmerge :: 181 Current cache size :: 0
+18/09/26 15:05:30 DEBUG UnsafeMemoryManager: main Freeing memory of size: 105available memory:  536870836
+18/09/26 15:05:30 DEBUG UnsafeMemoryManager: main Freeing memory of size: 76available memory:  536870912
+18/09/26 15:05:30 INFO CarbonLRUCache: main Removed entry from InMemory lru cache :: /home/target/store/default/stored_as_carbondata_table/Fact/Part0/Segment_0/0_1537954529044.carbonindexmerge
+```
 
 ## Getting tablestatus.lock issues When loading data
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/b3a5e3a8/docs/sdk-guide.md
----------------------------------------------------------------------
diff --git a/docs/sdk-guide.md b/docs/sdk-guide.md
index d1e4bc5..be42b3f 100644
--- a/docs/sdk-guide.md
+++ b/docs/sdk-guide.md
@@ -181,22 +181,31 @@ public class TestSdkJson {
 ```
 
 ## Datatypes Mapping
-Each of SQL data types are mapped into data types of SDK. Following are the mapping:
-
-| SQL DataTypes | Mapped SDK DataTypes |
-|---------------|----------------------|
-| BOOLEAN | DataTypes.BOOLEAN |
-| SMALLINT | DataTypes.SHORT |
-| INTEGER | DataTypes.INT |
-| BIGINT | DataTypes.LONG |
-| DOUBLE | DataTypes.DOUBLE |
-| VARCHAR | DataTypes.STRING |
-| FLOAT | DataTypes.FLOAT |
-| BYTE | DataTypes.BYTE |
-| DATE | DataTypes.DATE |
-| TIMESTAMP | DataTypes.TIMESTAMP |
-| STRING | DataTypes.STRING |
-| DECIMAL | DataTypes.createDecimalType(precision, scale) |
+Each of SQL data types and Avro Data Types are mapped into data types of SDK. Following are the mapping:
+
+| SQL DataTypes | Avro DataTypes | Mapped SDK DataTypes |
+|---------------|----------------|----------------------|
+| BOOLEAN | BOOLEAN | DataTypes.BOOLEAN |
+| SMALLINT |  -  | DataTypes.SHORT |
+| INTEGER | INTEGER | DataTypes.INT |
+| BIGINT | LONG | DataTypes.LONG |
+| DOUBLE | DOUBLE | DataTypes.DOUBLE |
+| VARCHAR |  -  | DataTypes.STRING |
+| FLOAT | FLOAT | DataTypes.FLOAT |
+| BYTE |  -  | DataTypes.BYTE |
+| DATE | DATE | DataTypes.DATE |
+| TIMESTAMP |  -  | DataTypes.TIMESTAMP |
+| STRING | STRING | DataTypes.STRING |
+| DECIMAL | DECIMAL | DataTypes.createDecimalType(precision, scale) |
+| ARRAY | ARRAY | DataTypes.createArrayType(elementType) |
+| STRUCT | RECORD | DataTypes.createStructType(fields) |
+|  -  | ENUM | DataTypes.STRING |
+|  -  | UNION | DataTypes.createStructType(types) |
+|  -  | MAP | DataTypes.createMapType(keyType, valueType) |
+|  -  | TimeMillis | DataTypes.INT |
+|  -  | TimeMicros | DataTypes.LONG |
+|  -  | TimestampMillis | DataTypes.TIMESTAMP |
+|  -  | TimestampMicros | DataTypes.TIMESTAMP |
 
 **NOTE:**
  1. Carbon Supports below logical types of AVRO.
@@ -209,12 +218,22 @@ Each of SQL data types are mapped into data types of SDK. Following are the mapp
  c. Timestamp (microsecond precision)
     The timestamp-micros logical type represents an instant on the global timeline, independent of a particular time zone or calendar, with a precision of one microsecond.
     A timestamp-micros logical type annotates an Avro long, where the long stores the number of microseconds from the unix epoch, 1 January 1970 00:00:00.000000 UTC.
+ d. Decimal
+    The decimal logical type represents an arbitrary-precision signed decimal number of the form unscaled × 10-scale.
+    A decimal logical type annotates Avro bytes or fixed types. The byte array must contain the two's-complement representation of the unscaled integer value in big-endian byte order. The scale is fixed, and is specified using an attribute.
+ e. Time (millisecond precision)
+    The time-millis logical type represents a time of day, with no reference to a particular calendar, time zone or date, with a precision of one millisecond.
+    A time-millis logical type annotates an Avro int, where the int stores the number of milliseconds after midnight, 00:00:00.000.
+ f. Time (microsecond precision)
+    The time-micros logical type represents a time of day, with no reference to a particular calendar, time zone or date, with a precision of one microsecond.
+    A time-micros logical type annotates an Avro long, where the long stores the number of microseconds after midnight, 00:00:00.000000.
+
     
     Currently the values of logical types are not validated by carbon. 
     Expect that avro record passed by the user is already validated by avro record generator tools.    
  2. If the string data is more than 32K in length, use withTableProperties() with "long_string_columns" property
-    or directly use DataTypes.VARCHAR if it is carbon schema.      
-
+    or directly use DataTypes.VARCHAR if it is carbon schema.
+ 3. Avro Bytes, Fixed and Duration data types are not yet supported.
 ## Run SQL on files directly
 Instead of creating table and query it, you can also query that file directly with SQL.
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/b3a5e3a8/docs/supported-data-types-in-carbondata.md
----------------------------------------------------------------------
diff --git a/docs/supported-data-types-in-carbondata.md b/docs/supported-data-types-in-carbondata.md
index fd13079..daf1acf 100644
--- a/docs/supported-data-types-in-carbondata.md
+++ b/docs/supported-data-types-in-carbondata.md
@@ -45,6 +45,7 @@
   * Complex Types
     * arrays: ARRAY``<data_type>``
     * structs: STRUCT``<col_name : data_type COMMENT col_comment, ...>``
+    * maps: MAP``<primitive_type, data_type>``
     
     **NOTE**: Only 2 level complex type schema is supported for now.
 


[05/45] carbondata git commit: [DOC] Add spark carbon file format documentation

Posted by ra...@apache.org.
[DOC] Add spark carbon file format documentation

Add spark carbon file format documentation

This closes #2757


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

Branch: refs/heads/branch-1.5
Commit: d84cd817a8ed90e79c12dcee7334b6e52d030982
Parents: 13ecc9e
Author: ravipesala <ra...@gmail.com>
Authored: Tue Sep 25 16:19:16 2018 +0530
Committer: kunal642 <ku...@gmail.com>
Committed: Wed Sep 26 16:11:55 2018 +0530

----------------------------------------------------------------------
 docs/carbon-as-spark-datasource-guide.md | 100 ++++++++++++++++++++++++++
 1 file changed, 100 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/d84cd817/docs/carbon-as-spark-datasource-guide.md
----------------------------------------------------------------------
diff --git a/docs/carbon-as-spark-datasource-guide.md b/docs/carbon-as-spark-datasource-guide.md
new file mode 100644
index 0000000..1d286cf
--- /dev/null
+++ b/docs/carbon-as-spark-datasource-guide.md
@@ -0,0 +1,100 @@
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one or more 
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership. 
+    The ASF licenses this file to you under the Apache License, Version 2.0
+    (the "License"); you may not use this file except in compliance with 
+    the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+    
+    Unless required by applicable law or agreed to in writing, software 
+    distributed under the License is distributed on an "AS IS" BASIS, 
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and 
+    limitations under the License.
+-->
+
+# Carbon as Spark's datasource guide
+
+Carbon fileformat can be integrated to Spark using datasource to read and write data without using CarbonSession.
+
+
+# Create Table with DDL
+
+Carbon table can be created with spark's datasource DDL syntax as follows.
+
+```
+ CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db_name.]table_name
+     [(col_name1 col_type1 [COMMENT col_comment1], ...)]
+     USING carbon
+     [OPTIONS (key1=val1, key2=val2, ...)]
+     [PARTITIONED BY (col_name1, col_name2, ...)]
+     [CLUSTERED BY (col_name3, col_name4, ...) INTO num_buckets BUCKETS]
+     [LOCATION path]
+     [COMMENT table_comment]
+     [TBLPROPERTIES (key1=val1, key2=val2, ...)]
+     [AS select_statement]
+``` 
+
+## Supported OPTIONS
+
+| Property | Default Value | Description |
+|-----------|--------------|------------|
+| table_blocksize | 1024 | Size of blocks to write onto hdfs |
+| table_blocklet_size | 64 | Size of blocklet to write |
+| local_dictionary_threshold | 10000 | Cardinality upto which the local dictionary can be generated  |
+| local_dictionary_enable | false | Enable local dictionary generation  |
+| sort_columns | all dimensions are sorted | comma separated string columns which to include in sort and its order of sort |
+| sort_scope | local_sort | Sort scope of the load.Options include no sort, local sort ,batch sort and global sort |
+| long_string_columns | null | comma separated string columns which are more than 32k length |
+
+## Example 
+
+```
+ CREATE TABLE CARBON_TABLE (NAME  STRING) USING CARBON OPTIONS(‘table_block_size’=’256’)
+```
+
+Note: User can only apply the features of what spark datasource like parquet supports. It cannot support the features of carbon session like IUD, compaction etc. 
+
+# Using DataFrame
+
+Carbon format can be used in dataframe also using the following way.
+
+Write carbon using dataframe 
+```
+df.write.format("carbon").save(path)
+```
+
+Read carbon using dataframe
+```
+val df = spark.read.format("carbon").load(path)
+```
+
+## Example
+
+```
+import org.apache.spark.sql.SparkSession
+
+val spark = SparkSession
+  .builder()
+  .appName("Spark SQL basic example")
+  .config("spark.some.config.option", "some-value")
+  .getOrCreate()
+
+// For implicit conversions like converting RDDs to DataFrames
+import spark.implicits._
+val df = spark.sparkContext.parallelize(1 to 10 * 10 * 1000)
+     .map(x => (r.nextInt(100000), "name" + x % 8, "city" + x % 50, BigDecimal.apply(x % 60)))
+      .toDF("ID", "name", "city", "age")
+      
+// Write to carbon format      
+df.write.format("carbon").save("/user/person_table")
+
+// Read carbon using dataframe
+val dfread = spark.read.format("carbon").load("/user/person_table")
+dfread.show()
+```
+
+Reference : [list of carbon properties](./configuration-parameters.md)
+


[24/45] carbondata git commit: [CARBONDATA-2952] Provide CarbonReader C++ interface for SDK

Posted by ra...@apache.org.
[CARBONDATA-2952] Provide CarbonReader C++ interface for SDK

1.init carbonreader,config data path and tablename
2.config ak sk endpoing for S3
3.configure projection
4.build carbon reader
5.hasNext
6.readNextRow
7.close

This closes #2738


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

Branch: refs/heads/branch-1.5
Commit: e9a198ab65c7bc9642e8a3186072f39aa2e5f0f6
Parents: d8a51c9
Author: xubo245 <xu...@huawei.com>
Authored: Thu Sep 20 18:35:34 2018 +0800
Committer: kunal642 <ku...@gmail.com>
Committed: Wed Oct 3 17:53:35 2018 +0530

----------------------------------------------------------------------
 .gitignore                                      |   1 +
 .../core/constants/CarbonCommonConstants.java   |   4 +
 docs/CSDK-guide.md                              | 197 ++++++++++++++++++
 examples/spark2/pom.xml                         |   2 +-
 pom.xml                                         |   2 +
 store/CSDK/CMakeLists.txt                       |  17 ++
 store/CSDK/CarbonReader.cpp                     | 103 ++++++++++
 store/CSDK/CarbonReader.h                       | 105 ++++++++++
 store/CSDK/main.cpp                             | 200 +++++++++++++++++++
 store/sdk/pom.xml                               |   5 +
 .../carbondata/sdk/file/CarbonReader.java       |  30 +++
 .../sdk/file/CarbonReaderBuilder.java           |  16 ++
 12 files changed, 681 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/e9a198ab/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 5d66a40..00e4934 100644
--- a/.gitignore
+++ b/.gitignore
@@ -12,6 +12,7 @@
 .settings
 .cache
 target/
+store/CSDK/cmake-build-debug/*
 .project
 .classpath
 metastore_db/

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e9a198ab/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 faad0dc..46139c9 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
@@ -507,6 +507,10 @@ public final class CarbonCommonConstants {
    */
   public static final String FILE_SEPARATOR = "/";
   /**
+   * ARRAY separator
+   */
+  public static final String ARRAY_SEPARATOR = "\001";
+  /**
    * MAX_QUERY_EXECUTION_TIME
    */
   @CarbonProperty

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e9a198ab/docs/CSDK-guide.md
----------------------------------------------------------------------
diff --git a/docs/CSDK-guide.md b/docs/CSDK-guide.md
new file mode 100644
index 0000000..c4f4a31
--- /dev/null
+++ b/docs/CSDK-guide.md
@@ -0,0 +1,197 @@
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one or more 
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership. 
+    The ASF licenses this file to you under the Apache License, Version 2.0
+    (the "License"); you may not use this file except in compliance with 
+    the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+    
+    Unless required by applicable law or agreed to in writing, software 
+    distributed under the License is distributed on an "AS IS" BASIS, 
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and 
+    limitations under the License.
+-->
+
+# CSDK Guide
+
+CarbonData CSDK provides C++ interface to write and read carbon file. 
+CSDK use JNI to invoke java SDK in C++ code.
+
+
+# CSDK Reader
+This CSDK reader reads CarbonData file and carbonindex file at a given path.
+External client can make use of this reader to read CarbonData files in C++ 
+code and without CarbonSession.
+
+
+In the carbon jars package, there exist a carbondata-sdk.jar, 
+including SDK reader for CSDK.
+## Quick example
+```
+// 1. init JVM
+JavaVM *jvm;
+JNIEnv *initJVM() {
+    JNIEnv *env;
+    JavaVMInitArgs vm_args;
+    int parNum = 3;
+    int res;
+    JavaVMOption options[parNum];
+
+    options[0].optionString = "-Djava.compiler=NONE";
+    options[1].optionString = "-Djava.class.path=../../sdk/target/carbondata-sdk.jar";
+    options[2].optionString = "-verbose:jni";
+    vm_args.version = JNI_VERSION_1_8;
+    vm_args.nOptions = parNum;
+    vm_args.options = options;
+    vm_args.ignoreUnrecognized = JNI_FALSE;
+
+    res = JNI_CreateJavaVM(&jvm, (void **) &env, &vm_args);
+    if (res < 0) {
+        fprintf(stderr, "\nCan't create Java VM\n");
+        exit(1);
+    }
+
+    return env;
+}
+
+// 2. create carbon reader and read data 
+// 2.1 read data from local disk
+/**
+ * test read data from local disk, without projection
+ *
+ * @param env  jni env
+ * @return
+ */
+bool readFromLocalWithoutProjection(JNIEnv *env) {
+
+    CarbonReader carbonReaderClass;
+    carbonReaderClass.builder(env, "../resources/carbondata", "test");
+    carbonReaderClass.build();
+
+    while (carbonReaderClass.hasNext()) {
+        jobjectArray row = carbonReaderClass.readNextRow();
+        jsize length = env->GetArrayLength(row);
+        int j = 0;
+        for (j = 0; j < length; j++) {
+            jobject element = env->GetObjectArrayElement(row, j);
+            char *str = (char *) env->GetStringUTFChars((jstring) element, JNI_FALSE);
+            printf("%s\t", str);
+        }
+        printf("\n");
+    }
+    carbonReaderClass.close();
+}
+
+// 2.2 read data from S3
+
+/**
+ * read data from S3
+ * parameter is ak sk endpoint
+ *
+ * @param env jni env
+ * @param argv argument vector
+ * @return
+ */
+bool readFromS3(JNIEnv *env, char *argv[]) {
+    CarbonReader reader;
+
+    char *args[3];
+    // "your access key"
+    args[0] = argv[1];
+    // "your secret key"
+    args[1] = argv[2];
+    // "your endPoint"
+    args[2] = argv[3];
+
+    reader.builder(env, "s3a://sdk/WriterOutput", "test");
+    reader.withHadoopConf(3, args);
+    reader.build();
+    printf("\nRead data from S3:\n");
+    while (reader.hasNext()) {
+        jobjectArray row = reader.readNextRow();
+        jsize length = env->GetArrayLength(row);
+
+        int j = 0;
+        for (j = 0; j < length; j++) {
+            jobject element = env->GetObjectArrayElement(row, j);
+            char *str = (char *) env->GetStringUTFChars((jstring) element, JNI_FALSE);
+            printf("%s\t", str);
+        }
+        printf("\n");
+    }
+
+    reader.close();
+}
+
+// 3. destory JVM
+    (jvm)->DestroyJavaVM();
+```
+Find example code at main.cpp of CSDK module
+
+## API List
+```
+    /**
+     * create a CarbonReaderBuilder object for building carbonReader,
+     * CarbonReaderBuilder object  can configure different parameter
+     *
+     * @param env JNIEnv
+     * @param path data store path
+     * @param tableName table name
+     * @return CarbonReaderBuilder object
+     */
+    jobject builder(JNIEnv *env, char *path, char *tableName);
+
+    /**
+     * Configure the projection column names of carbon reader
+     *
+     * @param argc argument counter
+     * @param argv argument vector
+     * @return CarbonReaderBuilder object
+     */
+    jobject projection(int argc, char *argv[]);
+
+    /**
+     *  build carbon reader with argument vector
+     *  it support multiple parameter
+     *  like: key=value
+     *  for example: fs.s3a.access.key=XXXX, XXXX is user's access key value
+     *
+     * @param argc argument counter
+     * @param argv argument vector
+     * @return CarbonReaderBuilder object
+     **/
+    jobject withHadoopConf(int argc, char *argv[]);
+
+    /**
+     * build carbonReader object for reading data
+     * it support read data from load disk
+     *
+     * @return carbonReader object
+     */
+    jobject build();
+
+    /**
+     * Whether it has next row data
+     *
+     * @return boolean value, if it has next row, return true. if it hasn't next row, return false.
+     */
+    jboolean hasNext();
+
+    /**
+     * read next row from data
+     *
+     * @return object array of one row
+     */
+    jobjectArray readNextRow();
+
+    /**
+     * close the carbon reader
+     *
+     * @return  boolean value
+     */
+    jboolean close();
+
+```

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e9a198ab/examples/spark2/pom.xml
----------------------------------------------------------------------
diff --git a/examples/spark2/pom.xml b/examples/spark2/pom.xml
index aa7b7c5..8dbab15 100644
--- a/examples/spark2/pom.xml
+++ b/examples/spark2/pom.xml
@@ -80,7 +80,7 @@
     <dependency>
       <groupId>org.apache.httpcomponents</groupId>
       <artifactId>httpclient</artifactId>
-      <version>4.2</version>
+      <version>${httpclient.version}</version>
     </dependency>
     <dependency>
       <groupId>org.scalatest</groupId>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e9a198ab/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 7b1d487..3dcce8d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -113,6 +113,7 @@
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <snappy.version>1.1.2.6</snappy.version>
     <hadoop.version>2.7.2</hadoop.version>
+    <httpclient.version>4.2.5</httpclient.version>
     <scala.binary.version>2.11</scala.binary.version>
     <scala.version>2.11.8</scala.version>
     <hadoop.deps.scope>compile</hadoop.deps.scope>
@@ -483,6 +484,7 @@
       <id>hadoop-2.8</id>
       <properties>
         <hadoop.version>2.8.3</hadoop.version>
+        <httpclient.version>4.5.2</httpclient.version>
       </properties>
     </profile>
     <profile>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e9a198ab/store/CSDK/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/store/CSDK/CMakeLists.txt b/store/CSDK/CMakeLists.txt
new file mode 100644
index 0000000..c3c57a0
--- /dev/null
+++ b/store/CSDK/CMakeLists.txt
@@ -0,0 +1,17 @@
+cmake_minimum_required (VERSION 2.8)
+project (CJDK)
+set(CMAKE_BUILD_TYPE Debug)
+SET (CMAKE_INSTALL_RPATH_USE_LINK_PATH true)
+
+find_package (JNI REQUIRED)
+
+include_directories (${JNI_INCLUDE_DIRS})
+
+set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++11")
+set (SOURCE_FILES CarbonReader.cpp CarbonReader.h main.cpp)
+
+add_executable (CJDK ${SOURCE_FILES})
+get_filename_component (JAVA_JVM_LIBRARY_DIR ${JAVA_JVM_LIBRARY} DIRECTORY)
+message (${JAVA_JVM_LIBRARY_DIR})
+target_link_libraries (CJDK ${JAVA_JVM_LIBRARY} )
+

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e9a198ab/store/CSDK/CarbonReader.cpp
----------------------------------------------------------------------
diff --git a/store/CSDK/CarbonReader.cpp b/store/CSDK/CarbonReader.cpp
new file mode 100644
index 0000000..0400957
--- /dev/null
+++ b/store/CSDK/CarbonReader.cpp
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "CarbonReader.h"
+#include <jni.h>
+
+jobject CarbonReader::builder(JNIEnv *env, char *path, char *tableName) {
+
+    jniEnv = env;
+    jclass carbonReaderClass = env->FindClass("org/apache/carbondata/sdk/file/CarbonReader");
+    jmethodID carbonReaderBuilderID = env->GetStaticMethodID(carbonReaderClass, "builder",
+        "(Ljava/lang/String;Ljava/lang/String;)Lorg/apache/carbondata/sdk/file/CarbonReaderBuilder;");
+    jstring jpath = env->NewStringUTF(path);
+    jstring jtableName = env->NewStringUTF(tableName);
+    jvalue args[2];
+    args[0].l = jpath;
+    args[1].l = jtableName;
+    carbonReaderBuilderObject = env->CallStaticObjectMethodA(carbonReaderClass, carbonReaderBuilderID, args);
+    return carbonReaderBuilderObject;
+}
+
+jobject CarbonReader::builder(JNIEnv *env, char *path) {
+    jniEnv = env;
+    jclass carbonReaderClass = env->FindClass("org/apache/carbondata/sdk/file/CarbonReader");
+    jmethodID carbonReaderBuilderID = env->GetStaticMethodID(carbonReaderClass, "builder",
+        "(Ljava/lang/String;)Lorg/apache/carbondata/sdk/file/CarbonReaderBuilder;");
+    jstring jpath = env->NewStringUTF(path);
+    jvalue args[1];
+    args[0].l = jpath;
+    carbonReaderBuilderObject = env->CallStaticObjectMethodA(carbonReaderClass, carbonReaderBuilderID, args);
+    return carbonReaderBuilderObject;
+}
+
+jobject CarbonReader::projection(int argc, char *argv[]) {
+    jclass carbonReaderBuilderClass = jniEnv->GetObjectClass(carbonReaderBuilderObject);
+    jmethodID buildID = jniEnv->GetMethodID(carbonReaderBuilderClass, "projection",
+        "([Ljava/lang/String;)Lorg/apache/carbondata/sdk/file/CarbonReaderBuilder;");
+    jclass objectArrayClass = jniEnv->FindClass("Ljava/lang/String;");
+    jobjectArray array = jniEnv->NewObjectArray(argc, objectArrayClass, NULL);
+    for (int i = 0; i < argc; ++i) {
+        jstring value = jniEnv->NewStringUTF(argv[i]);
+        jniEnv->SetObjectArrayElement(array, i, value);
+    }
+
+    jvalue args[1];
+    args[0].l = array;
+    carbonReaderBuilderObject = jniEnv->CallObjectMethodA(carbonReaderBuilderObject, buildID, args);
+    return carbonReaderBuilderObject;
+}
+
+jobject CarbonReader::withHadoopConf(char *key, char *value) {
+    jclass carbonReaderBuilderClass = jniEnv->GetObjectClass(carbonReaderBuilderObject);
+    jmethodID buildID = jniEnv->GetMethodID(carbonReaderBuilderClass, "withHadoopConf",
+        "(Ljava/lang/String;Ljava/lang/String;)Lorg/apache/carbondata/sdk/file/CarbonReaderBuilder;");
+
+    jvalue args[2];
+    args[0].l = jniEnv->NewStringUTF(key);
+    args[1].l = jniEnv->NewStringUTF(value);
+    carbonReaderBuilderObject = jniEnv->CallObjectMethodA(carbonReaderBuilderObject, buildID, args);
+    return carbonReaderBuilderObject;
+}
+
+jobject CarbonReader::build() {
+    jclass carbonReaderBuilderClass = jniEnv->GetObjectClass(carbonReaderBuilderObject);
+    jmethodID buildID = jniEnv->GetMethodID(carbonReaderBuilderClass, "build",
+        "()Lorg/apache/carbondata/sdk/file/CarbonReader;");
+    carbonReaderObject = jniEnv->CallObjectMethod(carbonReaderBuilderObject, buildID);
+    return carbonReaderObject;
+}
+
+jboolean CarbonReader::hasNext() {
+    jclass carbonReader = jniEnv->GetObjectClass(carbonReaderObject);
+    jmethodID hasNextID = jniEnv->GetMethodID(carbonReader, "hasNext", "()Z");
+    unsigned char hasNext = jniEnv->CallBooleanMethod(carbonReaderObject, hasNextID);
+    return hasNext;
+}
+
+jobjectArray CarbonReader::readNextRow() {
+    jclass carbonReader = jniEnv->GetObjectClass(carbonReaderObject);
+    jmethodID readNextRow2ID = jniEnv->GetMethodID(carbonReader, "readNextStringRow", "()[Ljava/lang/Object;");
+    jobjectArray row = (jobjectArray) jniEnv->CallObjectMethod(carbonReaderObject, readNextRow2ID);
+    return row;
+}
+
+jboolean CarbonReader::close() {
+    jclass carbonReader = jniEnv->GetObjectClass(carbonReaderObject);
+    jmethodID closeID = jniEnv->GetMethodID(carbonReader, "close", "()V");
+    jniEnv->CallBooleanMethod(carbonReaderObject, closeID);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e9a198ab/store/CSDK/CarbonReader.h
----------------------------------------------------------------------
diff --git a/store/CSDK/CarbonReader.h b/store/CSDK/CarbonReader.h
new file mode 100644
index 0000000..861f704
--- /dev/null
+++ b/store/CSDK/CarbonReader.h
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <jni.h>
+
+class CarbonReader {
+public:
+    /**
+     * jni env
+     */
+    JNIEnv *jniEnv;
+
+    /**
+     * carbonReaderBuilder object for building carbonReader
+     * it can configure some operation
+     */
+    jobject carbonReaderBuilderObject;
+
+    /**
+     * carbonReader object for reading data
+     */
+    jobject carbonReaderObject;
+
+    /**
+     * create a CarbonReaderBuilder object for building carbonReader,
+     * CarbonReaderBuilder object  can configure different parameter
+     *
+     * @param env JNIEnv
+     * @param path data store path
+     * @param tableName table name
+     * @return CarbonReaderBuilder object
+     */
+    jobject builder(JNIEnv *env, char *path, char *tableName);
+
+    /**
+     * create a CarbonReaderBuilder object for building carbonReader,
+     * CarbonReaderBuilder object  can configure different parameter
+     *
+     * @param env JNIEnv
+     * @param path data store path
+     * @return CarbonReaderBuilder object
+     * */
+    jobject builder(JNIEnv *env, char *path);
+
+    /**
+     * Configure the projection column names of carbon reader
+     *
+     * @param argc argument counter
+     * @param argv argument vector
+     * @return CarbonReaderBuilder object
+     */
+    jobject projection(int argc, char *argv[]);
+
+    /**
+     * configure parameter, including ak,sk and endpoint
+     *
+     * @param key key word
+     * @param value value
+     * @return CarbonReaderBuilder object
+     */
+    jobject withHadoopConf(char *key, char *value);
+
+    /**
+     * build carbonReader object for reading data
+     * it support read data from load disk
+     *
+     * @return carbonReader object
+     */
+    jobject build();
+
+    /**
+     * Whether it has next row data
+     *
+     * @return boolean value, if it has next row, return true. if it hasn't next row, return false.
+     */
+    jboolean hasNext();
+
+    /**
+     * read next row from data
+     *
+     * @return object array of one row
+     */
+    jobjectArray readNextRow();
+
+    /**
+     * close the carbon reader
+     *
+     * @return  boolean value
+     */
+    jboolean close();
+};

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e9a198ab/store/CSDK/main.cpp
----------------------------------------------------------------------
diff --git a/store/CSDK/main.cpp b/store/CSDK/main.cpp
new file mode 100644
index 0000000..a8d1a87
--- /dev/null
+++ b/store/CSDK/main.cpp
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <stdio.h>
+#include <jni.h>
+#include <stdlib.h>
+#include <iostream>
+#include <unistd.h>
+#include "CarbonReader.h"
+
+using namespace std;
+
+JavaVM *jvm;
+
+/**
+ * init jvm
+ *
+ * @return
+ */
+JNIEnv *initJVM() {
+    JNIEnv *env;
+    JavaVMInitArgs vm_args;
+    int parNum = 3;
+    int res;
+    JavaVMOption options[parNum];
+
+    options[0].optionString = "-Djava.compiler=NONE";
+    options[1].optionString = "-Djava.class.path=../../sdk/target/carbondata-sdk.jar";
+    options[2].optionString = "-verbose:jni";
+    vm_args.version = JNI_VERSION_1_8;
+    vm_args.nOptions = parNum;
+    vm_args.options = options;
+    vm_args.ignoreUnrecognized = JNI_FALSE;
+
+    res = JNI_CreateJavaVM(&jvm, (void **) &env, &vm_args);
+    if (res < 0) {
+        fprintf(stderr, "\nCan't create Java VM\n");
+        exit(1);
+    }
+
+    return env;
+}
+
+/**
+ * test read data from local disk, without projection
+ *
+ * @param env  jni env
+ * @return
+ */
+bool readFromLocalWithoutProjection(JNIEnv *env) {
+
+    CarbonReader carbonReaderClass;
+    carbonReaderClass.builder(env, "../resources/carbondata");
+    carbonReaderClass.build();
+
+    printf("\nRead data from local  without projection:\n");
+
+    while (carbonReaderClass.hasNext()) {
+        jobjectArray row = carbonReaderClass.readNextRow();
+        jsize length = env->GetArrayLength(row);
+
+        int j = 0;
+        for (j = 0; j < length; j++) {
+            jobject element = env->GetObjectArrayElement(row, j);
+            char *str = (char *) env->GetStringUTFChars((jstring) element, JNI_FALSE);
+            printf("%s\t", str);
+        }
+        printf("\n");
+    }
+
+    carbonReaderClass.close();
+}
+
+/**
+ * test read data from local disk
+ *
+ * @param env  jni env
+ * @return
+ */
+bool readFromLocal(JNIEnv *env) {
+
+    CarbonReader reader;
+    reader.builder(env, "../resources/carbondata", "test");
+
+    char *argv[11];
+    argv[0] = "stringField";
+    argv[1] = "shortField";
+    argv[2] = "intField";
+    argv[3] = "longField";
+    argv[4] = "doubleField";
+    argv[5] = "boolField";
+    argv[6] = "dateField";
+    argv[7] = "timeField";
+    argv[8] = "decimalField";
+    argv[9] = "varcharField";
+    argv[10] = "arrayField";
+    reader.projection(11, argv);
+
+    reader.build();
+
+    printf("\nRead data from local:\n");
+
+    while (reader.hasNext()) {
+        jobjectArray row = reader.readNextRow();
+        jsize length = env->GetArrayLength(row);
+
+        int j = 0;
+        for (j = 0; j < length; j++) {
+            jobject element = env->GetObjectArrayElement(row, j);
+            char *str = (char *) env->GetStringUTFChars((jstring) element, JNI_FALSE);
+            printf("%s\t", str);
+        }
+        printf("\n");
+    }
+
+    reader.close();
+}
+
+/**
+ * read data from S3
+ * parameter is ak sk endpoint
+ *
+ * @param env jni env
+ * @param argv argument vector
+ * @return
+ */
+bool readFromS3(JNIEnv *env, char *argv[]) {
+    CarbonReader reader;
+
+    char *args[3];
+    // "your access key"
+    args[0] = argv[1];
+    // "your secret key"
+    args[1] = argv[2];
+    // "your endPoint"
+    args[2] = argv[3];
+
+    reader.builder(env, "s3a://sdk/WriterOutput", "test");
+    reader.withHadoopConf("fs.s3a.access.key", argv[1]);
+    reader.withHadoopConf("fs.s3a.secret.key", argv[2]);
+    reader.withHadoopConf("fs.s3a.endpoint", argv[3]);
+    reader.build();
+    printf("\nRead data from S3:\n");
+    while (reader.hasNext()) {
+        jobjectArray row = reader.readNextRow();
+        jsize length = env->GetArrayLength(row);
+
+        int j = 0;
+        for (j = 0; j < length; j++) {
+            jobject element = env->GetObjectArrayElement(row, j);
+            char *str = (char *) env->GetStringUTFChars((jstring) element, JNI_FALSE);
+            printf("%s\t", str);
+        }
+        printf("\n");
+    }
+
+    reader.close();
+}
+
+/**
+ * This a example for C++ interface to read carbon file
+ * If you want to test read data fromS3, please input the parameter: ak sk endpoint
+ *
+ * @param argc argument counter
+ * @param argv argument vector
+ * @return
+ */
+int main(int argc, char *argv[]) {
+    // init jvm
+    JNIEnv *env;
+    env = initJVM();
+
+    if (argc > 3) {
+        readFromS3(env, argv);
+    } else {
+        readFromLocalWithoutProjection(env);
+        readFromLocal(env);
+    }
+    cout << "destory jvm\n\n";
+    (jvm)->DestroyJavaVM();
+
+    cout << "\nfinish destory jvm";
+    fprintf(stdout, "Java VM destory.\n");
+    return 0;
+}
+

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e9a198ab/store/sdk/pom.xml
----------------------------------------------------------------------
diff --git a/store/sdk/pom.xml b/store/sdk/pom.xml
index ea720a2..2424113 100644
--- a/store/sdk/pom.xml
+++ b/store/sdk/pom.xml
@@ -39,6 +39,11 @@
       <artifactId>hadoop-aws</artifactId>
       <version>${hadoop.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.httpcomponents</groupId>
+      <artifactId>httpclient</artifactId>
+      <version>${httpclient.version}</version>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e9a198ab/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReader.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReader.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReader.java
index be809e6..1f1794c 100644
--- a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReader.java
+++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReader.java
@@ -24,6 +24,7 @@ import java.util.List;
 
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.common.annotations.InterfaceStability;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.util.CarbonTaskInfo;
 import org.apache.carbondata.core.util.ThreadLocalTaskInfo;
 
@@ -91,6 +92,35 @@ public class CarbonReader<T> {
   }
 
   /**
+   * Read and return next string row object
+   * limitation: only single dimension Array is supported
+   * TODO: support didfferent data type
+   */
+  public Object[] readNextStringRow() throws IOException, InterruptedException {
+    validateReader();
+    T t = currentReader.getCurrentValue();
+    Object[] objects = (Object[]) t;
+    String[] strings = new String[objects.length];
+    for (int i = 0; i < objects.length; i++) {
+      if (objects[i] instanceof Object[]) {
+        Object[] arrayString = (Object[]) objects[i];
+        StringBuffer stringBuffer = new StringBuffer();
+        stringBuffer.append(String.valueOf(arrayString[0]));
+        if (arrayString.length > 1) {
+          for (int j = 1; j < arrayString.length; j++) {
+            stringBuffer.append(CarbonCommonConstants.ARRAY_SEPARATOR)
+                .append(String.valueOf(arrayString[j]));
+          }
+        }
+        strings[i] = stringBuffer.toString();
+      } else {
+        strings[i] = String.valueOf(objects[i]);
+      }
+    }
+    return strings;
+  }
+
+  /**
    * Return a new {@link CarbonReaderBuilder} instance
    *
    * @param tablePath table store path

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e9a198ab/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReaderBuilder.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReaderBuilder.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReaderBuilder.java
index af3480f..151d57c 100644
--- a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReaderBuilder.java
+++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReaderBuilder.java
@@ -102,6 +102,22 @@ public class CarbonReaderBuilder {
   }
 
   /**
+   * configure hadoop configuration with key value
+   *
+   * @param key   key word
+   * @param value value
+   * @return this object
+   */
+  public CarbonReaderBuilder withHadoopConf(String key, String value) {
+    if (this.hadoopConf == null) {
+      this.hadoopConf = new Configuration();
+
+    }
+    this.hadoopConf.set(key, value);
+    return this;
+  }
+
+  /**
    * Build CarbonReader
    *
    * @param <T>


[30/45] carbondata git commit: [CARBONDATA-2976] Support dumping column chunk metadata in CarbonCli

Posted by ra...@apache.org.
[CARBONDATA-2976] Support dumping column chunk metadata in CarbonCli

By using -k option, CarbonCli will print all column chunk/page metadata for the specified column
For example, java CarbonCli -cmd summary -p /home/root1/bin /home/root1/.local/bin /usr/local/sbin /usr/local/bin /usr/sbin /usr/bin /sbin /bin /usr/games /usr/local/games /snap/bin /usr/lib/jvm/java-8-oracle/bin /usr/lib/jvm/java-8-oracle/db/bin /usr/lib/jvm/java-8-oracle/jre/bin -c name -k
will output:

## Page Meta for column 'name' in file /Users/jacky/code/carbondata/tools/cli/CarbonCliTest/part-0-138391629343461_batchno0-0-null-138390048546321.carbondata
Blocklet 0:
Page 0 (offset 0, length 12049): DataChunk2(chunk_meta:ChunkCompressionMeta(compression_codec:DEPRECATED, total_uncompressed_size:0, total_compressed_size:0, compressor_name:snappy), rowMajor:false, data_page_length:12039, rowid_page_length:10, presence:PresenceMeta(represents_presence:false, present_bit_stream:00), sort_state:SORT_EXPLICIT, encoders:[INVERTED_INDEX], encoder_meta:[], min_max:BlockletMinMaxIndex(min_values:[72 6F 62 6F 74 30], max_values:[72 6F 62 6F 74 30], min_max_presence:[true]), numberOfRowsInpage:32000)
Page 1 (offset 12049, length 12049): DataChunk2(chunk_meta:ChunkCompressionMeta(compression_codec:DEPRECATED, total_uncompressed_size:0, total_compressed_size:0, compressor_name:snappy), rowMajor:false, data_page_length:12039, rowid_page_length:10, presence:PresenceMeta(represents_presence:false, present_bit_stream:00), sort_state:SORT_EXPLICIT, encoders:[INVERTED_INDEX], encoder_meta:[], min_max:BlockletMinMaxIndex(min_values:[72 6F 62 6F 74 30], max_values:[72 6F 62 6F 74 30], min_max_presence:[true]), numberOfRowsInpage:32000)
Page 2 (offset 24098, length 12049): DataChunk2(chunk_meta:ChunkCompressionMeta(compression_codec:DEPRECATED, total_uncompressed_size:0, total_compressed_size:0, compressor_name:snappy), rowMajor:false, data_page_length:12039, rowid_page_length:10, presence:PresenceMeta(represents_presence:false, present_bit_stream:00), sort_state:SORT_EXPLICIT, encoders:[INVERTED_INDEX], encoder_meta:[], min_max:BlockletMinMaxIndex(min_values:[72 6F 62 6F 74 30], max_values:[72 6F 62 6F 74 30], min_max_presence:[true]), numberOfRowsInpage:32000)

This closes #2771


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

Branch: refs/heads/branch-1.5
Commit: d8003a31c602807f00d438d8be392992cb0955ac
Parents: fa9c832
Author: Jacky Li <ja...@qq.com>
Authored: Wed Sep 26 23:51:34 2018 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Wed Oct 3 20:17:04 2018 +0530

----------------------------------------------------------------------
 .../org/apache/carbondata/tool/CarbonCli.java   |  2 +
 .../org/apache/carbondata/tool/DataFile.java    |  8 +++-
 .../org/apache/carbondata/tool/DataSummary.java | 45 ++++++++++++++++++--
 .../apache/carbondata/tool/CarbonCliTest.java   | 13 ++++++
 4 files changed, 63 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/d8003a31/tools/cli/src/main/java/org/apache/carbondata/tool/CarbonCli.java
----------------------------------------------------------------------
diff --git a/tools/cli/src/main/java/org/apache/carbondata/tool/CarbonCli.java b/tools/cli/src/main/java/org/apache/carbondata/tool/CarbonCli.java
index 5725f8e..f1baa92 100644
--- a/tools/cli/src/main/java/org/apache/carbondata/tool/CarbonCli.java
+++ b/tools/cli/src/main/java/org/apache/carbondata/tool/CarbonCli.java
@@ -65,6 +65,7 @@ public class CarbonCli {
     Option segment = new Option("m", "showSegment", false, "print segment information");
     Option tblProperties = new Option("t", "tblProperties", false, "print table properties");
     Option detail = new Option("b", "blocklet", false, "print blocklet size detail");
+    Option columnMeta = new Option("k", "columnChunkMeta", false, "print column chunk meta");
     Option columnName = OptionBuilder
         .withArgName("column name")
         .hasArg()
@@ -82,6 +83,7 @@ public class CarbonCli {
     options.addOption(segment);
     options.addOption(tblProperties);
     options.addOption(detail);
+    options.addOption(columnMeta);
     options.addOption(columnName);
     return options;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d8003a31/tools/cli/src/main/java/org/apache/carbondata/tool/DataFile.java
----------------------------------------------------------------------
diff --git a/tools/cli/src/main/java/org/apache/carbondata/tool/DataFile.java b/tools/cli/src/main/java/org/apache/carbondata/tool/DataFile.java
index da81d84..039401e 100644
--- a/tools/cli/src/main/java/org/apache/carbondata/tool/DataFile.java
+++ b/tools/cli/src/main/java/org/apache/carbondata/tool/DataFile.java
@@ -321,6 +321,8 @@ class DataFile {
     // they are set after calculation in DataSummary
     double minPercentage, maxPercentage;
 
+    DataChunk3 dataChunk;
+
     /**
      * Constructor
      * @param blockletInfo blocklet info which this column chunk belongs to
@@ -338,7 +340,7 @@ class DataFile {
       ByteBuffer buffer = fileReader.readByteBuffer(
           filePath, blockletInfo.column_data_chunks_offsets.get(columnIndex),
           blockletInfo.column_data_chunks_length.get(columnIndex));
-      DataChunk3 dataChunk = CarbonUtil.readDataChunk3(new ByteArrayInputStream(buffer.array()));
+      dataChunk = CarbonUtil.readDataChunk3(new ByteArrayInputStream(buffer.array()));
       this.localDict = dataChunk.isSetLocal_dictionary();
       if (this.localDict) {
         String compressorName = CarbonMetadataUtil.getCompressorNameFromChunkMeta(
@@ -376,6 +378,10 @@ class DataFile {
       return column.getDataType();
     }
 
+    public DataChunk3 getDataChunk3() {
+      return dataChunk;
+    }
+
     byte[] min(byte[] minValue) {
       if (minValue == null) {
         return min;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d8003a31/tools/cli/src/main/java/org/apache/carbondata/tool/DataSummary.java
----------------------------------------------------------------------
diff --git a/tools/cli/src/main/java/org/apache/carbondata/tool/DataSummary.java b/tools/cli/src/main/java/org/apache/carbondata/tool/DataSummary.java
index 6463977..5f1fb68 100644
--- a/tools/cli/src/main/java/org/apache/carbondata/tool/DataSummary.java
+++ b/tools/cli/src/main/java/org/apache/carbondata/tool/DataSummary.java
@@ -38,6 +38,8 @@ import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.format.BlockletInfo3;
+import org.apache.carbondata.format.DataChunk2;
+import org.apache.carbondata.format.DataChunk3;
 import org.apache.carbondata.format.FileFooter3;
 import org.apache.carbondata.format.FileHeader;
 import org.apache.carbondata.format.TableInfo;
@@ -91,6 +93,9 @@ class DataSummary implements Command {
     if (line.hasOption("c")) {
       String columName = line.getOptionValue("c");
       printColumnStats(columName);
+      if (line.hasOption("k")) {
+        printColumnChunkMeta(columName);
+      }
     }
   }
 
@@ -217,13 +222,13 @@ class DataSummary implements Command {
     throw new RuntimeException("schema for column " + columnName + " not found");
   }
 
+  // true if blockled stats are collected
+  private boolean collected = false;
+
   private void printColumnStats(String columnName) throws IOException, MemoryException {
     out.println();
     out.println("## Column Statistics for '" + columnName + "'");
-    for (DataFile dataFile : dataFiles.values()) {
-      dataFile.initAllBlockletStats(columnName);
-    }
-    collectAllBlockletStats(dataFiles.values());
+    collectStats(columnName);
 
     int columnIndex = getColumnIndex(columnName);
     String[] header = new String[]{"BLK", "BLKLT", "Meta Size", "Data Size",
@@ -260,6 +265,38 @@ class DataSummary implements Command {
     printer.printFormatted(out);
   }
 
+  private void collectStats(String columnName) throws IOException, MemoryException {
+    if (!collected) {
+      for (DataFile dataFile : dataFiles.values()) {
+        dataFile.initAllBlockletStats(columnName);
+      }
+      collectAllBlockletStats(dataFiles.values());
+      collected = true;
+    }
+  }
+
+  private void printColumnChunkMeta(String columnName) throws IOException, MemoryException {
+    out.println();
+    DataFile file = dataFiles.entrySet().iterator().next().getValue();
+    out.println("## Page Meta for column '" + columnName + "' in file " + file.getFilePath());
+    collectStats(columnName);
+    for (int i = 0; i < file.getAllBlocklets().size(); i++) {
+      DataFile.Blocklet blocklet = file.getAllBlocklets().get(i);
+      DataChunk3 dataChunk3 = blocklet.getColumnChunk().getDataChunk3();
+      List<DataChunk2> dataChunk2List = dataChunk3.getData_chunk_list();
+      out.println(String.format("Blocklet %d:", i));
+
+      // There will be many pages, for debugging purpose,
+      // just print 3 page for each blocklet is enough
+      for (int j = 0; j < dataChunk2List.size() && j < 3; j++) {
+        out.println(String.format("Page %d (offset %d, length %d): %s",
+            j, dataChunk3.page_offset.get(j), dataChunk3.page_length.get(j),
+            dataChunk2List.get(j).toString()));
+      }
+      out.println("\n");
+    }
+  }
+
   private void collectAllBlockletStats(Collection<DataFile> dataFiles) {
     // shard name mapping to blocklets belong to same shard
     Map<String, List<DataFile.Blocklet>> shards = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d8003a31/tools/cli/src/test/java/org/apache/carbondata/tool/CarbonCliTest.java
----------------------------------------------------------------------
diff --git a/tools/cli/src/test/java/org/apache/carbondata/tool/CarbonCliTest.java b/tools/cli/src/test/java/org/apache/carbondata/tool/CarbonCliTest.java
index fcd46c8..e526131 100644
--- a/tools/cli/src/test/java/org/apache/carbondata/tool/CarbonCliTest.java
+++ b/tools/cli/src/test/java/org/apache/carbondata/tool/CarbonCliTest.java
@@ -184,6 +184,19 @@ public class CarbonCliTest {
   }
 
   @Test
+  public void testSummaryPageMeta() {
+    String[] args = { "-cmd", "summary", "-p", path, "-c", "name", "-k"};
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+    PrintStream stream = new PrintStream(out);
+    CarbonCli.run(args, stream);
+    String output = new String(out.toByteArray());
+    Assert.assertTrue(
+        output.contains(
+            "Blocklet 0:\n"
+           + "Page 0 (offset 0, length 12049): DataChunk2(chunk_meta:ChunkCompressionMeta(compression_codec:DEPRECATED, total_uncompressed_size:0, total_compressed_size:0, compressor_name:snappy), rowMajor:false, data_page_length:12039, rowid_page_length:10, presence:PresenceMeta(represents_presence:false, present_bit_stream:00), sort_state:SORT_EXPLICIT, encoders:[INVERTED_INDEX], encoder_meta:[], min_max:BlockletMinMaxIndex(min_values:[72 6F 62 6F 74 30], max_values:[72 6F 62 6F 74 30], min_max_presence:[true]), numberOfRowsInpage:32000)"));
+  }
+
+  @Test
   public void testBenchmark() {
     String[] args = {"-cmd", "benchmark", "-p", path, "-a", "-c", "name"};
     ByteArrayOutputStream out = new ByteArrayOutputStream();


[26/45] carbondata git commit: Problem: Preaggregate select queries require table path of parent table to access dictionary files. Therefore in executor CarbonMetadata class was used to get parent table object. As CarbonMetadata class is only meant to be

Posted by ra...@apache.org.
Problem:
Preaggregate select queries require table path of parent table to access dictionary files. Therefore in executor CarbonMetadata class was used to get parent table object. As CarbonMetadata class is only meant to be used in driver and is not filled with carbontable objects for select queries therefore the query was throwing NPE.

Solution:
Pass parent table path from driver to executor by adding a new variable in RelationIdentifier. This will not be written to thrift, instead will be used to carry tablePath property from driver to executor.

This closes #2786


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

Branch: refs/heads/branch-1.5
Commit: 9ca985f0d96380d96dae6fdab2d4ee014d5ac345
Parents: 682160f
Author: kunal642 <ku...@gmail.com>
Authored: Fri Sep 28 17:41:32 2018 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Wed Oct 3 20:02:49 2018 +0530

----------------------------------------------------------------------
 .../core/metadata/schema/table/RelationIdentifier.java  | 12 ++++++++++++
 .../carbondata/core/scan/executor/util/QueryUtil.java   | 10 ++--------
 .../spark/sql/CarbonDatasourceHadoopRelation.scala      | 12 ++++++++++++
 3 files changed, 26 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/9ca985f0/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java
index 9a1dad1..0e8042d 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java
@@ -32,6 +32,8 @@ public class RelationIdentifier implements Serializable, Writable {
 
   private String tableId;
 
+  private String tablePath = "";
+
   public RelationIdentifier(String databaseName, String tableName, String tableId) {
     this.databaseName = databaseName;
     this.tableName = tableName;
@@ -50,16 +52,26 @@ public class RelationIdentifier implements Serializable, Writable {
     return tableId;
   }
 
+  public String getTablePath() {
+    return tablePath;
+  }
+
+  public void setTablePath(String tablePath) {
+    this.tablePath = tablePath;
+  }
+
   @Override public void write(DataOutput out) throws IOException {
     out.writeUTF(databaseName);
     out.writeUTF(tableName);
     out.writeUTF(tableId);
+    out.writeUTF(tablePath);
   }
 
   @Override public void readFields(DataInput in) throws IOException {
     this.databaseName = in.readUTF();
     this.tableName = in.readUTF();
     this.tableId = in.readUTF();
+    this.tablePath = in.readUTF();
   }
 
   @Override public boolean equals(Object o) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/9ca985f0/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
index 9fb0857..7849d10 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
@@ -393,12 +393,6 @@ public class QueryUtil {
 
   public static AbsoluteTableIdentifier getTableIdentifierForColumn(
       CarbonDimension carbonDimension) {
-    RelationIdentifier parentRelationIdentifier =
-        carbonDimension.getColumnSchema().getParentColumnTableRelations().get(0)
-            .getRelationIdentifier();
-    String parentTablePath = CarbonMetadata.getInstance()
-        .getCarbonTable(parentRelationIdentifier.getDatabaseName(),
-            parentRelationIdentifier.getTableName()).getTablePath();
     RelationIdentifier relation = carbonDimension.getColumnSchema()
         .getParentColumnTableRelations()
         .get(0)
@@ -406,8 +400,8 @@ public class QueryUtil {
     String parentTableName = relation.getTableName();
     String parentDatabaseName = relation.getDatabaseName();
     String parentTableId = relation.getTableId();
-    return AbsoluteTableIdentifier.from(parentTablePath, parentDatabaseName, parentTableName,
-        parentTableId);
+    return AbsoluteTableIdentifier.from(relation.getTablePath(), parentDatabaseName,
+        parentTableName, parentTableId);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/9ca985f0/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
index 8a0404c..04ec75d 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
@@ -17,6 +17,7 @@
 
 package org.apache.spark.sql
 
+import scala.collection.JavaConverters._
 import scala.collection.mutable.{ArrayBuffer, ListBuffer}
 import scala.util.control.Breaks._
 
@@ -35,6 +36,7 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.indexstore.PartitionSpec
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension
 import org.apache.carbondata.core.scan.expression.Expression
 import org.apache.carbondata.core.scan.expression.logical.AndExpression
 import org.apache.carbondata.hadoop.CarbonProjection
@@ -80,6 +82,16 @@ case class CarbonDatasourceHadoopRelation(
 
     val projection = new CarbonProjection
 
+    if (carbonTable.isChildDataMap) {
+      val parentTableIdentifier = carbonTable.getTableInfo.getParentRelationIdentifiers.get(0)
+      val path = CarbonEnv.getCarbonTable(Some(parentTableIdentifier.getDatabaseName),
+        parentTableIdentifier.getTableName)(sparkSession).getTablePath
+      for (carbonDimension: CarbonDimension <- carbonTable.getAllDimensions.asScala) {
+        carbonDimension.getColumnSchema.getParentColumnTableRelations.get(0)
+          .getRelationIdentifier.setTablePath(path)
+      }
+    }
+
     // As Filter pushdown for Complex datatype is not supported, if filter is applied on complex
     // column, then Projection pushdown on Complex Columns will not take effect. Hence, check if
     // filter contains Struct Complex Column.


[25/45] carbondata git commit: [CARBONDATA-2979] select count fails when carbondata file is written through SDK and read through sparkfileformat for complex datatype map(struct->array->map)

Posted by ra...@apache.org.
[CARBONDATA-2979] select count fails when carbondata file is written through SDK and read through sparkfileformat for complex datatype map(struct->array->map)

Problem
Select query failed issue for map type when data is loaded using avro SDK and external table using carbon file format is used to query the data

Analysis
When data is loaded through Avro SDK which has a schema of type struct<array>, fieldName was hard coded to val because of which during query the schema written in the file footer and schema inferred for the external table had a mismatch which lead to failure.

Solution
Instead of hard coding the field value as val use the given field name in the schema

This closes #2774


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

Branch: refs/heads/branch-1.5
Commit: 682160fa1bbde5f13c8a28e0114d3f18e5ffaf79
Parents: e9a198a
Author: manishgupta88 <to...@gmail.com>
Authored: Thu Sep 27 18:02:34 2018 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Wed Oct 3 19:57:50 2018 +0530

----------------------------------------------------------------------
 .../datasource/SparkCarbonDataSourceTest.scala  | 63 +++++++++++++++++++-
 .../sql/carbondata/datasource/TestUtil.scala    | 56 ++++++++++++++++-
 .../carbondata/sdk/file/AvroCarbonWriter.java   | 11 ++--
 3 files changed, 122 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/682160fa/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/SparkCarbonDataSourceTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/SparkCarbonDataSourceTest.scala b/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/SparkCarbonDataSourceTest.scala
index 3be8cb3..37677d0 100644
--- a/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/SparkCarbonDataSourceTest.scala
+++ b/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/SparkCarbonDataSourceTest.scala
@@ -1117,11 +1117,11 @@ class SparkCarbonDataSourceTest extends FunSuite with BeforeAndAfterAll {
   }
 
   private def createParquetTable {
-    FileFactory.deleteAllCarbonFilesOfDir(FileFactory.getCarbonFile(s"$warehouse1/../warehouse2"))
+    val path = FileFactory.getUpdatedFilePath(s"$warehouse1/../warehouse2")
+    FileFactory.deleteAllCarbonFilesOfDir(FileFactory.getCarbonFile(s"$path"))
     spark.sql(s"create table par_table(male boolean, age int, height double, name string, address " +
               s"string," +
-              s"salary long, floatField float, bytefield byte) using parquet location " +
-              s"'$warehouse1/../warehouse2'")
+              s"salary long, floatField float, bytefield byte) using parquet location '$path'")
     (0 to 10).foreach {
       i => spark.sql(s"insert into par_table select 'true','$i', ${i.toDouble / 2}, 'name$i', " +
                      s"'address$i', ${i*100}, $i.$i, '$i'")
@@ -1181,6 +1181,63 @@ class SparkCarbonDataSourceTest extends FunSuite with BeforeAndAfterAll {
     }
   }
 
+  def buildStructSchemaWithNestedArrayOfMapTypeAsValue(writerPath: String, rows: Int): Unit = {
+    FileFactory.deleteAllFilesOfDir(new File(writerPath))
+    val mySchema =
+      """
+        |{
+        |  "name": "address",
+        |  "type": "record",
+        |  "fields": [
+        |    {
+        |      "name": "name",
+        |      "type": "string"
+        |    },
+        |    {
+        |      "name": "age",
+        |      "type": "int"
+        |    },
+        |    {
+        |      "name": "structRecord",
+        |      "type": {
+        |        "type": "record",
+        |        "name": "my_address",
+        |        "fields": [
+        |          {
+        |            "name": "street",
+        |            "type": "string"
+        |          },
+        |          {
+        |            "name": "houseDetails",
+        |            "type": {
+        |               "type": "array",
+        |               "items": {
+        |                   "name": "memberDetails",
+        |                   "type": "map",
+        |                   "values": "string"
+        |                }
+        |             }
+        |          }
+        |        ]
+        |      }
+        |    }
+        |  ]
+        |}
+      """.stripMargin
+    val json = """ {"name":"bob", "age":10, "structRecord": {"street":"street1", "houseDetails": [{"101": "Rahul", "102": "Pawan"}]}} """.stripMargin
+    TestUtil.WriteFilesWithAvroWriter(writerPath, rows, mySchema, json)
+  }
+
+  test("test external table with struct type with value as nested struct<array<map>> type") {
+    val writerPath: String = FileFactory.getUpdatedFilePath(warehouse1 + "/sdk1")
+    val rowCount = 3
+    buildStructSchemaWithNestedArrayOfMapTypeAsValue(writerPath, rowCount)
+    spark.sql("drop table if exists carbon_external")
+    spark.sql(s"create table carbon_external using carbon location '$writerPath'")
+    assert(spark.sql("select * from carbon_external").count() == rowCount)
+    spark.sql("drop table if exists carbon_external")
+  }
+
   test("test byte and float for multiple pages") {
     val path = new File(warehouse1+"/sdk1").getAbsolutePath
     FileFactory.deleteAllFilesOfDir(new File(warehouse1+"/sdk1"))

http://git-wip-us.apache.org/repos/asf/carbondata/blob/682160fa/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/TestUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/TestUtil.scala b/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/TestUtil.scala
index b9185aa..f2285d6 100644
--- a/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/TestUtil.scala
+++ b/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/TestUtil.scala
@@ -16,17 +16,23 @@
  */
 package org.apache.spark.sql.carbondata.datasource
 
-import java.io.File
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, File, InputStream}
 
 import scala.collection.JavaConverters._
 
+import org.apache.avro
+import org.apache.avro.file.DataFileWriter
+import org.apache.avro.generic.{GenericDatumReader, GenericDatumWriter, GenericRecord}
+import org.apache.avro.io.{DecoderFactory, Encoder}
 import org.apache.spark.sql.carbondata.execution.datasources.CarbonFileIndexReplaceRule
 import org.apache.spark.sql.{DataFrame, Row, SparkSession}
 import org.apache.spark.sql.catalyst.plans.logical
 import org.apache.spark.sql.catalyst.util.sideBySide
+import org.junit.Assert
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.sdk.file.CarbonWriter
 
 object TestUtil {
 
@@ -134,4 +140,52 @@ object TestUtil {
     }
   }
 
+  def WriteFilesWithAvroWriter(writerPath: String,
+      rows: Int,
+      mySchema: String,
+      json: String) = {
+    // conversion to GenericData.Record
+    val nn = new avro.Schema.Parser().parse(mySchema)
+    val record = jsonToAvro(json, mySchema)
+    try {
+      val writer = CarbonWriter.builder
+        .outputPath(writerPath)
+        .uniqueIdentifier(System.currentTimeMillis()).withAvroInput(nn).build()
+      var i = 0
+      while (i < rows) {
+        writer.write(record)
+        i = i + 1
+      }
+      writer.close()
+    }
+    catch {
+      case e: Exception => {
+        e.printStackTrace()
+        Assert.fail(e.getMessage)
+      }
+    }
+  }
+
+  private def jsonToAvro(json: String, avroSchema: String): GenericRecord = {
+    var input: InputStream = null
+    var writer: DataFileWriter[GenericRecord] = null
+    var encoder: Encoder = null
+    var output: ByteArrayOutputStream = null
+    try {
+      val schema = new org.apache.avro.Schema.Parser().parse(avroSchema)
+      val reader = new GenericDatumReader[GenericRecord](schema)
+      input = new ByteArrayInputStream(json.getBytes())
+      output = new ByteArrayOutputStream()
+      val din = new DataInputStream(input)
+      writer = new DataFileWriter[GenericRecord](new GenericDatumWriter[GenericRecord]())
+      writer.create(schema, output)
+      val decoder = DecoderFactory.get().jsonDecoder(schema, din)
+      var datum: GenericRecord = reader.read(null, decoder)
+      return datum
+    } finally {
+      input.close()
+      writer.close()
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/682160fa/store/sdk/src/main/java/org/apache/carbondata/sdk/file/AvroCarbonWriter.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/AvroCarbonWriter.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/AvroCarbonWriter.java
index ab1e154..d19a96d 100644
--- a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/AvroCarbonWriter.java
+++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/AvroCarbonWriter.java
@@ -625,7 +625,8 @@ public class AvroCarbonWriter extends CarbonWriter {
       case ARRAY:
         // recursively get the sub fields
         // array will have only one sub field.
-        DataType subType = getMappingDataTypeForCollectionRecord(childSchema.getElementType());
+        DataType subType =
+            getMappingDataTypeForCollectionRecord(fieldName, childSchema.getElementType());
         if (subType != null) {
           return (new StructField(fieldName, DataTypes.createArrayType(subType)));
         } else {
@@ -661,7 +662,8 @@ public class AvroCarbonWriter extends CarbonWriter {
     }
   }
 
-  private static DataType getMappingDataTypeForCollectionRecord(Schema childSchema) {
+  private static DataType getMappingDataTypeForCollectionRecord(String fieldName,
+      Schema childSchema) {
     LogicalType logicalType = childSchema.getLogicalType();
     switch (childSchema.getType()) {
       case BOOLEAN:
@@ -700,7 +702,7 @@ public class AvroCarbonWriter extends CarbonWriter {
         return DataTypes.FLOAT;
       case MAP:
         // recursively get the sub fields
-        StructField mapField = prepareSubFields("val", childSchema);
+        StructField mapField = prepareSubFields(fieldName, childSchema);
         if (mapField != null) {
           return mapField.getDataType();
         }
@@ -717,7 +719,8 @@ public class AvroCarbonWriter extends CarbonWriter {
         return DataTypes.createStructType(structSubFields);
       case ARRAY:
         // array will have only one sub field.
-        DataType subType = getMappingDataTypeForCollectionRecord(childSchema.getElementType());
+        DataType subType =
+            getMappingDataTypeForCollectionRecord(fieldName, childSchema.getElementType());
         if (subType != null) {
           return DataTypes.createArrayType(subType);
         } else {


[23/45] carbondata git commit: [CARBONDATA-2982] CarbonSchemaReader support array

Posted by ra...@apache.org.
[CARBONDATA-2982] CarbonSchemaReader support array<string>

This PR fix the issue and change :
org.apache.carbondata.sdk.file.CarbonSchemaReader#readSchemaInDataFile
org.apache.carbondata.sdk.file.CarbonSchemaReader#readSchemaInIndexFile

This PR remove child schema

This closes #2780


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

Branch: refs/heads/branch-1.5
Commit: d8a51c9bf314fb1cd5f6112e66eb04e776a0553d
Parents: 7d1fcb3
Author: xubo245 <xu...@huawei.com>
Authored: Fri Sep 28 11:47:22 2018 +0800
Committer: manishgupta88 <to...@gmail.com>
Committed: Wed Oct 3 16:24:38 2018 +0530

----------------------------------------------------------------------
 .../examples/sdk/CarbonReaderExample.java       |  4 +-
 .../carbondata/examplesCI/RunExamples.scala     |  5 ++
 .../carbondata/sdk/file/CarbonSchemaReader.java | 14 +++-
 .../carbondata/sdk/file/CarbonReaderTest.java   | 86 ++++++++++++++++++++
 4 files changed, 105 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/d8a51c9b/examples/spark2/src/main/java/org/apache/carbondata/examples/sdk/CarbonReaderExample.java
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/java/org/apache/carbondata/examples/sdk/CarbonReaderExample.java b/examples/spark2/src/main/java/org/apache/carbondata/examples/sdk/CarbonReaderExample.java
index 9e80567..ef4ae7a 100644
--- a/examples/spark2/src/main/java/org/apache/carbondata/examples/sdk/CarbonReaderExample.java
+++ b/examples/spark2/src/main/java/org/apache/carbondata/examples/sdk/CarbonReaderExample.java
@@ -122,10 +122,11 @@ public class CarbonReaderExample {
                 for (int j = 0; j < arr.length; j++) {
                     System.out.print(arr[j] + " ");
                 }
+                assert (arr[0].equals("Hello"));
+                assert (arr[3].equals("Carbon"));
                 System.out.println();
                 i++;
             }
-            System.out.println("\nFinished");
             reader.close();
 
             // Read data
@@ -148,7 +149,6 @@ public class CarbonReaderExample {
                     row[5], row[6], row[7], row[8], row[9], row[10]));
                 i++;
             }
-            System.out.println("\nFinished");
             reader2.close();
             FileUtils.deleteDirectory(new File(path));
         } catch (Throwable e) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d8a51c9b/examples/spark2/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala b/examples/spark2/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala
index 2b9b999..6a13dc3 100644
--- a/examples/spark2/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala
+++ b/examples/spark2/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala
@@ -23,6 +23,7 @@ import org.scalatest.BeforeAndAfterAll
 import org.apache.carbondata.examples._
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.examples.sdk.CarbonReaderExample
 
 /**
  * Test suite for examples
@@ -113,4 +114,8 @@ class RunExamples extends QueryTest with BeforeAndAfterAll {
   test("ExternalTableExample") {
     ExternalTableExample.exampleBody(spark)
   }
+
+  test("CarbonReaderExample") {
+    CarbonReaderExample.main(null)
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d8a51c9b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonSchemaReader.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonSchemaReader.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonSchemaReader.java
index d8882bc..e84a25a 100644
--- a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonSchemaReader.java
+++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonSchemaReader.java
@@ -65,7 +65,15 @@ public class CarbonSchemaReader {
    */
   public static Schema readSchemaInDataFile(String dataFilePath) throws IOException {
     CarbonHeaderReader reader = new CarbonHeaderReader(dataFilePath);
-    return new Schema(reader.readSchema());
+    List<ColumnSchema> columnSchemaList = new ArrayList<ColumnSchema>();
+    List<ColumnSchema> schemaList = reader.readSchema();
+    for (int i = 0; i < schemaList.size(); i++) {
+      ColumnSchema columnSchema = schemaList.get(i);
+      if (!(columnSchema.getColumnName().contains("."))) {
+        columnSchemaList.add(columnSchema);
+      }
+    }
+    return new Schema(columnSchemaList);
   }
 
   /**
@@ -97,7 +105,9 @@ public class CarbonSchemaReader {
       List<org.apache.carbondata.format.ColumnSchema> table_columns =
           readIndexHeader.getTable_columns();
       for (org.apache.carbondata.format.ColumnSchema columnSchema : table_columns) {
-        columnSchemaList.add(thriftColumnSchemaToWrapperColumnSchema(columnSchema));
+        if (!(columnSchema.column_name.contains("."))) {
+          columnSchemaList.add(thriftColumnSchemaToWrapperColumnSchema(columnSchema));
+        }
       }
       return new Schema(columnSchemaList);
     } finally {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d8a51c9b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java
index 8c5ffe5..8d95456 100644
--- a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java
+++ b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java
@@ -1435,5 +1435,91 @@ public class CarbonReaderTest extends TestCase {
     FileUtils.deleteDirectory(new File("./testWriteFiles"));
   }
 
+  @Test
+  public void testReadSchemaFromDataFileArrayString() {
+    String path = "./testWriteFiles";
+    try {
+      FileUtils.deleteDirectory(new File(path));
 
+      Field[] fields = new Field[11];
+      fields[0] = new Field("stringField", DataTypes.STRING);
+      fields[1] = new Field("shortField", DataTypes.SHORT);
+      fields[2] = new Field("intField", DataTypes.INT);
+      fields[3] = new Field("longField", DataTypes.LONG);
+      fields[4] = new Field("doubleField", DataTypes.DOUBLE);
+      fields[5] = new Field("boolField", DataTypes.BOOLEAN);
+      fields[6] = new Field("dateField", DataTypes.DATE);
+      fields[7] = new Field("timeField", DataTypes.TIMESTAMP);
+      fields[8] = new Field("decimalField", DataTypes.createDecimalType(8, 2));
+      fields[9] = new Field("varcharField", DataTypes.VARCHAR);
+      fields[10] = new Field("arrayField", DataTypes.createArrayType(DataTypes.STRING));
+      Map<String, String> map = new HashMap<>();
+      map.put("complex_delimiter_level_1", "#");
+      CarbonWriter writer = CarbonWriter.builder()
+          .outputPath(path)
+          .withLoadOptions(map)
+          .withCsvInput(new Schema(fields)).build();
+
+      for (int i = 0; i < 10; i++) {
+        String[] row2 = new String[]{
+            "robot" + (i % 10),
+            String.valueOf(i % 10000),
+            String.valueOf(i),
+            String.valueOf(Long.MAX_VALUE - i),
+            String.valueOf((double) i / 2),
+            String.valueOf(true),
+            "2019-03-02",
+            "2019-02-12 03:03:34",
+            "12.345",
+            "varchar",
+            "Hello#World#From#Carbon"
+        };
+        writer.write(row2);
+      }
+      writer.close();
+
+      File[] dataFiles = new File(path).listFiles(new FilenameFilter() {
+        @Override
+        public boolean accept(File dir, String name) {
+          if (name == null) {
+            return false;
+          }
+          return name.endsWith("carbondata");
+        }
+      });
+      if (dataFiles == null || dataFiles.length < 1) {
+        throw new RuntimeException("Carbon index file not exists.");
+      }
+      Schema schema = CarbonSchemaReader
+          .readSchemaInDataFile(dataFiles[0].getAbsolutePath())
+          .asOriginOrder();
+      // Transform the schema
+      String[] strings = new String[schema.getFields().length];
+      for (int i = 0; i < schema.getFields().length; i++) {
+        strings[i] = (schema.getFields())[i].getFieldName();
+      }
+
+      // Read data
+      CarbonReader reader = CarbonReader
+          .builder(path, "_temp")
+          .projection(strings)
+          .build();
+
+      int i = 0;
+      while (reader.hasNext()) {
+        Object[] row = (Object[]) reader.readNextRow();
+        assert (row[0].equals("robot" + i));
+        assert (row[2].equals(i));
+        assert (row[6].equals(17957));
+        Object[] arr = (Object[]) row[10];
+        assert (arr[0].equals("Hello"));
+        assert (arr[3].equals("Carbon"));
+        i++;
+      }
+      reader.close();
+      FileUtils.deleteDirectory(new File(path));
+    } catch (Throwable e) {
+      e.printStackTrace();
+    }
+  }
 }


[20/45] carbondata git commit: [CARBONDATA-2980][BloomDataMap] Fix bug in clearing bloomindex cache when recreating table and datamap

Posted by ra...@apache.org.
[CARBONDATA-2980][BloomDataMap] Fix bug in clearing bloomindex cache when recreating table and datamap

We use shard path as a part of key for bloomindex cache. However the
path separator in windows is different from that in linux, which causes
the cache not being cleaned when we clear the cache. (When we loading
the cache, the path separator is '/' while dropping the cache, the path
separator is '\')

Here we fixed the bug by uniforming the path separator while clearing
the cache.

This closes #2778


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

Branch: refs/heads/branch-1.5
Commit: 1c1ced32d122ba8ce7cbad4fd29f778f5dbb4871
Parents: 9ae91cc
Author: xuchuanyin <xu...@hust.edu.cn>
Authored: Sat Sep 29 14:03:09 2018 +0800
Committer: manishgupta88 <to...@gmail.com>
Committed: Sun Sep 30 12:19:56 2018 +0530

----------------------------------------------------------------------
 .../datamap/bloom/BloomCoarseGrainDataMapFactory.java           | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c1ced32/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapFactory.java
----------------------------------------------------------------------
diff --git a/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapFactory.java b/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapFactory.java
index 8c74c94..8974918 100644
--- a/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapFactory.java
+++ b/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapFactory.java
@@ -235,13 +235,13 @@ public class BloomCoarseGrainDataMapFactory extends DataMapFactory<CoarseGrainDa
       } else if (carbonFile.getName().equals(BloomIndexFileStore.MERGE_INPROGRESS_FILE)) {
         mergeShardInprogress = true;
       } else if (carbonFile.isDirectory()) {
-        shardPaths.add(carbonFile.getAbsolutePath());
+        shardPaths.add(FileFactory.getPath(carbonFile.getAbsolutePath()).toString());
       }
     }
     if (mergeShardFile != null && !mergeShardInprogress) {
       // should only get one shard path if mergeShard is generated successfully
       shardPaths.clear();
-      shardPaths.add(mergeShardFile.getAbsolutePath());
+      shardPaths.add(FileFactory.getPath(mergeShardFile.getAbsolutePath()).toString());
     }
     return shardPaths;
   }
@@ -349,6 +349,7 @@ public class BloomCoarseGrainDataMapFactory extends DataMapFactory<CoarseGrainDa
             FileFactory.getFileType(datamapPath));
         CarbonUtil.deleteFoldersAndFilesSilent(file);
       }
+      clear(segment);
     } catch (InterruptedException ex) {
       throw new IOException("Failed to delete datamap for segment_" + segment.getSegmentNo());
     }


[39/45] carbondata git commit: [CARBONDATA-2991]NegativeArraySizeException during query execution

Posted by ra...@apache.org.
[CARBONDATA-2991]NegativeArraySizeException during query execution

Issue :- During Query Execution sometime NegativeArraySizeException Exception in Some Tasks . And sometime Executor is lost (JVM crash)

Root Cause :- It is because existing memoryblock is removed while it was in-use. This happened because duplicate taskid generated. Sometime freed same memory addresses are assigned to another task which will initialize memory block to0 and this cause NegativeSizeArrayException whereas sometime freed memory will not be used any task of executor process but running task will try to access it and as that address is not part of process so JVM crash will happen.

Solution :- Change taskID generation to UUID based instead of System.nanoTime()

This closes #2796


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

Branch: refs/heads/branch-1.5
Commit: d3927172f19b2251e77b840e53d8678cba2a38bd
Parents: 30adaa8
Author: BJangir <ba...@gmail.com>
Authored: Wed Oct 3 23:05:42 2018 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Oct 4 21:11:09 2018 +0530

----------------------------------------------------------------------
 .../unsafe/UnsafeAbstractDimensionDataChunkStore.java  |  2 +-
 .../core/datastore/page/UnsafeFixLengthColumnPage.java |  2 +-
 .../core/datastore/page/VarLengthColumnPageBase.java   |  2 +-
 .../core/indexstore/AbstractMemoryDMStore.java         |  2 +-
 .../carbondata/core/memory/IntPointerBuffer.java       |  4 ++--
 .../carbondata/core/memory/UnsafeMemoryManager.java    | 12 ++++++------
 .../core/memory/UnsafeSortMemoryManager.java           | 13 +++++++------
 .../apache/carbondata/core/util/CarbonTaskInfo.java    |  6 +++---
 .../org/apache/carbondata/core/util/CarbonUtil.java    | 10 ++++++++++
 .../carbondata/core/util/ThreadLocalTaskInfo.java      |  3 ++-
 .../org/apache/carbondata/spark/rdd/CarbonRDD.scala    |  2 +-
 .../org/apache/carbondata/spark/util/CommonUtil.scala  |  2 +-
 .../loading/sort/unsafe/UnsafeCarbonRowPage.java       |  4 ++--
 .../loading/sort/unsafe/UnsafeSortDataRows.java        |  2 +-
 .../org/apache/carbondata/sdk/file/CarbonReader.java   |  3 ++-
 .../carbondata/store/worker/SearchRequestHandler.java  |  9 +++------
 16 files changed, 44 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/d3927172/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeAbstractDimensionDataChunkStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeAbstractDimensionDataChunkStore.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeAbstractDimensionDataChunkStore.java
index 940ca1a..89bce2d 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeAbstractDimensionDataChunkStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeAbstractDimensionDataChunkStore.java
@@ -62,7 +62,7 @@ public abstract class UnsafeAbstractDimensionDataChunkStore implements Dimension
    */
   protected boolean isMemoryOccupied;
 
-  private final long taskId = ThreadLocalTaskInfo.getCarbonTaskInfo().getTaskId();
+  private final String taskId = ThreadLocalTaskInfo.getCarbonTaskInfo().getTaskId();
 
   /**
    * Constructor

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d3927172/core/src/main/java/org/apache/carbondata/core/datastore/page/UnsafeFixLengthColumnPage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/UnsafeFixLengthColumnPage.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/UnsafeFixLengthColumnPage.java
index 9e0eb8d..7df29df 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/UnsafeFixLengthColumnPage.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/UnsafeFixLengthColumnPage.java
@@ -51,7 +51,7 @@ public class UnsafeFixLengthColumnPage extends ColumnPage {
   // size of the allocated memory, in bytes
   private int capacity;
 
-  private final long taskId = ThreadLocalTaskInfo.getCarbonTaskInfo().getTaskId();
+  private final String taskId = ThreadLocalTaskInfo.getCarbonTaskInfo().getTaskId();
 
   private static final int byteBits = DataTypes.BYTE.getSizeBits();
   private static final int shortBits = DataTypes.SHORT.getSizeBits();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d3927172/core/src/main/java/org/apache/carbondata/core/datastore/page/VarLengthColumnPageBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/VarLengthColumnPageBase.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/VarLengthColumnPageBase.java
index 39395c3..35d0009 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/VarLengthColumnPageBase.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/VarLengthColumnPageBase.java
@@ -45,7 +45,7 @@ public abstract class VarLengthColumnPageBase extends ColumnPage {
 
   static final double FACTOR = 1.25;
 
-  final long taskId = ThreadLocalTaskInfo.getCarbonTaskInfo().getTaskId();
+  final String taskId = ThreadLocalTaskInfo.getCarbonTaskInfo().getTaskId();
 
   // memory allocated by Unsafe
   MemoryBlock memoryBlock;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d3927172/core/src/main/java/org/apache/carbondata/core/indexstore/AbstractMemoryDMStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/AbstractMemoryDMStore.java b/core/src/main/java/org/apache/carbondata/core/indexstore/AbstractMemoryDMStore.java
index 5880943..5e08cf9 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/AbstractMemoryDMStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/AbstractMemoryDMStore.java
@@ -31,7 +31,7 @@ public abstract class AbstractMemoryDMStore implements Serializable {
 
   protected boolean isMemoryFreed;
 
-  protected final long taskId = ThreadLocalTaskInfo.getCarbonTaskInfo().getTaskId();
+  protected final String taskId = ThreadLocalTaskInfo.getCarbonTaskInfo().getTaskId();
 
   public abstract void addIndexRow(CarbonRowSchema[] schema, DataMapRow indexRow)
       throws MemoryException;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d3927172/core/src/main/java/org/apache/carbondata/core/memory/IntPointerBuffer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/memory/IntPointerBuffer.java b/core/src/main/java/org/apache/carbondata/core/memory/IntPointerBuffer.java
index 5cd8e8d..58d873c 100644
--- a/core/src/main/java/org/apache/carbondata/core/memory/IntPointerBuffer.java
+++ b/core/src/main/java/org/apache/carbondata/core/memory/IntPointerBuffer.java
@@ -36,9 +36,9 @@ public class IntPointerBuffer {
 
   private MemoryBlock pointerMemoryBlock;
 
-  private long taskId;
+  private String taskId;
 
-  public IntPointerBuffer(long taskId) {
+  public IntPointerBuffer(String taskId) {
     // TODO can be configurable, it is initial size and it can grow automatically.
     this.length = 100000;
     pointerBlock = new int[length];

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d3927172/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java b/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java
index 4efea1a..048c058 100644
--- a/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java
@@ -39,7 +39,7 @@ public class UnsafeMemoryManager {
   private static boolean offHeap = Boolean.parseBoolean(CarbonProperties.getInstance()
       .getProperty(CarbonCommonConstants.ENABLE_OFFHEAP_SORT,
           CarbonCommonConstants.ENABLE_OFFHEAP_SORT_DEFAULT));
-  private static Map<Long,Set<MemoryBlock>> taskIdToMemoryBlockMap;
+  private static Map<String,Set<MemoryBlock>> taskIdToMemoryBlockMap;
   static {
     long size = 0L;
     String defaultWorkingMemorySize = null;
@@ -107,7 +107,7 @@ public class UnsafeMemoryManager {
         .info("Working Memory manager is created with size " + totalMemory + " with " + memoryType);
   }
 
-  private synchronized MemoryBlock allocateMemory(MemoryType memoryType, long taskId,
+  private synchronized MemoryBlock allocateMemory(MemoryType memoryType, String taskId,
       long memoryRequested) {
     if (memoryUsed + memoryRequested <= totalMemory) {
       MemoryBlock allocate = getMemoryAllocator(memoryType).allocate(memoryRequested);
@@ -128,7 +128,7 @@ public class UnsafeMemoryManager {
     return null;
   }
 
-  public synchronized void freeMemory(long taskId, MemoryBlock memoryBlock) {
+  public synchronized void freeMemory(String taskId, MemoryBlock memoryBlock) {
     if (taskIdToMemoryBlockMap.containsKey(taskId)) {
       taskIdToMemoryBlockMap.get(taskId).remove(memoryBlock);
     }
@@ -144,7 +144,7 @@ public class UnsafeMemoryManager {
     }
   }
 
-  public synchronized void freeMemoryAll(long taskId) {
+  public synchronized void freeMemoryAll(String taskId) {
     Set<MemoryBlock> memoryBlockSet = null;
     memoryBlockSet = taskIdToMemoryBlockMap.remove(taskId);
     long occuppiedMemory = 0;
@@ -181,12 +181,12 @@ public class UnsafeMemoryManager {
   /**
    * It tries to allocate memory of `size` bytes, keep retry until it allocates successfully.
    */
-  public static MemoryBlock allocateMemoryWithRetry(long taskId, long size)
+  public static MemoryBlock allocateMemoryWithRetry(String taskId, long size)
       throws MemoryException {
     return allocateMemoryWithRetry(INSTANCE.memoryType, taskId, size);
   }
 
-  public static MemoryBlock allocateMemoryWithRetry(MemoryType memoryType, long taskId,
+  public static MemoryBlock allocateMemoryWithRetry(MemoryType memoryType, String taskId,
       long size) throws MemoryException {
     MemoryBlock baseBlock = null;
     int tries = 0;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d3927172/core/src/main/java/org/apache/carbondata/core/memory/UnsafeSortMemoryManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/memory/UnsafeSortMemoryManager.java b/core/src/main/java/org/apache/carbondata/core/memory/UnsafeSortMemoryManager.java
index d8d4f81..67b8d43 100644
--- a/core/src/main/java/org/apache/carbondata/core/memory/UnsafeSortMemoryManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/memory/UnsafeSortMemoryManager.java
@@ -49,7 +49,7 @@ public class UnsafeSortMemoryManager {
   /**
    * map to keep taskid to memory blocks
    */
-  private static Map<Long, Set<MemoryBlock>> taskIdToMemoryBlockMap;
+  private static Map<String, Set<MemoryBlock>> taskIdToMemoryBlockMap;
 
   /**
    * singleton instance
@@ -142,7 +142,7 @@ public class UnsafeSortMemoryManager {
     }
   }
 
-  public synchronized void freeMemory(long taskId, MemoryBlock memoryBlock) {
+  public synchronized void freeMemory(String taskId, MemoryBlock memoryBlock) {
     if (taskIdToMemoryBlockMap.containsKey(taskId)) {
       taskIdToMemoryBlockMap.get(taskId).remove(memoryBlock);
     }
@@ -164,7 +164,7 @@ public class UnsafeSortMemoryManager {
    * when in case of task failure we need to clear all the memory occupied
    * @param taskId
    */
-  public synchronized void freeMemoryAll(long taskId) {
+  public synchronized void freeMemoryAll(String taskId) {
     Set<MemoryBlock> memoryBlockSet = null;
     memoryBlockSet = taskIdToMemoryBlockMap.remove(taskId);
     long occuppiedMemory = 0;
@@ -196,7 +196,7 @@ public class UnsafeSortMemoryManager {
    * @param memoryRequested
    * @return memory block
    */
-  public synchronized MemoryBlock allocateMemoryLazy(long taskId, long memoryRequested) {
+  public synchronized MemoryBlock allocateMemoryLazy(String taskId, long memoryRequested) {
     MemoryBlock allocate = allocator.allocate(memoryRequested);
     Set<MemoryBlock> listOfMemoryBlock = taskIdToMemoryBlockMap.get(taskId);
     if (null == listOfMemoryBlock) {
@@ -210,7 +210,8 @@ public class UnsafeSortMemoryManager {
   /**
    * It tries to allocate memory of `size` bytes, keep retry until it allocates successfully.
    */
-  public static MemoryBlock allocateMemoryWithRetry(long taskId, long size) throws MemoryException {
+  public static MemoryBlock allocateMemoryWithRetry(String taskId, long size)
+          throws MemoryException {
     MemoryBlock baseBlock = null;
     int tries = 0;
     while (tries < 100) {
@@ -232,7 +233,7 @@ public class UnsafeSortMemoryManager {
     return baseBlock;
   }
 
-  private synchronized MemoryBlock allocateMemory(long taskId, long memoryRequested) {
+  private synchronized MemoryBlock allocateMemory(String taskId, long memoryRequested) {
     if (memoryUsed + memoryRequested <= totalMemory) {
       MemoryBlock allocate = allocator.allocate(memoryRequested);
       memoryUsed += allocate.size();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d3927172/core/src/main/java/org/apache/carbondata/core/util/CarbonTaskInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonTaskInfo.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonTaskInfo.java
index d3e4d7a..19e0186 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonTaskInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonTaskInfo.java
@@ -28,13 +28,13 @@ public class CarbonTaskInfo implements Serializable {
    */
   private static final long serialVersionUID = 1L;
 
-  public long taskId;
+  public String taskId;
 
-  public long getTaskId() {
+  public String getTaskId() {
     return taskId;
   }
 
-  public void setTaskId(long taskId) {
+  public void setTaskId(String taskId) {
     this.taskId = taskId;
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d3927172/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 03054bf..937c222 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
@@ -3358,4 +3358,14 @@ public final class CarbonUtil {
     }
     return columnIndexTemp;
   }
+
+  /**
+   * Below method is to generateUUID (Random Based)
+   * later it will be extened for TimeBased,NameBased
+   *
+   * @return UUID as String
+   */
+  public static String generateUUID() {
+    return UUID.randomUUID().toString();
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d3927172/core/src/main/java/org/apache/carbondata/core/util/ThreadLocalTaskInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/ThreadLocalTaskInfo.java b/core/src/main/java/org/apache/carbondata/core/util/ThreadLocalTaskInfo.java
index 1e83e0d..d590612 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/ThreadLocalTaskInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/ThreadLocalTaskInfo.java
@@ -16,6 +16,7 @@
  */
 package org.apache.carbondata.core.util;
 
+
 /**
  * Class to keep all the thread local variable for task
  */
@@ -30,7 +31,7 @@ public class ThreadLocalTaskInfo {
   public static CarbonTaskInfo getCarbonTaskInfo() {
     if (null == threadLocal.get()) {
       CarbonTaskInfo carbonTaskInfo = new CarbonTaskInfo();
-      carbonTaskInfo.setTaskId(System.nanoTime());
+      carbonTaskInfo.setTaskId(CarbonUtil.generateUUID());
       ThreadLocalTaskInfo.setCarbonTaskInfo(carbonTaskInfo);
     }
     return threadLocal.get();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d3927172/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
index 096041f..87d8f50 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
@@ -71,7 +71,7 @@ abstract class CarbonRDD[T: ClassTag](
     ThreadLocalSessionInfo.setCarbonSessionInfo(carbonSessionInfo)
     TaskMetricsMap.threadLocal.set(Thread.currentThread().getId)
     val carbonTaskInfo = new CarbonTaskInfo
-    carbonTaskInfo.setTaskId(System.nanoTime)
+    carbonTaskInfo.setTaskId(CarbonUtil.generateUUID())
     ThreadLocalTaskInfo.setCarbonTaskInfo(carbonTaskInfo)
     carbonSessionInfo.getSessionParams.getAddedProps.asScala.
       map(f => CarbonProperties.getInstance().addProperty(f._1, f._2))

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d3927172/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
index 49b17fb..f6e2b94 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
@@ -639,7 +639,7 @@ object CommonUtil {
    * Method to clear the memory for a task
    * if present
    */
-  def clearUnsafeMemory(taskId: Long) {
+  def clearUnsafeMemory(taskId: String) {
     UnsafeMemoryManager.
       INSTANCE.freeMemoryAll(taskId)
     UnsafeSortMemoryManager.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d3927172/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java
index 0cef908..a480cf7 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java
@@ -46,14 +46,14 @@ public class UnsafeCarbonRowPage {
 
   private MemoryManagerType managerType;
 
-  private long taskId;
+  private String taskId;
 
   private TableFieldStat tableFieldStat;
   private SortStepRowHandler sortStepRowHandler;
   private boolean convertNoSortFields;
 
   public UnsafeCarbonRowPage(TableFieldStat tableFieldStat, MemoryBlock memoryBlock,
-      boolean saveToDisk, long taskId) {
+      boolean saveToDisk, String taskId) {
     this.tableFieldStat = tableFieldStat;
     this.sortStepRowHandler = new SortStepRowHandler(tableFieldStat);
     this.saveToDisk = saveToDisk;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d3927172/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java
index 64d941b..9c23dde 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java
@@ -93,7 +93,7 @@ public class UnsafeSortDataRows {
    */
   private Semaphore semaphore;
 
-  private final long taskId;
+  private final String taskId;
 
   public UnsafeSortDataRows(SortParameters parameters,
       UnsafeIntermediateMerger unsafeInMemoryIntermediateFileMerger, int inMemoryChunkSize) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d3927172/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReader.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReader.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReader.java
index 1f1794c..37a8779 100644
--- a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReader.java
+++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReader.java
@@ -26,6 +26,7 @@ import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.common.annotations.InterfaceStability;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.util.CarbonTaskInfo;
+import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.ThreadLocalTaskInfo;
 
 import org.apache.hadoop.mapreduce.RecordReader;
@@ -58,7 +59,7 @@ public class CarbonReader<T> {
     this.index = 0;
     this.currentReader = readers.get(0);
     CarbonTaskInfo carbonTaskInfo = new CarbonTaskInfo();
-    carbonTaskInfo.setTaskId(System.nanoTime());
+    carbonTaskInfo.setTaskId(CarbonUtil.generateUUID());
     ThreadLocalTaskInfo.setCarbonTaskInfo(carbonTaskInfo);
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d3927172/store/search/src/main/java/org/apache/carbondata/store/worker/SearchRequestHandler.java
----------------------------------------------------------------------
diff --git a/store/search/src/main/java/org/apache/carbondata/store/worker/SearchRequestHandler.java b/store/search/src/main/java/org/apache/carbondata/store/worker/SearchRequestHandler.java
index 7a764dd..2d3e5fe 100644
--- a/store/search/src/main/java/org/apache/carbondata/store/worker/SearchRequestHandler.java
+++ b/store/search/src/main/java/org/apache/carbondata/store/worker/SearchRequestHandler.java
@@ -18,11 +18,7 @@
 package org.apache.carbondata.store.worker;
 
 import java.io.IOException;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Objects;
+import java.util.*;
 
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.common.logging.LogService;
@@ -48,6 +44,7 @@ import org.apache.carbondata.core.scan.model.QueryModelBuilder;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
 import org.apache.carbondata.core.util.CarbonTaskInfo;
+import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.ThreadLocalTaskInfo;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.hadoop.CarbonInputSplit;
@@ -110,7 +107,7 @@ public class SearchRequestHandler {
   private List<CarbonRow> handleRequest(SearchRequest request)
       throws IOException, InterruptedException {
     CarbonTaskInfo carbonTaskInfo = new CarbonTaskInfo();
-    carbonTaskInfo.setTaskId(System.nanoTime());
+    carbonTaskInfo.setTaskId(CarbonUtil.generateUUID());
     ThreadLocalTaskInfo.setCarbonTaskInfo(carbonTaskInfo);
     TableInfo tableInfo = request.tableInfo();
     CarbonTable table = CarbonTable.buildFromTableInfo(tableInfo);


[14/45] carbondata git commit: [CARBONDATA-2972] Debug Logs and function added for Adaptive Encoding

Posted by ra...@apache.org.
[CARBONDATA-2972] Debug Logs and function added for Adaptive Encoding

Added a function to get the type of encoding used. Added the debug log for checking which type of encoding is used

This closes #2758


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

Branch: refs/heads/branch-1.5
Commit: 54bcf49638262af82583d930632018da6c73c8c5
Parents: 1b4109d
Author: Manish Nalla <ma...@gmail.com>
Authored: Tue Sep 25 17:44:49 2018 +0530
Committer: manishgupta88 <to...@gmail.com>
Committed: Thu Sep 27 17:16:37 2018 +0530

----------------------------------------------------------------------
 .../core/datastore/page/encoding/ColumnPageEncoder.java      | 8 ++++++++
 .../org/apache/carbondata/processing/store/TablePage.java    | 3 ++-
 2 files changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/54bcf496/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoder.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoder.java
index 3067823..44e7192 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoder.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoder.java
@@ -78,6 +78,14 @@ public abstract class ColumnPageEncoder {
     }
   }
 
+  public Encoding getEncodingType() {
+    List<Encoding> currEncodingList = getEncodingList();
+    if (CarbonUtil.isEncodedWithMeta(currEncodingList)) {
+      return currEncodingList.get(0);
+    }
+    return null;
+  }
+
   /**
    * Return a encoded column page by encoding the input page
    * The encoded binary data and metadata are wrapped in encoding column page

http://git-wip-us.apache.org/repos/asf/carbondata/blob/54bcf496/processing/src/main/java/org/apache/carbondata/processing/store/TablePage.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/TablePage.java b/processing/src/main/java/org/apache/carbondata/processing/store/TablePage.java
index 791b4c6..82129db 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/TablePage.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/TablePage.java
@@ -424,7 +424,8 @@ public class TablePage {
                   "Encoder result ---> Source data type: " + noDictDimensionPages[noDictIndex]
                       .getDataType().getName() + " Destination data type: " + targetDataType
                       .getName() + " for the column: " + noDictDimensionPages[noDictIndex]
-                      .getColumnSpec().getFieldName());
+                      .getColumnSpec().getFieldName() + " having encoding type: "
+                      + columnPageEncoder.getEncodingType());
             }
           }
           noDictIndex++;


[03/45] carbondata git commit: [CARBONDATA-2965] support Benchmark command in CarbonCli

Posted by ra...@apache.org.
[CARBONDATA-2965] support Benchmark command in CarbonCli

A new command called "benchmark" is added in CarbonCli tool to output the scan performance of the specified file and column.
Example usage:
```bash
shell>java -jar carbondata-cli.jar org.apache.carbondata.CarbonCli -cmd benchmark -p hdfs://carbon1:9000/carbon.store/tpchcarbon_base/lineitem/ -a -c l_comment
```
will scan output the scan time of l_comment column in first file in the input folder and prints: (or using -f option to provide the data file instead of folder)

```
ReadHeaderAndFooter takes 12,598 us
ConvertFooter takes 4,712 us
ReadAllMetaAndConvertFooter takes 8,039 us

Scan column 'l_comment'
Blocklet#0: ColumnChunkIO takes 222,609 us
Blocklet#0: DecompressPage takes 111,985 us
Blocklet#1: ColumnChunkIO takes 186,522 us
Blocklet#1: DecompressPage takes 89,132 us
Blocklet#2: ColumnChunkIO takes 209,129 us
Blocklet#2: DecompressPage takes 84,051 us
```
This closes #2755


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

Branch: refs/heads/branch-1.5
Commit: e07df44a1db52304c54ab4e379f28b0f026449fd
Parents: 49f6715
Author: Jacky Li <ja...@qq.com>
Authored: Sun Sep 23 00:01:04 2018 +0800
Committer: xuchuanyin <xu...@hust.edu.cn>
Committed: Wed Sep 26 15:47:37 2018 +0800

----------------------------------------------------------------------
 .../core/util/DataFileFooterConverterV3.java    |   6 +-
 pom.xml                                         |   7 +-
 tools/cli/pom.xml                               |   5 +
 .../org/apache/carbondata/tool/CarbonCli.java   |  90 ++++----
 .../org/apache/carbondata/tool/Command.java     |  28 +++
 .../org/apache/carbondata/tool/DataFile.java    |  94 +++++++--
 .../org/apache/carbondata/tool/DataSummary.java | 188 ++++++-----------
 .../apache/carbondata/tool/FileCollector.java   | 147 +++++++++++++
 .../apache/carbondata/tool/ScanBenchmark.java   | 205 +++++++++++++++++++
 .../apache/carbondata/tool/CarbonCliTest.java   |  94 +++++----
 10 files changed, 622 insertions(+), 242 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/e07df44a/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverterV3.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverterV3.java b/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverterV3.java
index 41e22fd..438e3e3 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverterV3.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverterV3.java
@@ -59,12 +59,16 @@ public class DataFileFooterConverterV3 extends AbstractDataFileFooterConverter {
    */
   @Override public DataFileFooter readDataFileFooter(TableBlockInfo tableBlockInfo)
       throws IOException {
-    DataFileFooter dataFileFooter = new DataFileFooter();
     CarbonHeaderReader carbonHeaderReader = new CarbonHeaderReader(tableBlockInfo.getFilePath());
     FileHeader fileHeader = carbonHeaderReader.readHeader();
     CarbonFooterReaderV3 reader =
         new CarbonFooterReaderV3(tableBlockInfo.getFilePath(), tableBlockInfo.getBlockOffset());
     FileFooter3 footer = reader.readFooterVersion3();
+    return convertDataFileFooter(fileHeader, footer);
+  }
+
+  public DataFileFooter convertDataFileFooter(FileHeader fileHeader, FileFooter3 footer) {
+    DataFileFooter dataFileFooter = new DataFileFooter();
     dataFileFooter.setVersionId(ColumnarFormatVersion.valueOf((short) fileHeader.getVersion()));
     dataFileFooter.setNumberOfRows(footer.getNum_rows());
     dataFileFooter.setSegmentInfo(getSegmentInfo(footer.getSegment_info()));

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e07df44a/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index eff438b..00a5287 100644
--- a/pom.xml
+++ b/pom.xml
@@ -106,6 +106,7 @@
     <module>store/sdk</module>
     <module>store/search</module>
     <module>assembly</module>
+    <module>tools/cli</module>
   </modules>
 
   <properties>
@@ -718,12 +719,6 @@
         <module>datamap/mv/core</module>
       </modules>
     </profile>
-    <profile>
-      <id>tools</id>
-      <modules>
-        <module>tools/cli</module>
-      </modules>
-    </profile>
   </profiles>
 
 </project>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e07df44a/tools/cli/pom.xml
----------------------------------------------------------------------
diff --git a/tools/cli/pom.xml b/tools/cli/pom.xml
index 0d00438..60e69dc 100644
--- a/tools/cli/pom.xml
+++ b/tools/cli/pom.xml
@@ -25,6 +25,11 @@
       <version>${project.version}</version>
     </dependency>
     <dependency>
+      <groupId>javax.servlet</groupId>
+      <artifactId>servlet-api</artifactId>
+      <version>2.5</version>
+    </dependency>
+    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e07df44a/tools/cli/src/main/java/org/apache/carbondata/tool/CarbonCli.java
----------------------------------------------------------------------
diff --git a/tools/cli/src/main/java/org/apache/carbondata/tool/CarbonCli.java b/tools/cli/src/main/java/org/apache/carbondata/tool/CarbonCli.java
index effb139..5725f8e 100644
--- a/tools/cli/src/main/java/org/apache/carbondata/tool/CarbonCli.java
+++ b/tools/cli/src/main/java/org/apache/carbondata/tool/CarbonCli.java
@@ -47,11 +47,16 @@ public class CarbonCli {
         .withDescription("the path which contains carbondata files, nested folder is supported")
         .withLongOpt("path")
         .create("p");
+    Option file = OptionBuilder.withArgName("file")
+        .hasArg()
+        .withDescription("the carbondata file path")
+        .withLongOpt("file")
+        .create("f");
 
     Option command = OptionBuilder
         .withArgName("command name")
         .hasArg()
-        .withDescription("command to execute, supported commands are: summary")
+        .withDescription("command to execute, supported commands are: summary, benchmark")
         .isRequired(true)
         .create("cmd");
 
@@ -70,6 +75,7 @@ public class CarbonCli {
     Options options = new Options();
     options.addOption(help);
     options.addOption(path);
+    options.addOption(file);
     options.addOption(command);
     options.addOption(all);
     options.addOption(schema);
@@ -87,71 +93,49 @@ public class CarbonCli {
   static void run(String[] args, PrintStream out) {
     Options options = buildOptions();
     CommandLineParser parser = new PosixParser();
-    try {
-      CommandLine line = parser.parse(options, args);
-      if (line.hasOption("h")) {
-        printHelp(options);
-        return;
-      }
-
-      String cmd = line.getOptionValue("cmd");
-      if (cmd.equalsIgnoreCase("summary")) {
-        runSummaryCommand(line, options, out);
-      } else {
-        out.println("command " + cmd + " is not supported");
-        printHelp(options);
-        return;
-      }
 
-      out.flush();
+    CommandLine line;
+    try {
+      line = parser.parse(options, args);
     } catch (ParseException exp) {
       out.println("Parsing failed. Reason: " + exp.getMessage());
-    } catch (IOException | MemoryException e) {
-      out.println(out);
+      return;
     }
-  }
 
-  private static void printHelp(Options options) {
-    HelpFormatter formatter = new HelpFormatter();
-    formatter.printHelp("CarbonCli", options);
-  }
+    if (line.hasOption("h")) {
+      printHelp(options);
+      return;
+    }
 
-  private static void runSummaryCommand(CommandLine line, Options options, PrintStream out)
-      throws IOException, MemoryException {
     String path = "";
     if (line.hasOption("p")) {
       path = line.getOptionValue("path");
+    }
+    out.println("Input Folder: " + path);
+
+    String cmd = line.getOptionValue("cmd");
+    Command command;
+    if (cmd.equalsIgnoreCase("summary")) {
+      command = new DataSummary(path, out);
+    } else if (cmd.equalsIgnoreCase("benchmark")) {
+      command = new ScanBenchmark(path, out);
     } else {
-      System.err.println("path is required");
+      out.println("command " + cmd + " is not supported");
       printHelp(options);
       return;
     }
-    DataSummary summary = new DataSummary(path, out);
-    if (summary.isEmpty()) {
-      System.out.println("no data file found");
-      return;
-    }
-    out.println("Input Folder: " + path);
-    summary.printBasic();
-    boolean printAll = false;
-    if (line.hasOption("a")) {
-      printAll = true;
-    }
-    if (line.hasOption("s") || printAll) {
-      summary.printSchema();
-    }
-    if (line.hasOption("m") || printAll) {
-      summary.printSegments();
-    }
-    if (line.hasOption("t") || printAll) {
-      summary.printTableProperties();
-    }
-    if (line.hasOption("b") || printAll) {
-      summary.printBlockletDetail();
-    }
-    if (line.hasOption("c")) {
-      String columName = line.getOptionValue("c");
-      summary.printColumnStats(columName);
+
+    try {
+      command.run(line);
+      out.flush();
+    } catch (IOException | MemoryException e) {
+      e.printStackTrace();
     }
   }
+
+  private static void printHelp(Options options) {
+    HelpFormatter formatter = new HelpFormatter();
+    formatter.printHelp("CarbonCli", options);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e07df44a/tools/cli/src/main/java/org/apache/carbondata/tool/Command.java
----------------------------------------------------------------------
diff --git a/tools/cli/src/main/java/org/apache/carbondata/tool/Command.java b/tools/cli/src/main/java/org/apache/carbondata/tool/Command.java
new file mode 100644
index 0000000..cb7d8df
--- /dev/null
+++ b/tools/cli/src/main/java/org/apache/carbondata/tool/Command.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.tool;
+
+import java.io.IOException;
+
+import org.apache.carbondata.core.memory.MemoryException;
+
+import org.apache.commons.cli.CommandLine;
+
+interface Command {
+  void run(CommandLine line) throws IOException, MemoryException;
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e07df44a/tools/cli/src/main/java/org/apache/carbondata/tool/DataFile.java
----------------------------------------------------------------------
diff --git a/tools/cli/src/main/java/org/apache/carbondata/tool/DataFile.java b/tools/cli/src/main/java/org/apache/carbondata/tool/DataFile.java
index ea67829..da81d84 100644
--- a/tools/cli/src/main/java/org/apache/carbondata/tool/DataFile.java
+++ b/tools/cli/src/main/java/org/apache/carbondata/tool/DataFile.java
@@ -55,6 +55,8 @@ import static org.apache.carbondata.core.constants.CarbonCommonConstants.FILE_SE
  * Contains information extracted from a .carbondata file
  */
 class DataFile {
+  private CarbonFile dataFile;
+
   // file full path
   private String filePath;
 
@@ -80,33 +82,37 @@ class DataFile {
 
   private FileHeader header;
   private FileFooter3 footer;
+  private long footerOffset;
   private List<ColumnSchema> schema;
   private List<Blocklet> blocklets;
 
-  DataFile(CarbonFile file) throws IOException {
-    this.fileSizeInBytes = file.getSize();
+  DataFile(CarbonFile dataFile) {
+    this.dataFile = dataFile;
+    this.filePath = dataFile.getPath();
+    this.fileSizeInBytes = dataFile.getSize();
+  }
 
+  void collectAllMeta() throws IOException {
     FileHeader header = null;
     FileFooter3 footer = null;
     try {
-      header = readHeader(file);
+      header = readHeader();
     } catch (IOException e) {
-      throw new IOException("failed to read header in " + file.getPath(), e);
+      throw new IOException("failed to read header in " + dataFile.getPath(), e);
     }
     if (header.isSetSync_marker()) {
       // if sync_marker is set, it is a streaming format file
       throw new UnsupportedOperationException("streaming file is not supported");
     }
     try {
-      footer = readFooter(file);
+      footer = readFooter();
     } catch (IOException e) {
-      throw new IOException("failed to read footer in " + file.getPath(), e);
+      throw new IOException("failed to read footer in " + dataFile.getPath(), e);
     }
 
-    this.filePath = file.getPath();
     this.header = header;
     this.footer = footer;
-    String filePath = file.getPath();
+    String filePath = dataFile.getPath();
     // folder path that contains this file
     String fileName = filePath.substring(filePath.lastIndexOf(FILE_SEPARATOR));
     this.shardName = CarbonTablePath.getShardName(fileName);
@@ -130,6 +136,7 @@ class DataFile {
       for (int i = 0; i < schema.size(); i++) {
         columnDataSize.add(blockletInfo3.column_data_chunks_offsets.get(i) - previousChunkOffset);
         columnMetaSize.add(blockletInfo3.column_data_chunks_length.get(i).longValue());
+        previousChunkOffset = blockletInfo3.column_data_chunks_offsets.get(i);
       }
       // last column chunk data size
       columnDataSize.add(fileSizeInBytes - footerSizeInBytes - previousChunkOffset);
@@ -146,17 +153,17 @@ class DataFile {
     assert (blockletSizeInBytes.size() == getNumBlocklets());
   }
 
-  private FileHeader readHeader(CarbonFile dataFile) throws IOException {
+  FileHeader readHeader() throws IOException {
     CarbonHeaderReader reader = new CarbonHeaderReader(dataFile.getPath());
     this.schema = reader.readSchema();
     return reader.readHeader();
   }
 
-  private FileFooter3 readFooter(CarbonFile dataFile) throws IOException {
+  FileFooter3 readFooter() throws IOException {
     this.fileReader = FileFactory.getFileHolder(FileFactory.getFileType(dataFile.getPath()));
     ByteBuffer buffer = fileReader.readByteBuffer(FileFactory.getUpdatedFilePath(
         dataFile.getPath()), dataFile.getSize() - 8, 8);
-    long footerOffset = buffer.getLong();
+    this.footerOffset = buffer.getLong();
     this.footerSizeInBytes = this.fileSizeInBytes - footerOffset;
     CarbonFooterReaderV3 footerReader =
         new CarbonFooterReaderV3(dataFile.getAbsolutePath(), footerOffset);
@@ -187,6 +194,53 @@ class DataFile {
     return schema;
   }
 
+  FileReader getFileReader() {
+    return fileReader;
+  }
+
+  long getFooterOffset() {
+    return footerOffset;
+  }
+
+  int getNumBlocklet() {
+    return blockletSizeInBytes.size();
+  }
+
+  long getFileSizeInBytes() {
+    return fileSizeInBytes;
+  }
+
+  int getColumnIndex(String columnName) {
+    List<ColumnSchema> columns = getSchema();
+    for (int i = 0; i < columns.size(); i++) {
+      if (columns.get(i).getColumnName().equalsIgnoreCase(columnName)) {
+        return i;
+      }
+    }
+    throw new IllegalArgumentException(columnName + " not found");
+  }
+
+  ColumnSchema getColumn(String columnName) {
+    List<ColumnSchema> columns = getSchema();
+    for (int i = 0; i < columns.size(); i++) {
+      if (columns.get(i).getColumnName().equalsIgnoreCase(columnName)) {
+        return columns.get(i);
+      }
+    }
+    throw new IllegalArgumentException(columnName + " not found");
+  }
+
+  int numDimensions() {
+    int numDimensions = 0;
+    List<ColumnSchema> columns = getSchema();
+    for (ColumnSchema column : columns) {
+      if (column.isDimensionColumn()) {
+        numDimensions++;
+      }
+    }
+    return numDimensions;
+  }
+
   private int getNumBlocklets() {
     return footer.blocklet_info_list3.size();
   }
@@ -396,28 +450,34 @@ class DataFile {
         if (column.isSortColumn()) {
           minValue = ByteUtil.toXorInt(min, 0, min.length);
           dataValue = ByteUtil.toXorInt(data, 0, data.length) - minValue;
-          factorValue = ByteUtil.toXorInt(max, 0, max.length) - ByteUtil.toXorInt(min, 0, min.length);
+          factorValue =
+              ByteUtil.toXorInt(max, 0, max.length) - ByteUtil.toXorInt(min, 0, min.length);
         } else {
           minValue = ByteUtil.toLong(min, 0, min.length);
           dataValue = ByteUtil.toLong(data, 0, data.length) - minValue;
-          factorValue = ByteUtil.toLong(max, 0, max.length) - ByteUtil.toLong(min, 0, min.length);
+          factorValue =
+              ByteUtil.toLong(max, 0, max.length) - ByteUtil.toLong(min, 0, min.length);
         }
       } else if (column.getDataType() == DataTypes.LONG) {
         minValue = ByteUtil.toLong(min, 0, min.length);
         dataValue = ByteUtil.toLong(data, 0, data.length) - minValue;
-        factorValue = ByteUtil.toLong(max, 0, max.length) - ByteUtil.toLong(min, 0, min.length);
+        factorValue =
+            ByteUtil.toLong(max, 0, max.length) - ByteUtil.toLong(min, 0, min.length);
       } else if (column.getDataType() == DataTypes.DATE) {
         minValue = ByteUtil.toInt(min, 0, min.length);
         dataValue = ByteUtil.toInt(data, 0, data.length) - minValue;
-        factorValue = ByteUtil.toInt(max, 0, max.length) - ByteUtil.toInt(min, 0, min.length);
+        factorValue =
+            ByteUtil.toInt(max, 0, max.length) - ByteUtil.toInt(min, 0, min.length);
       } else if (column.getDataType() == DataTypes.TIMESTAMP) {
         minValue = ByteUtil.toLong(min, 0, min.length);
         dataValue = ByteUtil.toLong(data, 0, data.length) - minValue;
-        factorValue = ByteUtil.toLong(max, 0, max.length) - ByteUtil.toLong(min, 0, min.length);
+        factorValue =
+            ByteUtil.toLong(max, 0, max.length) - ByteUtil.toLong(min, 0, min.length);
       } else if (column.getDataType() == DataTypes.DOUBLE) {
         minValue = ByteUtil.toDouble(min, 0, min.length);
         dataValue = ByteUtil.toDouble(data, 0, data.length) - minValue;
-        factorValue = ByteUtil.toDouble(max, 0, max.length) - ByteUtil.toDouble(min, 0, min.length);
+        factorValue =
+            ByteUtil.toDouble(max, 0, max.length) - ByteUtil.toDouble(min, 0, min.length);
       } else {
         throw new UnsupportedOperationException("data type: " + column.getDataType());
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e07df44a/tools/cli/src/main/java/org/apache/carbondata/tool/DataSummary.java
----------------------------------------------------------------------
diff --git a/tools/cli/src/main/java/org/apache/carbondata/tool/DataSummary.java b/tools/cli/src/main/java/org/apache/carbondata/tool/DataSummary.java
index 7ca6951..6463977 100644
--- a/tools/cli/src/main/java/org/apache/carbondata/tool/DataSummary.java
+++ b/tools/cli/src/main/java/org/apache/carbondata/tool/DataSummary.java
@@ -20,15 +20,12 @@ package org.apache.carbondata.tool;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.nio.charset.Charset;
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 import org.apache.carbondata.common.Strings;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
@@ -40,7 +37,6 @@ import org.apache.carbondata.core.reader.CarbonHeaderReader;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.format.BlockletInfo3;
 import org.apache.carbondata.format.FileFooter3;
 import org.apache.carbondata.format.FileHeader;
@@ -48,135 +44,89 @@ import org.apache.carbondata.format.TableInfo;
 
 import static org.apache.carbondata.core.constants.CarbonCommonConstants.DEFAULT_CHARSET;
 
+import org.apache.commons.cli.CommandLine;
+
 /**
  * Data Summary command implementation for {@link CarbonCli}
  */
-class DataSummary {
+class DataSummary implements Command {
   private String dataFolder;
   private PrintStream out;
 
-  private long numBlock;
-  private long numShard;
-  private long numBlocklet;
-  private long numPage;
-  private long numRow;
-  private long totalDataSize;
-
   // file path mapping to file object
-  private LinkedHashMap<String, DataFile> dataFiles = new LinkedHashMap<>();
-  private CarbonFile tableStatusFile;
-  private CarbonFile schemaFile;
+  private LinkedHashMap<String, DataFile> dataFiles;
 
-  DataSummary(String dataFolder, PrintStream out) throws IOException {
+  DataSummary(String dataFolder, PrintStream out) {
     this.dataFolder = dataFolder;
     this.out = out;
-    collectDataFiles();
-  }
-
-  private boolean isColumnarFile(String fileName) {
-    // if the timestamp in file name is "0", it is a streaming file
-    return fileName.endsWith(CarbonTablePath.CARBON_DATA_EXT) &&
-        !CarbonTablePath.DataFileUtil.getTimeStampFromFileName(fileName).equals("0");
   }
 
-  private boolean isStreamFile(String fileName) {
-    // if the timestamp in file name is "0", it is a streaming file
-    return fileName.endsWith(CarbonTablePath.CARBON_DATA_EXT) &&
-        CarbonTablePath.DataFileUtil.getTimeStampFromFileName(fileName).equals("0");
-  }
-
-  private void collectDataFiles() throws IOException {
-    Set<String> shards = new HashSet<>();
-    CarbonFile folder = FileFactory.getCarbonFile(dataFolder);
-    List<CarbonFile> files = folder.listFiles(true);
-    List<DataFile> unsortedFiles = new ArrayList<>();
-    for (CarbonFile file : files) {
-      if (isColumnarFile(file.getName())) {
-        DataFile dataFile = new DataFile(file);
-        unsortedFiles.add(dataFile);
-        collectNum(dataFile.getFooter());
-        shards.add(dataFile.getShardName());
-        totalDataSize += file.getSize();
-      } else if (file.getName().endsWith(CarbonTablePath.TABLE_STATUS_FILE)) {
-        tableStatusFile = file;
-      } else if (file.getName().startsWith(CarbonTablePath.SCHEMA_FILE)) {
-        schemaFile = file;
-      } else if (isStreamFile(file.getName())) {
-        out.println("WARN: input path contains streaming file, this tool does not support it yet, "
-            + "skipping it...");
-      }
+  @Override
+  public void run(CommandLine line) throws IOException, MemoryException {
+    FileCollector collector = new FileCollector(out);
+    collector.collectFiles(dataFolder);
+    collector.printBasicStats();
+    if (collector.getNumDataFiles() == 0) {
+      return;
     }
-    unsortedFiles.sort((o1, o2) -> {
-      if (o1.getShardName().equalsIgnoreCase(o2.getShardName())) {
-        return Integer.parseInt(o1.getPartNo()) - Integer.parseInt(o2.getPartNo());
-      } else {
-        return o1.getShardName().compareTo(o2.getShardName());
+    dataFiles = collector.getDataFiles();
+    boolean printAll = false;
+    if (line.hasOption("a")) {
+      printAll = true;
+    }
+    if (line.hasOption("s") || printAll) {
+      if (dataFiles.size() > 0) {
+        printSchema(dataFiles.entrySet().iterator().next().getValue());
       }
-    });
-    for (DataFile collectedFile : unsortedFiles) {
-      this.dataFiles.put(collectedFile.getFilePath(), collectedFile);
     }
-    numShard = shards.size();
-  }
-
-  private void collectNum(FileFooter3 footer) {
-    numBlock++;
-    numBlocklet += footer.blocklet_index_list.size();
-    numRow += footer.num_rows;
-    for (BlockletInfo3 blockletInfo3 : footer.blocklet_info_list3) {
-      numPage += blockletInfo3.number_number_of_pages;
+    if (line.hasOption("m") || printAll) {
+      printSegments(collector.getTableStatusFile());
+    }
+    if (line.hasOption("t") || printAll) {
+      printTableProperties(collector.getSchemaFile());
+    }
+    if (line.hasOption("b") || printAll) {
+      printBlockletDetail();
+    }
+    if (line.hasOption("c")) {
+      String columName = line.getOptionValue("c");
+      printColumnStats(columName);
     }
   }
 
-  void printBasic() {
-    out.println("## Summary");
-    out.println(
-        String.format("total: %,d blocks, %,d shards, %,d blocklets, %,d pages, %,d rows, %s",
-            numBlock, numShard, numBlocklet, numPage, numRow, Strings.formatSize(totalDataSize)));
-    out.println(
-        String.format("avg: %s/block, %s/blocklet, %,d rows/block, %,d rows/blocklet",
-            Strings.formatSize(totalDataSize / numBlock),
-            Strings.formatSize(totalDataSize / numBlocklet),
-            numRow / numBlock,
-            numRow / numBlocklet));
-  }
-
-  void printSchema() throws IOException {
-    if (dataFiles.size() > 0) {
-      String firstFile = dataFiles.keySet().iterator().next();
-      CarbonFile file = FileFactory.getCarbonFile(firstFile);
-      out.println();
-      out.println("## Schema");
-      out.println(String.format("schema in %s", file.getName()));
-      CarbonHeaderReader reader = new CarbonHeaderReader(file.getPath());
-      FileHeader header = reader.readHeader();
-      out.println("version: V" + header.version);
-      out.println("timestamp: " + new java.sql.Timestamp(header.time_stamp));
-      List<ColumnSchema> columns = reader.readSchema();
-      TablePrinter printer = new TablePrinter(
-          new String[]{"Column Name", "Data Type", "Column Type",
-              "SortColumn", "Encoding", "Ordinal", "Id"});
-      for (ColumnSchema column : columns) {
-        String shortColumnId = "NA";
-        if (column.getColumnUniqueId() != null && column.getColumnUniqueId().length() > 4) {
-          shortColumnId = "*" +
-              column.getColumnUniqueId().substring(column.getColumnUniqueId().length() - 4);
-        }
-        printer.addRow(new String[]{
-            column.getColumnName(),
-            column.getDataType().getName(),
-            column.isDimensionColumn() ? "dimension" : "measure",
-            String.valueOf(column.isSortColumn()),
-            column.getEncodingList().toString(),
-            Integer.toString(column.getSchemaOrdinal()),
-            shortColumnId
-        });
+  private void printSchema(DataFile dataFile) throws IOException {
+    CarbonFile file = FileFactory.getCarbonFile(dataFile.getFilePath());
+    out.println();
+    out.println("## Schema");
+    out.println(String.format("schema in %s", file.getName()));
+    CarbonHeaderReader reader = new CarbonHeaderReader(file.getPath());
+    FileHeader header = reader.readHeader();
+    out.println("version: V" + header.version);
+    out.println("timestamp: " + new java.sql.Timestamp(header.time_stamp));
+    List<ColumnSchema> columns = reader.readSchema();
+    TablePrinter printer = new TablePrinter(
+        new String[]{"Column Name", "Data Type", "Column Type",
+            "SortColumn", "Encoding", "Ordinal", "Id"});
+    for (ColumnSchema column : columns) {
+      String shortColumnId = "NA";
+      if (column.getColumnUniqueId() != null && column.getColumnUniqueId().length() > 4) {
+        shortColumnId = "*" +
+            column.getColumnUniqueId().substring(column.getColumnUniqueId().length() - 4);
       }
-      printer.printFormatted(out);
+      printer.addRow(new String[]{
+          column.getColumnName(),
+          column.getDataType().getName(),
+          column.isDimensionColumn() ? "dimension" : "measure",
+          String.valueOf(column.isSortColumn()),
+          column.getEncodingList().toString(),
+          Integer.toString(column.getSchemaOrdinal()),
+          shortColumnId
+      });
     }
+    printer.printFormatted(out);
   }
 
-  void printSegments() throws IOException {
+  private void printSegments(CarbonFile tableStatusFile) throws IOException {
     out.println();
     out.println("## Segment");
     if (tableStatusFile != null) {
@@ -215,7 +165,7 @@ class DataSummary {
     }
   }
 
-  void printTableProperties() throws IOException {
+  private void printTableProperties(CarbonFile schemaFile) throws IOException {
     out.println();
     out.println("## Table Properties");
     if (schemaFile != null) {
@@ -235,7 +185,7 @@ class DataSummary {
     }
   }
 
-  void printBlockletDetail() {
+  private void printBlockletDetail() {
     out.println();
     out.println("## Block Detail");
 
@@ -262,17 +212,12 @@ class DataSummary {
 
   private int getColumnIndex(String columnName) {
     if (dataFiles.size() > 0) {
-      List<ColumnSchema> columns = dataFiles.entrySet().iterator().next().getValue().getSchema();
-      for (int i = 0; i < columns.size(); i++) {
-        if (columns.get(i).getColumnName().equalsIgnoreCase(columnName)) {
-          return i;
-        }
-      }
+      return dataFiles.entrySet().iterator().next().getValue().getColumnIndex(columnName);
     }
     throw new RuntimeException("schema for column " + columnName + " not found");
   }
 
-  void printColumnStats(String columnName) throws IOException, MemoryException {
+  private void printColumnStats(String columnName) throws IOException, MemoryException {
     out.println();
     out.println("## Column Statistics for '" + columnName + "'");
     for (DataFile dataFile : dataFiles.values()) {
@@ -354,7 +299,4 @@ class DataSummary {
     }
   }
 
-  public boolean isEmpty() {
-    return dataFiles.size() == 0;
-  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e07df44a/tools/cli/src/main/java/org/apache/carbondata/tool/FileCollector.java
----------------------------------------------------------------------
diff --git a/tools/cli/src/main/java/org/apache/carbondata/tool/FileCollector.java b/tools/cli/src/main/java/org/apache/carbondata/tool/FileCollector.java
new file mode 100644
index 0000000..6e3297f
--- /dev/null
+++ b/tools/cli/src/main/java/org/apache/carbondata/tool/FileCollector.java
@@ -0,0 +1,147 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.tool;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.carbondata.common.Strings;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+import org.apache.carbondata.format.BlockletInfo3;
+import org.apache.carbondata.format.FileFooter3;
+
+/**
+ * A helper to collect all data files, schema file, table status file in a given folder
+ */
+class FileCollector {
+
+  private long numBlock;
+  private long numShard;
+  private long numBlocklet;
+  private long numPage;
+  private long numRow;
+  private long totalDataSize;
+
+  // file path mapping to file object
+  private LinkedHashMap<String, DataFile> dataFiles = new LinkedHashMap<>();
+  private CarbonFile tableStatusFile;
+  private CarbonFile schemaFile;
+
+  private PrintStream out;
+
+  FileCollector(PrintStream out) {
+    this.out = out;
+  }
+
+  void collectFiles(String dataFolder) throws IOException {
+    Set<String> shards = new HashSet<>();
+    CarbonFile folder = FileFactory.getCarbonFile(dataFolder);
+    List<CarbonFile> files = folder.listFiles(true);
+    List<DataFile> unsortedFiles = new ArrayList<>();
+    for (CarbonFile file : files) {
+      if (isColumnarFile(file.getName())) {
+        DataFile dataFile = new DataFile(file);
+        dataFile.collectAllMeta();
+        unsortedFiles.add(dataFile);
+        collectNum(dataFile.getFooter());
+        shards.add(dataFile.getShardName());
+        totalDataSize += file.getSize();
+      } else if (file.getName().endsWith(CarbonTablePath.TABLE_STATUS_FILE)) {
+        tableStatusFile = file;
+      } else if (file.getName().startsWith(CarbonTablePath.SCHEMA_FILE)) {
+        schemaFile = file;
+      } else if (isStreamFile(file.getName())) {
+        out.println("WARN: input path contains streaming file, this tool does not support it yet, "
+            + "skipping it...");
+      }
+    }
+    unsortedFiles.sort((o1, o2) -> {
+      if (o1.getShardName().equalsIgnoreCase(o2.getShardName())) {
+        return Integer.parseInt(o1.getPartNo()) - Integer.parseInt(o2.getPartNo());
+      } else {
+        return o1.getShardName().compareTo(o2.getShardName());
+      }
+    });
+    for (DataFile collectedFile : unsortedFiles) {
+      this.dataFiles.put(collectedFile.getFilePath(), collectedFile);
+    }
+    numShard = shards.size();
+  }
+
+  private void collectNum(FileFooter3 footer) {
+    numBlock++;
+    numBlocklet += footer.blocklet_index_list.size();
+    numRow += footer.num_rows;
+    for (BlockletInfo3 blockletInfo3 : footer.blocklet_info_list3) {
+      numPage += blockletInfo3.number_number_of_pages;
+    }
+  }
+
+  private boolean isColumnarFile(String fileName) {
+    // if the timestamp in file name is "0", it is a streaming file
+    return fileName.endsWith(CarbonTablePath.CARBON_DATA_EXT) &&
+        !CarbonTablePath.DataFileUtil.getTimeStampFromFileName(fileName).equals("0");
+  }
+
+  private boolean isStreamFile(String fileName) {
+    // if the timestamp in file name is "0", it is a streaming file
+    return fileName.endsWith(CarbonTablePath.CARBON_DATA_EXT) &&
+        CarbonTablePath.DataFileUtil.getTimeStampFromFileName(fileName).equals("0");
+  }
+
+  // return file path mapping to file object
+  LinkedHashMap<String, DataFile> getDataFiles() {
+    return dataFiles;
+  }
+
+  CarbonFile getTableStatusFile() {
+    return tableStatusFile;
+  }
+
+  CarbonFile getSchemaFile() {
+    return schemaFile;
+  }
+
+  int getNumDataFiles() {
+    return dataFiles.size();
+  }
+
+  void printBasicStats() {
+    if (dataFiles.size() == 0) {
+      System.out.println("no data file found");
+      return;
+    }
+    out.println("## Summary");
+    out.println(
+        String.format("total: %,d blocks, %,d shards, %,d blocklets, %,d pages, %,d rows, %s",
+            numBlock, numShard, numBlocklet, numPage, numRow, Strings.formatSize(totalDataSize)));
+    out.println(
+        String.format("avg: %s/block, %s/blocklet, %,d rows/block, %,d rows/blocklet",
+            Strings.formatSize((float) totalDataSize / numBlock),
+            Strings.formatSize((float) totalDataSize / numBlocklet),
+            numRow / numBlock,
+            numRow / numBlocklet));
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e07df44a/tools/cli/src/main/java/org/apache/carbondata/tool/ScanBenchmark.java
----------------------------------------------------------------------
diff --git a/tools/cli/src/main/java/org/apache/carbondata/tool/ScanBenchmark.java b/tools/cli/src/main/java/org/apache/carbondata/tool/ScanBenchmark.java
new file mode 100644
index 0000000..805d4e8
--- /dev/null
+++ b/tools/cli/src/main/java/org/apache/carbondata/tool/ScanBenchmark.java
@@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.tool;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.carbondata.common.Strings;
+import org.apache.carbondata.core.datastore.block.BlockletInfos;
+import org.apache.carbondata.core.datastore.block.TableBlockInfo;
+import org.apache.carbondata.core.datastore.chunk.AbstractRawColumnChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
+import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
+import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
+import org.apache.carbondata.core.datastore.chunk.reader.CarbonDataReaderFactory;
+import org.apache.carbondata.core.datastore.chunk.reader.DimensionColumnChunkReader;
+import org.apache.carbondata.core.datastore.chunk.reader.MeasureColumnChunkReader;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.datastore.page.ColumnPage;
+import org.apache.carbondata.core.memory.MemoryException;
+import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
+import org.apache.carbondata.core.metadata.blocklet.BlockletInfo;
+import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
+import org.apache.carbondata.core.util.DataFileFooterConverterV3;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+import org.apache.carbondata.format.FileFooter3;
+import org.apache.carbondata.format.FileHeader;
+
+import org.apache.commons.cli.CommandLine;
+
+class ScanBenchmark implements Command {
+
+  private String dataFolder;
+  private PrintStream out;
+  private DataFile file;
+
+  ScanBenchmark(String dataFolder, PrintStream out) {
+    this.dataFolder = dataFolder;
+    this.out = out;
+  }
+
+  @Override
+  public void run(CommandLine line) throws IOException, MemoryException {
+    if (line.hasOption("f")) {
+      String filePath = line.getOptionValue("f");
+      file = new DataFile(FileFactory.getCarbonFile(filePath));
+    } else {
+      FileCollector collector = new FileCollector(out);
+      collector.collectFiles(dataFolder);
+      if (collector.getNumDataFiles() == 0) {
+        return;
+      }
+      Map<String, DataFile> dataFiles = collector.getDataFiles();
+      file = dataFiles.entrySet().iterator().next().getValue();
+    }
+
+    out.println("\n## Benchmark");
+    AtomicReference<FileHeader> fileHeaderRef = new AtomicReference<>();
+    AtomicReference<FileFooter3> fileFoorterRef = new AtomicReference<>();
+    AtomicReference<DataFileFooter> convertedFooterRef = new AtomicReference<>();
+
+    // benchmark read header and footer time
+    benchmarkOperation("ReadHeaderAndFooter", () -> {
+      fileHeaderRef.set(file.readHeader());
+      fileFoorterRef.set(file.readFooter());
+    });
+    FileHeader fileHeader = fileHeaderRef.get();
+    FileFooter3 fileFooter = fileFoorterRef.get();
+
+    // benchmark convert footer
+    benchmarkOperation("ConvertFooter", () -> {
+      convertFooter(fileHeader, fileFooter);
+    });
+
+    // benchmark read all meta and convert footer
+    benchmarkOperation("ReadAllMetaAndConvertFooter", () -> {
+      DataFileFooter footer = readAndConvertFooter(file);
+      convertedFooterRef.set(footer);
+    });
+
+    if (line.hasOption("c")) {
+      String columnName = line.getOptionValue("c");
+      out.println("\nScan column '" + columnName + "'");
+
+      DataFileFooter footer = convertedFooterRef.get();
+      AtomicReference<AbstractRawColumnChunk> columnChunk = new AtomicReference<>();
+      int columnIndex = file.getColumnIndex(columnName);
+      boolean dimension = file.getColumn(columnName).isDimensionColumn();
+      for (int i = 0; i < footer.getBlockletList().size(); i++) {
+        int blockletId = i;
+        out.println(String.format("Blocklet#%d: total size %s, %,d pages, %,d rows",
+            blockletId,
+            Strings.formatSize(file.getColumnDataSizeInBytes(blockletId, columnIndex)),
+            footer.getBlockletList().get(blockletId).getNumberOfPages(),
+            footer.getBlockletList().get(blockletId).getNumberOfRows()));
+        benchmarkOperation("\tColumnChunk IO", () -> {
+          columnChunk.set(readBlockletColumnChunkIO(footer, blockletId, columnIndex, dimension));
+        });
+
+        if (dimensionColumnChunkReader != null) {
+          benchmarkOperation("\tDecompress Pages", () -> {
+            decompressDimensionPages(columnChunk.get(),
+                footer.getBlockletList().get(blockletId).getNumberOfPages());
+          });
+        } else {
+          benchmarkOperation("\tDecompress Pages", () -> {
+            decompressMeasurePages(columnChunk.get(),
+                footer.getBlockletList().get(blockletId).getNumberOfPages());
+          });
+        }
+      }
+    }
+
+  }
+
+  interface Operation {
+    void run() throws IOException, MemoryException;
+  }
+
+  private void benchmarkOperation(String opName, Operation op) throws IOException, MemoryException {
+    long start, end;
+    start = System.nanoTime();
+    op.run();
+    end = System.nanoTime();
+    out.println(String.format("%s takes %,d us", opName, (end - start) / 1000));
+  }
+
+  private DataFileFooter readAndConvertFooter(DataFile file) throws IOException {
+    int numBlocklets = file.getNumBlocklet();
+    BlockletInfos blockletInfos = new BlockletInfos(numBlocklets, 0, numBlocklets);
+    String segmentId = CarbonTablePath.DataFileUtil.getSegmentNo(file.getFilePath());
+    TableBlockInfo blockInfo =
+        new TableBlockInfo(file.getFilePath(), file.getFooterOffset(),
+            segmentId, new String[]{"localhost"}, file.getFileSizeInBytes(),
+            blockletInfos, ColumnarFormatVersion.V3, new String[0]);
+
+    DataFileFooterConverterV3 converter = new DataFileFooterConverterV3();
+    return converter.readDataFileFooter(blockInfo);
+  }
+
+  private DataFileFooter convertFooter(FileHeader fileHeader, FileFooter3 fileFooter) {
+    DataFileFooterConverterV3 converter = new DataFileFooterConverterV3();
+    return converter.convertDataFileFooter(fileHeader, fileFooter);
+  }
+
+  private DimensionColumnChunkReader dimensionColumnChunkReader;
+  private MeasureColumnChunkReader measureColumnChunkReader;
+
+  private AbstractRawColumnChunk readBlockletColumnChunkIO(
+      DataFileFooter footer, int blockletId, int columnIndex, boolean dimension)
+      throws IOException {
+    BlockletInfo blockletInfo = footer.getBlockletList().get(blockletId);
+    if (dimension) {
+      dimensionColumnChunkReader = CarbonDataReaderFactory.getInstance()
+          .getDimensionColumnChunkReader(ColumnarFormatVersion.V3, blockletInfo,
+              footer.getSegmentInfo().getColumnCardinality(), file.getFilePath(), false);
+      return dimensionColumnChunkReader.readRawDimensionChunk(file.getFileReader(), columnIndex);
+    } else {
+      columnIndex = columnIndex - file.numDimensions();
+      assert (columnIndex >= 0);
+      measureColumnChunkReader = CarbonDataReaderFactory.getInstance()
+          .getMeasureColumnChunkReader(ColumnarFormatVersion.V3, blockletInfo,
+              file.getFilePath(), false);
+      return measureColumnChunkReader.readRawMeasureChunk(file.getFileReader(), columnIndex);
+    }
+  }
+
+  private DimensionColumnPage[] decompressDimensionPages(
+      AbstractRawColumnChunk rawColumnChunk, int numPages) throws IOException, MemoryException {
+    DimensionColumnPage[] pages = new DimensionColumnPage[numPages];
+    for (int i = 0; i < pages.length; i++) {
+      pages[i] = dimensionColumnChunkReader.decodeColumnPage(
+          (DimensionRawColumnChunk) rawColumnChunk, i);
+    }
+    return pages;
+  }
+
+  private ColumnPage[] decompressMeasurePages(
+      AbstractRawColumnChunk rawColumnChunk, int numPages) throws IOException, MemoryException {
+    ColumnPage[] pages = new ColumnPage[numPages];
+    for (int i = 0; i < pages.length; i++) {
+      pages[i] = measureColumnChunkReader.decodeColumnPage(
+          (MeasureRawColumnChunk) rawColumnChunk, i);
+    }
+    return pages;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e07df44a/tools/cli/src/test/java/org/apache/carbondata/tool/CarbonCliTest.java
----------------------------------------------------------------------
diff --git a/tools/cli/src/test/java/org/apache/carbondata/tool/CarbonCliTest.java b/tools/cli/src/test/java/org/apache/carbondata/tool/CarbonCliTest.java
index 0d0d6b5..4dc34c4 100644
--- a/tools/cli/src/test/java/org/apache/carbondata/tool/CarbonCliTest.java
+++ b/tools/cli/src/test/java/org/apache/carbondata/tool/CarbonCliTest.java
@@ -45,9 +45,9 @@ public class CarbonCliTest {
     fields[0] = new Field("name", DataTypes.STRING);
     fields[1] = new Field("age", DataTypes.INT);
 
-    TestUtil.writeFilesAndVerify(5000000, new Schema(fields), path, new String[]{"age"},
+    TestUtil.writeFilesAndVerify(5000000, new Schema(fields), path, new String[]{"name"},
         true, 3, 8, true);
-    TestUtil.writeFilesAndVerify(5000000, new Schema(fields), path, new String[]{"age"},
+    TestUtil.writeFilesAndVerify(5000000, new Schema(fields), path, new String[]{"name"},
         true, 3, 8, true);
   }
 
@@ -69,7 +69,7 @@ public class CarbonCliTest {
   }
 
   @Test
-  public void testOutputIndividual() {
+  public void testSummaryOutputIndividual() {
     String[] args = {"-cmd", "summary", "-p", path};
     ByteArrayOutputStream out = new ByteArrayOutputStream();
     PrintStream stream = new PrintStream(out);
@@ -79,20 +79,19 @@ public class CarbonCliTest {
         output.contains(
             "Input Folder: ./CarbonCliTest\n"
           + "## Summary\n"
-          + "total: 6 blocks, 2 shards, 12 blocklets, 314 pages, 10,000,000 rows, 30.72MB\n"
-          + "avg: 5.12MB/block, 2.56MB/blocklet, 1,666,666 rows/block, 833,333 rows/blocklet"));
+          + "total: 6 blocks, 2 shards, 14 blocklets, 314 pages, 10,000,000 rows, 32.26MB\n"
+          + "avg: 5.38MB/block, 2.30MB/blocklet, 1,666,666 rows/block, 714,285 rows/blocklet"));
 
     String[] args2 = {"-cmd", "summary", "-p", path, "-s"};
     out = new ByteArrayOutputStream();
     stream = new PrintStream(out);
     CarbonCli.run(args2, stream);
     output = new String(out.toByteArray());
-
     Assert.assertTrue(
         output.contains(
             "Column Name  Data Type  Column Type  SortColumn  Encoding          Ordinal  Id  \n"
-          + "age          INT        dimension    true        [INVERTED_INDEX]  1        NA  \n"
-          + "name         STRING     dimension    false       [INVERTED_INDEX]  0        NA  \n"));
+          + "name         STRING     dimension    true        [INVERTED_INDEX]  0        NA  \n"
+          + "age          INT        measure      false       []                1        NA  "));
 
     String[] args3 = {"-cmd", "summary", "-p", path, "-t"};
     out = new ByteArrayOutputStream();
@@ -113,38 +112,37 @@ public class CarbonCliTest {
     stream = new PrintStream(out);
     CarbonCli.run(args4, stream);
     output = new String(out.toByteArray());
-
     Assert.assertTrue(
         output.contains(
-            "BLK  BLKLT  NumPages  NumRows  Size    \n"
-          + "0    0      29        928,000  2.60MB  \n"
-          + "0    1      29        928,000  2.60MB  \n"
-          + "1    0      29        928,000  2.60MB  \n"
-          + "1    1      29        928,000  2.60MB  \n"
-          + "2    0      22        704,000  2.54MB  \n"
-          + "2    1      19        584,000  2.43MB  "));
+            "BLK  BLKLT  NumPages  NumRows  Size      \n"
+          + "0    0      25        800,000  2.58MB    \n"
+          + "0    1      25        800,000  2.58MB    \n"
+          + "1    0      25        800,000  2.58MB    \n"
+          + "1    1      25        800,000  2.58MB    \n"
+          + "2    0      25        800,000  2.58MB    \n"
+          + "2    1      25        800,000  2.58MB    \n"
+          + "2    2      7         200,000  660.74KB  "));
 
     String[] args5 = {"-cmd", "summary", "-p", path, "-c", "name"};
     out = new ByteArrayOutputStream();
     stream = new PrintStream(out);
     CarbonCli.run(args5, stream);
     output = new String(out.toByteArray());
-
     Assert.assertTrue(
         output.contains(
             "BLK  BLKLT  Meta Size  Data Size  LocalDict  DictEntries  DictSize  AvgPageSize  Min%    Max%    \n"
-          + "0    0      1.82KB     5.19MB     false      0            0.0B      11.96KB      robot0  robot9  \n"
-          + "0    1      1.82KB     2.60MB     false      0            0.0B      11.96KB      robot0  robot9  \n"
-          + "1    0      1.82KB     5.19MB     false      0            0.0B      11.96KB      robot0  robot9  \n"
-          + "1    1      1.82KB     2.60MB     false      0            0.0B      11.96KB      robot0  robot9  \n"
-          + "2    0      1.38KB     4.97MB     false      0            0.0B      11.92KB      robot0  robot9  \n"
-          + "2    1      1.19KB     2.43MB     false      0            0.0B      11.42KB      robot0  robot9  \n"));
+          + "0    0      1.72KB     295.89KB   false      0            0.0B      11.77KB      robot0  robot1  \n"
+          + "0    1      1.72KB     295.89KB   false      0            0.0B      11.77KB      robot1  robot3  \n"
+          + "1    0      1.72KB     295.89KB   false      0            0.0B      11.77KB      robot3  robot4  \n"
+          + "1    1      1.72KB     295.89KB   false      0            0.0B      11.77KB      robot4  robot6  \n"
+          + "2    0      1.72KB     295.89KB   false      0            0.0B      11.77KB      robot6  robot7  \n"
+          + "2    1      1.72KB     295.89KB   false      0            0.0B      11.77KB      robot8  robot9  \n"
+          + "2    2      492.0B     74.03KB    false      0            0.0B      10.51KB      robot9  robot9  "));
   }
 
   @Test
-  public void testOutputAll() {
+  public void testSummaryOutputAll() {
     String[] args = {"-cmd", "summary", "-p", path, "-a", "-c", "age"};
-
     ByteArrayOutputStream out = new ByteArrayOutputStream();
     PrintStream stream = new PrintStream(out);
     CarbonCli.run(args, stream);
@@ -153,14 +151,14 @@ public class CarbonCliTest {
         output.contains(
             "Input Folder: ./CarbonCliTest\n"
           + "## Summary\n"
-          + "total: 6 blocks, 2 shards, 12 blocklets, 314 pages, 10,000,000 rows, 30.72MB\n"
-          + "avg: 5.12MB/block, 2.56MB/blocklet, 1,666,666 rows/block, 833,333 rows/blocklet"));
+          + "total: 6 blocks, 2 shards, 14 blocklets, 314 pages, 10,000,000 rows, 32.26MB\n"
+          + "avg: 5.38MB/block, 2.30MB/blocklet, 1,666,666 rows/block, 714,285 rows/blocklet\n"));
 
     Assert.assertTrue(
         output.contains(
             "Column Name  Data Type  Column Type  SortColumn  Encoding          Ordinal  Id  \n"
-          + "age          INT        dimension    true        [INVERTED_INDEX]  1        NA  \n"
-          + "name         STRING     dimension    false       [INVERTED_INDEX]  0        NA  \n"));
+          + "name         STRING     dimension    true        [INVERTED_INDEX]  0        NA  \n"
+          + "age          INT        measure      false       []                1        NA  \n"));
 
     Assert.assertTrue(
         output.contains(
@@ -172,23 +170,35 @@ public class CarbonCliTest {
 
     Assert.assertTrue(
         output.contains(
-            "BLK  BLKLT  NumPages  NumRows  Size    \n"
-          + "0    0      29        928,000  2.60MB  \n"
-          + "0    1      29        928,000  2.60MB  \n"
-          + "1    0      29        928,000  2.60MB  \n"
-          + "1    1      29        928,000  2.60MB  \n"
-          + "2    0      22        704,000  2.54MB  \n"
-          + "2    1      19        584,000  2.43MB  "));
+            "BLK  BLKLT  NumPages  NumRows  Size      \n"
+          + "0    0      25        800,000  2.58MB    \n"
+          + "0    1      25        800,000  2.58MB    \n"
+          + "1    0      25        800,000  2.58MB    \n"
+          + "1    1      25        800,000  2.58MB    \n"
+          + "2    0      25        800,000  2.58MB    \n"
+          + "2    1      25        800,000  2.58MB    \n"
+          + "2    2      7         200,000  660.74KB  "));
 
     Assert.assertTrue(
         output.contains(
           "BLK  BLKLT  Meta Size  Data Size  LocalDict  DictEntries  DictSize  AvgPageSize  Min%  Max%   \n"
-        + "0    0      1.81KB     2.26MB     false      0            0.0B      79.61KB      0.0   15.5   \n"
-        + "0    1      1.81KB     2.26MB     false      0            0.0B      79.60KB      15.5  30.9   \n"
-        + "1    0      1.81KB     2.26MB     false      0            0.0B      79.62KB      30.9  46.4   \n"
-        + "1    1      1.81KB     2.26MB     false      0            0.0B      79.60KB      46.4  61.9   \n"
-        + "2    0      1.37KB     2.28MB     false      0            0.0B      106.11KB     61.9  80.5   \n"
-        + "2    1      1.19KB     2.22MB     false      0            0.0B      119.55KB     80.5  100.0  "));
+        + "0    0      2.90KB     4.87MB     false      0            0.0B      93.76KB      0.0   100.0  \n"
+        + "0    1      2.90KB     2.29MB     false      0            0.0B      93.76KB      0.0   100.0  \n"
+        + "1    0      2.90KB     4.87MB     false      0            0.0B      93.76KB      0.0   100.0  \n"
+        + "1    1      2.90KB     2.29MB     false      0            0.0B      93.76KB      0.0   100.0  \n"
+        + "2    0      2.90KB     5.52MB     false      0            0.0B      93.76KB      0.0   100.0  \n"
+        + "2    1      2.90KB     2.94MB     false      0            0.0B      93.76KB      0.0   100.0  \n"
+        + "2    2      830.0B     586.81KB   false      0            0.0B      83.71KB      0.0   100.0 "));
+  }
+
+  @Test
+  public void testBenchmark() {
+    String[] args = {"-cmd", "benchmark", "-p", path, "-a", "-c", "name"};
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+    PrintStream stream = new PrintStream(out);
+    CarbonCli.run(args, stream);
+    String output = new String(out.toByteArray());
+    System.out.println(output);
   }
 
   @After


[18/45] carbondata git commit: [HOTFIX][Streaming] Avoid throwing NPE during deleting the streaming lock file

Posted by ra...@apache.org.
[HOTFIX][Streaming] Avoid throwing NPE during deleting the streaming lock file

We should check whether the lock file is exists or not before deleting it.
If the lock file is exists, need to delete it.
If the lock file is not exists, no need to do anything.

This closes #2775


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

Branch: refs/heads/branch-1.5
Commit: c016361639df899eeefa956e2dcb23fa962e6e7f
Parents: 8284d9e
Author: QiangCai <qi...@qq.com>
Authored: Thu Sep 27 20:32:40 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Fri Sep 28 19:39:03 2018 +0800

----------------------------------------------------------------------
 .../management/CarbonAlterTableCompactionCommand.scala       | 8 ++++++--
 .../spark/carbondata/TestStreamingTableOperation.scala       | 3 +++
 2 files changed, 9 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0163616/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
index 8b6dabd..b699ec1 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
@@ -334,8 +334,12 @@ case class CarbonAlterTableCompactionCommand(
     val streamingLock = CarbonLockFactory.getCarbonLockObj(
       carbonTable.getTableInfo.getOrCreateAbsoluteTableIdentifier,
       LockUsage.STREAMING_LOCK)
-    if (!FileFactory.getCarbonFile(streamingLock.getLockFilePath).delete()) {
-       LOGGER.warn("failed to delete lock file: " + streamingLock.getLockFilePath)
+    val lockFile =
+      FileFactory.getCarbonFile(streamingLock.getLockFilePath, FileFactory.getConfiguration)
+    if (lockFile.exists()) {
+      if (!lockFile.delete()) {
+        LOGGER.warn("failed to delete lock file: " + streamingLock.getLockFilePath)
+      }
     }
     try {
       if (streamingLock.lockWithRetries()) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c0163616/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
index c4e3517..43c1e5a 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
@@ -1506,6 +1506,9 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
   }
 
   test("auto hand off, close and reopen streaming table") {
+    sql("alter table streaming.stream_table_reopen compact 'close_streaming'")
+    sql("ALTER TABLE streaming.stream_table_reopen SET TBLPROPERTIES('streaming'='true')")
+
     executeStreamingIngest(
       tableName = "stream_table_reopen",
       batchNums = 2,


[27/45] carbondata git commit: [HOTFIX] carbon reader support open another reader without closing previous reader

Posted by ra...@apache.org.
[HOTFIX] carbon reader support open another reader without closing previous reader

[HOTFIX] carbon reader support open another reader without closing previous reader

This closes #2790


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

Branch: refs/heads/branch-1.5
Commit: 6ef4e46b0df2d3954788222943df09bf07fdb120
Parents: 9ca985f
Author: ajantha-bhat <aj...@gmail.com>
Authored: Fri Sep 28 20:24:39 2018 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Wed Oct 3 20:06:05 2018 +0530

----------------------------------------------------------------------
 .../hadoop/api/CarbonInputFormat.java           | 26 ++++----------------
 .../sdk/file/CarbonReaderBuilder.java           |  7 ++----
 .../carbondata/sdk/file/CarbonReaderTest.java   |  3 +--
 3 files changed, 8 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/6ef4e46b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java
index db93cbd..ed82e13 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java
@@ -774,29 +774,13 @@ m filterExpression
    */
   public String[] projectAllColumns(CarbonTable carbonTable) {
     List<ColumnSchema> colList = carbonTable.getTableInfo().getFactTable().getListOfColumns();
-    List<String> projectColumn = new ArrayList<>();
-    // childCount will recursively count the number of children for any parent
+    List<String> projectColumns = new ArrayList<>();
     // complex type and add just the parent column name while skipping the child columns.
-    int childDimCount = 0;
-    for (ColumnSchema cols : colList) {
-      if (cols.getSchemaOrdinal() != -1) {
-        if (childDimCount == 0) {
-          projectColumn.add(cols.getColumnName());
-        }
-        if (childDimCount > 0) {
-          childDimCount--;
-        }
-        if (cols.getDataType().isComplexType()) {
-          childDimCount += cols.getNumberOfChild();
-        }
+    for (ColumnSchema col : colList) {
+      if (!col.getColumnName().contains(".")) {
+        projectColumns.add(col.getColumnName());
       }
     }
-    String[] projectionColumns = new String[projectColumn.size()];
-    int i = 0;
-    for (String columnName : projectColumn) {
-      projectionColumns[i] = columnName;
-      i++;
-    }
-    return projectionColumns;
+    return projectColumns.toArray(new String[projectColumns.size()]);
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6ef4e46b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReaderBuilder.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReaderBuilder.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReaderBuilder.java
index 151d57c..9651a8f 100644
--- a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReaderBuilder.java
+++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReaderBuilder.java
@@ -131,11 +131,8 @@ public class CarbonReaderBuilder {
       hadoopConf = FileFactory.getConfiguration();
     }
     CarbonTable table;
-    if (filterExpression != null) {
-      table = CarbonTable.buildTable(tablePath, tableName, hadoopConf);
-    } else {
-      table = CarbonTable.buildDummyTable(tablePath);
-    }
+    // now always infer schema. TODO:Refactor in next version.
+    table = CarbonTable.buildTable(tablePath, tableName, hadoopConf);
     final CarbonFileInputFormat format = new CarbonFileInputFormat();
     final Job job = new Job(hadoopConf);
     format.setTableInfo(job.getConfiguration(), table.getTableInfo());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6ef4e46b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java
index 8d95456..ba8a49d 100644
--- a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java
+++ b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java
@@ -90,8 +90,6 @@ public class CarbonReaderTest extends TestCase {
     }
     Assert.assertEquals(i, 200);
 
-    reader.close();
-
     // Read again
     CarbonReader reader2 = CarbonReader
         .builder(path, "_temp")
@@ -108,6 +106,7 @@ public class CarbonReaderTest extends TestCase {
     }
     Assert.assertEquals(i, 200);
     reader2.close();
+    reader.close();
 
     FileUtils.deleteDirectory(new File(path));
   }


[40/45] carbondata git commit: [Documentation] Readme updated with latest topics and new TOC

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/ca30ad97/docs/datamap-developer-guide.md
----------------------------------------------------------------------
diff --git a/docs/datamap-developer-guide.md b/docs/datamap-developer-guide.md
index 6bac9b5..60f93df 100644
--- a/docs/datamap-developer-guide.md
+++ b/docs/datamap-developer-guide.md
@@ -6,7 +6,7 @@ Currently, there are two 2 types of DataMap supported:
 1. IndexDataMap: DataMap that leverages index to accelerate filter query
 2. MVDataMap: DataMap that leverages Materialized View to accelerate olap style query, like SPJG query (select, predicate, join, groupby)
 
-### DataMap provider
+### DataMap Provider
 When user issues `CREATE DATAMAP dm ON TABLE main USING 'provider'`, the corresponding DataMapProvider implementation will be created and initialized. 
 Currently, the provider string can be:
 1. preaggregate: A type of MVDataMap that do pre-aggregate of single table
@@ -15,5 +15,5 @@ Currently, the provider string can be:
 
 When user issues `DROP DATAMAP dm ON TABLE main`, the corresponding DataMapProvider interface will be called.
 
-Details about [DataMap Management](./datamap/datamap-management.md#datamap-management) and supported [DSL](./datamap/datamap-management.md#overview) are documented [here](./datamap/datamap-management.md).
+Click for more details about [DataMap Management](./datamap/datamap-management.md#datamap-management) and supported [DSL](./datamap/datamap-management.md#overview).
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ca30ad97/docs/datamap/bloomfilter-datamap-guide.md
----------------------------------------------------------------------
diff --git a/docs/datamap/bloomfilter-datamap-guide.md b/docs/datamap/bloomfilter-datamap-guide.md
index b2e7d60..fb244fe 100644
--- a/docs/datamap/bloomfilter-datamap-guide.md
+++ b/docs/datamap/bloomfilter-datamap-guide.md
@@ -15,7 +15,7 @@
     limitations under the License.
 -->
 
-# CarbonData BloomFilter DataMap (Alpha Feature)
+# CarbonData BloomFilter DataMap
 
 * [DataMap Management](#datamap-management)
 * [BloomFilter Datamap Introduction](#bloomfilter-datamap-introduction)
@@ -46,7 +46,7 @@ Showing all DataMaps on this table
   ```
 
 Disable Datamap
-> The datamap by default is enabled. To support tuning on query, we can disable a specific datamap during query to observe whether we can gain performance enhancement from it. This will only take effect current session.
+> The datamap by default is enabled. To support tuning on query, we can disable a specific datamap during query to observe whether we can gain performance enhancement from it. This is effective only for current session.
 
   ```
   // disable the datamap
@@ -82,7 +82,7 @@ and we always query on `id` and `name` with precise value.
 since `id` is in the sort_columns and it is orderd,
 query on it will be fast because CarbonData can skip all the irrelative blocklets.
 But queries on `name` may be bad since the blocklet minmax may not help,
-because in each blocklet the range of the value of `name` may be the same -- all from A*~z*.
+because in each blocklet the range of the value of `name` may be the same -- all from A* to z*.
 In this case, user can create a BloomFilter datamap on column `name`.
 Moreover, user can also create a BloomFilter datamap on the sort_columns.
 This is useful if user has too many segments and the range of the value of sort_columns are almost the same.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ca30ad97/docs/datamap/datamap-management.md
----------------------------------------------------------------------
diff --git a/docs/datamap/datamap-management.md b/docs/datamap/datamap-management.md
index bf52c05..ad8718a 100644
--- a/docs/datamap/datamap-management.md
+++ b/docs/datamap/datamap-management.md
@@ -66,9 +66,9 @@ If user create MV datamap without specifying `WITH DEFERRED REBUILD`, carbondata
 ### Automatic Refresh
 
 When user creates a datamap on the main table without using `WITH DEFERRED REBUILD` syntax, the datamap will be managed by system automatically.
-For every data load to the main table, system will immediately triger a load to the datamap automatically. These two data loading (to main table and datamap) is executed in a transactional manner, meaning that it will be either both success or neither success. 
+For every data load to the main table, system will immediately trigger a load to the datamap automatically. These two data loading (to main table and datamap) is executed in a transactional manner, meaning that it will be either both success or neither success. 
 
-The data loading to datamap is incremental based on Segment concept, avoiding a expesive total rebuild.
+The data loading to datamap is incremental based on Segment concept, avoiding a expensive total rebuild.
 
 If user perform following command on the main table, system will return failure. (reject the operation)
 
@@ -87,7 +87,7 @@ We do recommend you to use this management for index datamap.
 
 ### Manual Refresh
 
-When user creates a datamap specifying maunal refresh semantic, the datamap is created with status *disabled* and query will NOT use this datamap until user can issue REBUILD DATAMAP command to build the datamap. For every REBUILD DATAMAP command, system will trigger a full rebuild of the datamap. After rebuild is done, system will change datamap status to *enabled*, so that it can be used in query rewrite.
+When user creates a datamap specifying manual refresh semantic, the datamap is created with status *disabled* and query will NOT use this datamap until user can issue REBUILD DATAMAP command to build the datamap. For every REBUILD DATAMAP command, system will trigger a full rebuild of the datamap. After rebuild is done, system will change datamap status to *enabled*, so that it can be used in query rewrite.
 
 For every new data loading, data update, delete, the related datamap will be made *disabled*,
 which means that the following queries will not benefit from the datamap before it becomes *enabled* again.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ca30ad97/docs/datamap/lucene-datamap-guide.md
----------------------------------------------------------------------
diff --git a/docs/datamap/lucene-datamap-guide.md b/docs/datamap/lucene-datamap-guide.md
index 86b00e2..aa9c8d4 100644
--- a/docs/datamap/lucene-datamap-guide.md
+++ b/docs/datamap/lucene-datamap-guide.md
@@ -47,7 +47,7 @@ It will show all DataMaps created on main table.
 
 ## Lucene DataMap Introduction
   Lucene is a high performance, full featured text search engine. Lucene is integrated to carbon as
-  an index datamap and managed along with main tables by CarbonData.User can create lucene datamap 
+  an index datamap and managed along with main tables by CarbonData. User can create lucene datamap 
   to improve query performance on string columns which has content of more length. So, user can 
   search tokenized word or pattern of it using lucene query on text content.
   
@@ -95,7 +95,7 @@ As a technique for query acceleration, Lucene indexes cannot be queried directly
 Queries are to be made on main table. when a query with TEXT_MATCH('name:c10') or 
 TEXT_MATCH_WITH_LIMIT('name:n10',10)[the second parameter represents the number of result to be 
 returned, if user does not specify this value, all results will be returned without any limit] is 
-fired, two jobs are fired.The first job writes the temporary files in folder created at table level 
+fired, two jobs are fired. The first job writes the temporary files in folder created at table level 
 which contains lucene's seach results and these files will be read in second job to give faster 
 results. These temporary files will be cleared once the query finishes.
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ca30ad97/docs/datamap/preaggregate-datamap-guide.md
----------------------------------------------------------------------
diff --git a/docs/datamap/preaggregate-datamap-guide.md b/docs/datamap/preaggregate-datamap-guide.md
index 3a3efc2..eff601d 100644
--- a/docs/datamap/preaggregate-datamap-guide.md
+++ b/docs/datamap/preaggregate-datamap-guide.md
@@ -251,7 +251,7 @@ pre-aggregate tables. To further improve the query performance, compaction on pr
 can be triggered to merge the segments and files in the pre-aggregate tables. 
 
 ## Data Management with pre-aggregate tables
-In current implementation, data consistence need to be maintained for both main table and pre-aggregate
+In current implementation, data consistency needs to be maintained for both main table and pre-aggregate
 tables. Once there is pre-aggregate table created on the main table, following command on the main 
 table
 is not supported:

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ca30ad97/docs/ddl-of-carbondata.md
----------------------------------------------------------------------
diff --git a/docs/ddl-of-carbondata.md b/docs/ddl-of-carbondata.md
index 7cda9cd..22d754a 100644
--- a/docs/ddl-of-carbondata.md
+++ b/docs/ddl-of-carbondata.md
@@ -73,6 +73,7 @@ CarbonData DDL statements are documented here,which includes:
   [TBLPROPERTIES (property_name=property_value, ...)]
   [LOCATION 'path']
   ```
+  
   **NOTE:** CarbonData also supports "STORED AS carbondata" and "USING carbondata". Find example code at [CarbonSessionExample](https://github.com/apache/carbondata/blob/master/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala) in the CarbonData repo.
 ### Usage Guidelines
 
@@ -93,10 +94,10 @@ CarbonData DDL statements are documented here,which includes:
 | [streaming](#streaming)                                      | Whether the table is a streaming table                       |
 | [LOCAL_DICTIONARY_ENABLE](#local-dictionary-configuration)   | Enable local dictionary generation                           |
 | [LOCAL_DICTIONARY_THRESHOLD](#local-dictionary-configuration) | Cardinality upto which the local dictionary can be generated |
-| [LOCAL_DICTIONARY_INCLUDE](#local-dictionary-configuration)  | Columns for which local dictionary needs to be generated.Useful when local dictionary need not be generated for all string/varchar/char columns |
-| [LOCAL_DICTIONARY_EXCLUDE](#local-dictionary-configuration)  | Columns for which local dictionary generation should be skipped.Useful when local dictionary need not be generated for few string/varchar/char columns |
+| [LOCAL_DICTIONARY_INCLUDE](#local-dictionary-configuration)  | Columns for which local dictionary needs to be generated. Useful when local dictionary need not be generated for all string/varchar/char columns |
+| [LOCAL_DICTIONARY_EXCLUDE](#local-dictionary-configuration)  | Columns for which local dictionary generation should be skipped. Useful when local dictionary need not be generated for few string/varchar/char columns |
 | [COLUMN_META_CACHE](#caching-minmax-value-for-required-columns) | Columns whose metadata can be cached in Driver for efficient pruning and improved query performance |
-| [CACHE_LEVEL](#caching-at-block-or-blocklet-level)           | Column metadata caching level.Whether to cache column metadata of block or blocklet |
+| [CACHE_LEVEL](#caching-at-block-or-blocklet-level)           | Column metadata caching level. Whether to cache column metadata of block or blocklet |
 | [flat_folder](#support-flat-folder-same-as-hiveparquet)      | Whether to write all the carbondata files in a single folder.Not writing segments folder during incremental load |
 | [LONG_STRING_COLUMNS](#string-longer-than-32000-characters)  | Columns which are greater than 32K characters                |
 | [BUCKETNUMBER](#bucketing)                                   | Number of buckets to be created                              |
@@ -111,9 +112,10 @@ CarbonData DDL statements are documented here,which includes:
 
      ```
      TBLPROPERTIES ('DICTIONARY_INCLUDE'='column1, column2')
-	```
-	 NOTE: Dictionary Include/Exclude for complex child columns is not supported.
-	
+     ```
+     
+     **NOTE**: Dictionary Include/Exclude for complex child columns is not supported.
+     
    - ##### Inverted Index Configuration
 
      By default inverted index is enabled, it might help to improve compression ratio and query speed, especially for low cardinality columns which are in reward position.
@@ -128,7 +130,7 @@ CarbonData DDL statements are documented here,which includes:
      This property is for users to specify which columns belong to the MDK(Multi-Dimensions-Key) index.
      * If users don't specify "SORT_COLUMN" property, by default MDK index be built by using all dimension columns except complex data type column. 
      * If this property is specified but with empty argument, then the table will be loaded without sort.
-	 * This supports only string, date, timestamp, short, int, long, byte and boolean data types.
+     * This supports only string, date, timestamp, short, int, long, byte and boolean data types.
      Suggested use cases : Only build MDK index for required columns,it might help to improve the data loading performance.
 
      ```
@@ -136,7 +138,8 @@ CarbonData DDL statements are documented here,which includes:
      OR
      TBLPROPERTIES ('SORT_COLUMNS'='')
      ```
-     NOTE: Sort_Columns for Complex datatype columns is not supported.
+     
+     **NOTE**: Sort_Columns for Complex datatype columns is not supported.
 
    - ##### Sort Scope Configuration
    
@@ -147,10 +150,10 @@ CarbonData DDL statements are documented here,which includes:
      * BATCH_SORT: It increases the load performance but decreases the query performance if identified blocks > parallelism.
      * GLOBAL_SORT: It increases the query performance, especially high concurrent point query.
        And if you care about loading resources isolation strictly, because the system uses the spark GroupBy to sort data, the resource can be controlled by spark. 
-	
-	### Example:
 
-   ```
+    ### Example:
+
+    ```
     CREATE TABLE IF NOT EXISTS productSchema.productSalesTable (
       productNumber INT,
       productName STRING,
@@ -163,7 +166,7 @@ CarbonData DDL statements are documented here,which includes:
     STORED AS carbondata
     TBLPROPERTIES ('SORT_COLUMNS'='productName,storeCity',
                    'SORT_SCOPE'='NO_SORT')
-   ```
+    ```
 
    **NOTE:** CarbonData also supports "using carbondata". Find example code at [SparkSessionExample](https://github.com/apache/carbondata/blob/master/examples/spark2/src/main/scala/org/apache/carbondata/examples/SparkSessionExample.scala) in the CarbonData repo.
 
@@ -174,6 +177,7 @@ CarbonData DDL statements are documented here,which includes:
      ```
      TBLPROPERTIES ('TABLE_BLOCKSIZE'='512')
      ```
+ 
      **NOTE:** 512 or 512M both are accepted.
 
    - ##### Table Compaction Configuration
@@ -197,7 +201,7 @@ CarbonData DDL statements are documented here,which includes:
      
    - ##### Streaming
 
-     CarbonData supports streaming ingestion for real-time data. You can create the ‘streaming’ table using the following table properties.
+     CarbonData supports streaming ingestion for real-time data. You can create the 'streaming' table using the following table properties.
 
      ```
      TBLPROPERTIES ('streaming'='true')
@@ -247,8 +251,8 @@ CarbonData DDL statements are documented here,which includes:
 | ---------- | ------------- | ----------- |
 | LOCAL_DICTIONARY_ENABLE | false | Whether to enable local dictionary generation. **NOTE:** If this property is defined, it will override the value configured at system level by '***carbon.local.dictionary.enable***'.Local dictionary will be generated for all string/varchar/char columns unless LOCAL_DICTIONARY_INCLUDE, LOCAL_DICTIONARY_EXCLUDE is configured. |
 | LOCAL_DICTIONARY_THRESHOLD | 10000 | The maximum cardinality of a column upto which carbondata can try to generate local dictionary (maximum - 100000). **NOTE:** When LOCAL_DICTIONARY_THRESHOLD is defined for Complex columns, the count of distinct records of all child columns are summed up. |
-| LOCAL_DICTIONARY_INCLUDE | string/varchar/char columns| Columns for which Local Dictionary has to be generated.**NOTE:** Those string/varchar/char columns which are added into DICTIONARY_INCLUDE option will not be considered for local dictionary generation.This property needs to be configured only when local dictionary needs to be generated for few columns, skipping others.This property takes effect only when **LOCAL_DICTIONARY_ENABLE** is true or **carbon.local.dictionary.enable** is true |
-| LOCAL_DICTIONARY_EXCLUDE | none | Columns for which Local Dictionary need not be generated.This property needs to be configured only when local dictionary needs to be skipped for few columns, generating for others.This property takes effect only when **LOCAL_DICTIONARY_ENABLE** is true or **carbon.local.dictionary.enable** is true |
+| LOCAL_DICTIONARY_INCLUDE | string/varchar/char columns| Columns for which Local Dictionary has to be generated.**NOTE:** Those string/varchar/char columns which are added into DICTIONARY_INCLUDE option will not be considered for local dictionary generation. This property needs to be configured only when local dictionary needs to be generated for few columns, skipping others. This property takes effect only when **LOCAL_DICTIONARY_ENABLE** is true or **carbon.local.dictionary.enable** is true |
+| LOCAL_DICTIONARY_EXCLUDE | none | Columns for which Local Dictionary need not be generated. This property needs to be configured only when local dictionary needs to be skipped for few columns, generating for others. This property takes effect only when **LOCAL_DICTIONARY_ENABLE** is true or **carbon.local.dictionary.enable** is true |
 
    **Fallback behavior:** 
 
@@ -294,19 +298,19 @@ CarbonData DDL statements are documented here,which includes:
       * If you want no column min/max values to be cached in the driver.
 
       ```
-      COLUMN_META_CACHE=’’
+      COLUMN_META_CACHE=''
       ```
 
       * If you want only col1 min/max values to be cached in the driver.
 
       ```
-      COLUMN_META_CACHE=’col1’
+      COLUMN_META_CACHE='col1'
       ```
 
       * If you want min/max values to be cached in driver for all the specified columns.
 
       ```
-      COLUMN_META_CACHE=’col1,col2,col3,…’
+      COLUMN_META_CACHE='col1,col2,col3,…'
       ```
 
       Columns to be cached can be specified either while creating table or after creation of the table.
@@ -315,13 +319,13 @@ CarbonData DDL statements are documented here,which includes:
       Syntax:
 
       ```
-      CREATE TABLE [dbName].tableName (col1 String, col2 String, col3 int,…) STORED BY ‘carbondata’ TBLPROPERTIES (‘COLUMN_META_CACHE’=’col1,col2,…’)
+      CREATE TABLE [dbName].tableName (col1 String, col2 String, col3 int,…) STORED BY 'carbondata' TBLPROPERTIES ('COLUMN_META_CACHE'='col1,col2,…')
       ```
 
       Example:
 
       ```
-      CREATE TABLE employee (name String, city String, id int) STORED BY ‘carbondata’ TBLPROPERTIES (‘COLUMN_META_CACHE’=’name’)
+      CREATE TABLE employee (name String, city String, id int) STORED BY 'carbondata' TBLPROPERTIES ('COLUMN_META_CACHE'='name')
       ```
 
       After creation of table or on already created tables use the alter table command to configure the columns to be cached.
@@ -329,13 +333,13 @@ CarbonData DDL statements are documented here,which includes:
       Syntax:
 
       ```
-      ALTER TABLE [dbName].tableName SET TBLPROPERTIES (‘COLUMN_META_CACHE’=’col1,col2,…’)
+      ALTER TABLE [dbName].tableName SET TBLPROPERTIES ('COLUMN_META_CACHE'='col1,col2,…')
       ```
 
       Example:
 
       ```
-      ALTER TABLE employee SET TBLPROPERTIES (‘COLUMN_META_CACHE’=’city’)
+      ALTER TABLE employee SET TBLPROPERTIES ('COLUMN_META_CACHE'='city')
       ```
 
    - ##### Caching at Block or Blocklet Level
@@ -347,13 +351,13 @@ CarbonData DDL statements are documented here,which includes:
       *Configuration for caching in driver at Block level (default value).*
 
       ```
-      CACHE_LEVEL= ‘BLOCK’
+      CACHE_LEVEL= 'BLOCK'
       ```
 
       *Configuration for caching in driver at Blocklet level.*
 
       ```
-      CACHE_LEVEL= ‘BLOCKLET’
+      CACHE_LEVEL= 'BLOCKLET'
       ```
 
       Cache level can be specified either while creating table or after creation of the table.
@@ -362,13 +366,13 @@ CarbonData DDL statements are documented here,which includes:
       Syntax:
 
       ```
-      CREATE TABLE [dbName].tableName (col1 String, col2 String, col3 int,…) STORED BY ‘carbondata’ TBLPROPERTIES (‘CACHE_LEVEL’=’Blocklet’)
+      CREATE TABLE [dbName].tableName (col1 String, col2 String, col3 int,…) STORED BY 'carbondata' TBLPROPERTIES ('CACHE_LEVEL'='Blocklet')
       ```
 
       Example:
 
       ```
-      CREATE TABLE employee (name String, city String, id int) STORED BY ‘carbondata’ TBLPROPERTIES (‘CACHE_LEVEL’=’Blocklet’)
+      CREATE TABLE employee (name String, city String, id int) STORED BY 'carbondata' TBLPROPERTIES ('CACHE_LEVEL'='Blocklet')
       ```
 
       After creation of table or on already created tables use the alter table command to configure the cache level.
@@ -376,26 +380,27 @@ CarbonData DDL statements are documented here,which includes:
       Syntax:
 
       ```
-      ALTER TABLE [dbName].tableName SET TBLPROPERTIES (‘CACHE_LEVEL’=’Blocklet’)
+      ALTER TABLE [dbName].tableName SET TBLPROPERTIES ('CACHE_LEVEL'='Blocklet')
       ```
 
       Example:
 
       ```
-      ALTER TABLE employee SET TBLPROPERTIES (‘CACHE_LEVEL’=’Blocklet’)
+      ALTER TABLE employee SET TBLPROPERTIES ('CACHE_LEVEL'='Blocklet')
       ```
 
    - ##### Support Flat folder same as Hive/Parquet
 
-       This feature allows all carbondata and index files to keep directy under tablepath. Currently all carbondata/carbonindex files written under tablepath/Fact/Part0/Segment_NUM folder and it is not same as hive/parquet folder structure. This feature makes all files written will be directly under tablepath, it does not maintain any segment folder structure.This is useful for interoperability between the execution engines and plugin with other execution engines like hive or presto becomes easier.
+       This feature allows all carbondata and index files to keep directy under tablepath. Currently all carbondata/carbonindex files written under tablepath/Fact/Part0/Segment_NUM folder and it is not same as hive/parquet folder structure. This feature makes all files written will be directly under tablepath, it does not maintain any segment folder structure. This is useful for interoperability between the execution engines and plugin with other execution engines like hive or presto becomes easier.
 
        Following table property enables this feature and default value is false.
        ```
         'flat_folder'='true'
        ```
+   
        Example:
        ```
-       CREATE TABLE employee (name String, city String, id int) STORED BY ‘carbondata’ TBLPROPERTIES ('flat_folder'='true')
+       CREATE TABLE employee (name String, city String, id int) STORED BY 'carbondata' TBLPROPERTIES ('flat_folder'='true')
        ```
 
    - ##### String longer than 32000 characters
@@ -489,7 +494,7 @@ CarbonData DDL statements are documented here,which includes:
   This function allows user to create external table by specifying location.
   ```
   CREATE EXTERNAL TABLE [IF NOT EXISTS] [db_name.]table_name 
-  STORED AS carbondata LOCATION ‘$FilesPath’
+  STORED AS carbondata LOCATION '$FilesPath'
   ```
 
 ### Create external table on managed table data location.
@@ -542,7 +547,7 @@ CarbonData DDL statements are documented here,which includes:
 
 ### Example
   ```
-  CREATE DATABASE carbon LOCATION “hdfs://name_cluster/dir1/carbonstore”;
+  CREATE DATABASE carbon LOCATION "hdfs://name_cluster/dir1/carbonstore";
   ```
 
 ## TABLE MANAGEMENT  
@@ -600,21 +605,23 @@ CarbonData DDL statements are documented here,which includes:
      ```
      ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING) TBLPROPERTIES('DEFAULT.VALUE.a1'='10')
      ```
-      NOTE: Add Complex datatype columns is not supported.
+      **NOTE:** Add Complex datatype columns is not supported.
 
 Users can specify which columns to include and exclude for local dictionary generation after adding new columns. These will be appended with the already existing local dictionary include and exclude columns of main table respectively.
-  ```
+     ```
      ALTER TABLE carbon ADD COLUMNS (a1 STRING, b1 STRING) TBLPROPERTIES('LOCAL_DICTIONARY_INCLUDE'='a1','LOCAL_DICTIONARY_EXCLUDE'='b1')
-  ```
+     ```
 
    - ##### DROP COLUMNS
    
      This command is used to delete the existing column(s) in a table.
+     
      ```
      ALTER TABLE [db_name.]table_name DROP COLUMNS (col_name, ...)
      ```
 
      Examples:
+     
      ```
      ALTER TABLE carbon DROP COLUMNS (b1)
      OR
@@ -622,12 +629,14 @@ Users can specify which columns to include and exclude for local dictionary gene
      
      ALTER TABLE carbon DROP COLUMNS (c1,d1)
      ```
-     NOTE: Drop Complex child column is not supported.
+ 
+     **NOTE:** Drop Complex child column is not supported.
 
    - ##### CHANGE DATA TYPE
    
      This command is used to change the data type from INT to BIGINT or decimal precision from lower to higher.
      Change of decimal data type from lower precision to higher precision will only be supported for cases where there is no data loss.
+     
      ```
      ALTER TABLE [db_name.]table_name CHANGE col_name col_name changed_column_type
      ```
@@ -638,25 +647,31 @@ Users can specify which columns to include and exclude for local dictionary gene
      - **NOTE:** The allowed range is 38,38 (precision, scale) and is a valid upper case scenario which is not resulting in data loss.
 
      Example1:Changing data type of column a1 from INT to BIGINT.
+     
      ```
      ALTER TABLE test_db.carbon CHANGE a1 a1 BIGINT
      ```
      
      Example2:Changing decimal precision of column a1 from 10 to 18.
+     
      ```
      ALTER TABLE test_db.carbon CHANGE a1 a1 DECIMAL(18,2)
      ```
+ 
 - ##### MERGE INDEX
 
      This command is used to merge all the CarbonData index files (.carbonindex) inside a segment to a single CarbonData index merge file (.carbonindexmerge). This enhances the first query performance.
+     
      ```
       ALTER TABLE [db_name.]table_name COMPACT 'SEGMENT_INDEX'
      ```
 
       Examples:
-      ```
+      
+     ```
       ALTER TABLE test_db.carbon COMPACT 'SEGMENT_INDEX'
       ```
+      
       **NOTE:**
 
       * Merge index is not supported on streaming table.
@@ -726,10 +741,11 @@ Users can specify which columns to include and exclude for local dictionary gene
   ```
   CREATE TABLE IF NOT EXISTS productSchema.productSalesTable (
                                 productNumber Int COMMENT 'unique serial number for product')
-  COMMENT “This is table comment”
+  COMMENT "This is table comment"
    STORED AS carbondata
    TBLPROPERTIES ('DICTIONARY_INCLUDE'='productNumber')
   ```
+  
   You can also SET and UNSET table comment using ALTER command.
 
   Example to SET table comment:
@@ -775,8 +791,8 @@ Users can specify which columns to include and exclude for local dictionary gene
   PARTITIONED BY (productCategory STRING, productBatch STRING)
   STORED AS carbondata
   ```
-   NOTE: Hive partition is not supported on complex datatype columns.
-		
+   **NOTE:** Hive partition is not supported on complex datatype columns.
+
 
 #### Show Partitions
 
@@ -832,6 +848,7 @@ Users can specify which columns to include and exclude for local dictionary gene
   [TBLPROPERTIES ('PARTITION_TYPE'='HASH',
                   'NUM_PARTITIONS'='N' ...)]
   ```
+  
   **NOTE:** N is the number of hash partitions
 
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ca30ad97/docs/dml-of-carbondata.md
----------------------------------------------------------------------
diff --git a/docs/dml-of-carbondata.md b/docs/dml-of-carbondata.md
index 98bb132..db7c118 100644
--- a/docs/dml-of-carbondata.md
+++ b/docs/dml-of-carbondata.md
@@ -61,7 +61,7 @@ CarbonData DML statements are documented here,which includes:
 | [SORT_COLUMN_BOUNDS](#sort-column-bounds)               | How to parititon the sort columns to make the evenly distributed |
 | [SINGLE_PASS](#single_pass)                             | When to enable single pass data loading                      |
 | [BAD_RECORDS_LOGGER_ENABLE](#bad-records-handling)      | Whether to enable bad records logging                        |
-| [BAD_RECORD_PATH](#bad-records-handling)                | Bad records logging path.Useful when bad record logging is enabled |
+| [BAD_RECORD_PATH](#bad-records-handling)                | Bad records logging path. Useful when bad record logging is enabled |
 | [BAD_RECORDS_ACTION](#bad-records-handling)             | Behavior of data loading when bad record is found            |
 | [IS_EMPTY_DATA_BAD_RECORD](#bad-records-handling)       | Whether empty data of a column to be considered as bad record or not |
 | [GLOBAL_SORT_PARTITIONS](#global_sort_partitions)       | Number of partition to use for shuffling of data during sorting |
@@ -176,7 +176,7 @@ CarbonData DML statements are documented here,which includes:
 
     Range bounds for sort columns.
 
-    Suppose the table is created with 'SORT_COLUMNS'='name,id' and the range for name is aaa~zzz, the value range for id is 0~1000. Then during data loading, we can specify the following option to enhance data loading performance.
+    Suppose the table is created with 'SORT_COLUMNS'='name,id' and the range for name is aaa to zzz, the value range for id is 0 to 1000. Then during data loading, we can specify the following option to enhance data loading performance.
     ```
     OPTIONS('SORT_COLUMN_BOUNDS'='f,250;l,500;r,750')
     ```
@@ -186,7 +186,7 @@ CarbonData DML statements are documented here,which includes:
     * SORT_COLUMN_BOUNDS will be used only when the SORT_SCOPE is 'local_sort'.
     * Carbondata will use these bounds as ranges to process data concurrently during the final sort percedure. The records will be sorted and written out inside each partition. Since the partition is sorted, all records will be sorted.
     * Since the actual order and literal order of the dictionary column are not necessarily the same, we do not recommend you to use this feature if the first sort column is 'dictionary_include'.
-    * The option works better if your CPU usage during loading is low. If your system is already CPU tense, better not to use this option. Besides, it depends on the user to specify the bounds. If user does not know the exactly bounds to make the data distributed evenly among the bounds, loading performance will still be better than before or at least the same as before.
+    * The option works better if your CPU usage during loading is low. If your current system CPU usage is high, better not to use this option. Besides, it depends on the user to specify the bounds. If user does not know the exactly bounds to make the data distributed evenly among the bounds, loading performance will still be better than before or at least the same as before.
     * Users can find more information about this option in the description of PR1953.
 
   - ##### SINGLE_PASS:

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ca30ad97/docs/documentation.md
----------------------------------------------------------------------
diff --git a/docs/documentation.md b/docs/documentation.md
index 537a9d3..1b6726a 100644
--- a/docs/documentation.md
+++ b/docs/documentation.md
@@ -25,7 +25,7 @@ Apache CarbonData is a new big data file format for faster interactive query usi
 
 ## Getting Started
 
-**File Format Concepts:** Start with the basics of understanding the [CarbonData file format](./file-structure-of-carbondata.md#carbondata-file-format) and its [storage structure](./file-structure-of-carbondata.md).This will help to understand other parts of the documentation, including deployment, programming and usage guides. 
+**File Format Concepts:** Start with the basics of understanding the [CarbonData file format](./file-structure-of-carbondata.md#carbondata-file-format) and its [storage structure](./file-structure-of-carbondata.md). This will help to understand other parts of the documentation, including deployment, programming and usage guides. 
 
 **Quick Start:** [Run an example program](./quick-start-guide.md#installing-and-configuring-carbondata-to-run-locally-with-spark-shell) on your local machine or [study some examples](https://github.com/apache/carbondata/tree/master/examples/spark2/src/main/scala/org/apache/carbondata/examples).
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ca30ad97/docs/faq.md
----------------------------------------------------------------------
diff --git a/docs/faq.md b/docs/faq.md
index 3dee5a2..3ac9a0a 100644
--- a/docs/faq.md
+++ b/docs/faq.md
@@ -57,12 +57,12 @@ By default **carbon.badRecords.location** specifies the following location ``/op
 ## How to enable Bad Record Logging?
 While loading data we can specify the approach to handle Bad Records. In order to analyse the cause of the Bad Records the parameter ``BAD_RECORDS_LOGGER_ENABLE`` must be set to value ``TRUE``. There are multiple approaches to handle Bad Records which can be specified  by the parameter ``BAD_RECORDS_ACTION``.
 
-- To pad the incorrect values of the csv rows with NULL value and load the data in CarbonData, set the following in the query :
+- To pass the incorrect values of the csv rows with NULL value and load the data in CarbonData, set the following in the query :
 ```
 'BAD_RECORDS_ACTION'='FORCE'
 ```
 
-- To write the Bad Records without padding incorrect values with NULL in the raw csv (set in the parameter **carbon.badRecords.location**), set the following in the query :
+- To write the Bad Records without passing incorrect values with NULL in the raw csv (set in the parameter **carbon.badRecords.location**), set the following in the query :
 ```
 'BAD_RECORDS_ACTION'='REDIRECT'
 ```
@@ -199,7 +199,7 @@ select cntry,sum(gdp) from gdp21,pop1 where cntry=ctry group by cntry;
 ```
 
 ## Why all executors are showing success in Spark UI even after Dataload command failed at Driver side?
-Spark executor shows task as failed after the maximum number of retry attempts, but loading the data having bad records and BAD_RECORDS_ACTION (carbon.bad.records.action) is set as “FAIL” will attempt only once but will send the signal to driver as failed instead of throwing the exception to retry, as there is no point to retry if bad record found and BAD_RECORDS_ACTION is set to fail. Hence the Spark executor displays this one attempt as successful but the command has actually failed to execute. Task attempts or executor logs can be checked to observe the failure reason.
+Spark executor shows task as failed after the maximum number of retry attempts, but loading the data having bad records and BAD_RECORDS_ACTION (carbon.bad.records.action) is set as "FAIL" will attempt only once but will send the signal to driver as failed instead of throwing the exception to retry, as there is no point to retry if bad record found and BAD_RECORDS_ACTION is set to fail. Hence the Spark executor displays this one attempt as successful but the command has actually failed to execute. Task attempts or executor logs can be checked to observe the failure reason.
 
 ## Why different time zone result for select query output when query SDK writer output? 
 SDK writer is an independent entity, hence SDK writer can generate carbondata files from a non-cluster machine that has different time zones. But at cluster when those files are read, it always takes cluster time-zone. Hence, the value of timestamp and date datatype fields are not original value.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ca30ad97/docs/file-structure-of-carbondata.md
----------------------------------------------------------------------
diff --git a/docs/file-structure-of-carbondata.md b/docs/file-structure-of-carbondata.md
index ba9004c..9e656bb 100644
--- a/docs/file-structure-of-carbondata.md
+++ b/docs/file-structure-of-carbondata.md
@@ -48,7 +48,7 @@ The CarbonData files are stored in the location specified by the ***carbon.store
 
 ![File Directory Structure](../docs/images/2-1_1.png?raw=true)
 
-1. ModifiedTime.mdt records the timestamp of the metadata with the modification time attribute of the file. When the drop table and create table are used, the modification time of the file is updated.This is common to all databases and hence is kept in parallel to databases
+1. ModifiedTime.mdt records the timestamp of the metadata with the modification time attribute of the file. When the drop table and create table are used, the modification time of the file is updated. This is common to all databases and hence is kept in parallel to databases
 2. The **default** is the database name and contains the user tables.default is used when user doesn't specify any database name;else user configured database name will be the directory name. user_table is the table name.
 3. Metadata directory stores schema files, tablestatus and dictionary files (including .dict, .dictmeta and .sortindex). There are three types of metadata data information files.
 4. data and index files are stored under directory named **Fact**. The Fact directory has a Part0 partition directory, where 0 is the partition number.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ca30ad97/docs/how-to-contribute-to-apache-carbondata.md
----------------------------------------------------------------------
diff --git a/docs/how-to-contribute-to-apache-carbondata.md b/docs/how-to-contribute-to-apache-carbondata.md
index f64c948..8d6c891 100644
--- a/docs/how-to-contribute-to-apache-carbondata.md
+++ b/docs/how-to-contribute-to-apache-carbondata.md
@@ -48,7 +48,7 @@ alternatively, on the developer mailing list(dev@carbondata.apache.org).
 
 If there’s an existing JIRA issue for your intended contribution, please comment about your
 intended work. Once the work is understood, a committer will assign the issue to you.
-(If you don’t have a JIRA role yet, you’ll be added to the “contributor” role.) If an issue is
+(If you don’t have a JIRA role yet, you’ll be added to the "contributor" role.) If an issue is
 currently assigned, please check with the current assignee before reassigning.
 
 For moderate or large contributions, you should not start coding or writing a design doc unless
@@ -171,7 +171,7 @@ Our GitHub mirror automatically provides pre-commit testing coverage using Jenki
 Please make sure those tests pass,the contribution cannot be merged otherwise.
 
 #### LGTM
-Once the reviewer is happy with the change, they’ll respond with an LGTM (“looks good to me!”).
+Once the reviewer is happy with the change, they’ll respond with an LGTM ("looks good to me!").
 At this point, the committer will take over, possibly make some additional touch ups,
 and merge your changes into the codebase.
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ca30ad97/docs/introduction.md
----------------------------------------------------------------------
diff --git a/docs/introduction.md b/docs/introduction.md
index 434ccfa..e6c3372 100644
--- a/docs/introduction.md
+++ b/docs/introduction.md
@@ -18,15 +18,15 @@ CarbonData has
 
 ## CarbonData Features & Functions
 
-CarbonData has rich set of featues to support various use cases in Big Data analytics.The below table lists the major features supported by CarbonData.
+CarbonData has rich set of features to support various use cases in Big Data analytics. The below table lists the major features supported by CarbonData.
 
 
 
 ### Table Management
 
 - ##### DDL (Create, Alter,Drop,CTAS)
-
-​	CarbonData provides its own DDL to create and manage carbondata tables.These DDL conform to 			Hive,Spark SQL format and support additional properties and configuration to take advantages of CarbonData functionalities.
+  
+  CarbonData provides its own DDL to create and manage carbondata tables. These DDL conform to Hive,Spark SQL format and support additional properties and configuration to take advantages of CarbonData functionalities.
 
 - ##### DML(Load,Insert)
 
@@ -46,7 +46,7 @@ CarbonData has rich set of featues to support various use cases in Big Data anal
 
 - ##### Compaction
 
-  CarbonData manages incremental loads as segments.Compaction help to compact the growing number of segments and also to improve query filter pruning.
+  CarbonData manages incremental loads as segments. Compaction helps to compact the growing number of segments and also to improve query filter pruning.
 
 - ##### External Tables
 
@@ -56,11 +56,11 @@ CarbonData has rich set of featues to support various use cases in Big Data anal
 
 - ##### Pre-Aggregate
 
-  CarbonData has concept of datamaps to assist in pruning of data while querying so that performance is faster.Pre Aggregate tables are kind of datamaps which can improve the query performance by order of magnitude.CarbonData will automatically pre-aggregae the incremental data and re-write the query to automatically fetch from the most appropriate pre-aggregate table to serve the query faster.
+  CarbonData has concept of datamaps to assist in pruning of data while querying so that performance is faster.Pre Aggregate tables are kind of datamaps which can improve the query performance by order of magnitude.CarbonData will automatically pre-aggregate the incremental data and re-write the query to automatically fetch from the most appropriate pre-aggregate table to serve the query faster.
 
 - ##### Time Series
 
-  CarbonData has built in understanding of time order(Year, month,day,hour, minute,second).Time series is a pre-aggregate table which can automatically roll-up the data to the desired level during incremental load and serve the query from the most appropriate pre-aggregate table.
+  CarbonData has built in understanding of time order(Year, month,day,hour, minute,second). Time series is a pre-aggregate table which can automatically roll-up the data to the desired level during incremental load and serve the query from the most appropriate pre-aggregate table.
 
 - ##### Bloom filter
 
@@ -72,7 +72,7 @@ CarbonData has rich set of featues to support various use cases in Big Data anal
 
 - ##### MV (Materialized Views)
 
-  MVs are kind of pre-aggregate tables which can support efficent query re-write and processing.CarbonData provides MV which can rewrite query to fetch from any table(including non-carbondata tables).Typical usecase is to store the aggregated data of a non-carbondata fact table into carbondata and use mv to rewrite the query to fetch from carbondata.
+  MVs are kind of pre-aggregate tables which can support efficent query re-write and processing.CarbonData provides MV which can rewrite query to fetch from any table(including non-carbondata tables). Typical usecase is to store the aggregated data of a non-carbondata fact table into carbondata and use mv to rewrite the query to fetch from carbondata.
 
 ### Streaming
 
@@ -84,17 +84,17 @@ CarbonData has rich set of featues to support various use cases in Big Data anal
 
 - ##### CarbonData writer
 
-  CarbonData supports writing data from non-spark application using SDK.Users can use SDK to generate carbondata files from custom applications.Typical usecase is to write the streaming application plugged in to kafka and use carbondata as sink(target) table for storing.
+  CarbonData supports writing data from non-spark application using SDK.Users can use SDK to generate carbondata files from custom applications. Typical usecase is to write the streaming application plugged in to kafka and use carbondata as sink(target) table for storing.
 
 - ##### CarbonData reader
 
-  CarbonData supports reading of data from non-spark application using SDK.Users can use the SDK to read the carbondata files from their application and do custom processing.
+  CarbonData supports reading of data from non-spark application using SDK. Users can use the SDK to read the carbondata files from their application and do custom processing.
 
 ### Storage
 
 - ##### S3
 
-  CarbonData can write to S3, OBS or any cloud storage confirming to S3 protocol.CarbonData uses the HDFS api to write to cloud object stores.
+  CarbonData can write to S3, OBS or any cloud storage confirming to S3 protocol. CarbonData uses the HDFS api to write to cloud object stores.
 
 - ##### HDFS
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ca30ad97/docs/language-manual.md
----------------------------------------------------------------------
diff --git a/docs/language-manual.md b/docs/language-manual.md
index 9d3a9b9..79aad00 100644
--- a/docs/language-manual.md
+++ b/docs/language-manual.md
@@ -34,6 +34,8 @@ CarbonData has its own parser, in addition to Spark's SQL Parser, to parse and p
 - Data Manipulation Statements
   - [DML:](./dml-of-carbondata.md) [Load](./dml-of-carbondata.md#load-data), [Insert](./dml-of-carbondata.md#insert-data-into-carbondata-table), [Update](./dml-of-carbondata.md#update), [Delete](./dml-of-carbondata.md#delete)
   - [Segment Management](./segment-management-on-carbondata.md)
+- [CarbonData as Spark's Datasource](./carbon-as-spark-datasource-guide.md)
 - [Configuration Properties](./configuration-parameters.md)
 
 
+

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ca30ad97/docs/performance-tuning.md
----------------------------------------------------------------------
diff --git a/docs/performance-tuning.md b/docs/performance-tuning.md
index f56a63b..6c87ce9 100644
--- a/docs/performance-tuning.md
+++ b/docs/performance-tuning.md
@@ -140,7 +140,7 @@
 
 | Parameter | Default Value | Description/Tuning |
 |-----------|-------------|--------|
-|carbon.number.of.cores.while.loading|Default: 2.This value should be >= 2|Specifies the number of cores used for data processing during data loading in CarbonData. |
+|carbon.number.of.cores.while.loading|Default: 2. This value should be >= 2|Specifies the number of cores used for data processing during data loading in CarbonData. |
 |carbon.sort.size|Default: 100000. The value should be >= 100.|Threshold to write local file in sort step when loading data|
 |carbon.sort.file.write.buffer.size|Default:  50000.|DataOutputStream buffer. |
 |carbon.merge.sort.reader.thread|Default: 3 |Specifies the number of cores used for temp file merging during data loading in CarbonData.|
@@ -165,15 +165,15 @@
 |----------------------------------------------|-----------------------------------|---------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
 | carbon.sort.intermediate.files.limit | spark/carbonlib/carbon.properties | Data loading | During the loading of data, local temp is used to sort the data. This number specifies the minimum number of intermediate files after which the  merge sort has to be initiated. | Increasing the parameter to a higher value will improve the load performance. For example, when we increase the value from 20 to 100, it increases the data load performance from 35MB/S to more than 50MB/S. Higher values of this parameter consumes  more memory during the load. |
 | carbon.number.of.cores.while.loading | spark/carbonlib/carbon.properties | Data loading | Specifies the number of cores used for data processing during data loading in CarbonData. | If you have more number of CPUs, then you can increase the number of CPUs, which will increase the performance. For example if we increase the value from 2 to 4 then the CSV reading performance can increase about 1 times |
-| carbon.compaction.level.threshold | spark/carbonlib/carbon.properties | Data loading and Querying | For minor compaction, specifies the number of segments to be merged in stage 1 and number of compacted segments to be merged in stage 2. | Each CarbonData load will create one segment, if every load is small in size it will generate many small file over a period of time impacting the query performance. Configuring this parameter will merge the small segment to one big segment which will sort the data and improve the performance. For Example in one telecommunication scenario, the performance improves about 2 times after minor compaction. |
+| carbon.compaction.level.threshold | spark/carbonlib/carbon.properties | Data loading and Querying | For minor compaction, specifies the number of segments to be merged in stage 1 and number of compacted segments to be merged in stage 2. | Each CarbonData load will create one segment, if every load is small in size it will generate many small files over a period of time impacting the query performance. Configuring this parameter will merge the small segment to one big segment which will sort the data and improve the performance. For Example in one telecommunication scenario, the performance improves about 2 times after minor compaction. |
 | spark.sql.shuffle.partitions | spark/conf/spark-defaults.conf | Querying | The number of task started when spark shuffle. | The value can be 1 to 2 times as much as the executor cores. In an aggregation scenario, reducing the number from 200 to 32 reduced the query time from 17 to 9 seconds. |
 | spark.executor.instances/spark.executor.cores/spark.executor.memory | spark/conf/spark-defaults.conf | Querying | The number of executors, CPU cores, and memory used for CarbonData query. | In the bank scenario, we provide the 4 CPUs cores and 15 GB for each executor which can get good performance. This 2 value does not mean more the better. It needs to be configured properly in case of limited resources. For example, In the bank scenario, it has enough CPU 32 cores each node but less memory 64 GB each node. So we cannot give more CPU but less memory. For example, when 4 cores and 12GB for each executor. It sometimes happens GC during the query which impact the query performance very much from the 3 second to more than 15 seconds. In this scenario need to increase the memory or decrease the CPU cores. |
 | carbon.detail.batch.size | spark/carbonlib/carbon.properties | Data loading | The buffer size to store records, returned from the block scan. | In limit scenario this parameter is very important. For example your query limit is 1000. But if we set this value to 3000 that means we get 3000 records from scan but spark will only take 1000 rows. So the 2000 remaining are useless. In one Finance test case after we set it to 100, in the limit 1000 scenario the performance increase about 2 times in comparison to if we set this value to 12000. |
 | carbon.use.local.dir | spark/carbonlib/carbon.properties | Data loading | Whether use YARN local directories for multi-table load disk load balance | If this is set it to true CarbonData will use YARN local directories for multi-table load disk load balance, that will improve the data load performance. |
 | carbon.use.multiple.temp.dir | spark/carbonlib/carbon.properties | Data loading | Whether to use multiple YARN local directories during table data loading for disk load balance | After enabling 'carbon.use.local.dir', if this is set to true, CarbonData will use all YARN local directories during data load for disk load balance, that will improve the data load performance. Please enable this property when you encounter disk hotspot problem during data loading. |
-| carbon.sort.temp.compressor | spark/carbonlib/carbon.properties | Data loading | Specify the name of compressor to compress the intermediate sort temporary files during sort procedure in data loading. | The optional values are 'SNAPPY','GZIP','BZIP2','LZ4','ZSTD' and empty. By default, empty means that Carbondata will not compress the sort temp files. This parameter will be useful if you encounter disk bottleneck. |
-| carbon.load.skewedDataOptimization.enabled | spark/carbonlib/carbon.properties | Data loading | Whether to enable size based block allocation strategy for data loading. | When loading, carbondata will use file size based block allocation strategy for task distribution. It will make sure that all the executors process the same size of data -- It's useful if the size of your input data files varies widely, say 1MB~1GB. |
-| carbon.load.min.size.enabled | spark/carbonlib/carbon.properties | Data loading | Whether to enable node minumun input data size allocation strategy for data loading.| When loading, carbondata will use node minumun input data size allocation strategy for task distribution. It will make sure the node load the minimum amount of data -- It's useful if the size of your input data files very small, say 1MB~256MB,Avoid generating a large number of small files. |
+| carbon.sort.temp.compressor | spark/carbonlib/carbon.properties | Data loading | Specify the name of compressor to compress the intermediate sort temporary files during sort procedure in data loading. | The optional values are 'SNAPPY','GZIP','BZIP2','LZ4','ZSTD', and empty. By default, empty means that Carbondata will not compress the sort temp files. This parameter will be useful if you encounter disk bottleneck. |
+| carbon.load.skewedDataOptimization.enabled | spark/carbonlib/carbon.properties | Data loading | Whether to enable size based block allocation strategy for data loading. | When loading, carbondata will use file size based block allocation strategy for task distribution. It will make sure that all the executors process the same size of data -- It's useful if the size of your input data files varies widely, say 1MB to 1GB. |
+| carbon.load.min.size.enabled | spark/carbonlib/carbon.properties | Data loading | Whether to enable node minumun input data size allocation strategy for data loading.| When loading, carbondata will use node minumun input data size allocation strategy for task distribution. It will make sure the nodes load the minimum amount of data -- It's useful if the size of your input data files very small, say 1MB to 256MB,Avoid generating a large number of small files. |
 
   Note: If your CarbonData instance is provided only for query, you may specify the property 'spark.speculation=true' which is in conf directory of spark.
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ca30ad97/docs/quick-start-guide.md
----------------------------------------------------------------------
diff --git a/docs/quick-start-guide.md b/docs/quick-start-guide.md
index 0fdf055..fd535ae 100644
--- a/docs/quick-start-guide.md
+++ b/docs/quick-start-guide.md
@@ -16,7 +16,7 @@
 -->
 
 # Quick Start
-This tutorial provides a quick introduction to using CarbonData.To follow along with this guide, first download a packaged release of CarbonData from the [CarbonData website](https://dist.apache.org/repos/dist/release/carbondata/).Alternatively it can be created following [Building CarbonData](https://github.com/apache/carbondata/tree/master/build) steps.
+This tutorial provides a quick introduction to using CarbonData. To follow along with this guide, first download a packaged release of CarbonData from the [CarbonData website](https://dist.apache.org/repos/dist/release/carbondata/).Alternatively it can be created following [Building CarbonData](https://github.com/apache/carbondata/tree/master/build) steps.
 
 ##  Prerequisites
 * CarbonData supports Spark versions upto 2.2.1.Please download Spark package from [Spark website](https://spark.apache.org/downloads.html)
@@ -35,7 +35,7 @@ This tutorial provides a quick introduction to using CarbonData.To follow along
 
 ## Integration
 
-CarbonData can be integrated with Spark and Presto Execution Engines.The below documentation guides on Installing and Configuring with these execution engines.
+CarbonData can be integrated with Spark and Presto Execution Engines. The below documentation guides on Installing and Configuring with these execution engines.
 
 ### Spark
 
@@ -293,7 +293,7 @@ hdfs://<host_name>:port/user/hive/warehouse/carbon.store
 
 ## Installing and Configuring CarbonData on Presto
 
-**NOTE:** **CarbonData tables cannot be created nor loaded from Presto.User need to create CarbonData Table and load data into it
+**NOTE:** **CarbonData tables cannot be created nor loaded from Presto. User need to create CarbonData Table and load data into it
 either with [Spark](#installing-and-configuring-carbondata-to-run-locally-with-spark-shell) or [SDK](./sdk-guide.md).
 Once the table is created,it can be queried from Presto.**
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ca30ad97/docs/s3-guide.md
----------------------------------------------------------------------
diff --git a/docs/s3-guide.md b/docs/s3-guide.md
index a2e5f07..1121164 100644
--- a/docs/s3-guide.md
+++ b/docs/s3-guide.md
@@ -15,7 +15,7 @@
     limitations under the License.
 -->
 
-# S3 Guide (Alpha Feature 1.4.1)
+# S3 Guide
 
 Object storage is the recommended storage format in cloud as it can support storing large data 
 files. S3 APIs are widely used for accessing object stores. This can be 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ca30ad97/docs/streaming-guide.md
----------------------------------------------------------------------
diff --git a/docs/streaming-guide.md b/docs/streaming-guide.md
index 56e400e..714b07a 100644
--- a/docs/streaming-guide.md
+++ b/docs/streaming-guide.md
@@ -157,7 +157,7 @@ ALTER TABLE streaming_table SET TBLPROPERTIES('streaming'='true')
 At the begin of streaming ingestion, the system will try to acquire the table level lock of streaming.lock file. If the system isn't able to acquire the lock of this table, it will throw an InterruptedException.
 
 ## Create streaming segment
-The input data of streaming will be ingested into a segment of the CarbonData table, the status of this segment is streaming. CarbonData call it a streaming segment. The "tablestatus" file will record the segment status and data size. The user can use “SHOW SEGMENTS FOR TABLE tableName” to check segment status. 
+The streaming data will be ingested into a separate segment of carbondata table, this segment is termed as streaming segment. The status of this segment will be recorded as "streaming" in "tablestatus" file along with its data size. You can use "SHOW SEGMENTS FOR TABLE tableName" to check segment status. 
 
 After the streaming segment reaches the max size, CarbonData will change the segment status to "streaming finish" from "streaming", and create new "streaming" segment to continue to ingest streaming data.
 
@@ -352,7 +352,7 @@ Following example shows how to start a streaming ingest job
 
 In above example, two table is created: source and sink. The `source` table's format is `csv` and `sink` table format is `carbon`. Then a streaming job is created to stream data from source table to sink table.
 
-These two tables are normal carbon table, they can be queried independently.
+These two tables are normal carbon tables, they can be queried independently.
 
 
 
@@ -405,7 +405,7 @@ When this is issued, carbon will start a structured streaming job to do the stre
 
 - The sink table should have a TBLPROPERTY `'streaming'` equal to `true`, indicating it is a streaming table.
 - In the given STMPROPERTIES, user must specify `'trigger'`, its value must be `ProcessingTime` (In future, other value will be supported). User should also specify interval value for the streaming job.
-- If the schema specifid in sink table is different from CTAS, the streaming job will fail
+- If the schema specified in sink table is different from CTAS, the streaming job will fail
 
 For Kafka data source, create the source table by:
   ```SQL

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ca30ad97/docs/usecases.md
----------------------------------------------------------------------
diff --git a/docs/usecases.md b/docs/usecases.md
index 277c455..e8b98b5 100644
--- a/docs/usecases.md
+++ b/docs/usecases.md
@@ -39,7 +39,7 @@ These use cases can be broadly classified into below categories:
 
 ### Scenario
 
-User wants to analyse all the CHR(Call History Record) and MR(Measurement Records) of the mobile subscribers in order to identify the service failures within 10 secs.Also user wants to run machine learning models on the data to fairly estimate the reasons and time of probable failures and take action ahead to meet the SLA(Service Level Agreements) of VIP customers. 
+User wants to analyse all the CHR(Call History Record) and MR(Measurement Records) of the mobile subscribers in order to identify the service failures within 10 secs. Also user wants to run machine learning models on the data to fairly estimate the reasons and time of probable failures and take action ahead to meet the SLA(Service Level Agreements) of VIP customers. 
 
 ### Challenges
 
@@ -54,7 +54,7 @@ Setup a Hadoop + Spark + CarbonData cluster managed by YARN.
 
 Proposed the following configurations for CarbonData.(These tunings were proposed before CarbonData introduced SORT_COLUMNS parameter using which the sort order and schema order could be different.)
 
-Add the frequently used columns to the left of the table definition.Add it in the increasing order of cardinality.It was suggested to keep msisdn,imsi columns in the beginning of the schema.With latest CarbonData, SORT_COLUMNS needs to be configured msisdn,imsi in the beginning.
+Add the frequently used columns to the left of the table definition. Add it in the increasing order of cardinality. It was suggested to keep msisdn,imsi columns in the beginning of the schema. With latest CarbonData, SORT_COLUMNS needs to be configured msisdn,imsi in the beginning.
 
 Add timestamp column to the right of the schema as it is naturally increasing.
 
@@ -71,7 +71,7 @@ Apart from these, the following CarbonData configuration was suggested to be con
 | Data Loading | carbon.sort.size                        | 100000 | Number of records to sort at a time.More number of records configured will lead to increased memory foot print |
 | Data Loading | table_blocksize                         | 256  | To efficiently schedule multiple tasks during query |
 | Data Loading | carbon.sort.intermediate.files.limit    | 100    | Increased to 100 as number of cores are more.Can perform merging in backgorund.If less number of files to merge, sort threads would be idle |
-| Data Loading | carbon.use.local.dir                    | TRUE   | yarn application directory will be usually on a single disk.YARN would be configured with multiple disks to be used as temp or to assign randomly to applications.Using the yarn temp directory will allow carbon to use multiple disks and improve IO performance |
+| Data Loading | carbon.use.local.dir                    | TRUE   | yarn application directory will be usually on a single disk.YARN would be configured with multiple disks to be used as temp or to assign randomly to applications. Using the yarn temp directory will allow carbon to use multiple disks and improve IO performance |
 | Data Loading | carbon.use.multiple.temp.dir            | TRUE   | multiple disks to write sort files will lead to better IO and reduce the IO bottleneck |
 | Compaction | carbon.compaction.level.threshold       | 6,6    | Since frequent small loads, compacting more segments will give better query results |
 | Compaction | carbon.enable.auto.load.merge           | true   | Since data loading is small,auto compacting keeps the number of segments less and also compaction can complete in  time |
@@ -94,7 +94,7 @@ Apart from these, the following CarbonData configuration was suggested to be con
 
 ### Scenario
 
-User wants to analyse the person/vehicle movement and behavior during a certain time period.This output data needs to be joined with a external table for Human details extraction.The query will be run with different time period as filter to identify potential behavior mismatch.
+User wants to analyse the person/vehicle movement and behavior during a certain time period. This output data needs to be joined with a external table for Human details extraction. The query will be run with different time period as filter to identify potential behavior mismatch.
 
 ### Challenges
 
@@ -119,24 +119,24 @@ Use all columns are no-dictionary as the cardinality is high.
 | Configuration for | Parameter                               | Value                   | Description |
 | ------------------| --------------------------------------- | ----------------------- | ------------------|
 | Data Loading | carbon.graph.rowset.size                | 100000                  | Based on the size of each row, this determines the memory required during data loading.Higher value leads to increased memory foot print |
-| Data Loading | enable.unsafe.sort                      | TRUE                    | Temporary data generated during sort is huge which causes GC bottlenecks.Using unsafe reduces the pressure on GC |
-| Data Loading | enable.offheap.sort                     | TRUE                    | Temporary data generated during sort is huge which causes GC bottlenecks.Using offheap reduces the pressure on GC.offheap can be accessed through java unsafe.hence enable.unsafe.sort needs to be true |
+| Data Loading | enable.unsafe.sort                      | TRUE                    | Temporary data generated during sort is huge which causes GC bottlenecks. Using unsafe reduces the pressure on GC |
+| Data Loading | enable.offheap.sort                     | TRUE                    | Temporary data generated during sort is huge which causes GC bottlenecks. Using offheap reduces the pressure on GC.offheap can be accessed through java unsafe.hence enable.unsafe.sort needs to be true |
 | Data Loading | offheap.sort.chunk.size.in.mb           | 128                     | Size of memory to allocate for sorting.Can increase this based on the memory available |
 | Data Loading | carbon.number.of.cores.while.loading    | 12                      | Higher cores can improve data loading speed |
 | Data Loading | carbon.sort.size                        | 100000                  | Number of records to sort at a time.More number of records configured will lead to increased memory foot print |
-| Data Loading | table_blocksize                         | 512                     | To efficiently schedule multiple tasks during query.This size depends on data scenario.If data is such that the filters would select less number of blocklets to scan, keeping higher number works well.If the number blocklets to scan is more, better to reduce the size as more tasks can be scheduled in parallel. |
+| Data Loading | table_blocksize                         | 512                     | To efficiently schedule multiple tasks during query. This size depends on data scenario.If data is such that the filters would select less number of blocklets to scan, keeping higher number works well.If the number blocklets to scan is more, better to reduce the size as more tasks can be scheduled in parallel. |
 | Data Loading | carbon.sort.intermediate.files.limit    | 100                     | Increased to 100 as number of cores are more.Can perform merging in backgorund.If less number of files to merge, sort threads would be idle |
-| Data Loading | carbon.use.local.dir                    | TRUE                    | yarn application directory will be usually on a single disk.YARN would be configured with multiple disks to be used as temp or to assign randomly to applications.Using the yarn temp directory will allow carbon to use multiple disks and improve IO performance |
+| Data Loading | carbon.use.local.dir                    | TRUE                    | yarn application directory will be usually on a single disk.YARN would be configured with multiple disks to be used as temp or to assign randomly to applications. Using the yarn temp directory will allow carbon to use multiple disks and improve IO performance |
 | Data Loading | carbon.use.multiple.temp.dir            | TRUE                    | multiple disks to write sort files will lead to better IO and reduce the IO bottleneck |
-| Data Loading | sort.inmemory.size.in.mb                | 92160 | Memory allocated to do inmemory sorting.When more memory is available in the node, configuring this will retain more sort blocks in memory so that the merge sort is faster due to no/very less IO |
+| Data Loading | sort.inmemory.size.in.mb                | 92160 | Memory allocated to do inmemory sorting. When more memory is available in the node, configuring this will retain more sort blocks in memory so that the merge sort is faster due to no/very less IO |
 | Compaction | carbon.major.compaction.size            | 921600                  | Sum of several loads to combine into single segment |
 | Compaction | carbon.number.of.cores.while.compacting | 12                      | Higher number of cores can improve the compaction speed.Data size is huge.Compaction need to use more threads to speed up the process |
-| Compaction | carbon.enable.auto.load.merge           | FALSE                   | Doing auto minor compaction is costly process as data size is huge.Perform manual compaction when  the cluster is less loaded |
+| Compaction | carbon.enable.auto.load.merge           | FALSE                   | Doing auto minor compaction is costly process as data size is huge.Perform manual compaction when the cluster is less loaded |
 | Query | carbon.enable.vector.reader             | true                    | To fetch results faster, supporting spark vector processing will speed up the query |
-| Query | enable.unsafe.in.query.procressing      | true                    | Data that needs to be scanned in huge which in turn generates more short lived Java objects.This cause pressure of GC.using unsafe and offheap will reduce the GC overhead |
-| Query | use.offheap.in.query.processing         | true                    | Data that needs to be scanned in huge which in turn generates more short lived Java objects.This cause pressure of GC.using unsafe and offheap will reduce the GC overhead.offheap can be accessed through java unsafe.hence enable.unsafe.in.query.procressing needs to be true |
+| Query | enable.unsafe.in.query.procressing      | true                    | Data that needs to be scanned in huge which in turn generates more short lived Java objects. This cause pressure of GC.using unsafe and offheap will reduce the GC overhead |
+| Query | use.offheap.in.query.processing         | true                    | Data that needs to be scanned in huge which in turn generates more short lived Java objects. This cause pressure of GC.using unsafe and offheap will reduce the GC overhead.offheap can be accessed through java unsafe.hence enable.unsafe.in.query.procressing needs to be true |
 | Query | enable.unsafe.columnpage                | TRUE                    | Keep the column pages in offheap memory so that the memory overhead due to java object is less and also reduces GC pressure. |
-| Query | carbon.unsafe.working.memory.in.mb      | 10240                   | Amount of memory to use for offheap operations.Can increase this memory based on the data size |
+| Query | carbon.unsafe.working.memory.in.mb      | 10240                   | Amount of memory to use for offheap operations, you can increase this memory based on the data size |
 
 
 
@@ -177,7 +177,7 @@ Concurrent queries can be more due to the BI dashboard
 - Create pre-aggregate tables for non timestamp based group by queries
 - For queries containing group by date, create timeseries based Datamap(pre-aggregate) tables so that the data is rolled up during creation and fetch is faster
 - Reduce the Spark shuffle partitions.(In our configuration on 14 node cluster, it was reduced to 35 from default of 200)
-- Enable global dictionary for columns which have less cardinalities.Aggregation can be done on encoded data, there by improving the performance
+- Enable global dictionary for columns which have less cardinalities. Aggregation can be done on encoded data, there by improving the performance
 - For columns whose cardinality is high,enable the local dictionary so that store size is less and can take dictionary benefit for scan
 
 ## Handling near realtime data ingestion scenario
@@ -188,14 +188,14 @@ Need to support storing of continously arriving data and make it available immed
 
 ### Challenges
 
-When the data ingestion is near real time and the data needs to be available for query immediately, usual scenario is to do data loading in micro batches.But this causes the problem of generating many small files.This poses two problems:
+When the data ingestion is near real time and the data needs to be available for query immediately, usual scenario is to do data loading in micro batches.But this causes the problem of generating many small files. This poses two problems:
 
 1. Small file handling in HDFS is inefficient
 2. CarbonData will suffer in query performance as all the small files will have to be queried when filter is on non time column
 
 CarbonData will suffer in query performance as all the small files will have to be queried when filter is on non time column.
 
-Since data is continouly arriving, allocating resources for compaction might not be feasible.
+Since data is continously arriving, allocating resources for compaction might not be feasible.
 
 ### Goal
 


[10/45] carbondata git commit: [CARBONDATA-2971] Add shard info of blocklet for debugging

Posted by ra...@apache.org.
[CARBONDATA-2971] Add shard info of blocklet for debugging

add toString method to print both shard name and blocklet id for debugging.

This closes #2765


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

Branch: refs/heads/branch-1.5
Commit: 5c0da31a5a0afaf707455fa80ac431a082a57ec9
Parents: 3cd8b94
Author: Manhua <ke...@qq.com>
Authored: Wed Sep 26 10:34:54 2018 +0800
Committer: xuchuanyin <xu...@hust.edu.cn>
Committed: Thu Sep 27 11:37:56 2018 +0800

----------------------------------------------------------------------
 .../carbondata/core/indexstore/Blocklet.java    | 21 ++++++++++++++++----
 .../blockletindex/BlockletDataMapFactory.java   |  2 +-
 2 files changed, 18 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/5c0da31a/core/src/main/java/org/apache/carbondata/core/indexstore/Blocklet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/Blocklet.java b/core/src/main/java/org/apache/carbondata/core/indexstore/Blocklet.java
index c6e1681..3270d08 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/Blocklet.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/Blocklet.java
@@ -65,17 +65,20 @@ public class Blocklet implements Writable,Serializable {
     return filePath;
   }
 
-  @Override public void write(DataOutput out) throws IOException {
+  @Override
+  public void write(DataOutput out) throws IOException {
     out.writeUTF(filePath);
     out.writeUTF(blockletId);
   }
 
-  @Override public void readFields(DataInput in) throws IOException {
+  @Override
+  public void readFields(DataInput in) throws IOException {
     filePath = in.readUTF();
     blockletId = in.readUTF();
   }
 
-  @Override public boolean equals(Object o) {
+  @Override
+  public boolean equals(Object o) {
     if (this == o) return true;
     if (o == null || getClass() != o.getClass()) return false;
 
@@ -92,7 +95,17 @@ public class Blocklet implements Writable,Serializable {
         blocklet.blockletId == null;
   }
 
-  @Override public int hashCode() {
+  @Override
+  public String toString() {
+    final StringBuffer sb = new StringBuffer("Blocklet{");
+    sb.append("filePath='").append(filePath).append('\'');
+    sb.append(", blockletId='").append(blockletId).append('\'');
+    sb.append('}');
+    return sb.toString();
+  }
+
+  @Override
+  public int hashCode() {
     int result = filePath != null ? filePath.hashCode() : 0;
     result = 31 * result;
     if (compareBlockletIdForObjectMatching) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5c0da31a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
index e16c3cd..096a5e3 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
@@ -252,7 +252,7 @@ public class BlockletDataMapFactory extends CoarseGrainDataMapFactory
         }
       }
     }
-    throw new IOException("Blocklet with blockid " + blocklet.getBlockletId() + " not found ");
+    throw new IOException("Blocklet not found: " + blocklet.toString());
   }
 
 


[15/45] carbondata git commit: [DOC] Add kafka example in StreamSQL doc

Posted by ra...@apache.org.
[DOC] Add kafka example in StreamSQL doc

This closes #2769


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

Branch: refs/heads/branch-1.5
Commit: 629d62549860e037c2816702564713ce696419d5
Parents: 54bcf49
Author: Jacky Li <ja...@qq.com>
Authored: Wed Sep 26 21:56:52 2018 +0800
Committer: QiangCai <qi...@qq.com>
Committed: Fri Sep 28 11:52:41 2018 +0800

----------------------------------------------------------------------
 docs/streaming-guide.md | 33 +++++++++++++++++++++++++++------
 1 file changed, 27 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/629d6254/docs/streaming-guide.md
----------------------------------------------------------------------
diff --git a/docs/streaming-guide.md b/docs/streaming-guide.md
index 3b71662..56e400e 100644
--- a/docs/streaming-guide.md
+++ b/docs/streaming-guide.md
@@ -304,8 +304,9 @@ Following example shows how to start a streaming ingest job
          | register TIMESTAMP,
          | updated TIMESTAMP
          |)
-         |STORED BY carbondata
+         |STORED AS carbondata
          |TBLPROPERTIES (
+         | 'streaming'='source',
          | 'format'='csv',
          | 'path'='$csvDataDir'
          |)
@@ -324,7 +325,7 @@ Following example shows how to start a streaming ingest job
          | register TIMESTAMP,
          | updated TIMESTAMP
          |)
-         |STORED BY carbondata
+         |STORED AS carbondata
          |TBLPROPERTIES (
          |  'streaming'='true'
          |)
@@ -378,11 +379,14 @@ When this is issued, carbon will start a structured streaming job to do the stre
     name STRING,
     age INT
   )
-  STORED BY carbondata
+  STORED AS carbondata
   TBLPROPERTIES(
-    'format'='socket',
-    'host'='localhost',
-    'port'='8888'
+   'streaming'='source',
+   'format'='socket',
+   'host'='localhost',
+   'port'='8888',
+   'record_format'='csv', // can be csv or json, default is csv
+   'delimiter'='|'
   )
   ```
 
@@ -394,6 +398,7 @@ When this is issued, carbon will start a structured streaming job to do the stre
   	 .format("socket")
   	 .option("host", "localhost")
   	 .option("port", "8888")
+  	 .option("delimiter", "|")
   ```
 
 
@@ -402,6 +407,22 @@ When this is issued, carbon will start a structured streaming job to do the stre
 - In the given STMPROPERTIES, user must specify `'trigger'`, its value must be `ProcessingTime` (In future, other value will be supported). User should also specify interval value for the streaming job.
 - If the schema specifid in sink table is different from CTAS, the streaming job will fail
 
+For Kafka data source, create the source table by:
+  ```SQL
+  CREATE TABLE source(
+    name STRING,
+    age INT
+  )
+  STORED AS carbondata
+  TBLPROPERTIES(
+   'streaming'='source',
+   'format'='kafka',
+   'kafka.bootstrap.servers'='kafkaserver:9092',
+   'subscribe'='test'
+   'record_format'='csv', // can be csv or json, default is csv
+   'delimiter'='|'
+  )
+  ```
 
 
 ##### STOP STREAM


[04/45] carbondata git commit: [CARBONDATA-2968] Single pass load fails 2nd time in Spark submit execution due to port binding error

Posted by ra...@apache.org.
[CARBONDATA-2968] Single pass load fails 2nd time in Spark submit execution due to port binding error

Problem : In secure cluster setup, single pass load is failing in spark-submit after using the beeline.
Solution: It was happening because port was not getting updated and was not looking for the next empty port.
port variable was not changing.So modified that part and added log to diplay the port number.

This closes #2760


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

Branch: refs/heads/branch-1.5
Commit: 13ecc9e7a0a42ebf2f8417814c20474f3ce489f1
Parents: e07df44
Author: shardul-cr7 <sh...@gmail.com>
Authored: Tue Sep 25 19:55:19 2018 +0530
Committer: kumarvishal09 <ku...@gmail.com>
Committed: Wed Sep 26 14:16:21 2018 +0530

----------------------------------------------------------------------
 .../core/dictionary/server/NonSecureDictionaryServer.java      | 3 ++-
 .../spark/dictionary/server/SecureDictionaryServer.java        | 6 ++++--
 2 files changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/13ecc9e7/core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServer.java b/core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServer.java
index 95f3d69..dc2d211 100644
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServer.java
+++ b/core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServer.java
@@ -109,6 +109,7 @@ public class NonSecureDictionaryServer extends AbstractDictionaryServer
         });
         bootstrap.childOption(ChannelOption.SO_KEEPALIVE, true);
         String hostToBind = findLocalIpAddress(LOGGER);
+        //iteratively listening to newports
         InetSocketAddress address = hostToBind == null ?
             new InetSocketAddress(newPort) :
             new InetSocketAddress(hostToBind, newPort);
@@ -119,7 +120,7 @@ public class NonSecureDictionaryServer extends AbstractDictionaryServer
         this.host = hostToBind;
         break;
       } catch (Exception e) {
-        LOGGER.error(e, "Dictionary Server Failed to bind to port:");
+        LOGGER.error(e, "Dictionary Server Failed to bind to port:" + newPort);
         if (i == 9) {
           throw new RuntimeException("Dictionary Server Could not bind to any port");
         }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/13ecc9e7/integration/spark-common/src/main/java/org/apache/carbondata/spark/dictionary/server/SecureDictionaryServer.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/dictionary/server/SecureDictionaryServer.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/dictionary/server/SecureDictionaryServer.java
index f4948c4..995e520 100644
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/dictionary/server/SecureDictionaryServer.java
+++ b/integration/spark-common/src/main/java/org/apache/carbondata/spark/dictionary/server/SecureDictionaryServer.java
@@ -143,14 +143,16 @@ public class SecureDictionaryServer extends AbstractDictionaryServer implements
         TransportServerBootstrap bootstrap =
             new SaslServerBootstrap(transportConf, securityManager);
         String host = findLocalIpAddress(LOGGER);
-        context.createServer(host, port, Lists.<TransportServerBootstrap>newArrayList(bootstrap));
+        //iteratively listening to newports
+        context
+            .createServer(host, newPort, Lists.<TransportServerBootstrap>newArrayList(bootstrap));
         LOGGER.audit("Dictionary Server started, Time spent " + (System.currentTimeMillis() - start)
             + " Listening on port " + newPort);
         this.port = newPort;
         this.host = host;
         break;
       } catch (Exception e) {
-        LOGGER.error(e, "Dictionary Server Failed to bind to port:");
+        LOGGER.error(e, "Dictionary Server Failed to bind to port: " + newPort);
         if (i == 9) {
           throw new RuntimeException("Dictionary Server Could not bind to any port");
         }


[13/45] carbondata git commit: [CARBONDATA-2970]lock object creation fix for viewFS

Posted by ra...@apache.org.
[CARBONDATA-2970]lock object creation fix for viewFS

Problem
when default fs is set to ViewFS then the drop table and load fails with exception saying failed to get lock like meta.lock, tablestatus.lock.
This is because when getting locktypeObject we wre not checking for viewfs and we are returning it as local file system and failes while acquiring

Solution
Check for viewFS also when trying to get the lock object

This closes #2762


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

Branch: refs/heads/branch-1.5
Commit: 1b4109d5b2badc0c10d5522502bd799c6325263c
Parents: 5d17ff4
Author: akashrn5 <ak...@gmail.com>
Authored: Tue Sep 25 18:59:04 2018 +0530
Committer: kumarvishal09 <ku...@gmail.com>
Committed: Thu Sep 27 16:46:11 2018 +0530

----------------------------------------------------------------------
 .../java/org/apache/carbondata/core/locks/CarbonLockFactory.java  | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/1b4109d5/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockFactory.java b/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockFactory.java
index 91677a6..79bad6c 100644
--- a/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockFactory.java
@@ -71,7 +71,8 @@ public class CarbonLockFactory {
       lockTypeConfigured = CarbonCommonConstants.CARBON_LOCK_TYPE_S3;
       return new S3FileLock(absoluteLockPath,
           lockFile);
-    } else if (absoluteLockPath.startsWith(CarbonCommonConstants.HDFSURL_PREFIX)) {
+    } else if (absoluteLockPath.startsWith(CarbonCommonConstants.HDFSURL_PREFIX) || absoluteLockPath
+        .startsWith(CarbonCommonConstants.VIEWFSURL_PREFIX)) {
       lockTypeConfigured = CarbonCommonConstants.CARBON_LOCK_TYPE_HDFS;
       return new HdfsFileLock(absoluteLockPath, lockFile);
     } else {


[22/45] carbondata git commit: [HOTFIX] Fixed S3 metrics issue.

Posted by ra...@apache.org.
[HOTFIX] Fixed S3 metrics issue.

Problem: When data read from s3 it shows the data read as more than the size of carbon data total size.
Reason: It happens because carbondata uses dataInputStream.skip but in s3 interface it cannot handle properly
it reads in a loop and reads more data than required.
Solution: Use FSDataInputStream.seek instead of skip to fix this issue.

This closes #2789


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

Branch: refs/heads/branch-1.5
Commit: 7d1fcb3092a1e9da6c49f17c63c6217892e9e531
Parents: 2081bc8
Author: ravipesala <ra...@gmail.com>
Authored: Fri Sep 28 18:29:08 2018 +0530
Committer: kumarvishal09 <ku...@gmail.com>
Committed: Wed Oct 3 16:08:49 2018 +0530

----------------------------------------------------------------------
 .../datastore/filesystem/AbstractDFSCarbonFile.java |  7 +++++--
 .../apache/carbondata/core/reader/ThriftReader.java | 16 ++++++----------
 2 files changed, 11 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/7d1fcb30/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/AbstractDFSCarbonFile.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/AbstractDFSCarbonFile.java b/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/AbstractDFSCarbonFile.java
index b1e476b..c764430 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/AbstractDFSCarbonFile.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/AbstractDFSCarbonFile.java
@@ -327,8 +327,11 @@ public abstract class AbstractDFSCarbonFile implements CarbonFile {
       CompressionCodec codec = new CompressionCodecFactory(hadoopConf).getCodecByName(codecName);
       inputStream = codec.createInputStream(inputStream);
     }
-
-    return new DataInputStream(new BufferedInputStream(inputStream));
+    if (bufferSize <= 0 && inputStream instanceof FSDataInputStream) {
+      return (DataInputStream) inputStream;
+    } else {
+      return new DataInputStream(new BufferedInputStream(inputStream));
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7d1fcb30/core/src/main/java/org/apache/carbondata/core/reader/ThriftReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/reader/ThriftReader.java b/core/src/main/java/org/apache/carbondata/core/reader/ThriftReader.java
index 48d8345..f5ecda6 100644
--- a/core/src/main/java/org/apache/carbondata/core/reader/ThriftReader.java
+++ b/core/src/main/java/org/apache/carbondata/core/reader/ThriftReader.java
@@ -25,6 +25,7 @@ import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.util.CarbonUtil;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.thrift.TBase;
 import org.apache.thrift.TException;
 import org.apache.thrift.protocol.TCompactProtocol;
@@ -36,10 +37,6 @@ import org.apache.thrift.transport.TIOStreamTransport;
  */
 public class ThriftReader {
   /**
-   * buffer size
-   */
-  private static final int bufferSize = 2048;
-  /**
    * File containing the objects.
    */
   private String fileName;
@@ -101,7 +98,7 @@ public class ThriftReader {
   public void open() throws IOException {
     Configuration conf = configuration != null ? configuration : FileFactory.getConfiguration();
     FileFactory.FileType fileType = FileFactory.getFileType(fileName);
-    dataInputStream = FileFactory.getDataInputStream(fileName, fileType, bufferSize, conf);
+    dataInputStream = FileFactory.getDataInputStream(fileName, fileType, conf);
     binaryIn = new TCompactProtocol(new TIOStreamTransport(dataInputStream));
   }
 
@@ -109,7 +106,9 @@ public class ThriftReader {
    * This method will set the position of stream from where data has to be read
    */
   public void setReadOffset(long bytesToSkip) throws IOException {
-    if (dataInputStream.skip(bytesToSkip) != bytesToSkip) {
+    if (dataInputStream instanceof FSDataInputStream) {
+      ((FSDataInputStream)dataInputStream).seek(bytesToSkip);
+    } else if (dataInputStream.skip(bytesToSkip) != bytesToSkip) {
       throw new IOException("It doesn't set the offset properly");
     }
   }
@@ -118,10 +117,7 @@ public class ThriftReader {
    * Checks if another objects is available by attempting to read another byte from the stream.
    */
   public boolean hasNext() throws IOException {
-    dataInputStream.mark(1);
-    int val = dataInputStream.read();
-    dataInputStream.reset();
-    return val != -1;
+    return dataInputStream.available() > 0;
   }
 
   /**


[21/45] carbondata git commit: [CARBONDATA-2989] Upgrade spark integration version to 2.3.2

Posted by ra...@apache.org.
[CARBONDATA-2989] Upgrade spark integration version to 2.3.2

1.According to SPARK-PR#22346, change the parameter type from 'outputColumns: Seq[Attribute]' to 'outputColumnNames: Seq[String]' when call 'writeAndRead' method; 2.According to SPARK-PR#21815, there are some parameters added 'lazy', so move original class 'CarbonDataSourceScan' to src path 'commonTo2.1And2.2', and add a new class 'CarbonDataSourceScan' in src path 'spark2.3' which is added some lazy parameters. 3.Upgrade spark integration version to 2.3.2.

This closes #2779


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

Branch: refs/heads/branch-1.5
Commit: 2081bc87a5846055c861b28dfc1e3383c53e7ee0
Parents: 1c1ced3
Author: Zhang Zhichao <44...@qq.com>
Authored: Fri Sep 28 01:30:34 2018 +0800
Committer: chenliang613 <ch...@huawei.com>
Committed: Sun Sep 30 16:59:47 2018 +0800

----------------------------------------------------------------------
 .../testsuite/bigdecimal/TestBigDecimal.scala   |  2 +-
 .../spark/util/CarbonReflectionUtils.scala      |  6 +-
 integration/spark-datasource/pom.xml            |  2 +-
 integration/spark2/pom.xml                      |  5 +-
 .../strategy/CarbonDataSourceScan.scala         | 53 ++++++++++++++++++
 .../strategy/CarbonDataSourceScan.scala         | 53 ------------------
 .../strategy/CarbonDataSourceScan.scala         | 58 ++++++++++++++++++++
 pom.xml                                         |  4 +-
 8 files changed, 124 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/2081bc87/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/bigdecimal/TestBigDecimal.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/bigdecimal/TestBigDecimal.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/bigdecimal/TestBigDecimal.scala
index 1f7aafe..551b00b 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/bigdecimal/TestBigDecimal.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/bigdecimal/TestBigDecimal.scala
@@ -45,7 +45,7 @@ class TestBigDecimal extends QueryTest with BeforeAndAfterAll {
     sql("create table if not exists hiveTable(ID Int, date Timestamp, country String, name String, phonetype String, serialname String, salary Decimal(17,2))row format delimited fields terminated by ','")
     sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/decimalDataWithHeader.csv' into table carbonTable")
     sql(s"LOAD DATA local inpath '$resourcesPath/decimalDataWithoutHeader.csv' INTO table hiveTable")
-    sql("create table if not exists hiveBigDecimal(ID Int, date Timestamp, country String, name String, phonetype String, serialname String, salary decimal(27, 10))row format delimited fields terminated by ','")
+    sql("create table if not exists hiveBigDecimal(ID Int, date Timestamp, country String, name String, phonetype String, serialname String, salary decimal(30, 10))row format delimited fields terminated by ','")
     sql(s"LOAD DATA local inpath '$resourcesPath/decimalBoundaryDataHive.csv' INTO table hiveBigDecimal")
     sql("create table if not exists carbonBigDecimal_2 (ID Int, date Timestamp, country String, name String, phonetype String, serialname String, salary decimal(30, 10)) STORED BY 'org.apache.carbondata.format'")
     sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/decimalBoundaryDataCarbon.csv' into table carbonBigDecimal_2")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2081bc87/integration/spark-common/src/main/scala/org/apache/spark/util/CarbonReflectionUtils.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/util/CarbonReflectionUtils.scala b/integration/spark-common/src/main/scala/org/apache/spark/util/CarbonReflectionUtils.scala
index 9955286..0055e87 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/util/CarbonReflectionUtils.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/util/CarbonReflectionUtils.scala
@@ -294,9 +294,11 @@ object CarbonReflectionUtils {
         .getMethod("writeAndRead",
           classOf[SaveMode],
           classOf[LogicalPlan],
-          classOf[Seq[Attribute]],
+          classOf[Seq[String]],
           classOf[SparkPlan])
-      method.invoke(dataSourceObj, mode, query, query.output, physicalPlan)
+      // since spark 2.3.2 version (SPARK-PR#22346),
+      // change 'query.output' to 'query.output.map(_.name)'
+      method.invoke(dataSourceObj, mode, query, query.output.map(_.name), physicalPlan)
         .asInstanceOf[BaseRelation]
     } else {
       throw new UnsupportedOperationException("Spark version not supported")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2081bc87/integration/spark-datasource/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark-datasource/pom.xml b/integration/spark-datasource/pom.xml
index 9f0d3ff..002b9f3 100644
--- a/integration/spark-datasource/pom.xml
+++ b/integration/spark-datasource/pom.xml
@@ -278,7 +278,7 @@
     <profile>
       <id>spark-2.3</id>
       <properties>
-        <spark.version>2.3.1</spark.version>
+        <spark.version>2.3.2</spark.version>
         <scala.binary.version>2.11</scala.binary.version>
         <scala.version>2.11.8</scala.version>
       </properties>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2081bc87/integration/spark2/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark2/pom.xml b/integration/spark2/pom.xml
index 1eba780..b0d8bbe 100644
--- a/integration/spark2/pom.xml
+++ b/integration/spark2/pom.xml
@@ -283,6 +283,7 @@
                 <configuration>
                   <sources>
                     <source>src/main/spark2.1</source>
+                    <source>src/main/commonTo2.1And2.2</source>
                   </sources>
                 </configuration>
               </execution>
@@ -328,6 +329,7 @@
                   <sources>
                     <source>src/main/spark2.2</source>
                     <source>src/main/commonTo2.2And2.3</source>
+                    <source>src/main/commonTo2.1And2.2</source>
                   </sources>
                 </configuration>
               </execution>
@@ -339,7 +341,7 @@
     <profile>
       <id>spark-2.3</id>
       <properties>
-        <spark.version>2.3.1</spark.version>
+        <spark.version>2.3.2</spark.version>
         <scala.binary.version>2.11</scala.binary.version>
         <scala.version>2.11.8</scala.version>
       </properties>
@@ -352,6 +354,7 @@
               <excludes>
                 <exclude>src/main/spark2.1</exclude>
                 <exclude>src/main/spark2.2</exclude>
+                <exclude>src/main/commonTo2.1And2.2</exclude>
               </excludes>
             </configuration>
           </plugin>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2081bc87/integration/spark2/src/main/commonTo2.1And2.2/org/apache/spark/sql/execution/strategy/CarbonDataSourceScan.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/commonTo2.1And2.2/org/apache/spark/sql/execution/strategy/CarbonDataSourceScan.scala b/integration/spark2/src/main/commonTo2.1And2.2/org/apache/spark/sql/execution/strategy/CarbonDataSourceScan.scala
new file mode 100644
index 0000000..7605574
--- /dev/null
+++ b/integration/spark2/src/main/commonTo2.1And2.2/org/apache/spark/sql/execution/strategy/CarbonDataSourceScan.scala
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.execution.strategy
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.Partitioning
+import org.apache.spark.sql.execution.FileSourceScanExec
+import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation}
+
+/**
+ *  Physical plan node for scanning data. It is applied for both tables
+ *  USING carbondata and STORED AS CARBONDATA.
+ */
+class CarbonDataSourceScan(
+    override val output: Seq[Attribute],
+    val rdd: RDD[InternalRow],
+    @transient override val relation: HadoopFsRelation,
+    val partitioning: Partitioning,
+    override val metadata: Map[String, String],
+    identifier: Option[TableIdentifier],
+    @transient private val logicalRelation: LogicalRelation)
+  extends FileSourceScanExec(
+    relation,
+    output,
+    relation.dataSchema,
+    Seq.empty,
+    Seq.empty,
+    identifier) {
+
+  override val supportsBatch: Boolean = true
+
+  override val (outputPartitioning, outputOrdering): (Partitioning, Seq[SortOrder]) =
+    (partitioning, Nil)
+
+  override def inputRDDs(): Seq[RDD[InternalRow]] = rdd :: Nil
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2081bc87/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonDataSourceScan.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonDataSourceScan.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonDataSourceScan.scala
deleted file mode 100644
index 7605574..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonDataSourceScan.scala
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.spark.sql.execution.strategy
-
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
-import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder}
-import org.apache.spark.sql.catalyst.plans.physical.Partitioning
-import org.apache.spark.sql.execution.FileSourceScanExec
-import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation}
-
-/**
- *  Physical plan node for scanning data. It is applied for both tables
- *  USING carbondata and STORED AS CARBONDATA.
- */
-class CarbonDataSourceScan(
-    override val output: Seq[Attribute],
-    val rdd: RDD[InternalRow],
-    @transient override val relation: HadoopFsRelation,
-    val partitioning: Partitioning,
-    override val metadata: Map[String, String],
-    identifier: Option[TableIdentifier],
-    @transient private val logicalRelation: LogicalRelation)
-  extends FileSourceScanExec(
-    relation,
-    output,
-    relation.dataSchema,
-    Seq.empty,
-    Seq.empty,
-    identifier) {
-
-  override val supportsBatch: Boolean = true
-
-  override val (outputPartitioning, outputOrdering): (Partitioning, Seq[SortOrder]) =
-    (partitioning, Nil)
-
-  override def inputRDDs(): Seq[RDD[InternalRow]] = rdd :: Nil
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2081bc87/integration/spark2/src/main/spark2.3/org/apache/spark/sql/execution/strategy/CarbonDataSourceScan.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.3/org/apache/spark/sql/execution/strategy/CarbonDataSourceScan.scala b/integration/spark2/src/main/spark2.3/org/apache/spark/sql/execution/strategy/CarbonDataSourceScan.scala
new file mode 100644
index 0000000..5435f04
--- /dev/null
+++ b/integration/spark2/src/main/spark2.3/org/apache/spark/sql/execution/strategy/CarbonDataSourceScan.scala
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.execution.strategy
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder}
+import org.apache.spark.sql.catalyst.plans.physical.Partitioning
+import org.apache.spark.sql.execution.FileSourceScanExec
+import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation}
+
+/**
+ *  Physical plan node for scanning data. It is applied for both tables
+ *  USING carbondata and STORED AS CARBONDATA.
+ */
+class CarbonDataSourceScan(
+    override val output: Seq[Attribute],
+    val rdd: RDD[InternalRow],
+    @transient override val relation: HadoopFsRelation,
+    val partitioning: Partitioning,
+    val md: Map[String, String],
+    identifier: Option[TableIdentifier],
+    @transient private val logicalRelation: LogicalRelation)
+  extends FileSourceScanExec(
+    relation,
+    output,
+    relation.dataSchema,
+    Seq.empty,
+    Seq.empty,
+    identifier) {
+
+  // added lazy since spark 2.3.2 version (SPARK-PR#21815)
+  override lazy val supportsBatch: Boolean = true
+
+  // added lazy since spark 2.3.2 version (SPARK-PR#21815)
+  override lazy val (outputPartitioning, outputOrdering): (Partitioning, Seq[SortOrder]) =
+    (partitioning, Nil)
+
+  // added lazy since spark 2.3.2 version (SPARK-PR#21815)
+  override lazy val metadata: Map[String, String] = md
+
+  override def inputRDDs(): Seq[RDD[InternalRow]] = rdd :: Nil
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2081bc87/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 00a5287..7b1d487 100644
--- a/pom.xml
+++ b/pom.xml
@@ -522,6 +522,7 @@
                 <sourceDirectory>${basedir}/integration/spark2/src/main/scala</sourceDirectory>
                 <sourceDirectory>${basedir}/integration/spark2/src/main/java</sourceDirectory>
                 <sourceDirectory>${basedir}/integration/spark2/src/main/spark2.1</sourceDirectory>
+                <sourceDirectory>${basedir}/integration/spark2/src/main/commonTo2.1And2.2</sourceDirectory>
                 <sourceDirectory>${basedir}/integration/spark-common/src/main/scala</sourceDirectory>
                 <sourceDirectory>${basedir}/integration/spark-common/src/main/java</sourceDirectory>
                 <sourceDirectory>${basedir}/integration/spark-common-test/src/main/scala</sourceDirectory>
@@ -582,6 +583,7 @@
                 <sourceDirectory>${basedir}/hadoop/src/main/java</sourceDirectory>
                 <sourceDirectory>${basedir}/integration/spark2/src/main/scala</sourceDirectory>
                 <sourceDirectory>${basedir}/integration/spark2/src/main/spark2.2</sourceDirectory>
+                <sourceDirectory>${basedir}/integration/spark2/src/main/commonTo2.1And2.2</sourceDirectory>
                 <sourceDirectory>${basedir}/integration/spark2/src/main/commonTo2.2And2.3</sourceDirectory>
                 <sourceDirectory>${basedir}/integration/spark2/src/main/java</sourceDirectory>
                 <sourceDirectory>${basedir}/integration/spark-common/src/main/scala</sourceDirectory>
@@ -608,7 +610,7 @@
     <profile>
       <id>spark-2.3</id>
       <properties>
-        <spark.version>2.3.1</spark.version>
+        <spark.version>2.3.2</spark.version>
         <scala.binary.version>2.11</scala.binary.version>
         <scala.version>2.11.8</scala.version>
       </properties>


[38/45] carbondata git commit: [CARBONDATA-2985]Fix issues in Table level compaction and TableProperties

Posted by ra...@apache.org.
[CARBONDATA-2985]Fix issues in Table level compaction and TableProperties

Issue :-

If 2nd Level compaction is 1 like 2,1 or 6,1 then only 1st time compaction is done subsequent compaction are ignored . ( like if 2,1 is given then only 0.1 is segment is created and other segments are ignore forever )
Table level compaction does not support ,0 as 2nd level compaction value but system level compaction supports same.
Solution :-

if 2nd level compaction value is 1 then user does not want 2nd level compaction at all which mean 2nd level compaction can be set to 0.
remove check to support 2nd level compaction as 0 in table level.

This closes #2794


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

Branch: refs/heads/branch-1.5
Commit: 30adaa8c15e430b94bd1448969f50cb2451e1746
Parents: 396c26f
Author: BJangir <ba...@gmail.com>
Authored: Tue Oct 2 00:17:29 2018 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Oct 4 18:58:49 2018 +0530

----------------------------------------------------------------------
 .../TableLevelCompactionOptionTest.scala        | 84 ++++++++++++++++++++
 .../carbondata/spark/util/CommonUtil.scala      |  2 +-
 .../processing/merger/CarbonDataMergerUtil.java |  7 ++
 3 files changed, 92 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/30adaa8c/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/TableLevelCompactionOptionTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/TableLevelCompactionOptionTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/TableLevelCompactionOptionTest.scala
index 458d656..7b138f7 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/TableLevelCompactionOptionTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/TableLevelCompactionOptionTest.scala
@@ -271,4 +271,88 @@ class TableLevelCompactionOptionTest extends QueryTest
     assert(!segmentSequenceIds.contains("0.1"))
     assert(!segmentSequenceIds.contains("3.1"))
   }
+
+  test("AUTO MERGE TRUE:Verify 2nd Level compaction equals to 1"){
+    sql("DROP TABLE IF EXISTS tablecompaction_table")
+    sql(
+      """
+        |create table tablecompaction_table(
+        |name string,age int) stored by 'carbondata'
+        |tblproperties('AUTO_LOAD_MERGE'='true','COMPACTION_LEVEL_THRESHOLD'='2,1')
+      """.stripMargin)
+
+    for(i <-0 until 4){
+      sql("insert into tablecompaction_table select 'a',12")
+    }
+    var segments = sql("SHOW SEGMENTS FOR TABLE tablecompaction_table")
+    var segmentSequenceIds = segments.collect().map { each => (each.toSeq) (0) }
+    assert(segmentSequenceIds.size==6)
+    assert(segmentSequenceIds.contains("0.1"))
+    assert(segmentSequenceIds.contains("2.1"))
+  }
+
+  test("AUTO MERGE FALSE:Verify 2nd Level compaction equals to 1"){
+    sql("DROP TABLE IF EXISTS tablecompaction_table")
+    sql(
+      """
+        |create table tablecompaction_table(
+        |name string,age int) stored by 'carbondata'
+        |tblproperties('COMPACTION_LEVEL_THRESHOLD'='2,1')
+      """.stripMargin)
+
+    for(i <-0 until 4){
+      sql("insert into tablecompaction_table select 'a',12")
+    }
+    sql("alter table tablecompaction_table compact 'minor' ")
+    var segments = sql("SHOW SEGMENTS FOR TABLE tablecompaction_table")
+    var segmentSequenceIds = segments.collect().map { each => (each.toSeq) (0) }
+    assert(segmentSequenceIds.size==6)
+    assert(segmentSequenceIds.contains("0.1"))
+    assert(segmentSequenceIds.contains("2.1"))
+  }
+
+  // 2nd Level compaction value = 0 is supported by system level(like 6,0)
+  // same need to support for table level also
+  test("Verify 2nd Level compaction equals to 0"){
+    sql("DROP TABLE IF EXISTS tablecompaction_table")
+    sql(
+      """
+        |create table tablecompaction_table(
+        |name string,age int) stored by 'carbondata'
+        |tblproperties('AUTO_LOAD_MERGE'='true','COMPACTION_LEVEL_THRESHOLD'='2,0')
+      """.stripMargin)
+
+    for(i <-0 until 4){
+      sql("insert into tablecompaction_table select 'a',12")
+    }
+    var segments = sql("SHOW SEGMENTS FOR TABLE tablecompaction_table")
+    var segmentSequenceIds = segments.collect().map { each => (each.toSeq) (0) }
+    assert(segmentSequenceIds.size==6)
+    assert(segmentSequenceIds.contains("0.1"))
+    assert(segmentSequenceIds.contains("2.1"))
+  }
+
+  test("System Level:Verify 2nd Level compaction equals to 1"){
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.ENABLE_AUTO_LOAD_MERGE, "true")
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.COMPACTION_SEGMENT_LEVEL_THRESHOLD, "2,1")
+    sql("DROP TABLE IF EXISTS tablecompaction_table")
+    sql(
+      """
+        |create table tablecompaction_table(
+        |name string,age int) stored by 'carbondata'
+      """.stripMargin)
+
+    for(i <-0 until 4){
+      sql("insert into tablecompaction_table select 'a',12")
+    }
+    sql("alter table tablecompaction_table compact 'minor' ")
+    var segments = sql("SHOW SEGMENTS FOR TABLE tablecompaction_table")
+    var segmentSequenceIds = segments.collect().map { each => (each.toSeq) (0) }
+    assert(segmentSequenceIds.size==6)
+    assert(segmentSequenceIds.contains("0.1"))
+    assert(segmentSequenceIds.contains("2.1"))
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/30adaa8c/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
index c2f805d..49b17fb 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
@@ -455,7 +455,7 @@ object CommonUtil {
       try {
         val levels: Array[String] = regularedStr.split(",")
         val thresholds = regularedStr.split(",").map(levelThresholdStr => levelThresholdStr.toInt)
-        if (!thresholds.forall(t => t < 100 && t > 0)) {
+        if (!thresholds.forall(t => t < 100 && t >= 0)) {
           throw new MalformedCarbonCommandException(s"Invalid $tblPropName value found: " +
             s"$regularedStr, only int values separated by comma and between 0 " +
             s"and 100 are supported.")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/30adaa8c/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
index 2951283..5b001bf 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
@@ -744,6 +744,13 @@ public final class CarbonDataMergerUtil {
     if (size >= 2) {
       level1Size = noOfSegmentLevelsCount[0];
       level2Size = noOfSegmentLevelsCount[1];
+      /*
+      Ex. if segs => 0.1,2,3 and threshold =2,1
+      during 2nd time compaction,mergeCounter becomes 1 and we checks if mergeCounter==level2Size
+      then return mergedSegments which will return 0.1 and since only 1 segment(0.1) is identified ,
+      no segment would go for compaction .So change 2nd level threshold  to 0 if it is 1.
+       */
+      level2Size = level2Size == 1 ? 0 : level2Size;
     } else if (size == 1) {
       level1Size = noOfSegmentLevelsCount[0];
     }


[32/45] carbondata git commit: [CARBONDATA-2990] Fixed JVM crash when rebuilding bloom datamap

Posted by ra...@apache.org.
[CARBONDATA-2990] Fixed JVM crash when rebuilding bloom datamap

Problem: while rebuilding the datamap it access the datamap store so it builds datamap and store in unsafe onheap storage. But while closing the
reader it frees all memory acquired during that task. Since acquired memory is onheap but releasing the memory with offheap allocator it crashes the jvm.

Solution: Maintain the type of memory acquired in the memory block itself and get the allocator as per the memory type and release it.

This closes #2793


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

Branch: refs/heads/branch-1.5
Commit: 8fbd4a5f53070b3755f1f573b09e0066fa93a6ea
Parents: c3a8704
Author: ravipesala <ra...@gmail.com>
Authored: Sun Sep 30 11:27:57 2018 +0530
Committer: manishgupta88 <to...@gmail.com>
Committed: Thu Oct 4 14:39:25 2018 +0530

----------------------------------------------------------------------
 .../core/indexstore/UnsafeMemoryDMStore.java    | 14 +++----
 .../core/memory/HeapMemoryAllocator.java        |  5 ++-
 .../carbondata/core/memory/MemoryBlock.java     | 14 ++++++-
 .../carbondata/core/memory/MemoryType.java      | 23 ++++++++++
 .../core/memory/UnsafeMemoryAllocator.java      |  2 +-
 .../core/memory/UnsafeMemoryManager.java        | 44 +++++++++++---------
 6 files changed, 70 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/8fbd4a5f/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java b/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java
index 196559a..0db1b0a 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java
@@ -19,9 +19,9 @@ package org.apache.carbondata.core.indexstore;
 import org.apache.carbondata.core.indexstore.row.DataMapRow;
 import org.apache.carbondata.core.indexstore.row.UnsafeDataMapRow;
 import org.apache.carbondata.core.indexstore.schema.CarbonRowSchema;
-import org.apache.carbondata.core.memory.MemoryAllocator;
 import org.apache.carbondata.core.memory.MemoryBlock;
 import org.apache.carbondata.core.memory.MemoryException;
+import org.apache.carbondata.core.memory.MemoryType;
 import org.apache.carbondata.core.memory.UnsafeMemoryManager;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
@@ -51,7 +51,7 @@ public class UnsafeMemoryDMStore extends AbstractMemoryDMStore {
   public UnsafeMemoryDMStore() throws MemoryException {
     this.allocatedSize = capacity;
     this.memoryBlock =
-        UnsafeMemoryManager.allocateMemoryWithRetry(MemoryAllocator.HEAP, taskId, allocatedSize);
+        UnsafeMemoryManager.allocateMemoryWithRetry(MemoryType.ONHEAP, taskId, allocatedSize);
     this.pointers = new int[1000];
   }
 
@@ -74,10 +74,10 @@ public class UnsafeMemoryDMStore extends AbstractMemoryDMStore {
 
   private void increaseMemory(int requiredMemory) throws MemoryException {
     MemoryBlock newMemoryBlock = UnsafeMemoryManager
-        .allocateMemoryWithRetry(MemoryAllocator.HEAP, taskId, allocatedSize + requiredMemory);
+        .allocateMemoryWithRetry(MemoryType.ONHEAP, taskId, allocatedSize + requiredMemory);
     getUnsafe().copyMemory(this.memoryBlock.getBaseObject(), this.memoryBlock.getBaseOffset(),
         newMemoryBlock.getBaseObject(), newMemoryBlock.getBaseOffset(), runningLength);
-    UnsafeMemoryManager.INSTANCE.freeMemory(MemoryAllocator.HEAP, taskId, this.memoryBlock);
+    UnsafeMemoryManager.INSTANCE.freeMemory(taskId, this.memoryBlock);
     allocatedSize = allocatedSize + requiredMemory;
     this.memoryBlock = newMemoryBlock;
   }
@@ -190,10 +190,10 @@ public class UnsafeMemoryDMStore extends AbstractMemoryDMStore {
   public void finishWriting() throws MemoryException {
     if (runningLength < allocatedSize) {
       MemoryBlock allocate =
-          UnsafeMemoryManager.allocateMemoryWithRetry(MemoryAllocator.HEAP, taskId, runningLength);
+          UnsafeMemoryManager.allocateMemoryWithRetry(MemoryType.ONHEAP, taskId, runningLength);
       getUnsafe().copyMemory(memoryBlock.getBaseObject(), memoryBlock.getBaseOffset(),
           allocate.getBaseObject(), allocate.getBaseOffset(), runningLength);
-      UnsafeMemoryManager.INSTANCE.freeMemory(MemoryAllocator.HEAP, taskId, memoryBlock);
+      UnsafeMemoryManager.INSTANCE.freeMemory(taskId, memoryBlock);
       memoryBlock = allocate;
     }
     // Compact pointers.
@@ -206,7 +206,7 @@ public class UnsafeMemoryDMStore extends AbstractMemoryDMStore {
 
   public void freeMemory() {
     if (!isMemoryFreed) {
-      UnsafeMemoryManager.INSTANCE.freeMemory(MemoryAllocator.HEAP, taskId, memoryBlock);
+      UnsafeMemoryManager.INSTANCE.freeMemory(taskId, memoryBlock);
       isMemoryFreed = true;
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/8fbd4a5f/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java b/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
index d08f803..58162da 100644
--- a/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
+++ b/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
@@ -71,7 +71,8 @@ public class HeapMemoryAllocator implements MemoryAllocator {
             final long[] array = arrayReference.get();
             if (array != null) {
               assert (array.length * 8L >= size);
-              MemoryBlock memory = new MemoryBlock(array, CarbonUnsafe.LONG_ARRAY_OFFSET, size);
+              MemoryBlock memory =
+                  new MemoryBlock(array, CarbonUnsafe.LONG_ARRAY_OFFSET, size, MemoryType.ONHEAP);
               // reuse this MemoryBlock
               memory.setFreedStatus(false);
               return memory;
@@ -82,7 +83,7 @@ public class HeapMemoryAllocator implements MemoryAllocator {
       }
     }
     long[] array = new long[numWords];
-    return new MemoryBlock(array, CarbonUnsafe.LONG_ARRAY_OFFSET, size);
+    return new MemoryBlock(array, CarbonUnsafe.LONG_ARRAY_OFFSET, size, MemoryType.ONHEAP);
   }
 
   @Override public void free(MemoryBlock memory) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/8fbd4a5f/core/src/main/java/org/apache/carbondata/core/memory/MemoryBlock.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/memory/MemoryBlock.java b/core/src/main/java/org/apache/carbondata/core/memory/MemoryBlock.java
index 418ef89..87ae982 100644
--- a/core/src/main/java/org/apache/carbondata/core/memory/MemoryBlock.java
+++ b/core/src/main/java/org/apache/carbondata/core/memory/MemoryBlock.java
@@ -31,12 +31,18 @@ public class MemoryBlock extends MemoryLocation {
   /**
    * whether freed or not
    */
-  private boolean isFreed = false;
+  private boolean isFreed;
 
-  public MemoryBlock(@Nullable Object obj, long offset, long length) {
+  /**
+   * Whether it is offheap or onheap memory type
+   */
+  private MemoryType memoryType;
+
+  public MemoryBlock(@Nullable Object obj, long offset, long length, MemoryType memoryType) {
     super(obj, offset);
     this.length = length;
     this.isFreed = false;
+    this.memoryType = memoryType;
   }
 
   /**
@@ -53,4 +59,8 @@ public class MemoryBlock extends MemoryLocation {
   public void setFreedStatus(boolean freedStatus) {
     this.isFreed = freedStatus;
   }
+
+  public MemoryType getMemoryType() {
+    return memoryType;
+  }
 }

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

http://git-wip-us.apache.org/repos/asf/carbondata/blob/8fbd4a5f/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryAllocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryAllocator.java b/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryAllocator.java
index 67412ac..e596895 100644
--- a/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryAllocator.java
+++ b/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryAllocator.java
@@ -28,7 +28,7 @@ public class UnsafeMemoryAllocator implements MemoryAllocator {
     long address = CarbonUnsafe.getUnsafe().allocateMemory(size);
     // initializing memory with zero
     CarbonUnsafe.getUnsafe().setMemory(null, address, size, (byte) 0);
-    return new MemoryBlock(null, address, size);
+    return new MemoryBlock(null, address, size, MemoryType.OFFHEAP);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/carbondata/blob/8fbd4a5f/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java b/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java
index 703d57a..4efea1a 100644
--- a/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java
@@ -68,9 +68,9 @@ public class UnsafeMemoryManager {
       LOGGER.info("Invalid memory size value: " + defaultWorkingMemorySize);
     }
     long takenSize = size;
-    MemoryAllocator allocator;
+    MemoryType memoryType;
     if (offHeap) {
-      allocator = MemoryAllocator.UNSAFE;
+      memoryType = MemoryType.OFFHEAP;
       long defaultSize = Long.parseLong(CarbonCommonConstants.UNSAFE_WORKING_MEMORY_IN_MB_DEFAULT);
       if (takenSize < defaultSize) {
         takenSize = defaultSize;
@@ -86,9 +86,9 @@ public class UnsafeMemoryManager {
           takenSize = maxMemory;
         }
       }
-      allocator = MemoryAllocator.HEAP;
+      memoryType = MemoryType.ONHEAP;
     }
-    INSTANCE = new UnsafeMemoryManager(takenSize, allocator);
+    INSTANCE = new UnsafeMemoryManager(takenSize, memoryType);
     taskIdToMemoryBlockMap = new HashMap<>();
   }
 
@@ -98,19 +98,19 @@ public class UnsafeMemoryManager {
 
   private long memoryUsed;
 
-  private MemoryAllocator allocator;
+  private MemoryType memoryType;
 
-  private UnsafeMemoryManager(long totalMemory, MemoryAllocator allocator) {
+  private UnsafeMemoryManager(long totalMemory, MemoryType memoryType) {
     this.totalMemory = totalMemory;
-    this.allocator = allocator;
+    this.memoryType = memoryType;
     LOGGER
-        .info("Working Memory manager is created with size " + totalMemory + " with " + allocator);
+        .info("Working Memory manager is created with size " + totalMemory + " with " + memoryType);
   }
 
-  private synchronized MemoryBlock allocateMemory(MemoryAllocator memoryAllocator, long taskId,
+  private synchronized MemoryBlock allocateMemory(MemoryType memoryType, long taskId,
       long memoryRequested) {
     if (memoryUsed + memoryRequested <= totalMemory) {
-      MemoryBlock allocate = memoryAllocator.allocate(memoryRequested);
+      MemoryBlock allocate = getMemoryAllocator(memoryType).allocate(memoryRequested);
       memoryUsed += allocate.size();
       Set<MemoryBlock> listOfMemoryBlock = taskIdToMemoryBlockMap.get(taskId);
       if (null == listOfMemoryBlock) {
@@ -129,16 +129,11 @@ public class UnsafeMemoryManager {
   }
 
   public synchronized void freeMemory(long taskId, MemoryBlock memoryBlock) {
-    freeMemory(allocator, taskId, memoryBlock);
-  }
-
-  public synchronized void freeMemory(MemoryAllocator memoryAllocator, long taskId,
-      MemoryBlock memoryBlock) {
     if (taskIdToMemoryBlockMap.containsKey(taskId)) {
       taskIdToMemoryBlockMap.get(taskId).remove(memoryBlock);
     }
     if (!memoryBlock.isFreedStatus()) {
-      memoryAllocator.free(memoryBlock);
+      getMemoryAllocator(memoryBlock.getMemoryType()).free(memoryBlock);
       memoryUsed -= memoryBlock.size();
       memoryUsed = memoryUsed < 0 ? 0 : memoryUsed;
       if (LOGGER.isDebugEnabled()) {
@@ -160,7 +155,7 @@ public class UnsafeMemoryManager {
         memoryBlock = iterator.next();
         if (!memoryBlock.isFreedStatus()) {
           occuppiedMemory += memoryBlock.size();
-          allocator.free(memoryBlock);
+          getMemoryAllocator(memoryBlock.getMemoryType()).free(memoryBlock);
         }
       }
     }
@@ -188,15 +183,15 @@ public class UnsafeMemoryManager {
    */
   public static MemoryBlock allocateMemoryWithRetry(long taskId, long size)
       throws MemoryException {
-    return allocateMemoryWithRetry(INSTANCE.allocator, taskId, size);
+    return allocateMemoryWithRetry(INSTANCE.memoryType, taskId, size);
   }
 
-  public static MemoryBlock allocateMemoryWithRetry(MemoryAllocator memoryAllocator, long taskId,
+  public static MemoryBlock allocateMemoryWithRetry(MemoryType memoryType, long taskId,
       long size) throws MemoryException {
     MemoryBlock baseBlock = null;
     int tries = 0;
     while (tries < 300) {
-      baseBlock = INSTANCE.allocateMemory(memoryAllocator, taskId, size);
+      baseBlock = INSTANCE.allocateMemory(memoryType, taskId, size);
       if (baseBlock == null) {
         try {
           LOGGER.info("Memory is not available, retry after 500 millis");
@@ -217,6 +212,15 @@ public class UnsafeMemoryManager {
     return baseBlock;
   }
 
+  private MemoryAllocator getMemoryAllocator(MemoryType memoryType) {
+    switch (memoryType) {
+      case ONHEAP:
+        return MemoryAllocator.HEAP;
+      default:
+        return MemoryAllocator.UNSAFE;
+    }
+  }
+
   public static boolean isOffHeap() {
     return offHeap;
   }


[28/45] carbondata git commit: [CARBONDATA-2987] Data mismatch after compaction with measure sort columns

Posted by ra...@apache.org.
[CARBONDATA-2987] Data mismatch after compaction with measure sort columns

problem: Data mismatch after compaction with measure sort columns

root cause : In compaction flow (DictionaryBasedResultCollector), in ColumnPageWrapper inverted index mapping is not handled. Because of this row ID was wrong, row of no dictionary dimension columns gets data from other rows.
Hence the data mismatch.

solution: Handle inverted index mapping for DictionaryBasedResultCollector flow in ColumnPageWrapper

This closes #2784


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

Branch: refs/heads/branch-1.5
Commit: 0b16816da7b401318929bfe973dad4bf397e90d9
Parents: 6ef4e46
Author: ajantha-bhat <aj...@gmail.com>
Authored: Fri Sep 28 16:27:55 2018 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Wed Oct 3 20:09:13 2018 +0530

----------------------------------------------------------------------
 .../chunk/store/ColumnPageWrapper.java          | 52 +++++++----
 .../compaction/nodictionary_compaction.csv      |  3 +
 .../MajorCompactionWithMeasureSortColumns.scala | 97 ++++++++++++++++++++
 3 files changed, 136 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/0b16816d/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java
index 098287e..627c75f 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java
@@ -122,10 +122,19 @@ public class ColumnPageWrapper implements DimensionColumnPage {
   }
 
   @Override public byte[] getChunkData(int rowId) {
-    return getChunkData(rowId, false);
+    byte[] nullBitSet = getNullBitSet(rowId, columnPage.getColumnSpec().getColumnType());
+    if (nullBitSet != null) {
+      // if this row is null, return default null represent in byte array
+      return nullBitSet;
+    } else {
+      if (isExplicitSorted()) {
+        rowId = getInvertedReverseIndex(rowId);
+      }
+      return getChunkDataInBytes(rowId);
+    }
   }
 
-  private byte[] getChunkData(int rowId, boolean isRowIdChanged) {
+  private byte[] getChunkDataInBytes(int rowId) {
     ColumnType columnType = columnPage.getColumnSpec().getColumnType();
     DataType srcDataType = columnPage.getColumnSpec().getSchemaDataType();
     DataType targetDataType = columnPage.getDataType();
@@ -134,15 +143,6 @@ public class ColumnPageWrapper implements DimensionColumnPage {
           .getDictionaryValue(CarbonUtil.getSurrogateInternal(columnPage.getBytes(rowId), 0, 3));
     } else if ((columnType == ColumnType.COMPLEX_PRIMITIVE && isAdaptiveEncoded()) || (
         columnType == ColumnType.PLAIN_VALUE && DataTypeUtil.isPrimitiveColumn(srcDataType))) {
-      if (!isRowIdChanged && columnPage.getNullBits().get(rowId)
-          && columnType == ColumnType.COMPLEX_PRIMITIVE) {
-        // if this row is null, return default null represent in byte array
-        return CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY;
-      }
-      if (!isRowIdChanged && columnPage.getNullBits().get(rowId)) {
-        // if this row is null, return default null represent in byte array
-        return CarbonCommonConstants.EMPTY_BYTE_ARRAY;
-      }
       if (srcDataType == DataTypes.FLOAT) {
         float floatData = columnPage.getFloat(rowId);
         return ByteUtil.toXorBytes(floatData);
@@ -182,9 +182,6 @@ public class ColumnPageWrapper implements DimensionColumnPage {
         throw new RuntimeException("unsupported type: " + targetDataType);
       }
     } else if ((columnType == ColumnType.COMPLEX_PRIMITIVE && !isAdaptiveEncoded())) {
-      if (!isRowIdChanged && columnPage.getNullBits().get(rowId)) {
-        return CarbonCommonConstants.EMPTY_BYTE_ARRAY;
-      }
       if ((srcDataType == DataTypes.BYTE) || (srcDataType == DataTypes.BOOLEAN)) {
         byte[] out = new byte[1];
         out[0] = (columnPage.getByte(rowId));
@@ -205,6 +202,18 @@ public class ColumnPageWrapper implements DimensionColumnPage {
     }
   }
 
+  private byte[] getNullBitSet(int rowId, ColumnType columnType) {
+    if (columnPage.getNullBits().get(rowId) && columnType == ColumnType.COMPLEX_PRIMITIVE) {
+      // if this row is null, return default null represent in byte array
+      return CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY;
+    }
+    if (columnPage.getNullBits().get(rowId)) {
+      // if this row is null, return default null represent in byte array
+      return CarbonCommonConstants.EMPTY_BYTE_ARRAY;
+    }
+    return null;
+  }
+
   private Object getActualData(int rowId, boolean isRowIdChanged) {
     ColumnType columnType = columnPage.getColumnSpec().getColumnType();
     DataType srcDataType = columnPage.getColumnSpec().getSchemaDataType();
@@ -302,8 +311,19 @@ public class ColumnPageWrapper implements DimensionColumnPage {
 
   @Override
   public int compareTo(int rowId, byte[] compareValue) {
-    byte[] chunkData = this.getChunkData((int) rowId);
-    return ByteUtil.UnsafeComparer.INSTANCE.compareTo(chunkData, compareValue);
+    // rowId is the inverted index, but the null bitset is based on actual data
+    int nullBitSetRowId = rowId;
+    if (isExplicitSorted()) {
+      nullBitSetRowId = getInvertedReverseIndex(rowId);
+    }
+    byte[] nullBitSet = getNullBitSet(nullBitSetRowId, columnPage.getColumnSpec().getColumnType());
+    if (nullBitSet != null) {
+      // if this row is null, return default null represent in byte array
+      return ByteUtil.UnsafeComparer.INSTANCE.compareTo(nullBitSet, compareValue);
+    } else {
+      byte[] chunkData = this.getChunkDataInBytes(rowId);
+      return ByteUtil.UnsafeComparer.INSTANCE.compareTo(chunkData, compareValue);
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0b16816d/integration/spark-common-test/src/test/resources/compaction/nodictionary_compaction.csv
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/resources/compaction/nodictionary_compaction.csv b/integration/spark-common-test/src/test/resources/compaction/nodictionary_compaction.csv
new file mode 100644
index 0000000..2518fd2
--- /dev/null
+++ b/integration/spark-common-test/src/test/resources/compaction/nodictionary_compaction.csv
@@ -0,0 +1,3 @@
+code1,code2,country_code,category_id,product_id,date,count1,count2,count3
+"51job, Inc.",21695-534,FR,610,60,2017-11-27,4483,0,510
+Intercontinental Exchange Inc.,22100-020,TH,87,4,2017-10-16,2,647,69630

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0b16816d/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionWithMeasureSortColumns.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionWithMeasureSortColumns.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionWithMeasureSortColumns.scala
new file mode 100644
index 0000000..ff56619
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionWithMeasureSortColumns.scala
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.datacompaction
+
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+
+
+class MajorCompactionWithMeasureSortColumns extends QueryTest with BeforeAndAfterAll {
+
+  val csvFilePath = s"$resourcesPath/compaction/nodictionary_compaction.csv"
+  val backupDateFormat = CarbonProperties.getInstance()
+    .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
+      CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT)
+
+  override def beforeAll: Unit = {
+    sql("drop table if exists store")
+
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
+        CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT)
+  }
+
+  override def afterAll {
+    sql("drop table if exists  store")
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT, backupDateFormat)
+  }
+
+  test("test major compaction with measure sort columns") {
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_MAJOR_COMPACTION_SIZE, "1024")
+
+    val createStoreTableSql =
+      s"""
+         | CREATE TABLE IF NOT EXISTS store(
+         | code1 STRING,
+         | code2 STRING,
+         | country_code STRING,
+         | category_id INTEGER,
+         | product_id LONG,
+         | date DATE,
+         | count1 LONG,
+         | count2 LONG,
+         | count3 LONG
+         | )
+         | STORED BY 'carbondata'
+         | TBLPROPERTIES(
+         | 'SORT_COLUMNS'='code1, code2, country_code, date, category_id, product_id',
+         | 'SORT_SCOPE'='LOCAL_SORT',
+         | 'CACHE_LEVEL'='BLOCKLET'
+         | )
+      """.stripMargin
+    sql(createStoreTableSql)
+
+    sql(
+      s"""
+         | LOAD DATA LOCAL INPATH '$csvFilePath'
+         | INTO TABLE store
+         | OPTIONS('HEADER'='true', 'COMPLEX_DELIMITER_LEVEL_1'='#')
+       """.stripMargin).show(false)
+
+    sql(
+      s"""
+         | LOAD DATA LOCAL INPATH '$csvFilePath'
+         | INTO TABLE store
+         | OPTIONS('HEADER'='true', 'COMPLEX_DELIMITER_LEVEL_1'='#')
+       """.stripMargin).show(false)
+
+    val csvRows = sqlContext.sparkSession.read.option("header", "true")
+      .csv(csvFilePath).orderBy("code1")
+
+    sql("ALTER TABLE store COMPACT 'MAJOR'")
+
+    val answer = sql("select * from store ").orderBy("code1")
+    assert(answer.except(csvRows).count() == 0)
+    sql("drop table store")
+  }
+
+}


[36/45] carbondata git commit: [CARBONDATA-2986] Table Properties are lost when multiple driver concurrently

Posted by ra...@apache.org.
[CARBONDATA-2986] Table Properties are lost when multiple driver concurrently

Issue :- When concurrently multiple driver is creating table , for same table table properties are lost .

Root Cause :-Schema file is getting overwritten from CarbonRelation#createTableIfNotExists,because lookup of table is failed . this is happpened because concurrenly .mdt file is updated and current table is removed from cache org.apache.spark.sql.hive.CarbonFileMetastore#checkSchemasModifiedTimeAndReloadTable

Solution :- Since carbon table is already created and Schema file is already written so no need to do lookup again .

This closes #2785


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

Branch: refs/heads/branch-1.5
Commit: 3edea12a83e70dddb1eca271bf5660f73de272f5
Parents: 11bd0ad
Author: BJangir <ba...@gmail.com>
Authored: Fri Sep 28 17:17:30 2018 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Oct 4 18:05:06 2018 +0530

----------------------------------------------------------------------
 .../scala/org/apache/spark/sql/CarbonSource.scala  | 17 ++++++++++++++---
 1 file changed, 14 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/3edea12a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
index 16cee96..cd1087d 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
@@ -57,6 +57,7 @@ class CarbonSource extends CreatableRelationProvider with RelationProvider
   with SchemaRelationProvider with StreamSinkProvider with DataSourceRegister {
 
   override def shortName(): String = "carbondata"
+  private val LOGGER = LogServiceFactory.getLogService(CarbonSource.getClass.getName)
 
   // will be called if hive supported create table command is provided
   override def createRelation(sqlContext: SQLContext,
@@ -143,7 +144,7 @@ class CarbonSource extends CreatableRelationProvider with RelationProvider
       .exists(_.table.equalsIgnoreCase(tableName))) {
         getPathForTable(sqlContext.sparkSession, dbName, tableName, newParameters)
     } else {
-        createTableIfNotExists(sqlContext.sparkSession, newParameters, dataSchema)
+      createTableIfNotExists(sqlContext.sparkSession, dbName, tableName, newParameters, dataSchema)
     }
 
     CarbonDatasourceHadoopRelation(sqlContext.sparkSession, Array(path), updatedParams,
@@ -160,6 +161,8 @@ class CarbonSource extends CreatableRelationProvider with RelationProvider
 
   private def createTableIfNotExists(
       sparkSession: SparkSession,
+      dbName: String,
+      tableName: String,
       parameters: Map[String, String],
       dataSchema: StructType): (String, Map[String, String]) = {
 
@@ -167,10 +170,18 @@ class CarbonSource extends CreatableRelationProvider with RelationProvider
     val tableName: String = parameters.getOrElse("tableName", "").toLowerCase
 
     try {
-      val carbonTable = CarbonEnv.getCarbonTable(Some(dbName), tableName)(sparkSession)
-      (carbonTable.getTablePath, parameters)
+      if (!(parameters.contains("carbonSchemaPartsNo")
+        || parameters.contains("carbonschemapartsno"))) {
+        val carbonTable = CarbonEnv.getCarbonTable(Some(dbName), tableName)(sparkSession)
+        (carbonTable.getTablePath, parameters)
+      } else {
+        (getPathForTable(sparkSession, dbName, tableName, parameters))
+      }
+
     } catch {
       case _: NoSuchTableException =>
+        LOGGER.warn("Carbon Table [" +dbName +"] [" +tableName +"] is not found, " +
+          "Now existing Schema will be overwritten with default properties")
         val metaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
         val identifier = AbsoluteTableIdentifier.from(
           CarbonEnv.getTablePath(Some(dbName), tableName)(sparkSession),


[19/45] carbondata git commit: [CARBONDATA-2978] Fixed JVM crash issue when insert into carbon table from other carbon table

Posted by ra...@apache.org.
[CARBONDATA-2978] Fixed JVM crash issue when insert into carbon table from other carbon table

Problem:
When data is inserted from one carbon to other carbon table and unsafe load and query is enabled then JVM crash is happening.
Reason: When insert happens from one carbon table another table it uses same task and thread so it
gets the same taskid and at the unsafe manager tries to release all memory acquired by the task even though load happens on the task.

Solution:
Check the listeners and ignore cache clearing.

This closes #2773


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

Branch: refs/heads/branch-1.5
Commit: 9ae91cc5a9d683ef54550cfe7e65c4d63d5e5a24
Parents: c016361
Author: ravipesala <ra...@gmail.com>
Authored: Wed Sep 26 23:04:59 2018 +0530
Committer: kumarvishal09 <ku...@gmail.com>
Committed: Fri Sep 28 19:51:06 2018 +0530

----------------------------------------------------------------------
 .../hadoop/api/CarbonTableOutputFormat.java     | 35 +++++----
 .../InsertIntoNonCarbonTableTestCase.scala      | 79 +++++++++++++++++++-
 .../carbondata/spark/rdd/CarbonScanRDD.scala    | 76 ++++++++++++-------
 .../rdd/InsertTaskCompletionListener.scala      |  4 +-
 .../spark/rdd/QueryTaskCompletionListener.scala |  4 +-
 .../datasources/SparkCarbonFileFormat.scala     | 23 +++++-
 .../CarbonTaskCompletionListener.scala          | 72 ++++++++++++++++++
 7 files changed, 246 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/9ae91cc5/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableOutputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableOutputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableOutputFormat.java
index 28817e9..762983b 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableOutputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableOutputFormat.java
@@ -424,6 +424,8 @@ public class CarbonTableOutputFormat extends FileOutputFormat<NullWritable, Obje
 
     private Future future;
 
+    private boolean isClosed;
+
     public CarbonRecordWriter(CarbonOutputIteratorWrapper iteratorWrapper,
         DataLoadExecutor dataLoadExecutor, CarbonLoadModel loadModel, Future future,
         ExecutorService executorService) {
@@ -442,22 +444,25 @@ public class CarbonTableOutputFormat extends FileOutputFormat<NullWritable, Obje
     }
 
     @Override public void close(TaskAttemptContext taskAttemptContext) throws InterruptedException {
-      if (iteratorWrapper != null) {
-        iteratorWrapper.closeWriter(false);
-      }
-      try {
-        future.get();
-      } catch (ExecutionException e) {
-        LOG.error("Error while loading data", e);
-        throw new InterruptedException(e.getMessage());
-      } finally {
-        executorService.shutdownNow();
-        dataLoadExecutor.close();
-        ThreadLocalSessionInfo.unsetAll();
-        // clean up the folders and files created locally for data load operation
-        TableProcessingOperations.deleteLocalDataLoadFolderLocation(loadModel, false, false);
+      if (!isClosed) {
+        isClosed = true;
+        if (iteratorWrapper != null) {
+          iteratorWrapper.closeWriter(false);
+        }
+        try {
+          future.get();
+        } catch (ExecutionException e) {
+          LOG.error("Error while loading data", e);
+          throw new InterruptedException(e.getMessage());
+        } finally {
+          executorService.shutdownNow();
+          dataLoadExecutor.close();
+          ThreadLocalSessionInfo.unsetAll();
+          // clean up the folders and files created locally for data load operation
+          TableProcessingOperations.deleteLocalDataLoadFolderLocation(loadModel, false, false);
+        }
+        LOG.info("Closed writer task " + taskAttemptContext.getTaskAttemptID());
       }
-      LOG.info("Closed writer task " + taskAttemptContext.getTaskAttemptID());
     }
 
     public CarbonLoadModel getLoadModel() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/9ae91cc5/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/insertQuery/InsertIntoNonCarbonTableTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/insertQuery/InsertIntoNonCarbonTableTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/insertQuery/InsertIntoNonCarbonTableTestCase.scala
index a745672..a3fb11c 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/insertQuery/InsertIntoNonCarbonTableTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/insertQuery/InsertIntoNonCarbonTableTestCase.scala
@@ -18,10 +18,13 @@
  */
 package org.apache.carbondata.spark.testsuite.insertQuery
 
-import org.apache.spark.sql.Row
+import org.apache.spark.sql.{Row, SaveMode}
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+
 
 class InsertIntoNonCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
   override def beforeAll {
@@ -64,6 +67,8 @@ class InsertIntoNonCarbonTableTestCase extends QueryTest with BeforeAndAfterAll
       "Latest_webTypeDataVerNumber,Latest_operatorsVersion,Latest_phonePADPartitionedVersions," +
       "Latest_operatorId,gamePointDescription,gamePointId,contractNumber', " +
       "'bad_records_logger_enable'='false','bad_records_action'='FORCE')")
+    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.ENABLE_UNSAFE_IN_QUERY_EXECUTION, "true")
+    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.ENABLE_UNSAFE_COLUMN_PAGE, "true")
   }
 
   test("insert into hive") {
@@ -102,7 +107,79 @@ class InsertIntoNonCarbonTableTestCase extends QueryTest with BeforeAndAfterAll
     sql("drop table thive_cond")
   }
 
+  test("jvm crash when insert data from datasource table to session table") {
+    val spark = sqlContext.sparkSession
+    import spark.implicits._
+
+    import scala.util.Random
+    val r = new Random()
+    val df = spark.sparkContext.parallelize(1 to 10)
+      .map(x => (r.nextInt(100000), "name" + x % 8, "city" + x % 50, BigDecimal.apply(x % 60)))
+      .toDF("ID", "name", "city", "age")
+    spark.sql("DROP TABLE IF EXISTS personTable")
+    spark.sql("DROP TABLE IF EXISTS test_table")
+
+    df.write.format("carbon").saveAsTable("personTable")
+    spark.sql("create table test_table(ID int, name string, city string, age decimal) stored by 'carbondata' tblproperties('sort_columns'='ID')")
+    spark.sql("insert into test_table select * from personTable")
+    spark.sql("insert into test_table select * from personTable limit 2")
+
+    assert(spark.sql("select * from test_table").count() == 12)
+    spark.sql("DROP TABLE IF EXISTS personTable")
+    spark.sql("DROP TABLE IF EXISTS test_table")
+  }
+
+  test("jvm crash when insert data from datasource table to datasource table") {
+    val spark = sqlContext.sparkSession
+    import spark.implicits._
+
+    import scala.util.Random
+    val r = new Random()
+    val df = spark.sparkContext.parallelize(1 to 10)
+      .map(x => (r.nextInt(100000), "name" + x % 8, "city" + x % 50, BigDecimal.apply(x % 60)))
+      .toDF("ID", "name", "city", "age")
+    spark.sql("DROP TABLE IF EXISTS personTable")
+    spark.sql("DROP TABLE IF EXISTS test_table")
+
+    df.write.format("carbon").saveAsTable("personTable")
+    spark.sql("create table test_table(ID int, name string, city string, age decimal) using carbon")
+    spark.sql("insert into test_table select * from personTable")
+    spark.sql("insert into test_table select * from personTable limit 2")
+
+    assert(spark.sql("select * from test_table").count() == 12)
+    spark.sql("DROP TABLE IF EXISTS personTable")
+    spark.sql("DROP TABLE IF EXISTS test_table")
+  }
+
+  test("jvm crash when insert data from session table to datasource table") {
+    val spark = sqlContext.sparkSession
+    import spark.implicits._
+
+    import scala.util.Random
+    val r = new Random()
+    val df = spark.sparkContext.parallelize(1 to 10)
+      .map(x => (r.nextInt(100000), "name" + x % 8, "city" + x % 50, BigDecimal.apply(x % 60)))
+      .toDF("ID", "name", "city", "age")
+    spark.sql("DROP TABLE IF EXISTS personTable")
+    spark.sql("DROP TABLE IF EXISTS test_table")
+
+    df.write
+      .format("carbondata")
+      .option("tableName", "personTable")
+      .mode(SaveMode.Overwrite)
+      .save()
+    spark.sql("create table test_table(ID int, name string, city string, age decimal) using carbon")
+    spark.sql("insert into test_table select * from personTable")
+    spark.sql("insert into test_table select * from personTable limit 2")
+
+    assert(spark.sql("select * from test_table").count() == 12)
+    spark.sql("DROP TABLE IF EXISTS personTable")
+    spark.sql("DROP TABLE IF EXISTS test_table")
+  }
+
   override def afterAll {
+    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.ENABLE_UNSAFE_IN_QUERY_EXECUTION, CarbonCommonConstants.ENABLE_UNSAFE_IN_QUERY_EXECUTION_DEFAULTVALUE)
+    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.ENABLE_UNSAFE_COLUMN_PAGE, CarbonCommonConstants.ENABLE_UNSAFE_COLUMN_PAGE_DEFAULT)
     sql("DROP TABLE IF EXISTS TCarbonSource")
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/9ae91cc5/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
index eb7abbc..1a7eae2 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
@@ -35,6 +35,7 @@ import org.apache.spark._
 import org.apache.spark.deploy.SparkHadoopUtil
 import org.apache.spark.sql.hive.DistributionUtil
 import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.carbondata.execution.datasources.tasklisteners.CarbonLoadTaskCompletionListener
 import org.apache.spark.sql.execution.SQLExecution
 import org.apache.spark.sql.profiler.{GetPartition, Profiler, QueryTaskEnd}
 import org.apache.spark.sql.util.SparkSQLUtil.sessionState
@@ -470,39 +471,28 @@ class CarbonScanRDD[T: ClassTag](
       val recorder = CarbonTimeStatisticsFactory.createExecutorRecorder(model.getQueryId())
       model.setStatisticsRecorder(recorder)
 
-      // TODO: rewrite this logic to call free memory in FailureListener on failures. On success,
-      // TODO: no memory leak should be there, resources should be freed on success completion.
-      val onCompleteCallbacksField = context.getClass.getDeclaredField("onCompleteCallbacks")
-      onCompleteCallbacksField.setAccessible(true)
-      val listeners = onCompleteCallbacksField.get(context)
-        .asInstanceOf[ArrayBuffer[TaskCompletionListener]]
-
-      val isAdded = listeners.exists(p => p.isInstanceOf[InsertTaskCompletionListener])
-      model.setFreeUnsafeMemory(!isAdded)
-      // add task completion before calling initialize as initialize method will internally call
-      // for usage of unsafe method for processing of one blocklet and if there is any exception
-      // while doing that the unsafe memory occupied for that task will not get cleared
-      context.addTaskCompletionListener { new QueryTaskCompletionListener(!isAdded,
-        reader,
-        inputMetricsStats,
-        executionId,
-        taskId,
-        queryStartTime,
-        model.getStatisticsRecorder,
-        split,
-        queryId)
-      }
-      // initialize the reader
-      reader.initialize(inputSplit, attemptContext)
-
       new Iterator[Any] {
         private var havePair = false
         private var finished = false
+        private var first = true
 
         override def hasNext: Boolean = {
           if (context.isInterrupted) {
             throw new TaskKilledException
           }
+          if (first) {
+            first = false
+            addTaskCompletionListener(
+              split,
+              context,
+              queryStartTime,
+              executionId,
+              taskId,
+              model,
+              reader)
+            // initialize the reader
+            reader.initialize(inputSplit, attemptContext)
+          }
           if (!finished && !havePair) {
             finished = !reader.nextKeyValue
             havePair = !finished
@@ -534,6 +524,42 @@ class CarbonScanRDD[T: ClassTag](
     iterator.asInstanceOf[Iterator[T]]
   }
 
+  private def addTaskCompletionListener(split: Partition,
+      context: TaskContext,
+      queryStartTime: Long,
+      executionId: String,
+      taskId: Int,
+      model: QueryModel,
+      reader: RecordReader[Void, Object]) = {
+    // TODO: rewrite this logic to call free memory in FailureListener on failures and
+    // On success,
+    // TODO: no memory leak should be there, resources should be freed on
+    // success completion.
+    val onCompleteCallbacksField =
+    context.getClass.getDeclaredField("onCompleteCallbacks")
+    onCompleteCallbacksField.setAccessible(true)
+    val listeners = onCompleteCallbacksField.get(context)
+      .asInstanceOf[ArrayBuffer[TaskCompletionListener]]
+
+    val isAdded = listeners.exists(p => p.isInstanceOf[CarbonLoadTaskCompletionListener])
+    model.setFreeUnsafeMemory(!isAdded)
+    // add task completion before calling initialize as initialize method will internally
+    // call for usage of unsafe method for processing of one blocklet and if there is any
+    // exceptionwhile doing that the unsafe memory occupied for that task will not
+    // get cleared
+    context.addTaskCompletionListener {
+      new QueryTaskCompletionListener(!isAdded,
+        reader,
+        inputMetricsStats,
+        executionId,
+        taskId,
+        queryStartTime,
+        model.getStatisticsRecorder,
+        split,
+        queryId)
+    }
+  }
+
   private def close() {
     TaskMetricsMap.getInstance().updateReadBytes(Thread.currentThread().getId)
     inputMetricsStats.updateAndClose()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/9ae91cc5/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/InsertTaskCompletionListener.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/InsertTaskCompletionListener.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/InsertTaskCompletionListener.scala
index dfdbd19..7246645 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/InsertTaskCompletionListener.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/InsertTaskCompletionListener.scala
@@ -18,8 +18,8 @@
 package org.apache.carbondata.spark.rdd
 
 import org.apache.spark.TaskContext
+import org.apache.spark.sql.carbondata.execution.datasources.tasklisteners.CarbonLoadTaskCompletionListener
 import org.apache.spark.sql.execution.command.ExecutionErrors
-import org.apache.spark.util.TaskCompletionListener
 
 import org.apache.carbondata.core.util.ThreadLocalTaskInfo
 import org.apache.carbondata.processing.loading.{DataLoadExecutor, FailureCauses}
@@ -27,7 +27,7 @@ import org.apache.carbondata.spark.util.CommonUtil
 
 class InsertTaskCompletionListener(dataLoadExecutor: DataLoadExecutor,
     executorErrors: ExecutionErrors)
-  extends TaskCompletionListener {
+  extends CarbonLoadTaskCompletionListener {
   override def onTaskCompletion(context: TaskContext): Unit = {
     try {
       dataLoadExecutor.close()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/9ae91cc5/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/QueryTaskCompletionListener.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/QueryTaskCompletionListener.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/QueryTaskCompletionListener.scala
index e4cb3f8..97449c5 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/QueryTaskCompletionListener.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/QueryTaskCompletionListener.scala
@@ -21,8 +21,8 @@ import scala.collection.JavaConverters._
 
 import org.apache.hadoop.mapreduce.RecordReader
 import org.apache.spark.{Partition, TaskContext}
+import org.apache.spark.sql.carbondata.execution.datasources.tasklisteners.CarbonQueryTaskCompletionListener
 import org.apache.spark.sql.profiler.{Profiler, QueryTaskEnd}
-import org.apache.spark.util.TaskCompletionListener
 
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.memory.UnsafeMemoryManager
@@ -34,7 +34,7 @@ class QueryTaskCompletionListener(freeMemory: Boolean,
     var reader: RecordReader[Void, Object],
     inputMetricsStats: InitInputMetrics, executionId: String, taskId: Int, queryStartTime: Long,
     queryStatisticsRecorder: QueryStatisticsRecorder, split: Partition, queryId: String)
-  extends TaskCompletionListener {
+  extends CarbonQueryTaskCompletionListener {
   override def onTaskCompletion(context: TaskContext): Unit = {
     if (reader != null) {
       try {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/9ae91cc5/integration/spark-datasource/src/main/scala/org/apache/spark/sql/carbondata/execution/datasources/SparkCarbonFileFormat.scala
----------------------------------------------------------------------
diff --git a/integration/spark-datasource/src/main/scala/org/apache/spark/sql/carbondata/execution/datasources/SparkCarbonFileFormat.scala b/integration/spark-datasource/src/main/scala/org/apache/spark/sql/carbondata/execution/datasources/SparkCarbonFileFormat.scala
index a6965ac..53b1bb1 100644
--- a/integration/spark-datasource/src/main/scala/org/apache/spark/sql/carbondata/execution/datasources/SparkCarbonFileFormat.scala
+++ b/integration/spark-datasource/src/main/scala/org/apache/spark/sql/carbondata/execution/datasources/SparkCarbonFileFormat.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.sql.carbondata.execution.datasources
 
 import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.{FileStatus, Path}
@@ -29,6 +30,7 @@ import org.apache.spark.internal.Logging
 import org.apache.spark.memory.MemoryMode
 import org.apache.spark.sql._
 import org.apache.spark.sql.carbondata.execution.datasources.readsupport.SparkUnsafeRowReadSuport
+import org.apache.spark.sql.carbondata.execution.datasources.tasklisteners.{CarbonLoadTaskCompletionListener, CarbonLoadTaskCompletionListenerImpl, CarbonQueryTaskCompletionListener, CarbonQueryTaskCompletionListenerImpl}
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions.JoinedRow
 import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
@@ -37,7 +39,7 @@ import org.apache.spark.sql.execution.datasources._
 import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
 import org.apache.spark.sql.types._
 import org.apache.spark.sql.util.SparkTypeConverter
-import org.apache.spark.util.SerializableConfiguration
+import org.apache.spark.util.{SerializableConfiguration, TaskCompletionListener}
 
 import org.apache.carbondata.common.annotations.{InterfaceAudience, InterfaceStability}
 import org.apache.carbondata.common.logging.LogServiceFactory
@@ -174,6 +176,10 @@ class SparkCarbonFileFormat extends FileFormat
     private val recordWriter: RecordWriter[NullWritable, ObjectArrayWritable] =
       new CarbonTableOutputFormat().getRecordWriter(context)
 
+    Option(TaskContext.get()).foreach {c =>
+      c.addTaskCompletionListener(CarbonLoadTaskCompletionListenerImpl(recordWriter, context))
+    }
+
     /**
      * Write sparks internal row to carbondata record writer
      */
@@ -388,6 +394,15 @@ class SparkCarbonFileFormat extends FileFormat
         val model = format.createQueryModel(split, hadoopAttemptContext)
         model.setConverter(new SparkDataTypeConverterImpl)
         model.setPreFetchData(false)
+        var isAdded = false
+        Option(TaskContext.get()).foreach { context =>
+          val onCompleteCallbacksField = context.getClass.getDeclaredField("onCompleteCallbacks")
+          onCompleteCallbacksField.setAccessible(true)
+          val listeners = onCompleteCallbacksField.get(context)
+            .asInstanceOf[ArrayBuffer[TaskCompletionListener]]
+          isAdded = listeners.exists(p => p.isInstanceOf[CarbonLoadTaskCompletionListener])
+          model.setFreeUnsafeMemory(!isAdded)
+        }
         val carbonReader = if (readVector) {
           val vectorizedReader = new VectorizedCarbonRecordReader(model,
             null,
@@ -404,7 +419,11 @@ class SparkCarbonFileFormat extends FileFormat
         }
 
         val iter = new RecordReaderIterator(carbonReader)
-        Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => iter.close()))
+        Option(TaskContext.get()).foreach{context =>
+          context.addTaskCompletionListener(
+          CarbonQueryTaskCompletionListenerImpl(
+            iter.asInstanceOf[RecordReaderIterator[InternalRow]], !isAdded))
+        }
 
         if (carbonReader.isInstanceOf[VectorizedCarbonRecordReader] && readVector) {
           iter.asInstanceOf[Iterator[InternalRow]]

http://git-wip-us.apache.org/repos/asf/carbondata/blob/9ae91cc5/integration/spark-datasource/src/main/scala/org/apache/spark/sql/carbondata/execution/datasources/tasklisteners/CarbonTaskCompletionListener.scala
----------------------------------------------------------------------
diff --git a/integration/spark-datasource/src/main/scala/org/apache/spark/sql/carbondata/execution/datasources/tasklisteners/CarbonTaskCompletionListener.scala b/integration/spark-datasource/src/main/scala/org/apache/spark/sql/carbondata/execution/datasources/tasklisteners/CarbonTaskCompletionListener.scala
new file mode 100644
index 0000000..9d889d4
--- /dev/null
+++ b/integration/spark-datasource/src/main/scala/org/apache/spark/sql/carbondata/execution/datasources/tasklisteners/CarbonTaskCompletionListener.scala
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.carbondata.execution.datasources.tasklisteners
+
+import org.apache.hadoop.io.NullWritable
+import org.apache.hadoop.mapreduce.{RecordWriter, TaskAttemptContext}
+import org.apache.spark.TaskContext
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.execution.datasources.RecordReaderIterator
+import org.apache.spark.util.TaskCompletionListener
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.memory.UnsafeMemoryManager
+import org.apache.carbondata.core.util.ThreadLocalTaskInfo
+import org.apache.carbondata.hadoop.internal.ObjectArrayWritable
+
+/**
+ * Query completion listener
+ */
+trait CarbonQueryTaskCompletionListener extends TaskCompletionListener
+
+/**
+ * Load completion listener
+ */
+trait CarbonLoadTaskCompletionListener extends TaskCompletionListener
+
+case class CarbonQueryTaskCompletionListenerImpl(iter: RecordReaderIterator[InternalRow],
+    freeMemory: Boolean) extends CarbonQueryTaskCompletionListener {
+  override def onTaskCompletion(context: TaskContext): Unit = {
+    if (iter != null) {
+      try {
+        iter.close()
+      } catch {
+        case e: Exception =>
+          LogServiceFactory.getLogService(this.getClass.getCanonicalName).error(e)
+      }
+    }
+    if (freeMemory) {
+      UnsafeMemoryManager.INSTANCE
+        .freeMemoryAll(ThreadLocalTaskInfo.getCarbonTaskInfo.getTaskId)
+    }
+  }
+}
+
+case class CarbonLoadTaskCompletionListenerImpl(recordWriter: RecordWriter[NullWritable,
+  ObjectArrayWritable],
+    taskAttemptContext: TaskAttemptContext) extends CarbonLoadTaskCompletionListener {
+
+  override def onTaskCompletion(context: TaskContext): Unit = {
+    try {
+      recordWriter.close(taskAttemptContext)
+    } finally {
+      UnsafeMemoryManager.INSTANCE
+        .freeMemoryAll(ThreadLocalTaskInfo.getCarbonTaskInfo.getTaskId)
+    }
+  }
+}


[12/45] carbondata git commit: [HOTFIX] support "carbon.load.directWriteHdfs.enabled" for S3

Posted by ra...@apache.org.
[HOTFIX] support "carbon.load.directWriteHdfs.enabled" for S3

problem : Currently for s3, when the above carbon property is set. index file will not be written in the s3 store path due to bug in folder path.

Solution: file separator used is wrong. Need to fix it.
Also rename a carbon peroperty
"carbon.load.directWriteHdfs.enabled" to
"carbon.load.directWriteToStorePath.enabled"

This closes #2697


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

Branch: refs/heads/branch-1.5
Commit: 5d17ff40bdeeba64a8885fa2df427fbdec6a38ea
Parents: 2a4f530
Author: ajantha-bhat <aj...@gmail.com>
Authored: Thu Sep 6 16:47:22 2018 +0530
Committer: kunal642 <ku...@gmail.com>
Committed: Thu Sep 27 14:03:37 2018 +0530

----------------------------------------------------------------------
 .../constants/CarbonLoadOptionConstants.java    |  6 ++--
 docs/configuration-parameters.md                |  2 +-
 .../carbondata/examples/sdk/SDKS3Example.java   | 12 +++++++
 .../dataload/TestLoadDataGeneral.scala          |  8 ++---
 .../store/writer/AbstractFactDataWriter.java    | 38 ++++++++++----------
 5 files changed, 39 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/5d17ff40/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
index 3eab69d..82485ca 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
@@ -136,9 +136,9 @@ public final class CarbonLoadOptionConstants {
   public static final String SORT_COLUMN_BOUNDS_ROW_DELIMITER = ";";
 
   @CarbonProperty
-  public static final String ENABLE_CARBON_LOAD_DIRECT_WRITE_HDFS
-      = "carbon.load.directWriteHdfs.enabled";
-  public static final String ENABLE_CARBON_LOAD_DIRECT_WRITE_HDFS_DEFAULT = "false";
+  public static final String ENABLE_CARBON_LOAD_DIRECT_WRITE_TO_STORE_PATH
+      = "carbon.load.directWriteToStorePath.enabled";
+  public static final String ENABLE_CARBON_LOAD_DIRECT_WRITE_TO_STORE_PATH_DEFAULT = "false";
 
   /**
    * If the sort memory is insufficient, spill inmemory pages to disk.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5d17ff40/docs/configuration-parameters.md
----------------------------------------------------------------------
diff --git a/docs/configuration-parameters.md b/docs/configuration-parameters.md
index 662525b..9dd8164 100644
--- a/docs/configuration-parameters.md
+++ b/docs/configuration-parameters.md
@@ -90,7 +90,7 @@ This section provides the details of all the configurations required for the Car
 | carbon.prefetch.buffersize | 1000 | When the configuration ***carbon.merge.sort.prefetch*** is configured to true, we need to set the number of records that can be prefetched.This configuration is used specify the number of records to be prefetched.**NOTE: **Configuring more number of records to be prefetched increases memory footprint as more records will have to be kept in memory. |
 | load_min_size_inmb | 256 | This configuration is used along with ***carbon.load.min.size.enabled***.This determines the minimum size of input files to be considered for distribution among executors while data loading.**NOTE:** Refer to ***carbon.load.min.size.enabled*** for understanding when this configuration needs to be used and its advantages and disadvantages. |
 | carbon.load.sortmemory.spill.percentage | 0 | During data loading, some data pages are kept in memory upto memory configured in ***carbon.sort.storage.inmemory.size.inmb*** beyond which they are spilled to disk as intermediate temporary sort files.This configuration determines after what percentage data needs to be spilled to disk.**NOTE:** Without this configuration, when the data pages occupy upto configured memory, new data pages would be dumped to disk and old pages are still maintained in disk. |
-| carbon.load.directWriteHdfs.enabled | false | During data load all the carbondata files are written to local disk and finally copied to the target location in HDFS.Enabling this parameter will make carrbondata files to be written directly onto target HDFS location bypassing the local disk.**NOTE:** Writing directly to HDFS saves local disk IO(once for writing the files and again for copying to HDFS) there by improving the performance.But the drawback is when data loading fails or the application crashes, unwanted carbondata files will remain in the target HDFS location until it is cleared during next data load or by running *CLEAN FILES* DDL command |
+| carbon.load.directWriteToStorePath.enabled | false | During data load, all the carbondata files are written to local disk and finally copied to the target store location in HDFS/S3.Enabling this parameter will make carbondata files to be written directly onto target HDFS/S3 location bypassing the local disk.**NOTE:** Writing directly to HDFS/S3 saves local disk IO(once for writing the files and again for copying to HDFS/S3) there by improving the performance.But the drawback is when data loading fails or the application crashes, unwanted carbondata files will remain in the target HDFS/S3 location until it is cleared during next data load or by running *CLEAN FILES* DDL command |
 | carbon.options.serialization.null.format | \N | Based on the business scenarios, some columns might need to be loaded with null values.As null value cannot be written in csv files, some special characters might be adopted to specify null values.This configuration can be used to specify the null values format in the data being loaded. |
 | carbon.sort.storage.inmemory.size.inmb | 512 | CarbonData writes every ***carbon.sort.size*** number of records to intermediate temp files during data loading to ensure memory footprint is within limits.When ***enable.unsafe.sort*** configuration is enabled, instead of using ***carbon.sort.size*** which is based on rows count, size occupied in memory is used to determine when to flush data pages to intermediate temp files.This configuration determines the memory to be used for storing data pages in memory.**NOTE:** Configuring a higher values ensures more data is maintained in memory and hence increases data loading performance due to reduced or no IO.Based on the memory availability in the nodes of the cluster, configure the values accordingly. |
 | carbon.column.compressor | snappy | CarbonData will compress the column values using the compressor specified by this configuration. Currently CarbonData supports 'snappy' and 'zstd' compressors. |

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5d17ff40/examples/spark2/src/main/java/org/apache/carbondata/examples/sdk/SDKS3Example.java
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/java/org/apache/carbondata/examples/sdk/SDKS3Example.java b/examples/spark2/src/main/java/org/apache/carbondata/examples/sdk/SDKS3Example.java
index d4f49f5..bc0e280 100644
--- a/examples/spark2/src/main/java/org/apache/carbondata/examples/sdk/SDKS3Example.java
+++ b/examples/spark2/src/main/java/org/apache/carbondata/examples/sdk/SDKS3Example.java
@@ -19,10 +19,12 @@ package org.apache.carbondata.examples.sdk;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.constants.CarbonLoadOptionConstants;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.LiteralExpression;
 import org.apache.carbondata.core.scan.expression.conditional.EqualToExpression;
+import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.sdk.file.*;
 
 import org.apache.hadoop.conf.Configuration;
@@ -40,6 +42,12 @@ public class SDKS3Example {
             System.exit(0);
         }
 
+        String backupProperty = CarbonProperties.getInstance()
+            .getProperty(CarbonLoadOptionConstants.ENABLE_CARBON_LOAD_DIRECT_WRITE_TO_STORE_PATH,
+                CarbonLoadOptionConstants.ENABLE_CARBON_LOAD_DIRECT_WRITE_TO_STORE_PATH_DEFAULT);
+        CarbonProperties.getInstance()
+            .addProperty(CarbonLoadOptionConstants.ENABLE_CARBON_LOAD_DIRECT_WRITE_TO_STORE_PATH, "true");
+
         String path = "s3a://sdk/WriterOutput";
         if (args.length > 3) {
             path=args[3];
@@ -87,5 +95,9 @@ public class SDKS3Example {
         }
         System.out.println("\nFinished");
         reader.close();
+
+        CarbonProperties.getInstance()
+            .addProperty(CarbonLoadOptionConstants.ENABLE_CARBON_LOAD_DIRECT_WRITE_TO_STORE_PATH,
+                backupProperty);
     }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5d17ff40/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
index 8b51090..02abb8d 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
@@ -243,10 +243,10 @@ class TestLoadDataGeneral extends QueryTest with BeforeAndAfterEach {
 
   test("test data loading with directly writing fact data to hdfs") {
     val originStatus = CarbonProperties.getInstance().getProperty(
-      CarbonLoadOptionConstants.ENABLE_CARBON_LOAD_DIRECT_WRITE_HDFS,
-      CarbonLoadOptionConstants.ENABLE_CARBON_LOAD_DIRECT_WRITE_HDFS_DEFAULT)
+      CarbonLoadOptionConstants.ENABLE_CARBON_LOAD_DIRECT_WRITE_TO_STORE_PATH,
+      CarbonLoadOptionConstants.ENABLE_CARBON_LOAD_DIRECT_WRITE_TO_STORE_PATH_DEFAULT)
     CarbonProperties.getInstance().addProperty(
-      CarbonLoadOptionConstants.ENABLE_CARBON_LOAD_DIRECT_WRITE_HDFS, "true")
+      CarbonLoadOptionConstants.ENABLE_CARBON_LOAD_DIRECT_WRITE_TO_STORE_PATH, "true")
 
     val testData = s"$resourcesPath/sample.csv"
     sql(s"LOAD DATA LOCAL INPATH '$testData' into table loadtest")
@@ -256,7 +256,7 @@ class TestLoadDataGeneral extends QueryTest with BeforeAndAfterEach {
     )
 
     CarbonProperties.getInstance().addProperty(
-      CarbonLoadOptionConstants.ENABLE_CARBON_LOAD_DIRECT_WRITE_HDFS,
+      CarbonLoadOptionConstants.ENABLE_CARBON_LOAD_DIRECT_WRITE_TO_STORE_PATH,
       originStatus)
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5d17ff40/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java b/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
index 4afb3ef..37d33c2 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
@@ -66,9 +66,9 @@ public abstract class AbstractFactDataWriter implements CarbonFactDataWriter {
   protected WritableByteChannel fileChannel;
   protected long currentOffsetInFile;
   /**
-   * The path of CarbonData file to write in hdfs
+   * The path of CarbonData file to write in hdfs/s3
    */
-  private String carbonDataFileHdfsPath;
+  private String carbonDataFileStorePath;
   /**
    * The temp path of carbonData file used on executor
    */
@@ -145,9 +145,9 @@ public abstract class AbstractFactDataWriter implements CarbonFactDataWriter {
    */
   protected DataMapWriterListener listener;
   /**
-   * Whether directly write fact data to hdfs
+   * Whether directly write fact data to store path
    */
-  private boolean enableDirectlyWriteData2Hdfs = false;
+  private boolean enableDirectlyWriteDataToStorePath = false;
 
   protected ExecutorService fallbackExecutorService;
 
@@ -172,11 +172,11 @@ public abstract class AbstractFactDataWriter implements CarbonFactDataWriter {
 
     // whether to directly write fact data to HDFS
     String directlyWriteData2Hdfs = propInstance
-        .getProperty(CarbonLoadOptionConstants.ENABLE_CARBON_LOAD_DIRECT_WRITE_HDFS,
-            CarbonLoadOptionConstants.ENABLE_CARBON_LOAD_DIRECT_WRITE_HDFS_DEFAULT);
-    this.enableDirectlyWriteData2Hdfs = "TRUE".equalsIgnoreCase(directlyWriteData2Hdfs);
+        .getProperty(CarbonLoadOptionConstants.ENABLE_CARBON_LOAD_DIRECT_WRITE_TO_STORE_PATH,
+            CarbonLoadOptionConstants.ENABLE_CARBON_LOAD_DIRECT_WRITE_TO_STORE_PATH_DEFAULT);
+    this.enableDirectlyWriteDataToStorePath = "TRUE".equalsIgnoreCase(directlyWriteData2Hdfs);
 
-    if (enableDirectlyWriteData2Hdfs) {
+    if (enableDirectlyWriteDataToStorePath) {
       LOGGER.info("Carbondata will directly write fact data to HDFS.");
     } else {
       LOGGER.info("Carbondata will write temporary fact data to local disk.");
@@ -225,7 +225,7 @@ public abstract class AbstractFactDataWriter implements CarbonFactDataWriter {
     if ((currentFileSize + blockletSizeToBeAdded) >= blockSizeThreshold && currentFileSize != 0) {
       // set the current file size to zero
       String activeFile =
-          enableDirectlyWriteData2Hdfs ? carbonDataFileHdfsPath : carbonDataFileTempPath;
+          enableDirectlyWriteDataToStorePath ? carbonDataFileStorePath : carbonDataFileTempPath;
       LOGGER.info("Writing data to file as max file size reached for file: "
           + activeFile + ". Data block size: " + currentFileSize);
       // write meta data to end of the existing file
@@ -269,7 +269,7 @@ public abstract class AbstractFactDataWriter implements CarbonFactDataWriter {
   protected void commitCurrentFile(boolean copyInCurrentThread) {
     notifyDataMapBlockEnd();
     CarbonUtil.closeStreams(this.fileOutputStream, this.fileChannel);
-    if (!enableDirectlyWriteData2Hdfs) {
+    if (!enableDirectlyWriteDataToStorePath) {
       try {
         if (copyInCurrentThread) {
           CarbonUtil.copyCarbonDataFileToCarbonStorePath(carbonDataFileTempPath,
@@ -296,14 +296,14 @@ public abstract class AbstractFactDataWriter implements CarbonFactDataWriter {
         .getCarbonDataFileName(fileCount, model.getCarbonDataFileAttributes().getTaskId(),
             model.getBucketId(), model.getTaskExtension(),
             "" + model.getCarbonDataFileAttributes().getFactTimeStamp(), model.getSegmentId());
-    this.carbonDataFileHdfsPath = model.getCarbonDataDirectoryPath() + File.separator
+    this.carbonDataFileStorePath = model.getCarbonDataDirectoryPath() + File.separator
         + carbonDataFileName;
     try {
-      if (enableDirectlyWriteData2Hdfs) {
+      if (enableDirectlyWriteDataToStorePath) {
         // the block size will be twice the block_size specified by user to make sure that
         // one carbondata file only consists exactly one HDFS block.
         fileOutputStream = FileFactory
-            .getDataOutputStream(carbonDataFileHdfsPath, FileFactory.FileType.HDFS,
+            .getDataOutputStream(carbonDataFileStorePath, FileFactory.FileType.HDFS,
                 CarbonCommonConstants.BYTEBUFFER_SIZE, fileSizeInBytes * 2);
       } else {
         //each time we initialize writer, we choose a local temp location randomly
@@ -380,11 +380,11 @@ public abstract class AbstractFactDataWriter implements CarbonFactDataWriter {
     // get the block index info thrift
     List<BlockIndex> blockIndexThrift = CarbonMetadataUtil.getBlockIndexInfo(blockIndexInfoList);
     String indexFileName;
-    if (enableDirectlyWriteData2Hdfs) {
-      String rawFileName = model.getCarbonDataDirectoryPath() + File.separator + CarbonTablePath
-          .getCarbonIndexFileName(model.getCarbonDataFileAttributes().getTaskId(),
-              model.getBucketId(), model.getTaskExtension(),
-              "" + model.getCarbonDataFileAttributes().getFactTimeStamp(), model.getSegmentId());
+    if (enableDirectlyWriteDataToStorePath) {
+      String rawFileName = model.getCarbonDataDirectoryPath() + CarbonCommonConstants.FILE_SEPARATOR
+          + CarbonTablePath.getCarbonIndexFileName(model.getCarbonDataFileAttributes().getTaskId(),
+          model.getBucketId(), model.getTaskExtension(),
+          "" + model.getCarbonDataFileAttributes().getFactTimeStamp(), model.getSegmentId());
       indexFileName = FileFactory.getUpdatedFilePath(rawFileName, FileFactory.FileType.HDFS);
     } else {
       // randomly choose a temp location for index file
@@ -407,7 +407,7 @@ public abstract class AbstractFactDataWriter implements CarbonFactDataWriter {
       writer.writeThrift(blockIndex);
     }
     writer.close();
-    if (!enableDirectlyWriteData2Hdfs) {
+    if (!enableDirectlyWriteDataToStorePath) {
       CarbonUtil
           .copyCarbonDataFileToCarbonStorePath(indexFileName, model.getCarbonDataDirectoryPath(),
               fileSizeInBytes);


[42/45] carbondata git commit: [Documentation] Readme updated with latest topics and new TOC

Posted by ra...@apache.org.
[Documentation] Readme updated with latest topics and new TOC

Readme updated with the new structure
Formatting issue fixed
Review comments fixed

This closes #2788


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

Branch: refs/heads/branch-1.5
Commit: ca30ad97da020daceb49669fba454a4346241935
Parents: d392717
Author: sgururajshetty <sg...@gmail.com>
Authored: Fri Sep 28 19:13:08 2018 +0530
Committer: kunal642 <ku...@gmail.com>
Committed: Fri Oct 5 15:00:13 2018 +0530

----------------------------------------------------------------------
 README.md                                      |  33 ++--
 docs/carbon-as-spark-datasource-guide.md       |  29 ++--
 docs/configuration-parameters.md               | 158 ++++++++++----------
 docs/datamap-developer-guide.md                |   4 +-
 docs/datamap/bloomfilter-datamap-guide.md      |   6 +-
 docs/datamap/datamap-management.md             |   6 +-
 docs/datamap/lucene-datamap-guide.md           |   4 +-
 docs/datamap/preaggregate-datamap-guide.md     |   2 +-
 docs/ddl-of-carbondata.md                      |  97 +++++++-----
 docs/dml-of-carbondata.md                      |   6 +-
 docs/documentation.md                          |   2 +-
 docs/faq.md                                    |   6 +-
 docs/file-structure-of-carbondata.md           |   2 +-
 docs/how-to-contribute-to-apache-carbondata.md |   4 +-
 docs/introduction.md                           |  20 +--
 docs/language-manual.md                        |   2 +
 docs/performance-tuning.md                     |  10 +-
 docs/quick-start-guide.md                      |   6 +-
 docs/s3-guide.md                               |   2 +-
 docs/streaming-guide.md                        |   6 +-
 docs/usecases.md                               |  32 ++--
 21 files changed, 229 insertions(+), 208 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/ca30ad97/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index ba2cbf7..87bb71e 100644
--- a/README.md
+++ b/README.md
@@ -45,23 +45,26 @@ CarbonData file format is a columnar store in HDFS, it has many features that a
 CarbonData is built using Apache Maven, to [build CarbonData](https://github.com/apache/carbondata/blob/master/build)
 
 ## Online Documentation
+* [What is CarbonData](https://github.com/apache/carbondata/blob/master/docs/introduction.md)
 * [Quick Start](https://github.com/apache/carbondata/blob/master/docs/quick-start-guide.md)
-* [CarbonData File Structure](https://github.com/apache/carbondata/blob/master/docs/file-structure-of-carbondata.md)
-* [Data Types](https://github.com/apache/carbondata/blob/master/docs/supported-data-types-in-carbondata.md)
-* [Data Management on CarbonData](https://github.com/apache/carbondata/blob/master/docs/language-manual.md)
-* [Configuring Carbondata](https://github.com/apache/carbondata/blob/master/docs/configuration-parameters.md)
-* [Streaming Ingestion](https://github.com/apache/carbondata/blob/master/docs/streaming-guide.md)
-* [SDK Guide](https://github.com/apache/carbondata/blob/master/docs/sdk-guide.md)
-* [S3 Guide](https://github.com/apache/carbondata/blob/master/docs/s3-guide.md)
-* [DataMap Developer Guide](https://github.com/apache/carbondata/blob/master/docs/datamap-developer-guide.md)
-* [CarbonData DataMap Management](https://github.com/apache/carbondata/blob/master/docs/datamap/datamap-management.md)
-* [CarbonData BloomFilter DataMap](https://github.com/apache/carbondata/blob/master/docs/datamap/bloomfilter-datamap-guide.md)
-* [CarbonData Lucene DataMap](https://github.com/apache/carbondata/blob/master/docs/datamap/lucene-datamap-guide.md)
-* [CarbonData Pre-aggregate DataMap](https://github.com/apache/carbondata/blob/master/docs/datamap/preaggregate-datamap-guide.md)
-* [CarbonData Timeseries DataMap](https://github.com/apache/carbondata/blob/master/docs/datamap/timeseries-datamap-guide.md)
-* [Performance Tuning](https://github.com/apache/carbondata/blob/master/docs/performance-tuning.md)
-* [FAQ](https://github.com/apache/carbondata/blob/master/docs/faq.md)
 * [Use Cases](https://github.com/apache/carbondata/blob/master/docs/usecases.md)
+* [Language Reference](https://github.com/apache/carbondata/blob/master/docs/language-manual.md)
+ * [CarbonData Data Definition Language](https://github.com/apache/carbondata/blob/master/docs/ddl-of-carbondata.md) 
+ * [CarbonData Data Manipulation Language](https://github.com/apache/carbondata/blob/master/docs/dml-of-carbondata.md) 
+ * [CarbonData Streaming Ingestion](https://github.com/apache/carbondata/blob/master/docs/streaming-guide.md) 
+ * [Configuring CarbonData](https://github.com/apache/carbondata/blob/master/docs/configuration-parameters.md) 
+ * [DataMap Developer Guide](https://github.com/apache/carbondata/blob/master/docs/datamap-developer-guide.md) 
+ * [Data Types](https://github.com/apache/carbondata/blob/master/docs/supported-data-types-in-carbondata.md) 
+* [CarbonData DataMap Management](https://github.com/apache/carbondata/blob/master/docs/datamap-management.md) 
+ * [CarbonData BloomFilter DataMap](https://github.com/apache/carbondata/blob/master/docs/bloomfilter-datamap-guide.md) 
+ * [CarbonData Lucene DataMap](https://github.com/apache/carbondata/blob/master/docs/lucene-datamap-guide.md) 
+ * [CarbonData Pre-aggregate DataMap](https://github.com/apache/carbondata/blob/master/docs/preaggregate-datamap-guide.md) 
+ * [CarbonData Timeseries DataMap](https://github.com/apache/carbondata/blob/master/docs/timeseries-datamap-guide.md) 
+* [SDK Guide](https://github.com/apache/carbondata/blob/master/docs/sdk-guide.md) 
+* [Performance Tuning](https://github.com/apache/carbondata/blob/master/docs/performance-tuning.md) 
+* [S3 Storage](https://github.com/apache/carbondata/blob/master/docs/s3-guide.md) 
+* [Carbon as Spark's Datasource](https://github.com/apache/carbondata/blob/master/docs/carbon-as-spark-datasource-guide.md) 
+* [FAQs](https://github.com/apache/carbondata/blob/master/docs/faq.md) 
 
 ## Other Technical Material
 * [Apache CarbonData meetup material](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ca30ad97/docs/carbon-as-spark-datasource-guide.md
----------------------------------------------------------------------
diff --git a/docs/carbon-as-spark-datasource-guide.md b/docs/carbon-as-spark-datasource-guide.md
index 1d286cf..bc56a54 100644
--- a/docs/carbon-as-spark-datasource-guide.md
+++ b/docs/carbon-as-spark-datasource-guide.md
@@ -15,19 +15,20 @@
     limitations under the License.
 -->
 
-# Carbon as Spark's datasource guide
+# CarbonData as Spark's Datasource
 
-Carbon fileformat can be integrated to Spark using datasource to read and write data without using CarbonSession.
+The CarbonData fileformat is now integrated as Spark datasource for read and write operation without using CarbonSession. This is useful for users who wants to use carbondata as spark's data source. 
 
+**Note:** You can only apply the functions/features supported by spark datasource APIs, functionalities supported would be similar to Parquet. The carbon session features are not supported.
 
 # Create Table with DDL
 
-Carbon table can be created with spark's datasource DDL syntax as follows.
+Now you can create Carbon table using Spark's datasource DDL syntax.
 
 ```
  CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db_name.]table_name
      [(col_name1 col_type1 [COMMENT col_comment1], ...)]
-     USING carbon
+     USING CARBON
      [OPTIONS (key1=val1, key2=val2, ...)]
      [PARTITIONED BY (col_name1, col_name2, ...)]
      [CLUSTERED BY (col_name3, col_name4, ...) INTO num_buckets BUCKETS]
@@ -41,25 +42,23 @@ Carbon table can be created with spark's datasource DDL syntax as follows.
 
 | Property | Default Value | Description |
 |-----------|--------------|------------|
-| table_blocksize | 1024 | Size of blocks to write onto hdfs |
-| table_blocklet_size | 64 | Size of blocklet to write |
-| local_dictionary_threshold | 10000 | Cardinality upto which the local dictionary can be generated  |
-| local_dictionary_enable | false | Enable local dictionary generation  |
-| sort_columns | all dimensions are sorted | comma separated string columns which to include in sort and its order of sort |
-| sort_scope | local_sort | Sort scope of the load.Options include no sort, local sort ,batch sort and global sort |
-| long_string_columns | null | comma separated string columns which are more than 32k length |
+| table_blocksize | 1024 | Size of blocks to write onto hdfs. For  more details, see [Table Block Size Configuration](./ddl-of-carbondata.md#table-block-size-configuration). |
+| table_blocklet_size | 64 | Size of blocklet to write. |
+| local_dictionary_threshold | 10000 | Cardinality upto which the local dictionary can be generated. For  more details, see [Local Dictionary Configuration](./ddl-of-carbondata.md#local-dictionary-configuration). |
+| local_dictionary_enable | false | Enable local dictionary generation. For  more details, see [Local Dictionary Configuration](./ddl-of-carbondata.md#local-dictionary-configuration). |
+| sort_columns | all dimensions are sorted | Columns to include in sort and its order of sort. For  more details, see [Sort Columns Configuration](./ddl-of-carbondata.md#sort-columns-configuration). |
+| sort_scope | local_sort | Sort scope of the load.Options include no sort, local sort, batch sort, and global sort. For  more details, see [Sort Scope Configuration](./ddl-of-carbondata.md#sort-scope-configuration). |
+| long_string_columns | null | Comma separated string/char/varchar columns which are more than 32k length. For  more details, see [String longer than 32000 characters](./ddl-of-carbondata.md#string-longer-than-32000-characters). |
 
 ## Example 
 
 ```
- CREATE TABLE CARBON_TABLE (NAME  STRING) USING CARBON OPTIONS(‘table_block_size’=’256’)
+ CREATE TABLE CARBON_TABLE (NAME  STRING) USING CARBON OPTIONS('table_block_size'='256')
 ```
 
-Note: User can only apply the features of what spark datasource like parquet supports. It cannot support the features of carbon session like IUD, compaction etc. 
-
 # Using DataFrame
 
-Carbon format can be used in dataframe also using the following way.
+Carbon format can be used in dataframe also. Following are the ways to use carbon format in dataframe.
 
 Write carbon using dataframe 
 ```


[37/45] carbondata git commit: [CARBONDATA-2983][BloomDataMap] Change bloom query model to proceed multiple filter values

Posted by ra...@apache.org.
[CARBONDATA-2983][BloomDataMap] Change bloom query model to proceed multiple filter values

This PR is supposed to optimize the pruning for InExpression.
For a expression like colA in (1, 2, 3).
Previously it will create 3 bloom query model and for each query model it will iterate all the bloomfilters; But now we only need to generate 1 bloom query model.

Also this PR does an intersection for the pruned result generated by each expression. This optimization is important if we have multiple index columns and used them in one query.

This closes #2781


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

Branch: refs/heads/branch-1.5
Commit: 396c26f531192e3da8a233ca14024dd8899da52a
Parents: 3edea12
Author: Manhua <ke...@qq.com>
Authored: Fri Sep 28 11:39:39 2018 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Oct 4 18:13:12 2018 +0530

----------------------------------------------------------------------
 .../datamap/bloom/BloomCoarseGrainDataMap.java  | 98 ++++++++++++--------
 1 file changed, 59 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/396c26f5/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java
----------------------------------------------------------------------
diff --git a/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java b/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java
index ee71142..a5376be 100644
--- a/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java
+++ b/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java
@@ -21,15 +21,7 @@ import java.io.IOException;
 import java.io.UnsupportedEncodingException;
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TimeZone;
+import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.carbondata.common.annotations.InterfaceAudience;
@@ -47,7 +39,6 @@ import org.apache.carbondata.core.indexstore.Blocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.CarbonMetadata;
-import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
@@ -171,7 +162,7 @@ public class BloomCoarseGrainDataMap extends CoarseGrainDataMap {
   @Override
   public List<Blocklet> prune(FilterResolverIntf filterExp, SegmentProperties segmentProperties,
       List<PartitionSpec> partitions) throws IOException {
-    Set<Blocklet> hitBlocklets = new HashSet<>();
+    Set<Blocklet> hitBlocklets = null;
     if (filterExp == null) {
       // null is different from empty here. Empty means after pruning, no blocklet need to scan.
       return null;
@@ -185,6 +176,7 @@ public class BloomCoarseGrainDataMap extends CoarseGrainDataMap {
       throw new RuntimeException(e);
     }
     for (BloomQueryModel bloomQueryModel : bloomQueryModels) {
+      Set<Blocklet> tempHitBlockletsResult = new HashSet<>();
       LOGGER.debug("prune blocklet for query: " + bloomQueryModel);
       BloomCacheKeyValue.CacheKey cacheKey = new BloomCacheKeyValue.CacheKey(
           this.indexPath.toString(), bloomQueryModel.columnName);
@@ -195,17 +187,32 @@ public class BloomCoarseGrainDataMap extends CoarseGrainDataMap {
           // skip shard which has been pruned in Main datamap
           continue;
         }
-        boolean scanRequired = bloomFilter.membershipTest(new Key(bloomQueryModel.filterValue));
+        boolean scanRequired = false;
+        for (byte[] value: bloomQueryModel.filterValues) {
+          scanRequired = bloomFilter.membershipTest(new Key(value));
+          if (scanRequired) {
+            // if any filter value hit this bloomfilter
+            // no need to check other filter values
+            break;
+          }
+        }
         if (scanRequired) {
           LOGGER.debug(String.format("BloomCoarseGrainDataMap: Need to scan -> blocklet#%s",
               String.valueOf(bloomFilter.getBlockletNo())));
           Blocklet blocklet = new Blocklet(bloomFilter.getShardName(),
-                  String.valueOf(bloomFilter.getBlockletNo()));
-          hitBlocklets.add(blocklet);
+              String.valueOf(bloomFilter.getBlockletNo()));
+          tempHitBlockletsResult.add(blocklet);
         } else {
           LOGGER.debug(String.format("BloomCoarseGrainDataMap: Skip scan -> blocklet#%s",
               String.valueOf(bloomFilter.getBlockletNo())));
         }
+        // get intersect result between query models
+        // pre-condition: only And/In/EqualTo expression exists in single bloom datamap
+        if (null == hitBlocklets) {
+          hitBlocklets = tempHitBlockletsResult;
+        } else {
+          hitBlocklets.retainAll(tempHitBlockletsResult);
+        }
       }
     }
     return new ArrayList<>(hitBlocklets);
@@ -247,17 +254,17 @@ public class BloomCoarseGrainDataMap extends CoarseGrainDataMap {
       if (left instanceof ColumnExpression && right instanceof ListExpression) {
         column = ((ColumnExpression) left).getColumnName();
         if (this.name2Col.containsKey(column)) {
-          List<BloomQueryModel> models =
+          BloomQueryModel bloomQueryModel =
               buildQueryModelForIn((ColumnExpression) left, (ListExpression) right);
-          queryModels.addAll(models);
+          queryModels.add(bloomQueryModel);
         }
         return queryModels;
       } else if (left instanceof ListExpression && right instanceof ColumnExpression) {
         column = ((ColumnExpression) right).getColumnName();
         if (this.name2Col.containsKey(column)) {
-          List<BloomQueryModel> models =
+          BloomQueryModel bloomQueryModel =
               buildQueryModelForIn((ColumnExpression) right, (ListExpression) left);
-          queryModels.addAll(models);
+          queryModels.add(bloomQueryModel);
         }
         return queryModels;
       } else {
@@ -274,13 +281,15 @@ public class BloomCoarseGrainDataMap extends CoarseGrainDataMap {
     return queryModels;
   }
 
-  private BloomQueryModel buildQueryModelForEqual(ColumnExpression ce,
-      LiteralExpression le) throws DictionaryGenerationException, UnsupportedEncodingException {
-    String columnName = ce.getColumnName();
-    DataType dataType = ce.getDataType();
+  /**
+   * Here preprocessed NULL and date/timestamp data type.
+   *
+   * Note that if the datatype is date/timestamp, the expressionValue is long type.
+   */
+  private Object getLiteralExpValue(LiteralExpression le) {
     Object expressionValue = le.getLiteralExpValue();
     Object literalValue;
-    // note that if the datatype is date/timestamp, the expressionValue is long type.
+
     if (null == expressionValue) {
       literalValue = null;
     } else if (le.getLiteralExpDataType() == DataTypes.DATE) {
@@ -288,7 +297,6 @@ public class BloomCoarseGrainDataMap extends CoarseGrainDataMap {
       // the below settings are set statically according to DateDirectDirectionaryGenerator
       format.setLenient(false);
       format.setTimeZone(TimeZone.getTimeZone("GMT"));
-
       literalValue = format.format(new Date((long) expressionValue / 1000));
     } else if (le.getLiteralExpDataType() == DataTypes.TIMESTAMP) {
       DateFormat format =
@@ -299,28 +307,37 @@ public class BloomCoarseGrainDataMap extends CoarseGrainDataMap {
     } else {
       literalValue = expressionValue;
     }
+    return literalValue;
+  }
+
 
-    return buildQueryModelInternal(this.name2Col.get(columnName), literalValue, dataType);
+  private BloomQueryModel buildQueryModelForEqual(ColumnExpression ce,
+      LiteralExpression le) throws DictionaryGenerationException, UnsupportedEncodingException {
+    List<byte[]> filterValues = new ArrayList<>();
+    byte[] internalFilterValue = getInternalFilterValue(this.name2Col.get(ce.getColumnName()), le);
+    filterValues.add(internalFilterValue);
+    return new BloomQueryModel(ce.getColumnName(), filterValues);
   }
 
   /**
-   * for `in` expressions, we use `equal` to handle it.
    * Note that `in` operator needs at least one match not exactly match. since while doing pruning,
    * we collect all the blocklets that will match the querymodel, this will not be a problem.
    */
-  private List<BloomQueryModel> buildQueryModelForIn(ColumnExpression ce, ListExpression le)
+  private BloomQueryModel buildQueryModelForIn(ColumnExpression ce, ListExpression le)
       throws DictionaryGenerationException, UnsupportedEncodingException {
-    List<BloomQueryModel> queryModels = new ArrayList<>();
+    List<byte[]> filterValues = new ArrayList<>();
     for (Expression child : le.getChildren()) {
-      queryModels.add(buildQueryModelForEqual(ce, (LiteralExpression) child));
+      byte[] internalFilterValue = getInternalFilterValue(
+          this.name2Col.get(ce.getColumnName()), (LiteralExpression) child);
+      filterValues.add(internalFilterValue);
     }
-    return queryModels;
+    return new BloomQueryModel(ce.getColumnName(), filterValues);
   }
 
-  private BloomQueryModel buildQueryModelInternal(CarbonColumn carbonColumn,
-      Object filterLiteralValue, DataType filterValueDataType) throws
+  private byte[] getInternalFilterValue(CarbonColumn carbonColumn, LiteralExpression le) throws
       DictionaryGenerationException, UnsupportedEncodingException {
-    // convert the filter value to string and apply convertes on it to get carbon internal value
+    Object filterLiteralValue = getLiteralExpValue(le);
+    // convert the filter value to string and apply converters on it to get carbon internal value
     String strFilterValue = null;
     if (null != filterLiteralValue) {
       strFilterValue = String.valueOf(filterLiteralValue);
@@ -362,7 +379,7 @@ public class BloomCoarseGrainDataMap extends CoarseGrainDataMap {
     if (internalFilterValue.length == 0) {
       internalFilterValue = CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY;
     }
-    return new BloomQueryModel(carbonColumn.getColName(), internalFilterValue);
+    return internalFilterValue;
   }
 
   @Override
@@ -376,25 +393,28 @@ public class BloomCoarseGrainDataMap extends CoarseGrainDataMap {
 
   static class BloomQueryModel {
     private String columnName;
-    private byte[] filterValue;
+    private List<byte[]> filterValues;
 
     /**
      * represent an query model will be applyied on bloom index
      *
      * @param columnName bloom index column
-     * @param filterValue key for the bloom index,
+     * @param filterValues key for the bloom index,
      *                   this value is converted from user specified filter value in query
      */
-    private BloomQueryModel(String columnName, byte[] filterValue) {
+    private BloomQueryModel(String columnName, List<byte[]> filterValues) {
       this.columnName = columnName;
-      this.filterValue = filterValue;
+      this.filterValues = filterValues;
     }
 
     @Override
     public String toString() {
       final StringBuilder sb = new StringBuilder("BloomQueryModel{");
       sb.append("columnName='").append(columnName).append('\'');
-      sb.append(", filterValue=").append(Arrays.toString(filterValue));
+      sb.append(", filterValues=");
+      for (byte[] value: filterValues) {
+        sb.append(Arrays.toString(value));
+      }
       sb.append('}');
       return sb.toString();
     }


[35/45] carbondata git commit: [CARBONDATA-2992] Fixed Between Query Data Mismatch issue for timestamp data type

Posted by ra...@apache.org.
[CARBONDATA-2992] Fixed Between Query Data Mismatch issue for timestamp data type

Problem:
Between query is giving wrong result.
Root cause:
For timestamp time when filter is given in yyyy-mm-dd format instead of yyyy-mm-dd HH:MM:SS format it will add cast, In CastExpressionOptimization it is using SimpleDateFormat object to parse the filter value which is failing as filter values is not same.
Solution:
Use SPARK:DateTimeUtils.stringToTime method as spark is handling for above scenario.

This closes #2787


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

Branch: refs/heads/branch-1.5
Commit: 11bd0ade93a3ac72b42068c3b57ed8bb1203ab47
Parents: 6aa2a90
Author: kumarvishal09 <ku...@gmail.com>
Authored: Fri Sep 28 18:33:29 2018 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Oct 4 18:02:08 2018 +0530

----------------------------------------------------------------------
 .../src/test/resources/datedatafile.csv         |  7 ++
 .../src/test/resources/timestampdatafile.csv    |  7 ++
 ...imestampNoDictionaryColumnCastTestCase.scala | 80 ++++++++++++++++++++
 .../execution/CastExpressionOptimization.scala  | 67 +++++++++-------
 .../bloom/BloomCoarseGrainDataMapSuite.scala    | 47 +++++-------
 5 files changed, 152 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/11bd0ade/integration/spark-common-test/src/test/resources/datedatafile.csv
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/resources/datedatafile.csv b/integration/spark-common-test/src/test/resources/datedatafile.csv
new file mode 100644
index 0000000..43a615d
--- /dev/null
+++ b/integration/spark-common-test/src/test/resources/datedatafile.csv
@@ -0,0 +1,7 @@
+datetype1
+2018-09-11
+2018-09-12
+2018-09-13
+2018-09-14
+2018-09-15
+2018-09-16

http://git-wip-us.apache.org/repos/asf/carbondata/blob/11bd0ade/integration/spark-common-test/src/test/resources/timestampdatafile.csv
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/resources/timestampdatafile.csv b/integration/spark-common-test/src/test/resources/timestampdatafile.csv
new file mode 100644
index 0000000..473f330
--- /dev/null
+++ b/integration/spark-common-test/src/test/resources/timestampdatafile.csv
@@ -0,0 +1,7 @@
+timestamptype
+2018-09-11 00:00:00
+2018-09-12 00:00:00
+2018-09-13 00:00:00
+2018-09-14 00:00:00
+2018-09-15 00:00:00
+2018-09-16 00:00:00

http://git-wip-us.apache.org/repos/asf/carbondata/blob/11bd0ade/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/TimestampNoDictionaryColumnCastTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/TimestampNoDictionaryColumnCastTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/TimestampNoDictionaryColumnCastTestCase.scala
new file mode 100644
index 0000000..41c7005
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/TimestampNoDictionaryColumnCastTestCase.scala
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.spark.testsuite.directdictionary
+
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+
+/**
+ * Test Class for detailed query on timestamp datatypes
+ */
+class TimestampNoDictionaryColumnCastTestCase extends QueryTest with BeforeAndAfterAll {
+
+  override def beforeAll {
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
+        CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT)
+
+      sql("drop table if exists timestamp_nodictionary")
+    sql("drop table if exists datetype")
+      sql(
+        """
+         CREATE TABLE IF NOT EXISTS timestamp_nodictionary
+        (timestamptype timestamp) STORED BY 'carbondata'"""
+      )
+      val csvFilePath = s"$resourcesPath/timestampdatafile.csv"
+      sql(s"LOAD DATA LOCAL INPATH '$csvFilePath' into table timestamp_nodictionary")
+//
+    sql(
+      """
+         CREATE TABLE IF NOT EXISTS datetype
+        (datetype1 date) STORED BY 'carbondata'"""
+    )
+    val csvFilePath1 = s"$resourcesPath/datedatafile.csv"
+    sql(s"LOAD DATA LOCAL INPATH '$csvFilePath1' into table datetype")
+  }
+
+  test("select count(*) from timestamp_nodictionary where timestamptype BETWEEN '2018-09-11' AND '2018-09-16'") {
+    checkAnswer(
+      sql("select count(*) from timestamp_nodictionary where timestamptype BETWEEN '2018-09-11' AND '2018-09-16'"),
+      Seq(Row(6)
+      )
+    )
+  }
+//
+  test("select count(*) from datetype where datetype1 BETWEEN '2018-09-11' AND '2018-09-16'") {
+    checkAnswer(
+      sql("select count(*) from datetype where datetype1 BETWEEN '2018-09-11' AND '2018-09-16'"),
+      Seq(Row(6)
+      )
+    )
+  }
+
+  override def afterAll {
+    sql("drop table timestamp_nodictionary")
+    sql("drop table if exists datetype")
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/11bd0ade/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CastExpressionOptimization.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CastExpressionOptimization.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CastExpressionOptimization.scala
index 7e61814..57fb3f0 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CastExpressionOptimization.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CastExpressionOptimization.scala
@@ -29,53 +29,66 @@ import org.apache.spark.sql.FalseExpr
 import org.apache.spark.sql.sources
 import org.apache.spark.sql.types._
 import org.apache.spark.sql.CarbonExpressions.{MatchCast => Cast}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
 import org.apache.spark.sql.sources.Filter
+import org.apache.spark.unsafe.types.UTF8String
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
 
 object CastExpressionOptimization {
 
-
   def typeCastStringToLong(v: Any, dataType: DataType): Any = {
-    var parser: SimpleDateFormat = null
-    if (dataType == TimestampType) {
-      parser = new SimpleDateFormat(CarbonProperties.getInstance
-        .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-          CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT))
-    } else if (dataType == DateType) {
-      parser = new SimpleDateFormat(CarbonProperties.getInstance
-        .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
-          CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT))
-      parser.setTimeZone(TimeZone.getTimeZone("GMT"))
-    } else {
-      throw new UnsupportedOperationException("Unsupported DataType being evaluated.")
-    }
-    try {
-      val value = parser.parse(v.toString).getTime() * 1000L
-      value
-    } catch {
-      case e: ParseException =>
+    if (dataType == TimestampType || dataType == DateType) {
+      val value = if (dataType == TimestampType) {
+        DateTimeUtils.stringToTimestamp(UTF8String.fromString(v.toString))
+      } else {
+        None
+      }
+      if (value.isDefined) {
+        value.get
+      } else {
+        var parser: SimpleDateFormat = null
+        if (dataType == TimestampType) {
+          parser = new SimpleDateFormat(CarbonProperties.getInstance
+            .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+              CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT))
+        } else if (dataType == DateType) {
+          parser = new SimpleDateFormat(CarbonProperties.getInstance
+            .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
+              CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT))
+          parser.setTimeZone(TimeZone.getTimeZone("GMT"))
+        }
         try {
-          val parsenew: SimpleDateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSz")
-          parsenew.parse(v.toString).getTime() * 1000L
+          val value = parser.parse(v.toString).getTime() * 1000L
+          value
         } catch {
           case e: ParseException =>
-            val gmtDay = new SimpleDateFormat("yyyy-MM-dd", Locale.US)
-            gmtDay.setTimeZone(TimeZone.getTimeZone("GMT"))
             try {
-              gmtDay.parse(v.toString).getTime() * 1000L
+              val parsenew: SimpleDateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSz")
+              parsenew.parse(v.toString).getTime() * 1000L
             } catch {
               case e: ParseException =>
-                v
+                val gmtDay = new SimpleDateFormat("yyyy-MM-dd", Locale.US)
+                gmtDay.setTimeZone(TimeZone.getTimeZone("GMT"))
+                try {
+                  gmtDay.parse(v.toString).getTime() * 1000L
+                } catch {
+                  case e: ParseException =>
+                    v
+                  case e: Exception =>
+                    v
+                }
               case e: Exception =>
                 v
             }
           case e: Exception =>
             v
         }
-      case e: Exception =>
-        v
+      }
+    }
+    else {
+      throw new UnsupportedOperationException("Unsupported DataType being evaluated.")
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/11bd0ade/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapSuite.scala b/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapSuite.scala
index 3360530..3b5b5ca 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapSuite.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapSuite.scala
@@ -756,17 +756,10 @@ class BloomCoarseGrainDataMapSuite extends QueryTest with BeforeAndAfterAll with
   }
 
   test("test bloom datamap on all basic data types") {
-    val originTimestampFormat = CarbonProperties.getInstance().getProperty(
-      CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-      CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
-    val originDateFormat = CarbonProperties.getInstance().getProperty(
-      CarbonCommonConstants.CARBON_DATE_FORMAT,
-      CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT)
-
     CarbonProperties.getInstance().addProperty(
-      CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd HH:mm:ss")
+      CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
     CarbonProperties.getInstance().addProperty(
-      CarbonCommonConstants.CARBON_DATE_FORMAT, "yyyy/MM/dd")
+      CarbonCommonConstants.CARBON_DATE_FORMAT, CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT)
 
     val columnNames = "booleanField,shortField,intField,bigintField,doubleField,stringField," +
       "timestampField,decimalField,dateField,charField,floatField"
@@ -811,17 +804,17 @@ class BloomCoarseGrainDataMapSuite extends QueryTest with BeforeAndAfterAll with
     sql(
       s"""
          | INSERT INTO TABLE $bloomDMSampleTable
-         | VALUES(true,1,10,100,48.4,'spark','2015/4/23 12:01:01',1.23,'2015/4/23','aaa',2.5),
-         | (true,1,11,100,44.4,'flink','2015/5/23 12:01:03',23.23,'2015/5/23','ccc',2.15),
-         | (true,3,14,160,43.4,'hive','2015/7/26 12:01:06',3454.32,'2015/7/26','ff',5.5),
+         | VALUES(true,1,10,100,48.4,'spark','2015-4-23 12:01:01',1.23,'2015-4-23','aaa',2.5),
+         | (true,1,11,100,44.4,'flink','2015-5-23 12:01:03',23.23,'2015-5-23','ccc',2.15),
+         | (true,3,14,160,43.4,'hive','2015-7-26 12:01:06',3454.32,'2015-7-26','ff',5.5),
          | (NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL)
        """.stripMargin)
     sql(
       s"""
          | INSERT INTO TABLE $normalTable
-         | VALUES(true,1,10,100,48.4,'spark','2015/4/23 12:01:01',1.23,'2015/4/23','aaa',2.5),
-         | (true,1,11,100,44.4,'flink','2015/5/23 12:01:03',23.23,'2015/5/23','ccc',2.15),
-         | (true,3,14,160,43.4,'hive','2015/7/26 12:01:06',3454.32,'2015/7/26','ff',5.5),
+         | VALUES(true,1,10,100,48.4,'spark','2015-4-23 12:01:01',1.23,'2015-4-23','aaa',2.5),
+         | (true,1,11,100,44.4,'flink','2015-5-23 12:01:03',23.23,'2015-5-23','ccc',2.15),
+         | (true,3,14,160,43.4,'hive','2015-7-26 12:01:06',3454.32,'2015-7-26','ff',5.5),
          | (NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL)
        """.stripMargin)
 
@@ -838,17 +831,17 @@ class BloomCoarseGrainDataMapSuite extends QueryTest with BeforeAndAfterAll with
     sql(
       s"""
          | INSERT INTO TABLE $bloomDMSampleTable
-         | VALUES(true,1,10,100,48.4,'spark','2015/4/23 12:01:01',1.23,'2015/4/23','aaa',2.5),
-         | (true,1,11,100,44.4,'flink','2015/5/23 12:01:03',23.23,'2015/5/23','ccc',2.15),
-         | (true,3,14,160,43.4,'hive','2015/7/26 12:01:06',3454.32,'2015/7/26','ff',5.5),
+         | VALUES(true,1,10,100,48.4,'spark','2015-4-23 12:01:01',1.23,'2015-4-23','aaa',2.5),
+         | (true,1,11,100,44.4,'flink','2015-5-23 12:01:03',23.23,'2015-5-23','ccc',2.15),
+         | (true,3,14,160,43.4,'hive','2015-7-26 12:01:06',3454.32,'2015-7-26','ff',5.5),
          | (NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL)
        """.stripMargin)
     sql(
       s"""
          | INSERT INTO TABLE $normalTable
-         | VALUES(true,1,10,100,48.4,'spark','2015/4/23 12:01:01',1.23,'2015/4/23','aaa',2.5),
-         | (true,1,11,100,44.4,'flink','2015/5/23 12:01:03',23.23,'2015/5/23','ccc',2.15),
-         | (true,3,14,160,43.4,'hive','2015/7/26 12:01:06',3454.32,'2015/7/26','ff',5.5),
+         | VALUES(true,1,10,100,48.4,'spark','2015-4-23 12:01:01',1.23,'2015-4-23','aaa',2.5),
+         | (true,1,11,100,44.4,'flink','2015-5-23 12:01:03',23.23,'2015-5-23','ccc',2.15),
+         | (true,3,14,160,43.4,'hive','2015-7-26 12:01:06',3454.32,'2015-7-26','ff',5.5),
          | (NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL)
        """.stripMargin)
 
@@ -866,12 +859,12 @@ class BloomCoarseGrainDataMapSuite extends QueryTest with BeforeAndAfterAll with
     checkAnswer(sql(s"SELECT * FROM $bloomDMSampleTable WHERE stringField = 'spark'"),
       sql(s"SELECT * FROM $normalTable WHERE stringField = 'spark'"))
     checkAnswer(
-      sql(s"SELECT * FROM $bloomDMSampleTable WHERE timestampField = '2015/7/26 12:01:06'"),
-      sql(s"SELECT * FROM $normalTable WHERE timestampField = '2015/7/26 12:01:06'"))
+      sql(s"SELECT * FROM $bloomDMSampleTable WHERE timestampField = '2015-7-26 12:01:06'"),
+      sql(s"SELECT * FROM $normalTable WHERE timestampField = '2015-7-26 12:01:06'"))
     checkAnswer(sql(s"SELECT * FROM $bloomDMSampleTable WHERE decimalField = 23.23"),
       sql(s"SELECT * FROM $normalTable WHERE decimalField = 23.23"))
-    checkAnswer(sql(s"SELECT * FROM $bloomDMSampleTable WHERE dateField = '2015/4/23'"),
-      sql(s"SELECT * FROM $normalTable WHERE dateField = '2015/4/23'"))
+    checkAnswer(sql(s"SELECT * FROM $bloomDMSampleTable WHERE dateField = '2015-4-23'"),
+      sql(s"SELECT * FROM $normalTable WHERE dateField = '2015-4-23'"))
     checkAnswer(sql(s"SELECT * FROM $bloomDMSampleTable WHERE charField = 'ccc'"),
       sql(s"SELECT * FROM $normalTable WHERE charField = 'ccc'"))
     checkAnswer(sql(s"SELECT * FROM $bloomDMSampleTable WHERE floatField = 2.5"),
@@ -919,10 +912,6 @@ class BloomCoarseGrainDataMapSuite extends QueryTest with BeforeAndAfterAll with
     checkAnswer(sql(s"SELECT * FROM $bloomDMSampleTable WHERE floatField = 0"),
       sql(s"SELECT * FROM $normalTable WHERE floatField = 0"))
 
-    CarbonProperties.getInstance().addProperty(
-      CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, originTimestampFormat)
-    CarbonProperties.getInstance().addProperty(
-      CarbonCommonConstants.CARBON_DATE_FORMAT, originDateFormat)
   }
 
   test("test bloom datamap on multiple columns") {


[43/45] carbondata git commit: [CARBONDATA-2993] fix random NPE while concurrent loading

Posted by ra...@apache.org.
[CARBONDATA-2993] fix random NPE while concurrent loading

This closes #2797


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

Branch: refs/heads/branch-1.5
Commit: fa0882569872d3280807a5a57f36c4c43f48cc99
Parents: ca30ad9
Author: kunal642 <ku...@gmail.com>
Authored: Fri Oct 5 10:13:05 2018 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Fri Oct 5 15:31:33 2018 +0530

----------------------------------------------------------------------
 .../scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala   | 9 +++++----
 .../org/apache/carbondata/sdk/file/AvroCarbonWriter.java    | 2 +-
 2 files changed, 6 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa088256/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
index 87d8f50..3a02f85 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
@@ -47,8 +47,10 @@ abstract class CarbonRDD[T: ClassTag](
     info
   }
 
+  @transient val hadoopConf = SparkSQLUtil.sessionState(ss).newHadoopConf()
+
   val config: Broadcast[SerializableConfiguration] = sparkContext
-    .broadcast(new SerializableConfiguration(SparkSQLUtil.sessionState(ss).newHadoopConf()))
+    .broadcast(new SerializableConfiguration(hadoopConf))
 
   /** Construct an RDD with just a one-to-one dependency on one parent */
   def this(@transient sparkSession: SparkSession, @transient oneParent: RDD[_]) =
@@ -57,7 +59,7 @@ abstract class CarbonRDD[T: ClassTag](
   protected def internalGetPartitions: Array[Partition]
 
   override def getPartitions: Array[Partition] = {
-    ThreadLocalSessionInfo.setConfigurationToCurrentThread(config.value.value)
+    ThreadLocalSessionInfo.setConfigurationToCurrentThread(hadoopConf)
     internalGetPartitions
   }
 
@@ -66,8 +68,7 @@ abstract class CarbonRDD[T: ClassTag](
 
   final def compute(split: Partition, context: TaskContext): Iterator[T] = {
     TaskContext.get.addTaskCompletionListener(_ => ThreadLocalSessionInfo.unsetAll())
-    carbonSessionInfo.getNonSerializableExtraInfo.put("carbonConf", config
-      .value.value)
+    carbonSessionInfo.getNonSerializableExtraInfo.put("carbonConf", getConf)
     ThreadLocalSessionInfo.setCarbonSessionInfo(carbonSessionInfo)
     TaskMetricsMap.threadLocal.set(Thread.currentThread().getId)
     val carbonTaskInfo = new CarbonTaskInfo

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa088256/store/sdk/src/main/java/org/apache/carbondata/sdk/file/AvroCarbonWriter.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/AvroCarbonWriter.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/AvroCarbonWriter.java
index d19a96d..e4a65c0 100644
--- a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/AvroCarbonWriter.java
+++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/AvroCarbonWriter.java
@@ -520,7 +520,7 @@ public class AvroCarbonWriter extends CarbonWriter {
         // recursively get the sub fields
         ArrayList<StructField> arraySubField = new ArrayList<>();
         // array will have only one sub field.
-        StructField structField = prepareSubFields("val", childSchema.getElementType());
+        StructField structField = prepareSubFields(fieldName, childSchema.getElementType());
         if (structField != null) {
           arraySubField.add(structField);
           return new Field(fieldName, "array", arraySubField);


[17/45] carbondata git commit: [CARBONDATA-2974] Fixed multiple expressions issue on datamap chooser and bloom datamap

Posted by ra...@apache.org.
[CARBONDATA-2974] Fixed multiple expressions issue on datamap chooser and bloom datamap

DataMap framework provide a mechanism to composite expression and
forward it to corresponding datamap, in this way, the datamap can handle
the pruning in batch. But currently the expressions the framework
forwarded contains the one that cannot be supported by the datamap, so
here we optimize the datamap chooser.

We will composite the expression and wrap them into AndExpression. These
expressions are exactly the datamap wanted. The bloomfilter datamap
changed accordingly to handle the AndExpression.

This closes #2767


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

Branch: refs/heads/branch-1.5
Commit: 8284d9ed1fe60d8881788656b7f78c055f76e453
Parents: 8427771
Author: ravipesala <ra...@gmail.com>
Authored: Wed Sep 26 16:56:03 2018 +0530
Committer: xuchuanyin <xu...@hust.edu.cn>
Committed: Fri Sep 28 16:46:49 2018 +0800

----------------------------------------------------------------------
 .../carbondata/core/datamap/DataMapChooser.java | 76 ++++++++++----------
 .../datamap/bloom/BloomCoarseGrainDataMap.java  |  8 ++-
 .../bloom/BloomCoarseGrainDataMapSuite.scala    | 62 +++++++++++++++-
 3 files changed, 106 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/8284d9ed/core/src/main/java/org/apache/carbondata/core/datamap/DataMapChooser.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapChooser.java b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapChooser.java
index 68696cf..3b6537c 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapChooser.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapChooser.java
@@ -39,6 +39,7 @@ import org.apache.carbondata.core.scan.expression.logical.AndExpression;
 import org.apache.carbondata.core.scan.expression.logical.OrExpression;
 import org.apache.carbondata.core.scan.filter.intf.ExpressionType;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
+import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.TrueConditionalResolverImpl;
 
 /**
  * This chooser does 2 jobs.
@@ -123,9 +124,11 @@ public class DataMapChooser {
     if (resolverIntf != null) {
       Expression expression = resolverIntf.getFilterExpression();
       List<TableDataMap> datamaps = level == DataMapLevel.CG ? cgDataMaps : fgDataMaps;
-      ExpressionTuple tuple = selectDataMap(expression, datamaps, resolverIntf);
-      if (tuple.dataMapExprWrapper != null) {
-        return tuple.dataMapExprWrapper;
+      if (datamaps.size() > 0) {
+        ExpressionTuple tuple = selectDataMap(expression, datamaps, resolverIntf);
+        if (tuple.dataMapExprWrapper != null) {
+          return tuple.dataMapExprWrapper;
+        }
       }
     }
     return null;
@@ -177,34 +180,35 @@ public class DataMapChooser {
           // If both left and right has datamap then we can either merge both datamaps to single
           // datamap if possible. Otherwise apply AND expression.
           if (left.dataMapExprWrapper != null && right.dataMapExprWrapper != null) {
-            filterExpressionTypes.add(
-                left.dataMapExprWrapper.getFilterResolverIntf().getFilterExpression()
-                    .getFilterExpressionType());
-            filterExpressionTypes.add(
-                right.dataMapExprWrapper.getFilterResolverIntf().getFilterExpression()
-                    .getFilterExpressionType());
+            filterExpressionTypes.addAll(left.filterExpressionTypes);
+            filterExpressionTypes.addAll(right.filterExpressionTypes);
             List<ColumnExpression> columnExpressions = new ArrayList<>();
             columnExpressions.addAll(left.columnExpressions);
             columnExpressions.addAll(right.columnExpressions);
             // Check if we can merge them to single datamap.
             TableDataMap dataMap =
                 chooseDataMap(allDataMap, columnExpressions, filterExpressionTypes);
+            TrueConditionalResolverImpl resolver = new TrueConditionalResolverImpl(
+                new AndExpression(left.expression, right.expression), false,
+                true);
             if (dataMap != null) {
               ExpressionTuple tuple = new ExpressionTuple();
               tuple.columnExpressions = columnExpressions;
-              tuple.dataMapExprWrapper = new DataMapExprWrapperImpl(dataMap, filterResolverIntf);
+              tuple.dataMapExprWrapper = new DataMapExprWrapperImpl(dataMap, resolver);
+              tuple.expression = resolver.getFilterExpression();
               return tuple;
             } else {
               // Apply AND expression.
               ExpressionTuple tuple = new ExpressionTuple();
               tuple.columnExpressions = columnExpressions;
               tuple.dataMapExprWrapper = new AndDataMapExprWrapper(left.dataMapExprWrapper,
-                  right.dataMapExprWrapper, filterResolverIntf);
+                  right.dataMapExprWrapper, resolver);
+              tuple.expression = resolver.getFilterExpression();
               return tuple;
             }
-          } else if (left.dataMapExprWrapper != null && right.dataMapExprWrapper == null) {
+          } else if (left.dataMapExprWrapper != null) {
             return left;
-          } else if (left.dataMapExprWrapper == null && right.dataMapExprWrapper != null) {
+          } else if (right.dataMapExprWrapper != null) {
             return right;
           } else {
             return left;
@@ -218,33 +222,21 @@ public class DataMapChooser {
               filterResolverIntf.getLeft());
           ExpressionTuple right = selectDataMap(orExpression.getRight(), allDataMap,
               filterResolverIntf.getRight());
-          Set<ExpressionType> filterExpressionTypes = new HashSet<>();
           // If both left and right has datamap then we can either merge both datamaps to single
           // datamap if possible. Otherwise apply OR expression.
           if (left.dataMapExprWrapper != null && right.dataMapExprWrapper != null) {
-            filterExpressionTypes.add(
-                left.dataMapExprWrapper.getFilterResolverIntf().getFilterExpression()
-                    .getFilterExpressionType());
-            filterExpressionTypes.add(
-                right.dataMapExprWrapper.getFilterResolverIntf().getFilterExpression()
-                    .getFilterExpressionType());
+            TrueConditionalResolverImpl resolver = new TrueConditionalResolverImpl(
+                new OrExpression(left.expression, right.expression), false,
+                true);
             List<ColumnExpression> columnExpressions = new ArrayList<>();
             columnExpressions.addAll(left.columnExpressions);
             columnExpressions.addAll(right.columnExpressions);
-            TableDataMap dataMap =
-                chooseDataMap(allDataMap, columnExpressions, filterExpressionTypes);
-            if (dataMap != null) {
-              ExpressionTuple tuple = new ExpressionTuple();
-              tuple.columnExpressions = columnExpressions;
-              tuple.dataMapExprWrapper = new DataMapExprWrapperImpl(dataMap, filterResolverIntf);
-              return tuple;
-            } else {
-              ExpressionTuple tuple = new ExpressionTuple();
-              tuple.columnExpressions = columnExpressions;
-              tuple.dataMapExprWrapper = new OrDataMapExprWrapper(left.dataMapExprWrapper,
-                  right.dataMapExprWrapper, filterResolverIntf);
-              return tuple;
-            }
+            ExpressionTuple tuple = new ExpressionTuple();
+            tuple.columnExpressions = columnExpressions;
+            tuple.dataMapExprWrapper = new OrDataMapExprWrapper(left.dataMapExprWrapper,
+                right.dataMapExprWrapper, resolver);
+            tuple.expression = resolver.getFilterExpression();
+            return tuple;
           } else {
             left.dataMapExprWrapper = null;
             return left;
@@ -256,16 +248,22 @@ public class DataMapChooser {
         extractColumnExpression(expression, tuple.columnExpressions);
         Set<ExpressionType> filterExpressionTypes = new HashSet<>();
         filterExpressionTypes.add(expression.getFilterExpressionType());
+        TrueConditionalResolverImpl resolver = new TrueConditionalResolverImpl(
+            filterResolverIntf.getFilterExpression(), false,
+            true);
         TableDataMap dataMap =
             chooseDataMap(allDataMap, tuple.columnExpressions, filterExpressionTypes);
         if (dataMap != null) {
-          tuple.dataMapExprWrapper = new DataMapExprWrapperImpl(dataMap, filterResolverIntf);
+          tuple.dataMapExprWrapper = new DataMapExprWrapperImpl(dataMap, resolver);
+          tuple.filterExpressionTypes.addAll(filterExpressionTypes);
+          tuple.expression = filterResolverIntf.getFilterExpression();
         }
         return tuple;
     }
     return new ExpressionTuple();
   }
 
+
   private void extractColumnExpression(Expression expression,
       List<ColumnExpression> columnExpressions) {
     if (expression instanceof ColumnExpression) {
@@ -282,7 +280,9 @@ public class DataMapChooser {
       List<Expression> children = expression.getChildren();
       if (children != null && children.size() > 0) {
         for (Expression exp : children) {
-          extractColumnExpression(exp, columnExpressions);
+          if (exp != null && exp.getFilterExpressionType() != ExpressionType.UNKNOWN) {
+            extractColumnExpression(exp, columnExpressions);
+          }
         }
       }
     }
@@ -332,6 +332,10 @@ public class DataMapChooser {
 
     List<ColumnExpression> columnExpressions = new ArrayList<>();
 
+    Set<ExpressionType> filterExpressionTypes = new HashSet<>();
+
+    Expression expression;
+
   }
 
   private static class DataMapTuple implements Comparable<DataMapTuple> {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/8284d9ed/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java
----------------------------------------------------------------------
diff --git a/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java b/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java
index 344ec09..ee71142 100644
--- a/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java
+++ b/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java
@@ -59,6 +59,7 @@ import org.apache.carbondata.core.scan.expression.LiteralExpression;
 import org.apache.carbondata.core.scan.expression.conditional.EqualToExpression;
 import org.apache.carbondata.core.scan.expression.conditional.InExpression;
 import org.apache.carbondata.core.scan.expression.conditional.ListExpression;
+import org.apache.carbondata.core.scan.expression.logical.AndExpression;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -264,11 +265,12 @@ public class BloomCoarseGrainDataMap extends CoarseGrainDataMap {
         LOGGER.warn(errorMsg);
         throw new RuntimeException(errorMsg);
       }
+    }  else if (expression instanceof AndExpression) {
+      queryModels.addAll(createQueryModel(((AndExpression) expression).getLeft()));
+      queryModels.addAll(createQueryModel(((AndExpression) expression).getRight()));
+      return queryModels;
     }
 
-    for (Expression child : expression.getChildren()) {
-      queryModels.addAll(createQueryModel(child));
-    }
     return queryModels;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/8284d9ed/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapSuite.scala b/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapSuite.scala
index 0a22937..84edd73 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapSuite.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapSuite.scala
@@ -22,7 +22,7 @@ import java.util.UUID
 
 import scala.util.Random
 
-import org.apache.spark.sql.{CarbonSession, DataFrame}
+import org.apache.spark.sql.{CarbonSession, DataFrame, Row}
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
 
@@ -923,6 +923,66 @@ class BloomCoarseGrainDataMapSuite extends QueryTest with BeforeAndAfterAll with
       CarbonCommonConstants.CARBON_DATE_FORMAT, originDateFormat)
   }
 
+  test("test bloom datamap on multiple columns") {
+    sql("drop table if exists store")
+        sql(
+          s"""
+             |CREATE TABLE IF NOT EXISTS store(
+             | market_code STRING,
+             | device_code STRING,
+             | country_code STRING,
+             | category_id INTEGER,
+             | product_id string,
+             | date date,
+             | est_free_app_download LONG,
+             | est_paid_app_download LONG,
+             | est_revenue LONG
+             | )
+             | STORED BY 'carbondata'
+             | TBLPROPERTIES(
+             | 'SORT_COLUMNS'='market_code, device_code, country_code, category_id, date,product_id',
+             | 'NO_INVERTED_INDEX'='est_free_app_download, est_paid_app_download,est_revenue',
+             | 'DICTIONARY_INCLUDE' = 'market_code, device_code, country_code,category_id, product_id',
+             | 'SORT_SCOPE'='GLOBAL_SORT',
+             | 'CACHE_LEVEL'='BLOCKLET',  'TABLE_BLOCKSIZE'='256',
+             | 'GLOBAL_SORT_PARTITIONS'='2'
+             | )""".stripMargin)
+
+    sql(s"""insert into store values('a', 'ios-phone', 'EE', 100021, 590416158, '2016-09-01', 100, 200, 300)""")
+    sql(s"""insert into store values('b', 'ios-phone', 'EE', 100021, 590437560, '2016-09-03', 100, 200, 300)""")
+    sql(s"""insert into store values('a', 'ios-phone', 'EF', 100022, 590416159, '2016-09-04', 100, 200, 300)""")
+
+    sql(
+      s"""
+         |CREATE DATAMAP IF NOT EXISTS bloomfilter_all_dimensions ON TABLE store
+         | USING 'bloomfilter'
+         | DMPROPERTIES (
+         | 'INDEX_COLUMNS'='market_code, device_code, country_code, category_id, date,product_id',
+         | 'BLOOM_SIZE'='640000',
+         | 'BLOOM_FPP'='0.000001',
+         | 'BLOOM_COMPRESS'='true'
+         | )
+       """.stripMargin).show()
+
+    checkAnswer(sql(
+      s"""SELECT market_code, device_code, country_code,
+         |category_id, sum(est_free_app_download) FROM store WHERE date
+         |BETWEEN '2016-09-01' AND '2016-09-03' AND device_code='ios-phone'
+         |AND country_code='EE' AND category_id=100021 AND product_id IN (590416158, 590437560)
+         |GROUP BY date, market_code, device_code, country_code, category_id""".stripMargin),
+      Seq(Row("a", "ios-phone", "EE", 100021, 100), Row("b", "ios-phone", "EE", 100021, 100)))
+
+    assert(sql(
+      s"""SELECT market_code, device_code, country_code,
+         |category_id, sum(est_free_app_download) FROM store WHERE (device_code='ios-phone'
+         |AND country_code='EF') or (category_id=100021 AND product_id IN (590416158, 590437560))
+         |GROUP BY date, market_code, device_code, country_code, category_id""".stripMargin).collect().length == 3)
+
+    checkAnswer(sql("select device_code from store where product_id=590416158"), Seq(Row("ios-phone")))
+
+    sql("drop table if exists store")
+  }
+
   override protected def afterAll(): Unit = {
     // in case of search mode test case failed, stop search mode again
     if (carbonSession.isSearchModeEnabled) {


[29/45] carbondata git commit: [CARBONDATA-2984][Streaming] Fix NPE when there is no data in the task of a batch

Posted by ra...@apache.org.
[CARBONDATA-2984][Streaming] Fix NPE when there is no data in the task of a batch

Fix NPE when there is no data in the task of a batch

Streaming batch maybe has no data, so it doesn't require to append blocklet to streaming file. So it doesn't need to update min/max index of streaming file, just use min/max index of old file .

This closes #2782


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

Branch: refs/heads/branch-1.5
Commit: fa9c8323c11c083452d75886cbbdad1f23d6dfb7
Parents: 0b16816
Author: QiangCai <qi...@qq.com>
Authored: Fri Sep 28 14:48:39 2018 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Wed Oct 3 20:13:50 2018 +0530

----------------------------------------------------------------------
 .../TestStreamingTableOperation.scala           | 49 +++++++++++++++++++-
 .../streaming/CarbonStreamRecordWriter.java     |  5 +-
 .../streaming/segment/StreamSegment.java        | 15 ++++--
 3 files changed, 61 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa9c8323/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
index 43c1e5a..607c429 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
@@ -37,6 +37,7 @@ import org.scalatest.BeforeAndAfterAll
 import org.apache.carbondata.common.exceptions.NoSuchStreamException
 import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider.TIMESERIES
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.statusmanager.{FileFormat, SegmentStatus}
@@ -125,6 +126,8 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
 
     createTable(tableName = "agg_table", streaming = true, withBatchLoad = false)
 
+    createTable(tableName = "stream_table_empty", streaming = true, withBatchLoad = false)
+
     var csvDataDir = integrationPath + "/spark2/target/csvdatanew"
     generateCSVDataFile(spark, idStart = 10, rowNums = 5, csvDataDir)
     generateCSVDataFile(spark, idStart = 10, rowNums = 5, csvDataDir, SaveMode.Append)
@@ -213,6 +216,7 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
     sql("drop table if exists streaming.stream_table_reopen")
     sql("drop table if exists streaming.stream_table_drop")
     sql("drop table if exists streaming.agg_table_block")
+    sql("drop table if exists streaming.stream_table_empty")
   }
 
   // normal table not support streaming ingest
@@ -226,7 +230,7 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
       .asInstanceOf[CarbonRelation].metaData.carbonTable
     var server: ServerSocket = null
     try {
-      server = getServerSocket
+      server = getServerSocket()
       val thread1 = createWriteSocketThread(server, 2, 10, 1)
       thread1.start()
       // use thread pool to catch the exception of sink thread
@@ -2253,6 +2257,46 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
     sql("DROP TABLE IF EXISTS dim")
   }
 
+  // test empty batch
+  test("test empty batch") {
+    executeStreamingIngest(
+      tableName = "stream_table_empty",
+      batchNums = 1,
+      rowNumsEachBatch = 10,
+      intervalOfSource = 1,
+      intervalOfIngest = 3,
+      continueSeconds = 10,
+      generateBadRecords = false,
+      badRecordAction = "force",
+      autoHandoff = false
+    )
+    var result = sql("select count(*) from streaming.stream_table_empty").collect()
+    assert(result(0).getLong(0) == 10)
+
+    // clean checkpointDir and logDir
+    val carbonTable = CarbonEnv.getCarbonTable(Option("streaming"), "stream_table_empty")(spark)
+    FileFactory
+      .deleteAllFilesOfDir(new File(CarbonTablePath.getStreamingLogDir(carbonTable.getTablePath)))
+    FileFactory
+      .deleteAllFilesOfDir(new File(CarbonTablePath
+        .getStreamingCheckpointDir(carbonTable.getTablePath)))
+
+    // some batches don't have data
+    executeStreamingIngest(
+      tableName = "stream_table_empty",
+      batchNums = 1,
+      rowNumsEachBatch = 1,
+      intervalOfSource = 1,
+      intervalOfIngest = 1,
+      continueSeconds = 10,
+      generateBadRecords = false,
+      badRecordAction = "force",
+      autoHandoff = false
+    )
+    result = sql("select count(*) from streaming.stream_table_empty").collect()
+    assert(result(0).getLong(0) == 11)
+  }
+
   def createWriteSocketThread(
       serverSocket: ServerSocket,
       writeNums: Int,
@@ -2330,7 +2374,8 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
             .load()
 
           // Write data from socket stream to carbondata file
-          qry = readSocketDF.writeStream
+          // repartition to simulate an empty partition when readSocketDF has only one row
+          qry = readSocketDF.repartition(2).writeStream
             .format("carbondata")
             .trigger(ProcessingTime(s"$intervalSecond seconds"))
             .option("checkpointLocation", CarbonTablePath.getStreamingCheckpointDir(carbonTable.getTablePath))

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa9c8323/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamRecordWriter.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamRecordWriter.java b/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamRecordWriter.java
index 0d2a889..672f6a6 100644
--- a/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamRecordWriter.java
+++ b/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamRecordWriter.java
@@ -139,9 +139,7 @@ public class CarbonStreamRecordWriter extends RecordWriter<Void, Object> {
     segmentDir = CarbonTablePath.getSegmentPath(
         carbonTable.getAbsoluteTableIdentifier().getTablePath(), segmentId);
     fileName = CarbonTablePath.getCarbonDataFileName(0, taskNo, 0, 0, "0", segmentId);
-  }
 
-  private void initializeAtFirstRow() throws IOException, InterruptedException {
     // initialize metadata
     isNoDictionaryDimensionColumn =
         CarbonDataProcessorUtil.getNoDictionaryMapping(configuration.getDataFields());
@@ -153,6 +151,9 @@ public class CarbonStreamRecordWriter extends RecordWriter<Void, Object> {
       measureDataTypes[i] =
           dataFields[dimensionWithComplexCount + i].getColumn().getDataType();
     }
+  }
+
+  private void initializeAtFirstRow() throws IOException, InterruptedException {
     // initialize parser and converter
     rowParser = new RowParserImpl(dataFields, configuration);
     badRecordLogger = BadRecordsLoggerProvider.createBadRecordLogger(configuration);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa9c8323/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java b/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java
index 51417c4..6ee6876 100644
--- a/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java
+++ b/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java
@@ -458,6 +458,13 @@ public class StreamSegment {
       return;
     }
 
+    BlockletMinMaxIndex minMaxIndex = blockletIndex.getMinMaxIndex();
+    // if min/max of new blocklet is null, use min/max of old file
+    if (minMaxIndex == null) {
+      blockletIndex.setMinMaxIndex(fileIndex);
+      return;
+    }
+
     DataType[] msrDataTypes = blockletIndex.getMsrDataTypes();
     SerializableComparator[] comparators = new SerializableComparator[msrDataTypes.length];
     for (int index = 0; index < comparators.length; index++) {
@@ -465,11 +472,11 @@ public class StreamSegment {
     }
 
     // min value
-    byte[][] minValues = blockletIndex.getMinMaxIndex().getMinValues();
+    byte[][] minValues = minMaxIndex.getMinValues();
     byte[][] mergedMinValues = fileIndex.getMinValues();
     if (minValues == null || minValues.length == 0) {
       // use file index
-      blockletIndex.getMinMaxIndex().setMinValues(mergedMinValues);
+      minMaxIndex.setMinValues(mergedMinValues);
     } else if (mergedMinValues != null && mergedMinValues.length != 0) {
       if (minValues.length != mergedMinValues.length) {
         throw new IOException("the lengths of the min values should be same.");
@@ -494,10 +501,10 @@ public class StreamSegment {
     }
 
     // max value
-    byte[][] maxValues = blockletIndex.getMinMaxIndex().getMaxValues();
+    byte[][] maxValues = minMaxIndex.getMaxValues();
     byte[][] mergedMaxValues = fileIndex.getMaxValues();
     if (maxValues == null || maxValues.length == 0) {
-      blockletIndex.getMinMaxIndex().setMaxValues(mergedMaxValues);
+      minMaxIndex.setMaxValues(mergedMaxValues);
     } else if (mergedMaxValues != null && mergedMaxValues.length != 0) {
       if (maxValues.length != mergedMaxValues.length) {
         throw new IOException("the lengths of the max values should be same.");


[02/45] carbondata git commit: [CARBONDATA-2962]Even after carbon file is copied to targetfolder(local/hdfs), carbon files is not deleted from temp directory

Posted by ra...@apache.org.
[CARBONDATA-2962]Even after carbon file is copied to targetfolder(local/hdfs), carbon files is not deleted from temp directory

Problem:
Even after carbon file is copied to targetfolder(local/hdfs), carbon files is not deleted from temp directory.
Solution:
After copying Carbon data and index files from temp directory, delete those files.

This closes #2752


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

Branch: refs/heads/branch-1.5
Commit: 49f67153a21e5a0cb5705adeb0f056eef4d3ed25
Parents: 2ab2254
Author: Indhumathi27 <in...@gmail.com>
Authored: Mon Sep 24 12:28:47 2018 +0530
Committer: kumarvishal09 <ku...@gmail.com>
Committed: Wed Sep 26 12:35:24 2018 +0530

----------------------------------------------------------------------
 .../store/writer/AbstractFactDataWriter.java    | 20 ++++++++++++++------
 1 file changed, 14 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/49f67153/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java b/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
index ad0e8e0..4afb3ef 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
@@ -270,12 +270,18 @@ public abstract class AbstractFactDataWriter implements CarbonFactDataWriter {
     notifyDataMapBlockEnd();
     CarbonUtil.closeStreams(this.fileOutputStream, this.fileChannel);
     if (!enableDirectlyWriteData2Hdfs) {
-      if (copyInCurrentThread) {
-        CarbonUtil.copyCarbonDataFileToCarbonStorePath(carbonDataFileTempPath,
-            model.getCarbonDataDirectoryPath(), fileSizeInBytes);
-      } else {
-        executorServiceSubmitList.add(executorService.submit(
-            new CompleteHdfsBackendThread(carbonDataFileTempPath)));
+      try {
+        if (copyInCurrentThread) {
+          CarbonUtil.copyCarbonDataFileToCarbonStorePath(carbonDataFileTempPath,
+              model.getCarbonDataDirectoryPath(), fileSizeInBytes);
+          FileFactory
+              .deleteFile(carbonDataFileTempPath, FileFactory.getFileType(carbonDataFileTempPath));
+        } else {
+          executorServiceSubmitList
+              .add(executorService.submit(new CompleteHdfsBackendThread(carbonDataFileTempPath)));
+        }
+      } catch (IOException e) {
+        LOGGER.error("Failed to delete carbondata file from temp location" + e.getMessage());
       }
     }
   }
@@ -405,6 +411,7 @@ public abstract class AbstractFactDataWriter implements CarbonFactDataWriter {
       CarbonUtil
           .copyCarbonDataFileToCarbonStorePath(indexFileName, model.getCarbonDataDirectoryPath(),
               fileSizeInBytes);
+      FileFactory.deleteFile(indexFileName, FileFactory.getFileType(indexFileName));
     }
   }
 
@@ -470,6 +477,7 @@ public abstract class AbstractFactDataWriter implements CarbonFactDataWriter {
     public Void call() throws Exception {
       CarbonUtil.copyCarbonDataFileToCarbonStorePath(fileName, model.getCarbonDataDirectoryPath(),
           fileSizeInBytes);
+      FileFactory.deleteFile(fileName, FileFactory.getFileType(fileName));
       return null;
     }
   }


[16/45] carbondata git commit: [CARBONDATA-2818] Upgrade presto integration version to 0.210

Posted by ra...@apache.org.
[CARBONDATA-2818] Upgrade presto integration version to 0.210

Upgrade presto integration version to 0.210

This closes #2733


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

Branch: refs/heads/branch-1.5
Commit: 8427771fc22071099186d3310500d5fcd6c419a6
Parents: 629d625
Author: chenliang613 <ch...@huawei.com>
Authored: Wed Sep 19 08:18:28 2018 +0800
Committer: Raghunandan S <ca...@gmail.com>
Committed: Fri Sep 28 11:39:48 2018 +0530

----------------------------------------------------------------------
 docs/quick-start-guide.md                       | 29 ++++++++++++--------
 .../Presto_Cluster_Setup_For_Carbondata.md      | 24 ++++++++--------
 integration/presto/README.md                    | 12 +++++---
 integration/presto/pom.xml                      |  4 +--
 4 files changed, 40 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/8427771f/docs/quick-start-guide.md
----------------------------------------------------------------------
diff --git a/docs/quick-start-guide.md b/docs/quick-start-guide.md
index 37c398c..0fdf055 100644
--- a/docs/quick-start-guide.md
+++ b/docs/quick-start-guide.md
@@ -300,24 +300,24 @@ Once the table is created,it can be queried from Presto.**
 
 ### Installing Presto
 
- 1. Download the 0.187 version of Presto using:
-    `wget https://repo1.maven.org/maven2/com/facebook/presto/presto-server/0.187/presto-server-0.187.tar.gz`
+ 1. Download the 0.210 version of Presto using:
+    `wget https://repo1.maven.org/maven2/com/facebook/presto/presto-server/0.210/presto-server-0.210.tar.gz`
 
- 2. Extract Presto tar file: `tar zxvf presto-server-0.187.tar.gz`.
+ 2. Extract Presto tar file: `tar zxvf presto-server-0.210.tar.gz`.
 
  3. Download the Presto CLI for the coordinator and name it presto.
 
   ```
-    wget https://repo1.maven.org/maven2/com/facebook/presto/presto-cli/0.187/presto-cli-0.187-executable.jar
+    wget https://repo1.maven.org/maven2/com/facebook/presto/presto-cli/0.210/presto-cli-0.210-executable.jar
 
-    mv presto-cli-0.187-executable.jar presto
+    mv presto-cli-0.210-executable.jar presto
 
     chmod +x presto
   ```
 
 ### Create Configuration Files
 
-  1. Create `etc` folder in presto-server-0.187 directory.
+  1. Create `etc` folder in presto-server-0.210 directory.
   2. Create `config.properties`, `jvm.config`, `log.properties`, and `node.properties` files.
   3. Install uuid to generate a node.id.
 
@@ -363,10 +363,15 @@ Once the table is created,it can be queried from Presto.**
   coordinator=true
   node-scheduler.include-coordinator=false
   http-server.http.port=8086
-  query.max-memory=50GB
-  query.max-memory-per-node=2GB
+  query.max-memory=5GB
+  query.max-total-memory-per-node=5GB
+  query.max-memory-per-node=3GB
+  memory.heap-headroom-per-node=1GB
   discovery-server.enabled=true
-  discovery.uri=<coordinator_ip>:8086
+  discovery.uri=http://localhost:8086
+  task.max-worker-threads=4
+  optimizer.dictionary-aggregation=true
+  optimizer.optimize-hash-generation = false
   ```
 The options `node-scheduler.include-coordinator=false` and `coordinator=true` indicate that the node is the coordinator and tells the coordinator not to do any of the computation work itself and to use the workers.
 
@@ -383,7 +388,7 @@ Then, `query.max-memory=<30GB * number of nodes>`.
   ```
   coordinator=false
   http-server.http.port=8086
-  query.max-memory=50GB
+  query.max-memory=5GB
   query.max-memory-per-node=2GB
   discovery.uri=<coordinator_ip>:8086
   ```
@@ -405,12 +410,12 @@ Then, `query.max-memory=<30GB * number of nodes>`.
 ### Start Presto Server on all nodes
 
 ```
-./presto-server-0.187/bin/launcher start
+./presto-server-0.210/bin/launcher start
 ```
 To run it as a background process.
 
 ```
-./presto-server-0.187/bin/launcher run
+./presto-server-0.210/bin/launcher run
 ```
 To run it in foreground.
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/8427771f/integration/presto/Presto_Cluster_Setup_For_Carbondata.md
----------------------------------------------------------------------
diff --git a/integration/presto/Presto_Cluster_Setup_For_Carbondata.md b/integration/presto/Presto_Cluster_Setup_For_Carbondata.md
index 082b8fe..975e1fa 100644
--- a/integration/presto/Presto_Cluster_Setup_For_Carbondata.md
+++ b/integration/presto/Presto_Cluster_Setup_For_Carbondata.md
@@ -2,24 +2,24 @@
 
 ## Installing Presto
 
-  1. Download the 0.187 version of Presto using:
-  `wget https://repo1.maven.org/maven2/com/facebook/presto/presto-server/0.187/presto-server-0.187.tar.gz`
+  1. Download the 0.210 version of Presto using:
+  `wget https://repo1.maven.org/maven2/com/facebook/presto/presto-server/0.210/presto-server-0.210.tar.gz`
 
-  2. Extract Presto tar file: `tar zxvf presto-server-0.187.tar.gz`.
+  2. Extract Presto tar file: `tar zxvf presto-server-0.210.tar.gz`.
 
   3. Download the Presto CLI for the coordinator and name it presto.
 
   ```
-    wget https://repo1.maven.org/maven2/com/facebook/presto/presto-cli/0.187/presto-cli-0.187-executable.jar
+    wget https://repo1.maven.org/maven2/com/facebook/presto/presto-cli/0.210/presto-cli-0.210-executable.jar
 
-    mv presto-cli-0.187-executable.jar presto
+    mv presto-cli-0.210-executable.jar presto
 
     chmod +x presto
   ```
 
  ## Create Configuration Files
 
-  1. Create `etc` folder in presto-server-0.187 directory.
+  1. Create `etc` folder in presto-server-0.210 directory.
   2. Create `config.properties`, `jvm.config`, `log.properties`, and `node.properties` files.
   3. Install uuid to generate a node.id.
 
@@ -65,8 +65,10 @@
   coordinator=true
   node-scheduler.include-coordinator=false
   http-server.http.port=8086
-  query.max-memory=50GB
-  query.max-memory-per-node=2GB
+  query.max-memory=5GB
+  query.max-total-memory-per-node=5GB
+  query.max-memory-per-node=3GB
+  memory.heap-headroom-per-node=1GB
   discovery-server.enabled=true
   discovery.uri=<coordinator_ip>:8086
   ```
@@ -85,7 +87,7 @@ Then, `query.max-memory=<30GB * number of nodes>`.
   ```
   coordinator=false
   http-server.http.port=8086
-  query.max-memory=50GB
+  query.max-memory=5GB
   query.max-memory-per-node=2GB
   discovery.uri=<coordinator_ip>:8086
   ```
@@ -107,12 +109,12 @@ Then, `query.max-memory=<30GB * number of nodes>`.
 ## Start Presto Server on all nodes
 
 ```
-./presto-server-0.187/bin/launcher start
+./presto-server-0.210/bin/launcher start
 ```
 To run it as a background process.
 
 ```
-./presto-server-0.187/bin/launcher run
+./presto-server-0.210/bin/launcher run
 ```
 To run it in foreground.
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/8427771f/integration/presto/README.md
----------------------------------------------------------------------
diff --git a/integration/presto/README.md b/integration/presto/README.md
index 5a44f5c..ceab0fe 100644
--- a/integration/presto/README.md
+++ b/integration/presto/README.md
@@ -18,7 +18,7 @@
 Please follow the below steps to query carbondata in presto
 
 ### Config presto server
-* Download presto server (0.187 is suggested and supported) : https://repo1.maven.org/maven2/com/facebook/presto/presto-server/
+* Download presto server (0.210 is suggested and supported) : https://repo1.maven.org/maven2/com/facebook/presto/presto-server/
 * Finish presto configuration following https://prestodb.io/docs/current/installation/deployment.html.
   A configuration example:
   ```
@@ -27,10 +27,14 @@ Please follow the below steps to query carbondata in presto
   node-scheduler.include-coordinator=true
   http-server.http.port=8086
   query.max-memory=5GB
-  query.max-memory-per-node=1GB
+  query.max-total-memory-per-node=5GB
+  query.max-memory-per-node=3GB
+  memory.heap-headroom-per-node=1GB
   discovery-server.enabled=true
   discovery.uri=http://localhost:8086
-  reorder-joins=true
+  task.max-worker-threads=4
+  optimizer.dictionary-aggregation=true
+  optimizer.optimize-hash-generation = false
  
   
   jvm.config:
@@ -131,7 +135,7 @@ Load data statement in Spark can be used to create carbondata tables. And then y
 carbondata files.
 
 ### Query carbondata in CLI of presto
-* Download presto cli client following: https://prestodb.io/docs/current/installation/cli.html
+* Download presto cli client of version 0.210 : https://repo1.maven.org/maven2/com/facebook/presto/presto-cli
 
 * Start CLI:
   

http://git-wip-us.apache.org/repos/asf/carbondata/blob/8427771f/integration/presto/pom.xml
----------------------------------------------------------------------
diff --git a/integration/presto/pom.xml b/integration/presto/pom.xml
index 9a9bc55..72847e2 100644
--- a/integration/presto/pom.xml
+++ b/integration/presto/pom.xml
@@ -31,7 +31,7 @@
   <packaging>presto-plugin</packaging>
 
   <properties>
-    <presto.version>0.208</presto.version>
+    <presto.version>0.210</presto.version>
     <dev.path>${basedir}/../../dev</dev.path>
     <jacoco.append>true</jacoco.append>
   </properties>
@@ -455,7 +455,7 @@
       <groupId>org.antlr</groupId>
       <artifactId>antlr4-runtime</artifactId>
       <scope>test</scope>
-      <version>4.5.3</version>
+      <version>4.7.1</version>
     </dependency>
     <dependency>
       <groupId>com.google.code.findbugs</groupId>


[31/45] carbondata git commit: [HOTFIX] Changes to align printing of information in explain command based on enable.query.statistics flag

Posted by ra...@apache.org.
[HOTFIX] Changes to align printing of information in explain command based on enable.query.statistics flag

Now user need to set the flag enable.query.statistics = true to print the pruning and dataMap related information in the explain command.

This closes #2795


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

Branch: refs/heads/branch-1.5
Commit: c3a8704494d8d4cffbe8f35c43101ce248913965
Parents: d8003a3
Author: manishgupta88 <to...@gmail.com>
Authored: Wed Oct 3 18:13:45 2018 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Wed Oct 3 20:23:12 2018 +0530

----------------------------------------------------------------------
 .../apache/carbondata/core/profiler/ExplainCollector.java   | 9 ++++++++-
 docs/datamap/datamap-management.md                          | 2 +-
 .../cluster/sdv/generated/BloomFilterDataMapTestCase.scala  | 6 +++++-
 .../datamap/lucene/LuceneFineGrainDataMapSuite.scala        | 5 +++++
 .../lucene/LuceneFineGrainDataMapWithSearchModeSuite.scala  | 5 +++++
 .../preaggregate/TestPreAggregateTableSelection.scala       | 7 +++++++
 .../testsuite/createTable/TestRenameTableWithDataMap.scala  | 8 ++++++++
 .../spark/testsuite/datamap/CGDataMapTestCase.scala         | 5 +++++
 .../spark/testsuite/datamap/FGDataMapTestCase.scala         | 7 ++++++-
 .../src/main/scala/org/apache/spark/sql/CarbonSession.scala | 1 +
 .../sql/execution/command/table/CarbonExplainCommand.scala  | 6 +++++-
 .../datamap/bloom/BloomCoarseGrainDataMapSuite.scala        | 5 +++++
 12 files changed, 61 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3a87044/core/src/main/java/org/apache/carbondata/core/profiler/ExplainCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/profiler/ExplainCollector.java b/core/src/main/java/org/apache/carbondata/core/profiler/ExplainCollector.java
index 755c56a..8513dac 100644
--- a/core/src/main/java/org/apache/carbondata/core/profiler/ExplainCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/profiler/ExplainCollector.java
@@ -26,7 +26,9 @@ import java.util.Objects;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datamap.dev.expr.DataMapWrapperSimpleInfo;
+import org.apache.carbondata.core.util.CarbonProperties;
 
 /**
  * An information collector used for EXPLAIN command, to print out
@@ -52,7 +54,12 @@ public class ExplainCollector {
   }
 
   public static void setup() {
-    INSTANCE = new ExplainCollector();
+    boolean isQueryStatisticsEnabled = Boolean.parseBoolean(CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.ENABLE_QUERY_STATISTICS,
+            CarbonCommonConstants.ENABLE_QUERY_STATISTICS_DEFAULT));
+    if (isQueryStatisticsEnabled) {
+      INSTANCE = new ExplainCollector();
+    }
   }
 
   public static void remove() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3a87044/docs/datamap/datamap-management.md
----------------------------------------------------------------------
diff --git a/docs/datamap/datamap-management.md b/docs/datamap/datamap-management.md
index eee03a7..bf52c05 100644
--- a/docs/datamap/datamap-management.md
+++ b/docs/datamap/datamap-management.md
@@ -122,7 +122,7 @@ There is a DataMapCatalog interface to retrieve schema of all datamap, it can be
 
 How can user know whether datamap is used in the query?
 
-User can use EXPLAIN command to know, it will print out something like
+User can set enable.query.statistics = true and use EXPLAIN command to know, it will print out something like
 
 ```text
 == CarbonData Profiler ==

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3a87044/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BloomFilterDataMapTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BloomFilterDataMapTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BloomFilterDataMapTestCase.scala
index 8acbcd6..077e007 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BloomFilterDataMapTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BloomFilterDataMapTestCase.scala
@@ -31,7 +31,8 @@ class BloomFilterDataMapTestCase extends QueryTest with BeforeAndAfterEach with
       "yyyy-MM-dd")
     CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
       "yyyy-MM-dd HH:mm:ss")
-
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.ENABLE_QUERY_STATISTICS, "true")
   }
 
   override protected def afterAll(): Unit = {
@@ -39,6 +40,9 @@ class BloomFilterDataMapTestCase extends QueryTest with BeforeAndAfterEach with
       CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT)
     CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
       CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.ENABLE_QUERY_STATISTICS,
+        CarbonCommonConstants.ENABLE_QUERY_STATISTICS_DEFAULT)
   }
 
   private def createAllDataTypeTable(tableName: String): Unit = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3a87044/integration/spark-common-test/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala
index 2e3019a..a51294c 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapSuite.scala
@@ -41,6 +41,8 @@ class LuceneFineGrainDataMapSuite extends QueryTest with BeforeAndAfterAll {
   val file2 = resourcesPath + "/datamap_input.csv"
 
   override protected def beforeAll(): Unit = {
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.ENABLE_QUERY_STATISTICS, "true")
     new File(CarbonProperties.getInstance().getSystemFolderLocation).delete()
     LuceneFineGrainDataMapSuite.createFile(file2)
     sql("create database if not exists lucene")
@@ -921,6 +923,9 @@ class LuceneFineGrainDataMapSuite extends QueryTest with BeforeAndAfterAll {
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.USE_DISTRIBUTED_DATAMAP,
         originDistributedDatamapStatus)
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.ENABLE_QUERY_STATISTICS,
+        CarbonCommonConstants.ENABLE_QUERY_STATISTICS_DEFAULT)
   }
 }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3a87044/integration/spark-common-test/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapWithSearchModeSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapWithSearchModeSuite.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapWithSearchModeSuite.scala
index 369bed1..375495c 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapWithSearchModeSuite.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/datamap/lucene/LuceneFineGrainDataMapWithSearchModeSuite.scala
@@ -42,6 +42,8 @@ class LuceneFineGrainDataMapWithSearchModeSuite extends QueryTest with BeforeAnd
     sqlContext.sparkSession.asInstanceOf[CarbonSession].startSearchMode()
     CarbonProperties
       .getInstance().addProperty(CarbonCommonConstants.CARBON_SEARCH_QUERY_TIMEOUT, "100s")
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.ENABLE_QUERY_STATISTICS, "true")
     LuceneFineGrainDataMapSuite.createFile(file2, n)
     sql("create database if not exists lucene")
     sql("use lucene")
@@ -296,6 +298,9 @@ class LuceneFineGrainDataMapWithSearchModeSuite extends QueryTest with BeforeAnd
   }
 
   override protected def afterAll(): Unit = {
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.ENABLE_QUERY_STATISTICS,
+        CarbonCommonConstants.ENABLE_QUERY_STATISTICS_DEFAULT)
     LuceneFineGrainDataMapSuite.deleteFile(file2)
     sql("DROP TABLE IF EXISTS datamap_test")
     sql("DROP TABLE IF EXISTS datamap_test5")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3a87044/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala
index 0dc52ea..2d2bde1 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala
@@ -24,7 +24,9 @@ import org.apache.spark.sql.hive.CarbonRelation
 import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, Row}
 import org.scalatest.BeforeAndAfterAll
 
+import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider.TIMESERIES
+import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.spark.util.SparkQueryTest
 
 class TestPreAggregateTableSelection extends SparkQueryTest with BeforeAndAfterAll {
@@ -32,6 +34,8 @@ class TestPreAggregateTableSelection extends SparkQueryTest with BeforeAndAfterA
   val timeSeries = TIMESERIES.toString
 
   override def beforeAll: Unit = {
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.ENABLE_QUERY_STATISTICS, "true")
     sql("drop table if exists mainTable")
     sql("drop table if exists mainTableavg")
     sql("drop table if exists agg0")
@@ -454,6 +458,9 @@ class TestPreAggregateTableSelection extends SparkQueryTest with BeforeAndAfterA
     sql("DROP TABLE IF EXISTS mainTableavg")
     sql("DROP TABLE IF EXISTS filtertable")
     sql("DROP TABLE IF EXISTS grouptable")
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.ENABLE_QUERY_STATISTICS,
+        CarbonCommonConstants.ENABLE_QUERY_STATISTICS_DEFAULT)
   }
 
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3a87044/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestRenameTableWithDataMap.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestRenameTableWithDataMap.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestRenameTableWithDataMap.scala
index 18fb28f..78cf44b 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestRenameTableWithDataMap.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestRenameTableWithDataMap.scala
@@ -20,6 +20,9 @@ package org.apache.carbondata.spark.testsuite.createTable
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+
 /**
  * test functionality for alter table with datamap
  */
@@ -31,6 +34,8 @@ class TestRenameTableWithDataMap extends QueryTest with BeforeAndAfterAll {
     sql("DROP TABLE IF EXISTS carbon_table")
     sql("DROP TABLE IF EXISTS carbon_tb")
     sql("DROP TABLE IF EXISTS fact_table1")
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.ENABLE_QUERY_STATISTICS, "true")
   }
 
   test("Creating a bloomfilter datamap,then table rename") {
@@ -188,5 +193,8 @@ class TestRenameTableWithDataMap extends QueryTest with BeforeAndAfterAll {
     sql("DROP TABLE IF EXISTS carbon_table")
     sql("DROP TABLE IF EXISTS carbon_tb")
     sql("DROP TABLE IF EXISTS fact_table1")
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.ENABLE_QUERY_STATISTICS,
+        CarbonCommonConstants.ENABLE_QUERY_STATISTICS_DEFAULT)
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3a87044/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/CGDataMapTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/CGDataMapTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/CGDataMapTestCase.scala
index a4bc6f0..58f2542 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/CGDataMapTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/CGDataMapTestCase.scala
@@ -361,6 +361,8 @@ class CGDataMapTestCase extends QueryTest with BeforeAndAfterAll {
     //n should be about 5000000 of reset if size is default 1024
     val n = 150000
     CompactionSupportGlobalSortBigFileTest.createFile(file2, n * 4, n)
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.ENABLE_QUERY_STATISTICS, "true")
     sql("DROP TABLE IF EXISTS normal_test")
     sql(
       """
@@ -558,6 +560,9 @@ class CGDataMapTestCase extends QueryTest with BeforeAndAfterAll {
     sql("DROP TABLE IF EXISTS datamap_store_test")
     sql("DROP TABLE IF EXISTS datamap_store_test1")
     sql("DROP TABLE IF EXISTS datamap_store_test2")
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.ENABLE_QUERY_STATISTICS,
+        CarbonCommonConstants.ENABLE_QUERY_STATISTICS_DEFAULT)
   }
 
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3a87044/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/FGDataMapTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/FGDataMapTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/FGDataMapTestCase.scala
index 57b3672..f553a9d 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/FGDataMapTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/FGDataMapTestCase.scala
@@ -45,7 +45,7 @@ import org.apache.carbondata.core.scan.expression.Expression
 import org.apache.carbondata.core.scan.expression.conditional.EqualToExpression
 import org.apache.carbondata.core.scan.filter.intf.ExpressionType
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf
-import org.apache.carbondata.core.util.ByteUtil
+import org.apache.carbondata.core.util.{ByteUtil, CarbonProperties}
 import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.Event
 import org.apache.carbondata.spark.testsuite.datacompaction.CompactionSupportGlobalSortBigFileTest
@@ -437,6 +437,8 @@ class FGDataMapTestCase extends QueryTest with BeforeAndAfterAll {
     //n should be about 5000000 of reset if size is default 1024
     val n = 150000
     CompactionSupportGlobalSortBigFileTest.createFile(file2, n * 4, n)
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.ENABLE_QUERY_STATISTICS, "true")
     sql("DROP TABLE IF EXISTS normal_test")
     sql(
       """
@@ -569,5 +571,8 @@ class FGDataMapTestCase extends QueryTest with BeforeAndAfterAll {
     sql("DROP TABLE IF EXISTS normal_test")
     sql("DROP TABLE IF EXISTS datamap_test")
     sql("DROP TABLE IF EXISTS datamap_testFG")
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.ENABLE_QUERY_STATISTICS,
+        CarbonCommonConstants.ENABLE_QUERY_STATISTICS_DEFAULT)
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3a87044/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
index d6117de..838b28d 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
@@ -115,6 +115,7 @@ class CarbonSession(@transient val sc: SparkContext,
    */
   @InterfaceAudience.Developer(Array("DataMap"))
   def isDataMapHit(sqlStatement: String, dataMapName: String): Boolean = {
+    // explain command will output the dataMap information only if enable.query.statistics = true
     val message = sql(s"EXPLAIN $sqlStatement").collect()
     message(0).getString(0).contains(dataMapName)
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3a87044/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonExplainCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonExplainCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonExplainCommand.scala
index 0c2567a..cb402c7 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonExplainCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonExplainCommand.scala
@@ -52,7 +52,11 @@ case class CarbonExplainCommand(
     try {
       ExplainCollector.setup()
       queryExecution.toRdd.partitions
-      Seq(Row("== CarbonData Profiler ==\n" + ExplainCollector.getFormatedOutput))
+      if (ExplainCollector.enabled()) {
+        Seq(Row("== CarbonData Profiler ==\n" + ExplainCollector.getFormatedOutput))
+      } else {
+        Seq.empty
+      }
     } finally {
       ExplainCollector.remove()
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c3a87044/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapSuite.scala b/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapSuite.scala
index 84edd73..3360530 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapSuite.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapSuite.scala
@@ -41,6 +41,8 @@ class BloomCoarseGrainDataMapSuite extends QueryTest with BeforeAndAfterAll with
 
   override protected def beforeAll(): Unit = {
     new File(CarbonProperties.getInstance().getSystemFolderLocation).delete()
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.ENABLE_QUERY_STATISTICS, "true")
     createFile(bigFile, line = 50000)
     createFile(smallFile)
     sql(s"DROP TABLE IF EXISTS $normalTable")
@@ -992,6 +994,9 @@ class BloomCoarseGrainDataMapSuite extends QueryTest with BeforeAndAfterAll with
     deleteFile(smallFile)
     sql(s"DROP TABLE IF EXISTS $normalTable")
     sql(s"DROP TABLE IF EXISTS $bloomDMSampleTable")
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.ENABLE_QUERY_STATISTICS,
+        CarbonCommonConstants.ENABLE_QUERY_STATISTICS_DEFAULT)
   }
 
   private def createFile(fileName: String, line: Int = 10000, start: Int = 0) = {


[08/45] carbondata git commit: [CARBONDATA-2957][DOC] update doc for supporting compressor in table property

Posted by ra...@apache.org.
[CARBONDATA-2957][DOC] update doc for supporting compressor in table property

add doc for supporting compressor in table property

This closes #2744


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

Branch: refs/heads/branch-1.5
Commit: 3cd8b947c9d671f5d409f3b6c589c65afc2545ee
Parents: 3f99e9b
Author: xuchuanyin <xu...@hust.edu.cn>
Authored: Fri Sep 21 17:06:48 2018 +0800
Committer: kunal642 <ku...@gmail.com>
Committed: Wed Sep 26 18:13:57 2018 +0530

----------------------------------------------------------------------
 docs/ddl-of-carbondata.md | 26 ++++++++++++++++++++++++++
 1 file changed, 26 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/3cd8b947/docs/ddl-of-carbondata.md
----------------------------------------------------------------------
diff --git a/docs/ddl-of-carbondata.md b/docs/ddl-of-carbondata.md
index 5eeba86..7cda9cd 100644
--- a/docs/ddl-of-carbondata.md
+++ b/docs/ddl-of-carbondata.md
@@ -32,6 +32,7 @@ CarbonData DDL statements are documented here,which includes:
   * [Caching Level](#caching-at-block-or-blocklet-level)
   * [Hive/Parquet folder Structure](#support-flat-folder-same-as-hiveparquet)
   * [Extra Long String columns](#string-longer-than-32000-characters)
+  * [Compression for Table](#compression-for-table)
 * [CREATE TABLE AS SELECT](#create-table-as-select)
 * [CREATE EXTERNAL TABLE](#create-external-table)
   * [External Table on Transactional table location](#create-external-table-on-managed-table-data-location)
@@ -424,6 +425,31 @@ CarbonData DDL statements are documented here,which includes:
 
      **NOTE:** The LONG_STRING_COLUMNS can only be string/char/varchar columns and cannot be dictionary_include/sort_columns/complex columns.
 
+   - ##### Compression for table
+
+     Data compression is also supported by CarbonData.
+     By default, Snappy is used to compress the data. CarbonData also support ZSTD compressor.
+     User can specify the compressor in the table property:
+
+     ```
+     TBLPROPERTIES('carbon.column.compressor'='snappy')
+     ```
+     or
+     ```
+     TBLPROPERTIES('carbon.column.compressor'='zstd')
+     ```
+     If the compressor is configured, all the data loading and compaction will use that compressor.
+     If the compressor is not configured, the data loading and compaction will use the compressor from current system property.
+     In this scenario, the compressor for each load may differ if the system property is changed each time. This is helpful if you want to change the compressor for a table.
+     The corresponding system property is configured in carbon.properties file as below:
+     ```
+     carbon.column.compressor=snappy
+     ```
+     or
+     ```
+     carbon.column.compressor=zstd
+     ```
+
 ## CREATE TABLE AS SELECT
   This function allows user to create a Carbon table from any of the Parquet/Hive/Carbon table. This is beneficial when the user wants to create Carbon table from any other Parquet/Hive table and use the Carbon query engine to query and achieve better query results for cases where Carbon is faster than other file formats. Also this feature can be used for backing up the data.
 


[09/45] carbondata git commit: [CARBONDATA-2973] Added documentation for fallback condition for complex columns in local Dictionary

Posted by ra...@apache.org.
[CARBONDATA-2973] Added documentation for fallback condition for complex columns in local Dictionary

1. Added documentation for fallback condition for complex columns in local Dictionary
2. Added documentation for system level property" carbon.local.dictionary.decoder.fallback"

This closes #2766


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

Branch: refs/heads/branch-1.5
Commit: 3f99e9b7f87b387f03c1111b5bece2b2a8c5a50b
Parents: a9ddfbd
Author: praveenmeenakshi56 <pr...@gmail.com>
Authored: Wed Sep 26 12:40:37 2018 +0530
Committer: manishgupta88 <to...@gmail.com>
Committed: Wed Sep 26 18:14:44 2018 +0530

----------------------------------------------------------------------
 docs/configuration-parameters.md |  2 +-
 docs/ddl-of-carbondata.md        | 16 +++++++++++-----
 docs/faq.md                      |  2 +-
 3 files changed, 13 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/3f99e9b7/docs/configuration-parameters.md
----------------------------------------------------------------------
diff --git a/docs/configuration-parameters.md b/docs/configuration-parameters.md
index 7edae47..662525b 100644
--- a/docs/configuration-parameters.md
+++ b/docs/configuration-parameters.md
@@ -119,7 +119,7 @@ This section provides the details of all the configurations required for the Car
 
 | Parameter | Default Value | Description |
 |--------------------------------------|---------------|---------------------------------------------------|
-| carbon.max.driver.lru.cache.size | -1 | Maximum memory **(in MB)** upto which the driver process can cache the data (BTree and dictionary values). Beyond this, least recently used data will be removed from cache before loading new set of values.Default value of -1 means there is no memory limit for caching. Only integer values greater than 0 are accepted.**NOTE:** Minimum number of entries that needs to be removed from cache in order to load the new set of data is determined and unloaded.ie.,for example if 3 cache entries qualify for pre-emption, out of these, those entries that free up more cache memory is removed prior to others. Please refer [FAQs](./faq.md#how-to-check-LRU-cache-memory-footprint) for checking LRU cache memory footprint. |
+| carbon.max.driver.lru.cache.size | -1 | Maximum memory **(in MB)** upto which the driver process can cache the data (BTree and dictionary values). Beyond this, least recently used data will be removed from cache before loading new set of values.Default value of -1 means there is no memory limit for caching. Only integer values greater than 0 are accepted.**NOTE:** Minimum number of entries that needs to be removed from cache in order to load the new set of data is determined and unloaded.ie.,for example if 3 cache entries qualify for pre-emption, out of these, those entries that free up more cache memory is removed prior to others. Please refer [FAQs](./faq.md#how-to-check-lru-cache-memory-footprint) for checking LRU cache memory footprint. |
 | carbon.max.executor.lru.cache.size | -1 | Maximum memory **(in MB)** upto which the executor process can cache the data (BTree and reverse dictionary values).Default value of -1 means there is no memory limit for caching. Only integer values greater than 0 are accepted.**NOTE:** If this parameter is not configured, then the value of ***carbon.max.driver.lru.cache.size*** will be used. |
 | max.query.execution.time | 60 | Maximum time allowed for one query to be executed. The value is in minutes. |
 | carbon.enableMinMax | true | CarbonData maintains the metadata which enables to prune unnecessary files from being scanned as per the query conditions.To achieve pruning, Min,Max of each column is maintined.Based on the filter condition in the query, certain data can be skipped from scanning by matching the filter value against the min,max values of the column(s) present in that carbondata file.This pruing enhances query performance significantly. |

http://git-wip-us.apache.org/repos/asf/carbondata/blob/3f99e9b7/docs/ddl-of-carbondata.md
----------------------------------------------------------------------
diff --git a/docs/ddl-of-carbondata.md b/docs/ddl-of-carbondata.md
index 2a467a2..5eeba86 100644
--- a/docs/ddl-of-carbondata.md
+++ b/docs/ddl-of-carbondata.md
@@ -231,7 +231,13 @@ CarbonData DDL statements are documented here,which includes:
    
    * In case of multi-level complex dataType columns, primitive string/varchar/char columns are considered for local dictionary generation.
 
-   Local dictionary will have to be enabled explicitly during create table or by enabling the **system property** ***carbon.local.dictionary.enable***. By default, Local Dictionary will be disabled for the carbondata table.
+   System Level Properties for Local Dictionary: 
+   
+   
+   | Properties | Default value | Description |
+   | ---------- | ------------- | ----------- |
+   | carbon.local.dictionary.enable | false | By default, Local Dictionary will be disabled for the carbondata table. |
+   | carbon.local.dictionary.decoder.fallback | true | Page Level data will not be maintained for the blocklet. During fallback, actual data will be retrieved from the encoded page data using local dictionary. **NOTE:** Memory footprint decreases significantly as compared to when this property is set to false |
     
    Local Dictionary can be configured using the following properties during create table command: 
           
@@ -239,24 +245,24 @@ CarbonData DDL statements are documented here,which includes:
 | Properties | Default value | Description |
 | ---------- | ------------- | ----------- |
 | LOCAL_DICTIONARY_ENABLE | false | Whether to enable local dictionary generation. **NOTE:** If this property is defined, it will override the value configured at system level by '***carbon.local.dictionary.enable***'.Local dictionary will be generated for all string/varchar/char columns unless LOCAL_DICTIONARY_INCLUDE, LOCAL_DICTIONARY_EXCLUDE is configured. |
-| LOCAL_DICTIONARY_THRESHOLD | 10000 | The maximum cardinality of a column upto which carbondata can try to generate local dictionary (maximum - 100000) |
+| LOCAL_DICTIONARY_THRESHOLD | 10000 | The maximum cardinality of a column upto which carbondata can try to generate local dictionary (maximum - 100000). **NOTE:** When LOCAL_DICTIONARY_THRESHOLD is defined for Complex columns, the count of distinct records of all child columns are summed up. |
 | LOCAL_DICTIONARY_INCLUDE | string/varchar/char columns| Columns for which Local Dictionary has to be generated.**NOTE:** Those string/varchar/char columns which are added into DICTIONARY_INCLUDE option will not be considered for local dictionary generation.This property needs to be configured only when local dictionary needs to be generated for few columns, skipping others.This property takes effect only when **LOCAL_DICTIONARY_ENABLE** is true or **carbon.local.dictionary.enable** is true |
 | LOCAL_DICTIONARY_EXCLUDE | none | Columns for which Local Dictionary need not be generated.This property needs to be configured only when local dictionary needs to be skipped for few columns, generating for others.This property takes effect only when **LOCAL_DICTIONARY_ENABLE** is true or **carbon.local.dictionary.enable** is true |
 
    **Fallback behavior:** 
 
    * When the cardinality of a column exceeds the threshold, it triggers a fallback and the generated dictionary will be reverted and data loading will be continued without dictionary encoding.
+   
+   * In case of complex columns, fallback is triggered when the summation value of all child columns' distinct records exceeds the defined LOCAL_DICTIONARY_THRESHOLD value.
 
    **NOTE:** When fallback is triggered, the data loading performance will decrease as encoded data will be discarded and the actual data is written to the temporary sort files.
 
    **Points to be noted:**
 
-   1. Reduce Block size:
+   * Reduce Block size:
    
       Number of Blocks generated is less in case of Local Dictionary as compression ratio is high. This may reduce the number of tasks launched during query, resulting in degradation of query performance if the pruned blocks are less compared to the number of parallel tasks which can be run. So it is recommended to configure smaller block size which in turn generates more number of blocks.
       
-   2. All the page-level data for a blocklet needs to be maintained in memory until all the pages encoded for local dictionary is processed in order to handle fallback. Hence the memory required for local dictionary based table is more and this memory increase is proportional to number of columns. 
-      
 ### Example:
 
    ```

http://git-wip-us.apache.org/repos/asf/carbondata/blob/3f99e9b7/docs/faq.md
----------------------------------------------------------------------
diff --git a/docs/faq.md b/docs/faq.md
index dbf9155..3dee5a2 100644
--- a/docs/faq.md
+++ b/docs/faq.md
@@ -28,7 +28,7 @@
 * [Why aggregate query is not fetching data from aggregate table?](#why-aggregate-query-is-not-fetching-data-from-aggregate-table)
 * [Why all executors are showing success in Spark UI even after Dataload command failed at Driver side?](#why-all-executors-are-showing-success-in-spark-ui-even-after-dataload-command-failed-at-driver-side)
 * [Why different time zone result for select query output when query SDK writer output?](#why-different-time-zone-result-for-select-query-output-when-query-sdk-writer-output)
-* [How to check LRU cache memory footprint?](#how-to-check-LRU-cache-memory-footprint)
+* [How to check LRU cache memory footprint?](#how-to-check-lru-cache-memory-footprint)
 
 # TroubleShooting
 


[11/45] carbondata git commit: [HOTFIX] Fix NPE in LRU cache when entry from the same table is getting evicted to load another entry from same table

Posted by ra...@apache.org.
[HOTFIX] Fix NPE in LRU cache when entry from the same table is getting evicted to load another entry from same table

Problem
When driver LRU cache size is configured to a small value then on running concurrent queries sometimes while loading the block dataMap in LRU cache one of the dataMap entries from the same table is getting deleted because of shortage of space. Due to this in the flow after loading the dataMap cache NPE is thrown.
This is because when an cacheable entry is removed from LRU cache then invalidate is called on that cacheable entry to clear the unsafe memory used by that entry. Invalidate method makes the references null and clears the unsafe memory which leads to NPE when accessed again.

Solution
Currently dataMap cache uses unsafe offheap memory for datamap caching. To avoid this the code is modified to use unsafe with onheap so that JVM itself takes care of clearing the memory when required. We do not require to explicitly set the references to null.

This closes #2759


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

Branch: refs/heads/branch-1.5
Commit: 2a4f53001058346843e0248c60fee2943087efc9
Parents: 5c0da31
Author: manishgupta88 <to...@gmail.com>
Authored: Tue Sep 25 19:21:08 2018 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 27 12:08:28 2018 +0530

----------------------------------------------------------------------
 .../indexstore/BlockletDataMapIndexWrapper.java  |  1 -
 .../core/indexstore/SafeMemoryDMStore.java       |  1 -
 .../core/indexstore/UnsafeMemoryDMStore.java     | 16 +++++++++-------
 .../indexstore/blockletindex/BlockDataMap.java   |  2 --
 .../core/memory/HeapMemoryAllocator.java         |  8 ++++++--
 .../core/memory/UnsafeMemoryManager.java         | 19 +++++++++++++++----
 .../util/AbstractDataFileFooterConverter.java    |  4 ++--
 7 files changed, 32 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/2a4f5300/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexWrapper.java b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexWrapper.java
index 7b8a13b..33d69aa 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexWrapper.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexWrapper.java
@@ -70,7 +70,6 @@ public class BlockletDataMapIndexWrapper implements Cacheable, Serializable {
     for (DataMap dataMap : dataMaps) {
       dataMap.clear();
     }
-    dataMaps = null;
   }
 
   public List<BlockDataMap> getDataMaps() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2a4f5300/core/src/main/java/org/apache/carbondata/core/indexstore/SafeMemoryDMStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/SafeMemoryDMStore.java b/core/src/main/java/org/apache/carbondata/core/indexstore/SafeMemoryDMStore.java
index 0b3d4d8..042790f 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/SafeMemoryDMStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/SafeMemoryDMStore.java
@@ -62,7 +62,6 @@ public class SafeMemoryDMStore extends AbstractMemoryDMStore {
     if (!isMemoryFreed) {
       if (null != dataMapRows) {
         dataMapRows.clear();
-        dataMapRows = null;
       }
       isMemoryFreed = true;
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2a4f5300/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java b/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java
index 3e8ce12..196559a 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java
@@ -19,6 +19,7 @@ package org.apache.carbondata.core.indexstore;
 import org.apache.carbondata.core.indexstore.row.DataMapRow;
 import org.apache.carbondata.core.indexstore.row.UnsafeDataMapRow;
 import org.apache.carbondata.core.indexstore.schema.CarbonRowSchema;
+import org.apache.carbondata.core.memory.MemoryAllocator;
 import org.apache.carbondata.core.memory.MemoryBlock;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.memory.UnsafeMemoryManager;
@@ -49,7 +50,8 @@ public class UnsafeMemoryDMStore extends AbstractMemoryDMStore {
 
   public UnsafeMemoryDMStore() throws MemoryException {
     this.allocatedSize = capacity;
-    this.memoryBlock = UnsafeMemoryManager.allocateMemoryWithRetry(taskId, allocatedSize);
+    this.memoryBlock =
+        UnsafeMemoryManager.allocateMemoryWithRetry(MemoryAllocator.HEAP, taskId, allocatedSize);
     this.pointers = new int[1000];
   }
 
@@ -71,11 +73,11 @@ public class UnsafeMemoryDMStore extends AbstractMemoryDMStore {
   }
 
   private void increaseMemory(int requiredMemory) throws MemoryException {
-    MemoryBlock newMemoryBlock =
-        UnsafeMemoryManager.allocateMemoryWithRetry(taskId, allocatedSize + requiredMemory);
+    MemoryBlock newMemoryBlock = UnsafeMemoryManager
+        .allocateMemoryWithRetry(MemoryAllocator.HEAP, taskId, allocatedSize + requiredMemory);
     getUnsafe().copyMemory(this.memoryBlock.getBaseObject(), this.memoryBlock.getBaseOffset(),
         newMemoryBlock.getBaseObject(), newMemoryBlock.getBaseOffset(), runningLength);
-    UnsafeMemoryManager.INSTANCE.freeMemory(taskId, this.memoryBlock);
+    UnsafeMemoryManager.INSTANCE.freeMemory(MemoryAllocator.HEAP, taskId, this.memoryBlock);
     allocatedSize = allocatedSize + requiredMemory;
     this.memoryBlock = newMemoryBlock;
   }
@@ -188,10 +190,10 @@ public class UnsafeMemoryDMStore extends AbstractMemoryDMStore {
   public void finishWriting() throws MemoryException {
     if (runningLength < allocatedSize) {
       MemoryBlock allocate =
-          UnsafeMemoryManager.allocateMemoryWithRetry(taskId, runningLength);
+          UnsafeMemoryManager.allocateMemoryWithRetry(MemoryAllocator.HEAP, taskId, runningLength);
       getUnsafe().copyMemory(memoryBlock.getBaseObject(), memoryBlock.getBaseOffset(),
           allocate.getBaseObject(), allocate.getBaseOffset(), runningLength);
-      UnsafeMemoryManager.INSTANCE.freeMemory(taskId, memoryBlock);
+      UnsafeMemoryManager.INSTANCE.freeMemory(MemoryAllocator.HEAP, taskId, memoryBlock);
       memoryBlock = allocate;
     }
     // Compact pointers.
@@ -204,7 +206,7 @@ public class UnsafeMemoryDMStore extends AbstractMemoryDMStore {
 
   public void freeMemory() {
     if (!isMemoryFreed) {
-      UnsafeMemoryManager.INSTANCE.freeMemory(taskId, memoryBlock);
+      UnsafeMemoryManager.INSTANCE.freeMemory(MemoryAllocator.HEAP, taskId, memoryBlock);
       isMemoryFreed = true;
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2a4f5300/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java
index 0cf9914..d7b7977 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java
@@ -943,12 +943,10 @@ public class BlockDataMap extends CoarseGrainDataMap
   @Override public void clear() {
     if (memoryDMStore != null) {
       memoryDMStore.freeMemory();
-      memoryDMStore = null;
     }
     // clear task min/max unsafe memory
     if (null != taskSummaryDMStore) {
       taskSummaryDMStore.freeMemory();
-      taskSummaryDMStore = null;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2a4f5300/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java b/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
index d898dc9..d08f803 100644
--- a/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
+++ b/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
@@ -23,6 +23,7 @@ import java.util.LinkedList;
 import java.util.Map;
 import javax.annotation.concurrent.GuardedBy;
 
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.util.CarbonProperties;
 
 /**
@@ -39,8 +40,11 @@ public class HeapMemoryAllocator implements MemoryAllocator {
 
   public HeapMemoryAllocator() {
     poolingThresholdBytes = CarbonProperties.getInstance().getHeapMemoryPoolingThresholdBytes();
-    // if set 'poolingThresholdBytes' to -1, it should not go through the pooling mechanism.
-    if (poolingThresholdBytes == -1) {
+    boolean isDriver = Boolean.parseBoolean(CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.IS_DRIVER_INSTANCE, "false"));
+    // if set 'poolingThresholdBytes' to -1 or the object creation call is in driver,
+    // it should not go through the pooling mechanism.
+    if (poolingThresholdBytes == -1 || isDriver) {
       shouldPooling = false;
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2a4f5300/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java b/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java
index e3593c5..703d57a 100644
--- a/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java
@@ -107,9 +107,10 @@ public class UnsafeMemoryManager {
         .info("Working Memory manager is created with size " + totalMemory + " with " + allocator);
   }
 
-  private synchronized MemoryBlock allocateMemory(long taskId, long memoryRequested) {
+  private synchronized MemoryBlock allocateMemory(MemoryAllocator memoryAllocator, long taskId,
+      long memoryRequested) {
     if (memoryUsed + memoryRequested <= totalMemory) {
-      MemoryBlock allocate = allocator.allocate(memoryRequested);
+      MemoryBlock allocate = memoryAllocator.allocate(memoryRequested);
       memoryUsed += allocate.size();
       Set<MemoryBlock> listOfMemoryBlock = taskIdToMemoryBlockMap.get(taskId);
       if (null == listOfMemoryBlock) {
@@ -128,11 +129,16 @@ public class UnsafeMemoryManager {
   }
 
   public synchronized void freeMemory(long taskId, MemoryBlock memoryBlock) {
+    freeMemory(allocator, taskId, memoryBlock);
+  }
+
+  public synchronized void freeMemory(MemoryAllocator memoryAllocator, long taskId,
+      MemoryBlock memoryBlock) {
     if (taskIdToMemoryBlockMap.containsKey(taskId)) {
       taskIdToMemoryBlockMap.get(taskId).remove(memoryBlock);
     }
     if (!memoryBlock.isFreedStatus()) {
-      allocator.free(memoryBlock);
+      memoryAllocator.free(memoryBlock);
       memoryUsed -= memoryBlock.size();
       memoryUsed = memoryUsed < 0 ? 0 : memoryUsed;
       if (LOGGER.isDebugEnabled()) {
@@ -182,10 +188,15 @@ public class UnsafeMemoryManager {
    */
   public static MemoryBlock allocateMemoryWithRetry(long taskId, long size)
       throws MemoryException {
+    return allocateMemoryWithRetry(INSTANCE.allocator, taskId, size);
+  }
+
+  public static MemoryBlock allocateMemoryWithRetry(MemoryAllocator memoryAllocator, long taskId,
+      long size) throws MemoryException {
     MemoryBlock baseBlock = null;
     int tries = 0;
     while (tries < 300) {
-      baseBlock = INSTANCE.allocateMemory(taskId, size);
+      baseBlock = INSTANCE.allocateMemory(memoryAllocator, taskId, size);
       if (baseBlock == null) {
         try {
           LOGGER.info("Memory is not available, retry after 500 millis");

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2a4f5300/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java b/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
index b1dd580..601ce50 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
@@ -295,8 +295,8 @@ public abstract class AbstractDataFileFooterConverter {
       blockletMinMaxFlag = blockletIndexList.get(i).getMinMaxIndex().getIsMinMaxSet();
       for (int j = 0; j < maxValue.length; j++) {
         // can be null for stores < 1.5.0 version
-        if (null != blockletMinMaxFlag && !blockletMinMaxFlag[i]) {
-          blockMinMaxFlag[i] = blockletMinMaxFlag[i];
+        if (null != blockletMinMaxFlag && !blockletMinMaxFlag[j]) {
+          blockMinMaxFlag[j] = blockletMinMaxFlag[j];
           currentMaxValue[j] = new byte[0];
           currentMinValue[j] = new byte[0];
           continue;


[41/45] carbondata git commit: [Documentation] Readme updated with latest topics and new TOC

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/ca30ad97/docs/configuration-parameters.md
----------------------------------------------------------------------
diff --git a/docs/configuration-parameters.md b/docs/configuration-parameters.md
index 9dd8164..0a4565a 100644
--- a/docs/configuration-parameters.md
+++ b/docs/configuration-parameters.md
@@ -16,7 +16,7 @@
 -->
 
 # Configuring CarbonData
- This guide explains the configurations that can be used to tune CarbonData to achieve better performance.Most of the properties that control the internal settings have reasonable default values.They are listed along with the properties along with explanation.
+ This guide explains the configurations that can be used to tune CarbonData to achieve better performance.Most of the properties that control the internal settings have reasonable default values. They are listed along with the properties along with explanation.
 
  * [System Configuration](#system-configuration)
  * [Data Loading Configuration](#data-loading-configuration)
@@ -31,68 +31,68 @@ This section provides the details of all the configurations required for the Car
 
 | Property | Default Value | Description |
 |----------------------------|-------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| carbon.storelocation | spark.sql.warehouse.dir property value | Location where CarbonData will create the store, and write the data in its custom format. If not specified,the path defaults to spark.sql.warehouse.dir property. NOTE: Store location should be in HDFS. |
-| carbon.ddl.base.hdfs.url | (none) | To simplify and shorten the path to be specified in DDL/DML commands, this property is supported.This property is used to configure the HDFS relative path, the path configured in carbon.ddl.base.hdfs.url will be appended to the HDFS path configured in fs.defaultFS of core-site.xml. If this path is configured, then user need not pass the complete path while dataload. For example: If absolute path of the csv file is hdfs://10.18.101.155:54310/data/cnbc/2016/xyz.csv, the path "hdfs://10.18.101.155:54310" will come from property fs.defaultFS and user can configure the /data/cnbc/ as carbon.ddl.base.hdfs.url. Now while dataload user can specify the csv path as /2016/xyz.csv. |
-| carbon.badRecords.location | (none) | CarbonData can detect the records not conforming to defined table schema and isolate them as bad records.This property is used to specify where to store such bad records. |
-| carbon.streaming.auto.handoff.enabled | true | CarbonData supports storing of streaming data.To have high throughput for streaming, the data is written in Row format which is highly optimized for write, but performs poorly for query.When this property is true and when the streaming data size reaches ***carbon.streaming.segment.max.size***, CabonData will automatically convert the data to columnar format and optimize it for faster querying.**NOTE:** It is not recommended to keep the default value which is true. |
-| carbon.streaming.segment.max.size | 1024000000 | CarbonData writes streaming data in row format which is optimized for high write throughput.This property defines the maximum size of data to be held is row format, beyond which it will be converted to columnar format in order to support high performane query, provided ***carbon.streaming.auto.handoff.enabled*** is true. **NOTE:** Setting higher value will impact the streaming ingestion. The value has to be configured in bytes. |
-| carbon.query.show.datamaps | true | CarbonData stores datamaps as independent tables so as to allow independent maintenance to some extent.When this property is true,which is by default, show tables command will list all the tables including datatmaps(eg: Preaggregate table), else datamaps will be excluded from the table list.**NOTE:**  It is generally not required for the user to do any maintenance operations on these tables and hence not required to be seen.But it is shown by default so that user or admin can get clear understanding of the system for capacity planning. |
-| carbon.segment.lock.files.preserve.hours | 48 | In order to support parallel data loading onto the same table, CarbonData sequences(locks) at the granularity of segments.Operations affecting the segment(like IUD, alter) are blocked from parallel operations.This property value indicates the number of hours the segment lock files will be preserved after dataload. These lock files will be deleted with the clean command after the configured number of hours. |
-| carbon.timestamp.format | yyyy-MM-dd HH:mm:ss | CarbonData can understand data of timestamp type and process it in special manner.It can be so that the format of Timestamp data is different from that understood by CarbonData by default.This configuration allows users to specify the format of Timestamp in their data. |
+| carbon.storelocation | spark.sql.warehouse.dir property value | Location where CarbonData will create the store, and write the data in its custom format. If not specified,the path defaults to spark.sql.warehouse.dir property. **NOTE:** Store location should be in HDFS. |
+| carbon.ddl.base.hdfs.url | (none) | To simplify and shorten the path to be specified in DDL/DML commands, this property is supported. This property is used to configure the HDFS relative path, the path configured in carbon.ddl.base.hdfs.url will be appended to the HDFS path configured in fs.defaultFS of core-site.xml. If this path is configured, then user need not pass the complete path while dataload. For example: If absolute path of the csv file is hdfs://10.18.101.155:54310/data/cnbc/2016/xyz.csv, the path "hdfs://10.18.101.155:54310" will come from property fs.defaultFS and user can configure the /data/cnbc/ as carbon.ddl.base.hdfs.url. Now while dataload user can specify the csv path as /2016/xyz.csv. |
+| carbon.badRecords.location | (none) | CarbonData can detect the records not conforming to defined table schema and isolate them as bad records. This property is used to specify where to store such bad records. |
+| carbon.streaming.auto.handoff.enabled | true | CarbonData supports storing of streaming data. To have high throughput for streaming, the data is written in Row format which is highly optimized for write, but performs poorly for query. When this property is true and when the streaming data size reaches ***carbon.streaming.segment.max.size***, CabonData will automatically convert the data to columnar format and optimize it for faster querying.**NOTE:** It is not recommended to keep the default value which is true. |
+| carbon.streaming.segment.max.size | 1024000000 | CarbonData writes streaming data in row format which is optimized for high write throughput. This property defines the maximum size of data to be held is row format, beyond which it will be converted to columnar format in order to support high performance query, provided ***carbon.streaming.auto.handoff.enabled*** is true. **NOTE:** Setting higher value will impact the streaming ingestion. The value has to be configured in bytes. |
+| carbon.query.show.datamaps | true | CarbonData stores datamaps as independent tables so as to allow independent maintenance to some extent. When this property is true,which is by default, show tables command will list all the tables including datatmaps(eg: Preaggregate table), else datamaps will be excluded from the table list.**NOTE:**  It is generally not required for the user to do any maintenance operations on these tables and hence not required to be seen.But it is shown by default so that user or admin can get clear understanding of the system for capacity planning. |
+| carbon.segment.lock.files.preserve.hours | 48 | In order to support parallel data loading onto the same table, CarbonData sequences(locks) at the granularity of segments.Operations affecting the segment(like IUD, alter) are blocked from parallel operations. This property value indicates the number of hours the segment lock files will be preserved after dataload. These lock files will be deleted with the clean command after the configured number of hours. |
+| carbon.timestamp.format | yyyy-MM-dd HH:mm:ss | CarbonData can understand data of timestamp type and process it in special manner.It can be so that the format of Timestamp data is different from that understood by CarbonData by default. This configuration allows users to specify the format of Timestamp in their data. |
 | carbon.lock.type | LOCALLOCK | This configuration specifies the type of lock to be acquired during concurrent operations on table. There are following types of lock implementation: - LOCALLOCK: Lock is created on local file system as file. This lock is useful when only one spark driver (thrift server) runs on a machine and no other CarbonData spark application is launched concurrently. - HDFSLOCK: Lock is created on HDFS file system as file. This lock is useful when multiple CarbonData spark applications are launched and no ZooKeeper is running on cluster and HDFS supports file based locking. |
 | carbon.lock.path | TABLEPATH | This configuration specifies the path where lock files have to be created. Recommended to configure zookeeper lock type or configure HDFS lock path(to this property) in case of S3 file system as locking is not feasible on S3. |
-| carbon.unsafe.working.memory.in.mb | 512 | CarbonData supports storing data in off-heap memory for certain operations during data loading and query.This helps to avoid the Java GC and thereby improve the overall performance.The Minimum value recommeded is 512MB.Any value below this is reset to default value of 512MB.**NOTE:** The below formulas explain how to arrive at the off-heap size required.<u>Memory Required For Data Loading:</u>(*carbon.number.of.cores.while.loading*) * (Number of tables to load in parallel) * (*offheap.sort.chunk.size.inmb* + *carbon.blockletgroup.size.in.mb* + *carbon.blockletgroup.size.in.mb*/3.5 ). <u>Memory required for Query:</u>SPARK_EXECUTOR_INSTANCES * (*carbon.blockletgroup.size.in.mb* + *carbon.blockletgroup.size.in.mb* * 3.5) * spark.executor.cores |
+| carbon.unsafe.working.memory.in.mb | 512 | CarbonData supports storing data in off-heap memory for certain operations during data loading and query. This helps to avoid the Java GC and thereby improve the overall performance. The Minimum value recommeded is 512MB. Any value below this is reset to default value of 512MB. **NOTE:** The below formulas explain how to arrive at the off-heap size required.<u>Memory Required For Data Loading:</u>(*carbon.number.of.cores.while.loading*) * (Number of tables to load in parallel) * (*offheap.sort.chunk.size.inmb* + *carbon.blockletgroup.size.in.mb* + *carbon.blockletgroup.size.in.mb*/3.5 ). <u>Memory required for Query:</u>SPARK_EXECUTOR_INSTANCES * (*carbon.blockletgroup.size.in.mb* + *carbon.blockletgroup.size.in.mb* * 3.5) * spark.executor.cores |
 | carbon.unsafe.driver.working.memory.in.mb | 60% of JVM Heap Memory | CarbonData supports storing data in unsafe on-heap memory in driver for certain operations like insert into, query for loading datamap cache. The Minimum value recommended is 512MB. |
-| carbon.update.sync.folder | /tmp/carbondata | CarbonData maintains last modification time entries in modifiedTime.mdt to determine the schema changes and reload only when necessary.This configuration specifies the path where the file needs to be written. |
-| carbon.invisible.segments.preserve.count | 200 | CarbonData maintains each data load entry in tablestatus file. The entries from this file are not deleted for those segments that are compacted or dropped, but are made invisible.If the number of data loads are very high, the size and number of entries in tablestatus file can become too many causing unnecessary reading of all data.This configuration specifies the number of segment entries to be maintained afte they are compacted or dropped.Beyond this, the entries are moved to a separate history tablestatus file.**NOTE:** The entries in tablestatus file help to identify the operations performed on CarbonData table and is also used for checkpointing during various data manupulation operations.This is similar to AUDIT file maintaining all the operations and its status.Hence the entries are never deleted but moved to a separate history file. |
-| carbon.lock.retries | 3 | CarbonData ensures consistency of operations by blocking certain operations from running in parallel.In order to block the operations from running in parallel, lock is obtained on the table.This configuration specifies the maximum number of retries to obtain the lock for any operations other than load.**NOTE:** Data manupulation operations like Compaction,UPDATE,DELETE  or LOADING,UPDATE,DELETE are not allowed to run in parallel.How ever data loading can happen in parallel to compaction. |
-| carbon.lock.retry.timeout.sec | 5 | Specifies the interval between the retries to obtain the lock for any operation other than load.**NOTE:** Refer to ***carbon.lock.retries*** for understanding why CarbonData uses locks for operations. |
+| carbon.update.sync.folder | /tmp/carbondata | CarbonData maintains last modification time entries in modifiedTime.mdt to determine the schema changes and reload only when necessary. This configuration specifies the path where the file needs to be written. |
+| carbon.invisible.segments.preserve.count | 200 | CarbonData maintains each data load entry in tablestatus file. The entries from this file are not deleted for those segments that are compacted or dropped, but are made invisible. If the number of data loads are very high, the size and number of entries in tablestatus file can become too many causing unnecessary reading of all data. This configuration specifies the number of segment entries to be maintained afte they are compacted or dropped.Beyond this, the entries are moved to a separate history tablestatus file. **NOTE:** The entries in tablestatus file help to identify the operations performed on CarbonData table and is also used for checkpointing during various data manupulation operations. This is similar to AUDIT file maintaining all the operations and its status.Hence the entries are never deleted but moved to a separate history file. |
+| carbon.lock.retries | 3 | CarbonData ensures consistency of operations by blocking certain operations from running in parallel. In order to block the operations from running in parallel, lock is obtained on the table. This configuration specifies the maximum number of retries to obtain the lock for any operations other than load. **NOTE:** Data manupulation operations like Compaction,UPDATE,DELETE  or LOADING,UPDATE,DELETE are not allowed to run in parallel.How ever data loading can happen in parallel to compaction. |
+| carbon.lock.retry.timeout.sec | 5 | Specifies the interval between the retries to obtain the lock for any operation other than load. **NOTE:** Refer to ***carbon.lock.retries*** for understanding why CarbonData uses locks for operations. |
 
 ## Data Loading Configuration
 
 | Parameter | Default Value | Description |
 |--------------------------------------|---------------|----------------------------------------------------------------------------------------------------------------------|
-| carbon.number.of.cores.while.loading | 2 | Number of cores to be used while loading data.This also determines the number of threads to be used to read the input files (csv) in parallel.**NOTE:** This configured value is used in every data loading step to parallelize the operations. Configuring a higher value can lead to increased early thread pre-emption by OS and there by reduce the overall performance. |
+| carbon.number.of.cores.while.loading | 2 | Number of cores to be used while loading data. This also determines the number of threads to be used to read the input files (csv) in parallel.**NOTE:** This configured value is used in every data loading step to parallelize the operations. Configuring a higher value can lead to increased early thread pre-emption by OS and there by reduce the overall performance. |
 | carbon.sort.size | 100000 | Number of records to hold in memory to sort and write intermediate temp files.**NOTE:** Memory required for data loading increases with increase in configured value as each thread would cache configured number of records. |
 | carbon.global.sort.rdd.storage.level | MEMORY_ONLY | Storage level to persist dataset of RDD/dataframe when loading data with 'sort_scope'='global_sort', if user's executor has less memory, set this parameter to 'MEMORY_AND_DISK_SER' or other storage level to correspond to different environment. [See detail](http://spark.apache.org/docs/latest/rdd-programming-guide.html#rdd-persistence). |
 | carbon.load.global.sort.partitions | 0 | The Number of partitions to use when shuffling data for sort. Default value 0 means to use same number of map tasks as reduce tasks.**NOTE:** In general, it is recommended to have 2-3 tasks per CPU core in your cluster. |
-| carbon.options.bad.records.logger.enable | false | CarbonData can identify the records that are not conformant to schema and isolate them as bad records.Enabling this configuration will make CarbonData to log such bad records.**NOTE:** If the input data contains many bad records, logging them will slow down the over all data loading throughput.The data load operation status would depend on the configuration in ***carbon.bad.records.action***. |
-| carbon.bad.records.action | FAIL | CarbonData in addition to identifying the bad records, can take certain actions on such data.This configuration can have four types of actions for bad records namely FORCE, REDIRECT, IGNORE and FAIL. If set to FORCE then it auto-corrects the data by storing the bad records as NULL. If set to REDIRECT then bad records are written to the raw CSV instead of being loaded. If set to IGNORE then bad records are neither loaded nor written to the raw CSV. If set to FAIL then data loading fails if any bad records are found. |
+| carbon.options.bad.records.logger.enable | false | CarbonData can identify the records that are not conformant to schema and isolate them as bad records. Enabling this configuration will make CarbonData to log such bad records.**NOTE:** If the input data contains many bad records, logging them will slow down the over all data loading throughput. The data load operation status would depend on the configuration in ***carbon.bad.records.action***. |
+| carbon.bad.records.action | FAIL | CarbonData in addition to identifying the bad records, can take certain actions on such data. This configuration can have four types of actions for bad records namely FORCE, REDIRECT, IGNORE and FAIL. If set to FORCE then it auto-corrects the data by storing the bad records as NULL. If set to REDIRECT then bad records are written to the raw CSV instead of being loaded. If set to IGNORE then bad records are neither loaded nor written to the raw CSV. If set to FAIL then data loading fails if any bad records are found. |
 | carbon.options.is.empty.data.bad.record | false | Based on the business scenarios, empty("" or '' or ,,) data can be valid or invalid. This configuration controls how empty data should be treated by CarbonData. If false, then empty ("" or '' or ,,) data will not be considered as bad record and vice versa. |
 | carbon.options.bad.record.path | (none) | Specifies the HDFS path where bad records are to be stored. By default the value is Null. This path must to be configured by the user if ***carbon.options.bad.records.logger.enable*** is **true** or ***carbon.bad.records.action*** is **REDIRECT**. |
-| carbon.blockletgroup.size.in.mb | 64 | Please refer to [file-structure-of-carbondata](./file-structure-of-carbondata.md#carbondata-file-format) to understand the storage format of CarbonData.The data are read as a group of blocklets which are called blocklet groups. This parameter specifies the size of each blocklet group. Higher value results in better sequential IO access.The minimum value is 16MB, any value lesser than 16MB will reset to the default value (64MB).**NOTE:** Configuring a higher value might lead to poor performance as an entire blocklet group will have to read into memory before processing.For filter queries with limit, it is **not advisable** to have a bigger blocklet size.For Aggregation queries which need to return more number of rows,bigger blocklet size is advisable. |
-| carbon.sort.file.write.buffer.size | 16384 | CarbonData sorts and writes data to intermediate files to limit the memory usage.This configuration determines the buffer size to be used for reading and writing such files. **NOTE:** This configuration is useful to tune IO and derive optimal performance.Based on the OS and underlying harddisk type, these values can significantly affect the overall performance.It is ideal to tune the buffersize equivalent to the IO buffer size of the OS.Recommended range is between 10240 to 10485760 bytes. |
-| carbon.sort.intermediate.files.limit | 20 | CarbonData sorts and writes data to intermediate files to limit the memory usage.Before writing the target carbondat file, the data in these intermediate files needs to be sorted again so as to ensure the entire data in the data load is sorted.This configuration determines the minimum number of intermediate files after which merged sort is applied on them sort the data.**NOTE:** Intermediate merging happens on a separate thread in the background.Number of threads used is determined by ***carbon.merge.sort.reader.thread***.Configuring a low value will cause more time to be spent in merging these intermediate merged files which can cause more IO.Configuring a high value would cause not to use the idle threads to do intermediate sort merges.Range of recommended values are between 2 and 50 |
-| carbon.csv.read.buffersize.byte | 1048576 | CarbonData uses Hadoop InputFormat to read the csv files.This configuration value is used to pass buffer size as input for the Hadoop MR job when reading the csv files.This value is configured in bytes.**NOTE:** Refer to ***org.apache.hadoop.mapreduce.InputFormat*** documentation for additional information. |
-| carbon.merge.sort.reader.thread | 3 | CarbonData sorts and writes data to intermediate files to limit the memory usage.When the intermediate files reaches ***carbon.sort.intermediate.files.limit*** the files will be merged,the number of threads specified in this configuration will be used to read the intermediate files for performing merge sort.**NOTE:** Refer to ***carbon.sort.intermediate.files.limit*** for operation description.Configuring less  number of threads can cause merging to slow down over loading process where as configuring more number of threads can cause thread contention with threads in other data loading steps.Hence configure a fraction of ***carbon.number.of.cores.while.loading***. |
-| carbon.concurrent.lock.retries | 100 | CarbonData supports concurrent data loading onto same table.To ensure the loading status is correctly updated into the system,locks are used to sequence the status updation step.This configuration specifies the maximum number of retries to obtain the lock for updating the load status.**NOTE:** This value is high as more number of concurrent loading happens,more the chances of not able to obtain the lock when tried.Adjust this value according to the number of concurrent loading to be supported by the system. |
-| carbon.concurrent.lock.retry.timeout.sec | 1 | Specifies the interval between the retries to obtain the lock for concurrent operations.**NOTE:** Refer to ***carbon.concurrent.lock.retries*** for understanding why CarbonData uses locks during data loading operations. |
-| carbon.skip.empty.line | false | The csv files givent to CarbonData for loading can contain empty lines.Based on the business scenario, this empty line might have to be ignored or needs to be treated as NULL value for all columns.In order to define this business behavior, this configuration is provided.**NOTE:** In order to consider NULL values for non string columns and continue with data load, ***carbon.bad.records.action*** need to be set to **FORCE**;else data load will be failed as bad records encountered. |
-| carbon.enable.calculate.size | true | **For Load Operation**: Setting this property calculates the size of the carbon data file (.carbondata) and carbon index file (.carbonindex) for every load and updates the table status file. **For Describe Formatted**: Setting this property calculates the total size of the carbon data files and carbon index files for the respective table and displays in describe formatted command.**NOTE:** This is useful to determine the overall size of the carbondata table and also get an idea of how the table is growing in order to take up other backup strategy decisions. |
+| carbon.blockletgroup.size.in.mb | 64 | Please refer to [file-structure-of-carbondata](./file-structure-of-carbondata.md#carbondata-file-format) to understand the storage format of CarbonData. The data are read as a group of blocklets which are called blocklet groups. This parameter specifies the size of each blocklet group. Higher value results in better sequential IO access. The minimum value is 16MB, any value lesser than 16MB will reset to the default value (64MB).**NOTE:** Configuring a higher value might lead to poor performance as an entire blocklet group will have to read into memory before processing.For filter queries with limit, it is **not advisable** to have a bigger blocklet size. For Aggregation queries which need to return more number of rows,bigger blocklet size is advisable. |
+| carbon.sort.file.write.buffer.size | 16384 | CarbonData sorts and writes data to intermediate files to limit the memory usage. This configuration determines the buffer size to be used for reading and writing such files. **NOTE:** This configuration is useful to tune IO and derive optimal performance.Based on the OS and underlying harddisk type, these values can significantly affect the overall performance.It is ideal to tune the buffersize equivalent to the IO buffer size of the OS.Recommended range is between 10240 to 10485760 bytes. |
+| carbon.sort.intermediate.files.limit | 20 | CarbonData sorts and writes data to intermediate files to limit the memory usage. Before writing the target carbondat file, the data in these intermediate files needs to be sorted again so as to ensure the entire data in the data load is sorted. This configuration determines the minimum number of intermediate files after which merged sort is applied on them sort the data.**NOTE:** Intermediate merging happens on a separate thread in the background.Number of threads used is determined by ***carbon.merge.sort.reader.thread***.Configuring a low value will cause more time to be spent in merging these intermediate merged files which can cause more IO.Configuring a high value would cause not to use the idle threads to do intermediate sort merges.Range of recommended values are between 2 and 50 |
+| carbon.csv.read.buffersize.byte | 1048576 | CarbonData uses Hadoop InputFormat to read the csv files. This configuration value is used to pass buffer size as input for the Hadoop MR job when reading the csv files. This value is configured in bytes.**NOTE:** Refer to ***org.apache.hadoop.mapreduce.InputFormat*** documentation for additional information. |
+| carbon.merge.sort.reader.thread | 3 | CarbonData sorts and writes data to intermediate files to limit the memory usage. When the intermediate files reaches ***carbon.sort.intermediate.files.limit*** the files will be merged,the number of threads specified in this configuration will be used to read the intermediate files for performing merge sort.**NOTE:** Refer to ***carbon.sort.intermediate.files.limit*** for operation description.Configuring less  number of threads can cause merging to slow down over loading process where as configuring more number of threads can cause thread contention with threads in other data loading steps.Hence configure a fraction of ***carbon.number.of.cores.while.loading***. |
+| carbon.concurrent.lock.retries | 100 | CarbonData supports concurrent data loading onto same table. To ensure the loading status is correctly updated into the system,locks are used to sequence the status updation step. This configuration specifies the maximum number of retries to obtain the lock for updating the load status. **NOTE:** This value is high as more number of concurrent loading happens,more the chances of not able to obtain the lock when tried. Adjust this value according to the number of concurrent loading to be supported by the system. |
+| carbon.concurrent.lock.retry.timeout.sec | 1 | Specifies the interval between the retries to obtain the lock for concurrent operations. **NOTE:** Refer to ***carbon.concurrent.lock.retries*** for understanding why CarbonData uses locks during data loading operations. |
+| carbon.skip.empty.line | false | The csv files givent to CarbonData for loading can contain empty lines. Based on the business scenario, this empty line might have to be ignored or needs to be treated as NULL value for all columns.In order to define this business behavior, this configuration is provided.**NOTE:** In order to consider NULL values for non string columns and continue with data load, ***carbon.bad.records.action*** need to be set to **FORCE**;else data load will be failed as bad records encountered. |
+| carbon.enable.calculate.size | true | **For Load Operation**: Setting this property calculates the size of the carbon data file (.carbondata) and carbon index file (.carbonindex) for every load and updates the table status file. **For Describe Formatted**: Setting this property calculates the total size of the carbon data files and carbon index files for the respective table and displays in describe formatted command. **NOTE:** This is useful to determine the overall size of the carbondata table and also get an idea of how the table is growing in order to take up other backup strategy decisions. |
 | carbon.cutOffTimestamp | (none) | CarbonData has capability to generate the Dictionary values for the timestamp columns from the data itself without the need to store the computed dictionary values. This configuration sets the start date for calculating the timestamp. Java counts the number of milliseconds from start of "1970-01-01 00:00:00". This property is used to customize the start of position. For example "2000-01-01 00:00:00". **NOTE:** The date must be in the form ***carbon.timestamp.format***. CarbonData supports storing data for upto 68 years.For example, if the cut-off time is 1970-01-01 05:30:00, then data upto 2038-01-01 05:30:00 will be supported by CarbonData. |
-| carbon.timegranularity | SECOND | The configuration is used to specify the data granularity level such as DAY, HOUR, MINUTE, or SECOND.This helps to store more than 68 years of data into CarbonData. |
-| carbon.use.local.dir | false | CarbonData,during data loading, writes files to local temp directories before copying the files to HDFS.This configuration is used to specify whether CarbonData can write locally to tmp directory of the container or to the YARN application directory. |
-| carbon.use.multiple.temp.dir | false | When multiple disks are present in the system, YARN is generally configured with multiple disks to be used as temp directories for managing the containers.This configuration specifies whether to use multiple YARN local directories during data loading for disk IO load balancing.Enable ***carbon.use.local.dir*** for this configuration to take effect.**NOTE:** Data Loading is an IO intensive operation whose performance can be limited by the disk IO threshold, particularly during multi table concurrent data load.Configuring this parameter, balances the disk IO across multiple disks there by improving the over all load performance. |
-| carbon.sort.temp.compressor | (none) | CarbonData writes every ***carbon.sort.size*** number of records to intermediate temp files during data loading to ensure memory footprint is within limits.These temporary files cab be compressed and written in order to save the storage space.This configuration specifies the name of compressor to be used to compress the intermediate sort temp files during sort procedure in data loading.The valid values are 'SNAPPY','GZIP','BZIP2','LZ4','ZSTD' and empty. By default, empty means that Carbondata will not compress the sort temp files.**NOTE:** Compressor will be useful if you encounter disk bottleneck.Since the data needs to be compressed and decompressed,it involves additional CPU cycles,but is compensated by the high IO throughput due to less data to be written or read from the disks. |
-| carbon.load.skewedDataOptimization.enabled | false | During data loading,CarbonData would divide the number of blocks equally so as to ensure all executors process same number of blocks.This mechanism satisfies most of the scenarios and ensures maximum parallel processing for optimal data loading performance.In some business scenarios, there might be scenarios where the size of blocks vary significantly and hence some executors would have to do more work if they get blocks containing more data. This configuration enables size based block allocation strategy for data loading.When loading, carbondata will use file size based block allocation strategy for task distribution. It will make sure that all the executors process the same size of data.**NOTE:** This configuration is useful if the size of your input data files varies widely, say 1MB~1GB.For this configuration to work effectively,knowing the data pattern and size is important and necessary. |
-| carbon.load.min.size.enabled | false | During Data Loading, CarbonData would divide the number of files among the available executors to parallelize the loading operation.When the input data files are very small, this action causes to generate many small carbondata files.This configuration determines whether to enable node minumun input data size allocation strategy for data loading.It will make sure that the node load the minimum amount of data there by reducing number of carbondata files.**NOTE:** This configuration is useful if the size of the input data files are very small, like 1MB~256MB.Refer to ***load_min_size_inmb*** to configure the minimum size to be considered for splitting files among executors. |
-| enable.data.loading.statistics | false | CarbonData has extensive logging which would be useful for debugging issues related to performance or hard to locate issues.This configuration when made ***true*** would log additional data loading statistics information to more accurately locate the issues being debugged.**NOTE:** Enabling this would log more debug information to log files, there by increasing the log files size significantly in short span of time.It is advised to configure the log files size, retention of log files parameters in log4j properties appropriately.Also extensive logging is an increased IO operation and hence over all data loading performance might get reduced.Therefore it is recommened to enable this configuration only for the duration of debugging. |
-| carbon.dictionary.chunk.size | 10000 | CarbonData generates dictionary keys and writes them to separate dictionary file during data loading.To optimize the IO, this configuration determines the number of dictionary keys to be persisted to dictionary file at a time.**NOTE:** Writing to file also serves as a commit point to the dictionary generated.Increasing more values in memory causes more data loss during system or application failure.It is advised to alter this configuration judiciously. |
-| dictionary.worker.threads | 1 | CarbonData supports Optimized data loading by relying on a dictionary server.Dictionary server helps  to maintain dictionary values independent of the data loading and there by avoids reading the same input data multiples times.This configuration determines the number of concurrent dictionary generation or request that needs to be served by the dictionary server.**NOTE:** This configuration takes effect when ***carbon.options.single.pass*** is configured as true.Please refer to *carbon.options.single.pass*to understand how dictionary server optimizes data loading. |
-| enable.unsafe.sort | true | CarbonData supports unsafe operations of Java to avoid GC overhead for certain operations.This configuration enables to use unsafe functions in CarbonData.**NOTE:** For operations like data loading, which generates more short lived Java objects, Java GC can be a bottle neck.Using unsafe can overcome the GC overhead and improve the overall performance. |
-| enable.offheap.sort | true | CarbonData supports storing data in off-heap memory for certain operations during data loading and query.This helps to avoid the Java GC and thereby improve the overall performance.This configuration enables using off-heap memory for sorting of data during data loading.**NOTE:**  ***enable.unsafe.sort*** configuration needs to be configured to true for using off-heap |
-| enable.inmemory.merge.sort | false | CarbonData sorts and writes data to intermediate files to limit the memory usage.These intermediate files needs to be sorted again using merge sort before writing to the final carbondata file.Performing merge sort in memory would increase the sorting performance at the cost of increased memory footprint. This Configuration specifies to do in-memory merge sort or to do file based merge sort. |
-| carbon.load.sort.scope | LOCAL_SORT | CarbonData can support various sorting options to match the balance between load and query performance.LOCAL_SORT:All the data given to an executor in the single load is fully sorted and written to carondata files.Data loading performance is reduced a little as the entire data needs to be sorted in the executor.BATCH_SORT:Sorts the data in batches of configured size and writes to carbondata files.Data loading performance increases as the entire data need not be sorted.But query performance will get reduced due to false positives in block pruning and also due to more number of carbondata files written.Due to more number of carbondata files, if identified blocks > cluster parallelism, query performance and concurrency will get reduced.GLOBAL SORT:Entire data in the data load is fully sorted and written to carbondata files.Data loading perfromance would get reduced as the entire data needs to be sorted.But the query performance increases signific
 antly due to very less false positives and concurrency is also improved.**NOTE:** when BATCH_SORTis configured, it is recommended to keep ***carbon.load.batch.sort.size.inmb*** > ***carbon.blockletgroup.size.in.mb*** |
-| carbon.load.batch.sort.size.inmb | 0 | When  ***carbon.load.sort.scope*** is configured as ***BATCH_SORT***,This configuration needs to be added to specify the batch size for sorting and writing to carbondata files.**NOTE:** It is recommended to keep the value around 45% of ***carbon.sort.storage.inmemory.size.inmb*** to avoid spill to disk.Also it is recommended to keep the value higher than ***carbon.blockletgroup.size.in.mb***. Refer to *carbon.load.sort.scope* for more information on sort options and the advantages/disadvantges of each option. |
-| carbon.dictionary.server.port | 2030 | Single Pass Loading enables single job to finish data loading with dictionary generation on the fly. It enhances performance in the scenarios where the subsequent data loading after initial load involves fewer incremental updates on the dictionary.Single pass loading can be enabled using the option ***carbon.options.single.pass***.When this option is specified, a dictionary server will be internally started to handle the dictionary generation and query requests.This configuration specifies the port on which the server need to listen for incoming requests.Port value ranges between 0-65535 |
-| carbon.merge.sort.prefetch | true | CarbonData writes every ***carbon.sort.size*** number of records to intermediate temp files during data loading to ensure memory footprint is within limits.These intermediate temp files will have to be sorted using merge sort before writing into CarbonData format.This configuration enables pre fetching of data from these temp files in order to optimize IO and speed up data loading process. |
-| carbon.loading.prefetch | false | CarbonData uses univocity parser to read csv files.This configuration is used to inform the parser whether it can prefetch the data from csv files to speed up the reading.**NOTE:** Enabling prefetch improves the data loading performance, but needs higher memory to keep more records which are read ahead from disk. |
-| carbon.prefetch.buffersize | 1000 | When the configuration ***carbon.merge.sort.prefetch*** is configured to true, we need to set the number of records that can be prefetched.This configuration is used specify the number of records to be prefetched.**NOTE: **Configuring more number of records to be prefetched increases memory footprint as more records will have to be kept in memory. |
-| load_min_size_inmb | 256 | This configuration is used along with ***carbon.load.min.size.enabled***.This determines the minimum size of input files to be considered for distribution among executors while data loading.**NOTE:** Refer to ***carbon.load.min.size.enabled*** for understanding when this configuration needs to be used and its advantages and disadvantages. |
-| carbon.load.sortmemory.spill.percentage | 0 | During data loading, some data pages are kept in memory upto memory configured in ***carbon.sort.storage.inmemory.size.inmb*** beyond which they are spilled to disk as intermediate temporary sort files.This configuration determines after what percentage data needs to be spilled to disk.**NOTE:** Without this configuration, when the data pages occupy upto configured memory, new data pages would be dumped to disk and old pages are still maintained in disk. |
-| carbon.load.directWriteToStorePath.enabled | false | During data load, all the carbondata files are written to local disk and finally copied to the target store location in HDFS/S3.Enabling this parameter will make carbondata files to be written directly onto target HDFS/S3 location bypassing the local disk.**NOTE:** Writing directly to HDFS/S3 saves local disk IO(once for writing the files and again for copying to HDFS/S3) there by improving the performance.But the drawback is when data loading fails or the application crashes, unwanted carbondata files will remain in the target HDFS/S3 location until it is cleared during next data load or by running *CLEAN FILES* DDL command |
-| carbon.options.serialization.null.format | \N | Based on the business scenarios, some columns might need to be loaded with null values.As null value cannot be written in csv files, some special characters might be adopted to specify null values.This configuration can be used to specify the null values format in the data being loaded. |
-| carbon.sort.storage.inmemory.size.inmb | 512 | CarbonData writes every ***carbon.sort.size*** number of records to intermediate temp files during data loading to ensure memory footprint is within limits.When ***enable.unsafe.sort*** configuration is enabled, instead of using ***carbon.sort.size*** which is based on rows count, size occupied in memory is used to determine when to flush data pages to intermediate temp files.This configuration determines the memory to be used for storing data pages in memory.**NOTE:** Configuring a higher values ensures more data is maintained in memory and hence increases data loading performance due to reduced or no IO.Based on the memory availability in the nodes of the cluster, configure the values accordingly. |
+| carbon.timegranularity | SECOND | The configuration is used to specify the data granularity level such as DAY, HOUR, MINUTE, or SECOND. This helps to store more than 68 years of data into CarbonData. |
+| carbon.use.local.dir | false | CarbonData,during data loading, writes files to local temp directories before copying the files to HDFS. This configuration is used to specify whether CarbonData can write locally to tmp directory of the container or to the YARN application directory. |
+| carbon.use.multiple.temp.dir | false | When multiple disks are present in the system, YARN is generally configured with multiple disks to be used as temp directories for managing the containers. This configuration specifies whether to use multiple YARN local directories during data loading for disk IO load balancing.Enable ***carbon.use.local.dir*** for this configuration to take effect. **NOTE:** Data Loading is an IO intensive operation whose performance can be limited by the disk IO threshold, particularly during multi table concurrent data load.Configuring this parameter, balances the disk IO across multiple disks there by improving the over all load performance. |
+| carbon.sort.temp.compressor | (none) | CarbonData writes every ***carbon.sort.size*** number of records to intermediate temp files during data loading to ensure memory footprint is within limits. These temporary files can be compressed and written in order to save the storage space. This configuration specifies the name of compressor to be used to compress the intermediate sort temp files during sort procedure in data loading. The valid values are 'SNAPPY','GZIP','BZIP2','LZ4','ZSTD' and empty. By default, empty means that Carbondata will not compress the sort temp files. **NOTE:** Compressor will be useful if you encounter disk bottleneck.Since the data needs to be compressed and decompressed,it involves additional CPU cycles,but is compensated by the high IO throughput due to less data to be written or read from the disks. |
+| carbon.load.skewedDataOptimization.enabled | false | During data loading,CarbonData would divide the number of blocks equally so as to ensure all executors process same number of blocks. This mechanism satisfies most of the scenarios and ensures maximum parallel processing for optimal data loading performance.In some business scenarios, there might be scenarios where the size of blocks vary significantly and hence some executors would have to do more work if they get blocks containing more data. This configuration enables size based block allocation strategy for data loading. When loading, carbondata will use file size based block allocation strategy for task distribution. It will make sure that all the executors process the same size of data.**NOTE:** This configuration is useful if the size of your input data files varies widely, say 1MB to 1GB.For this configuration to work effectively,knowing the data pattern and size is important and necessary. |
+| carbon.load.min.size.enabled | false | During Data Loading, CarbonData would divide the number of files among the available executors to parallelize the loading operation. When the input data files are very small, this action causes to generate many small carbondata files. This configuration determines whether to enable node minumun input data size allocation strategy for data loading.It will make sure that the node load the minimum amount of data there by reducing number of carbondata files.**NOTE:** This configuration is useful if the size of the input data files are very small, like 1MB to 256MB.Refer to ***load_min_size_inmb*** to configure the minimum size to be considered for splitting files among executors. |
+| enable.data.loading.statistics | false | CarbonData has extensive logging which would be useful for debugging issues related to performance or hard to locate issues. This configuration when made ***true*** would log additional data loading statistics information to more accurately locate the issues being debugged. **NOTE:** Enabling this would log more debug information to log files, there by increasing the log files size significantly in short span of time.It is advised to configure the log files size, retention of log files parameters in log4j properties appropriately. Also extensive logging is an increased IO operation and hence over all data loading performance might get reduced. Therefore it is recommended to enable this configuration only for the duration of debugging. |
+| carbon.dictionary.chunk.size | 10000 | CarbonData generates dictionary keys and writes them to separate dictionary file during data loading. To optimize the IO, this configuration determines the number of dictionary keys to be persisted to dictionary file at a time. **NOTE:** Writing to file also serves as a commit point to the dictionary generated.Increasing more values in memory causes more data loss during system or application failure.It is advised to alter this configuration judiciously. |
+| dictionary.worker.threads | 1 | CarbonData supports Optimized data loading by relying on a dictionary server. Dictionary server helps to maintain dictionary values independent of the data loading and there by avoids reading the same input data multiples times. This configuration determines the number of concurrent dictionary generation or request that needs to be served by the dictionary server. **NOTE:** This configuration takes effect when ***carbon.options.single.pass*** is configured as true.Please refer to *carbon.options.single.pass*to understand how dictionary server optimizes data loading. |
+| enable.unsafe.sort | true | CarbonData supports unsafe operations of Java to avoid GC overhead for certain operations. This configuration enables to use unsafe functions in CarbonData. **NOTE:** For operations like data loading, which generates more short lived Java objects, Java GC can be a bottle neck. Using unsafe can overcome the GC overhead and improve the overall performance. |
+| enable.offheap.sort | true | CarbonData supports storing data in off-heap memory for certain operations during data loading and query. This helps to avoid the Java GC and thereby improve the overall performance. This configuration enables using off-heap memory for sorting of data during data loading.**NOTE:**  ***enable.unsafe.sort*** configuration needs to be configured to true for using off-heap |
+| enable.inmemory.merge.sort | false | CarbonData sorts and writes data to intermediate files to limit the memory usage. These intermediate files needs to be sorted again using merge sort before writing to the final carbondata file.Performing merge sort in memory would increase the sorting performance at the cost of increased memory footprint. This Configuration specifies to do in-memory merge sort or to do file based merge sort. |
+| carbon.load.sort.scope | LOCAL_SORT | CarbonData can support various sorting options to match the balance between load and query performance. LOCAL_SORT:All the data given to an executor in the single load is fully sorted and written to carbondata files. Data loading performance is reduced a little as the entire data needs to be sorted in the executor. BATCH_SORT:Sorts the data in batches of configured size and writes to carbondata files. Data loading performance increases as the entire data need not be sorted.But query performance will get reduced due to false positives in block pruning and also due to more number of carbondata files written.Due to more number of carbondata files, if identified blocks > cluster parallelism, query performance and concurrency will get reduced.GLOBAL SORT:Entire data in the data load is fully sorted and written to carbondata files. Data loading performance would get reduced as the entire data needs to be sorted.But the query performance increases si
 gnificantly due to very less false positives and concurrency is also improved. **NOTE:** when BATCH_SORT is configured, it is recommended to keep ***carbon.load.batch.sort.size.inmb*** > ***carbon.blockletgroup.size.in.mb*** |
+| carbon.load.batch.sort.size.inmb | 0 | When  ***carbon.load.sort.scope*** is configured as ***BATCH_SORT***, this configuration needs to be added to specify the batch size for sorting and writing to carbondata files. **NOTE:** It is recommended to keep the value around 45% of ***carbon.sort.storage.inmemory.size.inmb*** to avoid spill to disk. Also it is recommended to keep the value higher than ***carbon.blockletgroup.size.in.mb***. Refer to *carbon.load.sort.scope* for more information on sort options and the advantages/disadvantages of each option. |
+| carbon.dictionary.server.port | 2030 | Single Pass Loading enables single job to finish data loading with dictionary generation on the fly. It enhances performance in the scenarios where the subsequent data loading after initial load involves fewer incremental updates on the dictionary.Single pass loading can be enabled using the option ***carbon.options.single.pass***. When this option is specified, a dictionary server will be internally started to handle the dictionary generation and query requests. This configuration specifies the port on which the server need to listen for incoming requests.Port value ranges between 0-65535 |
+| carbon.merge.sort.prefetch | true | CarbonData writes every ***carbon.sort.size*** number of records to intermediate temp files during data loading to ensure memory footprint is within limits. These intermediate temp files will have to be sorted using merge sort before writing into CarbonData format. This configuration enables pre fetching of data from these temp files in order to optimize IO and speed up data loading process. |
+| carbon.loading.prefetch | false | CarbonData uses univocity parser to read csv files. This configuration is used to inform the parser whether it can prefetch the data from csv files to speed up the reading.**NOTE:** Enabling prefetch improves the data loading performance, but needs higher memory to keep more records which are read ahead from disk. |
+| carbon.prefetch.buffersize | 1000 | When the configuration ***carbon.merge.sort.prefetch*** is configured to true, we need to set the number of records that can be prefetched. This configuration is used specify the number of records to be prefetched.**NOTE: **Configuring more number of records to be prefetched increases memory footprint as more records will have to be kept in memory. |
+| load_min_size_inmb | 256 | This configuration is used along with ***carbon.load.min.size.enabled***. This determines the minimum size of input files to be considered for distribution among executors while data loading.**NOTE:** Refer to ***carbon.load.min.size.enabled*** for understanding when this configuration needs to be used and its advantages and disadvantages. |
+| carbon.load.sortmemory.spill.percentage | 0 | During data loading, some data pages are kept in memory upto memory configured in ***carbon.sort.storage.inmemory.size.inmb*** beyond which they are spilled to disk as intermediate temporary sort files. This configuration determines after what percentage data needs to be spilled to disk. **NOTE:** Without this configuration, when the data pages occupy upto configured memory, new data pages would be dumped to disk and old pages are still maintained in disk. |
+| carbon.load.directWriteToStorePath.enabled | false | During data load, all the carbondata files are written to local disk and finally copied to the target store location in HDFS/S3. Enabling this parameter will make carbondata files to be written directly onto target HDFS/S3 location bypassing the local disk.**NOTE:** Writing directly to HDFS/S3 saves local disk IO(once for writing the files and again for copying to HDFS/S3) there by improving the performance. But the drawback is when data loading fails or the application crashes, unwanted carbondata files will remain in the target HDFS/S3 location until it is cleared during next data load or by running *CLEAN FILES* DDL command |
+| carbon.options.serialization.null.format | \N | Based on the business scenarios, some columns might need to be loaded with null values. As null value cannot be written in csv files, some special characters might be adopted to specify null values. This configuration can be used to specify the null values format in the data being loaded. |
+| carbon.sort.storage.inmemory.size.inmb | 512 | CarbonData writes every ***carbon.sort.size*** number of records to intermediate temp files during data loading to ensure memory footprint is within limits. When ***enable.unsafe.sort*** configuration is enabled, instead of using ***carbon.sort.size*** which is based on rows count, size occupied in memory is used to determine when to flush data pages to intermediate temp files. This configuration determines the memory to be used for storing data pages in memory. **NOTE:** Configuring a higher value ensures more data is maintained in memory and hence increases data loading performance due to reduced or no IO.Based on the memory availability in the nodes of the cluster, configure the values accordingly. |
 | carbon.column.compressor | snappy | CarbonData will compress the column values using the compressor specified by this configuration. Currently CarbonData supports 'snappy' and 'zstd' compressors. |
 | carbon.minmax.allowed.byte.count | 200 | CarbonData will write the min max values for string/varchar types column using the byte count specified by this configuration. Max value is 1000 bytes(500 characters) and Min value is 10 bytes(5 characters). **NOTE:** This property is useful for reducing the store size thereby improving the query performance but can lead to query degradation if value is not configured properly. | |
 
@@ -100,53 +100,53 @@ This section provides the details of all the configurations required for the Car
 
 | Parameter | Default Value | Description |
 |-----------------------------------------------|---------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| carbon.number.of.cores.while.compacting | 2 | Number of cores to be used while compacting data.This also determines the number of threads to be used to read carbondata files in parallel. |
-| carbon.compaction.level.threshold | 4, 3 | Each CarbonData load will create one segment, if every load is small in size it will generate many small file over a period of time impacting the query performance.This configuration is for minor compaction which decides how many segments to be merged. Configuration is of the form (x,y). Compaction will be triggered for every x segments and form a single level 1 compacted segment.When the number of compacted level 1 segments reach y, compaction will be triggered again to merge them to form a single level 2 segment. For example: If it is set as 2, 3 then minor compaction will be triggered for every 2 segments. 3 is the number of level 1 compacted segments which is further compacted to new segment.**NOTE:** When ***carbon.enable.auto.load.merge*** is **true**, Configuring higher values cause overall data loading time to increase as compaction will be triggered after data loading is complete but status is not returned till compaction is comp
 lete. But compacting more number of segments can increase query performance.Hence optimal values needs to be configured based on the business scenario.Valid values are bwteen 0 to 100. |
-| carbon.major.compaction.size | 1024 | To improve query performance and All the segments can be merged and compacted to a single segment upto configured size.This Major compaction size can be configured using this parameter. Sum of the segments which is below this threshold will be merged. This value is expressed in MB. |
-| carbon.horizontal.compaction.enable | true | CarbonData supports DELETE/UPDATE functionality by creating delta data files for existing carbondata files.These delta files would grow as more number of DELETE/UPDATE operations are performed.Compaction of these delta files are termed as horizontal compaction.This configuration is used to turn ON/OFF horizontal compaction. After every DELETE and UPDATE statement, horizontal compaction may occur in case the delta (DELETE/ UPDATE) files becomes more than specified threshold.**NOTE: **Having many delta files will reduce the query performance as scan has to happen on all these files before the final state of data can be decided.Hence it is advisable to keep horizontal compaction enabled and configure reasonable values to ***carbon.horizontal.UPDATE.compaction.threshold*** and ***carbon.horizontal.DELETE.compaction.threshold*** |
+| carbon.number.of.cores.while.compacting | 2 | Number of cores to be used while compacting data. This also determines the number of threads to be used to read carbondata files in parallel. |
+| carbon.compaction.level.threshold | 4, 3 | Each CarbonData load will create one segment, if every load is small in size it will generate many small file over a period of time impacting the query performance. This configuration is for minor compaction which decides how many segments to be merged. Configuration is of the form (x,y). Compaction will be triggered for every x segments and form a single level 1 compacted segment. When the number of compacted level 1 segments reach y, compaction will be triggered again to merge them to form a single level 2 segment. For example: If it is set as 2, 3 then minor compaction will be triggered for every 2 segments. 3 is the number of level 1 compacted segments which is further compacted to new segment.**NOTE:** When ***carbon.enable.auto.load.merge*** is **true**, configuring higher values cause overall data loading time to increase as compaction will be triggered after data loading is complete but status is not returned till compaction is co
 mplete. But compacting more number of segments can increase query performance.Hence optimal values needs to be configured based on the business scenario. Valid values are between 0 to 100. |
+| carbon.major.compaction.size | 1024 | To improve query performance and all the segments can be merged and compacted to a single segment upto configured size. This Major compaction size can be configured using this parameter. Sum of the segments which is below this threshold will be merged. This value is expressed in MB. |
+| carbon.horizontal.compaction.enable | true | CarbonData supports DELETE/UPDATE functionality by creating delta data files for existing carbondata files. These delta files would grow as more number of DELETE/UPDATE operations are performed.Compaction of these delta files are termed as horizontal compaction. This configuration is used to turn ON/OFF horizontal compaction. After every DELETE and UPDATE statement, horizontal compaction may occur in case the delta (DELETE/ UPDATE) files becomes more than specified threshold.**NOTE: **Having many delta files will reduce the query performance as scan has to happen on all these files before the final state of data can be decided.Hence it is advisable to keep horizontal compaction enabled and configure reasonable values to ***carbon.horizontal.UPDATE.compaction.threshold*** and ***carbon.horizontal.DELETE.compaction.threshold*** |
 | carbon.horizontal.update.compaction.threshold | 1 | This configuration specifies the threshold limit on number of UPDATE delta files within a segment. In case the number of delta files goes beyond the threshold, the UPDATE delta files within the segment becomes eligible for horizontal compaction and are compacted into single UPDATE delta file.Values range between 1 to 10000. |
 | carbon.horizontal.delete.compaction.threshold | 1 | This configuration specifies the threshold limit on number of DELETE delta files within a block of a segment. In case the number of delta files goes beyond the threshold, the DELETE delta files for the particular block of the segment becomes eligible for horizontal compaction and are compacted into single DELETE delta file.Values range between 1 to 10000. |
-| carbon.update.segment.parallelism | 1 | CarbonData processes the UPDATE operations by grouping records belonging to a segment into a single executor task.When the amount of data to be updated is more, this behavior causes problems like restarting of executor due to low memory and data-spill related errors.This property specifies the parallelism for each segment during update.**NOTE:** It is recommended to set this value to a multiple of the number of executors for balance.Values range between 1 to 1000. |
+| carbon.update.segment.parallelism | 1 | CarbonData processes the UPDATE operations by grouping records belonging to a segment into a single executor task. When the amount of data to be updated is more, this behavior causes problems like restarting of executor due to low memory and data-spill related errors. This property specifies the parallelism for each segment during update.**NOTE:** It is recommended to set this value to a multiple of the number of executors for balance.Values range between 1 to 1000. |
 | carbon.numberof.preserve.segments | 0 | If the user wants to preserve some number of segments from being compacted then he can set this configuration. Example: carbon.numberof.preserve.segments = 2 then 2 latest segments will always be excluded from the compaction. No segments will be preserved by default.**NOTE:** This configuration is useful when the chances of input data can be wrong due to environment scenarios.Preserving some of the latest segments from being compacted can help to easily delete the wrongly loaded segments.Once compacted,it becomes more difficult to determine the exact data to be deleted(except when data is incrementing according to time) |
-| carbon.allowed.compaction.days | 0 | This configuration is used to control on the number of recent segments that needs to be compacted, ignoring the older ones.This congifuration is in days.For Example: If the configuration is 2, then the segments which are loaded in the time frame of past 2 days only will get merged. Segments which are loaded earlier than 2 days will not be merged. This configuration is disabled by default.**NOTE:** This configuration is useful when a bulk of history data is loaded into the carbondata.Query on this data is less frequent.In such cases involving these segments also into compacation will affect the resource consumption, increases overall compaction time. |
-| carbon.enable.auto.load.merge | false | Compaction can be automatically triggered once data load completes.This ensures that the segments are merged in time and thus query times doesnt increase with increase in segments.This configuration enables to do compaction along with data loading.**NOTE: **Compaction will be triggered once the data load completes.But the status of data load wait till the compaction is completed.Hence it might look like data loading time has increased, but thats not the case.Moreover failure of compaction will not affect the data loading status.If data load had completed successfully, the status would be updated and segments are committed.However, failure while data loading, will not trigger compaction and error is returned immediately. |
-| carbon.enable.page.level.reader.in.compaction|true|Enabling page level reader for compaction reduces the memory usage while compacting more number of segments. It allows reading only page by page instead of reading whole blocklet to memory.**NOTE:** Please refer to [file-structure-of-carbondata](./file-structure-of-carbondata.md#carbondata-file-format) to understand the storage format of CarbonData and concepts of pages.|
-| carbon.concurrent.compaction | true | Compaction of different tables can be executed concurrently.This configuration determines whether to compact all qualifying tables in parallel or not.**NOTE: **Compacting concurrently is a resource demanding operation and needs more resouces there by affecting the query performance also.This configuration is **deprecated** and might be removed in future releases. |
-| carbon.compaction.prefetch.enable | false | Compaction operation is similar to Query + data load where in data from qualifying segments are queried and data loading performed to generate a new single segment.This configuration determines whether to query ahead data from segments and feed it for data loading.**NOTE: **This configuration is disabled by default as it needs extra resources for querying ahead extra data.Based on the memory availability on the cluster, user can enable it to improve compaction performance. |
-| carbon.merge.index.in.segment | true | Each CarbonData file has a companion CarbonIndex file which maintains the metadata about the data.These CarbonIndex files are read and loaded into driver and is used subsequently for pruning of data during queries.These CarbonIndex files are very small in size(few KB) and are many.Reading many small files from HDFS is not efficient and leads to slow IO performance.Hence these CarbonIndex files belonging to a segment can be combined into  a single file and read once there by increasing the IO throughput.This configuration enables to merge all the CarbonIndex files into a single MergeIndex file upon data loading completion.**NOTE:** Reading a single big file is more efficient in HDFS and IO throughput is very high.Due to this the time needed to load the index files into memory when query is received for the first time on that table is significantly reduced and there by significantly reduces the delay in serving the first query. |
+| carbon.allowed.compaction.days | 0 | This configuration is used to control on the number of recent segments that needs to be compacted, ignoring the older ones. This configuration is in days.For Example: If the configuration is 2, then the segments which are loaded in the time frame of past 2 days only will get merged. Segments which are loaded earlier than 2 days will not be merged. This configuration is disabled by default.**NOTE:** This configuration is useful when a bulk of history data is loaded into the carbondata.Query on this data is less frequent.In such cases involving these segments also into compaction will affect the resource consumption, increases overall compaction time. |
+| carbon.enable.auto.load.merge | false | Compaction can be automatically triggered once data load completes. This ensures that the segments are merged in time and thus query times does not increase with increase in segments. This configuration enables to do compaction along with data loading.**NOTE: **Compaction will be triggered once the data load completes.But the status of data load wait till the compaction is completed.Hence it might look like data loading time has increased, but thats not the case.Moreover failure of compaction will not affect the data loading status.If data load had completed successfully, the status would be updated and segments are committed.However, failure while data loading, will not trigger compaction and error is returned immediately. |
+| carbon.enable.page.level.reader.in.compaction|true|Enabling page level reader for compaction reduces the memory usage while compacting more number of segments. It allows reading only page by page instead of reading whole blocklet to memory. **NOTE:** Please refer to [file-structure-of-carbondata](./file-structure-of-carbondata.md#carbondata-file-format) to understand the storage format of CarbonData and concepts of pages.|
+| carbon.concurrent.compaction | true | Compaction of different tables can be executed concurrently. This configuration determines whether to compact all qualifying tables in parallel or not. **NOTE: **Compacting concurrently is a resource demanding operation and needs more resources there by affecting the query performance also. This configuration is **deprecated** and might be removed in future releases. |
+| carbon.compaction.prefetch.enable | false | Compaction operation is similar to Query + data load where in data from qualifying segments are queried and data loading performed to generate a new single segment. This configuration determines whether to query ahead data from segments and feed it for data loading. **NOTE: **This configuration is disabled by default as it needs extra resources for querying extra data.Based on the memory availability on the cluster, user can enable it to improve compaction performance. |
+| carbon.merge.index.in.segment | true | Each CarbonData file has a companion CarbonIndex file which maintains the metadata about the data. These CarbonIndex files are read and loaded into driver and is used subsequently for pruning of data during queries. These CarbonIndex files are very small in size(few KB) and are many.Reading many small files from HDFS is not efficient and leads to slow IO performance.Hence these CarbonIndex files belonging to a segment can be combined into  a single file and read once there by increasing the IO throughput. This configuration enables to merge all the CarbonIndex files into a single MergeIndex file upon data loading completion.**NOTE:** Reading a single big file is more efficient in HDFS and IO throughput is very high.Due to this the time needed to load the index files into memory when query is received for the first time on that table is significantly reduced and there by significantly reduces the delay in serving the first query. |
 
 ## Query Configuration
 
 | Parameter | Default Value | Description |
 |--------------------------------------|---------------|---------------------------------------------------|
-| carbon.max.driver.lru.cache.size | -1 | Maximum memory **(in MB)** upto which the driver process can cache the data (BTree and dictionary values). Beyond this, least recently used data will be removed from cache before loading new set of values.Default value of -1 means there is no memory limit for caching. Only integer values greater than 0 are accepted.**NOTE:** Minimum number of entries that needs to be removed from cache in order to load the new set of data is determined and unloaded.ie.,for example if 3 cache entries qualify for pre-emption, out of these, those entries that free up more cache memory is removed prior to others. Please refer [FAQs](./faq.md#how-to-check-lru-cache-memory-footprint) for checking LRU cache memory footprint. |
-| carbon.max.executor.lru.cache.size | -1 | Maximum memory **(in MB)** upto which the executor process can cache the data (BTree and reverse dictionary values).Default value of -1 means there is no memory limit for caching. Only integer values greater than 0 are accepted.**NOTE:** If this parameter is not configured, then the value of ***carbon.max.driver.lru.cache.size*** will be used. |
+| carbon.max.driver.lru.cache.size | -1 | Maximum memory **(in MB)** upto which the driver process can cache the data (BTree and dictionary values). Beyond this, least recently used data will be removed from cache before loading new set of values.Default value of -1 means there is no memory limit for caching. Only integer values greater than 0 are accepted. **NOTE:** Minimum number of entries that needs to be removed from cache in order to load the new set of data is determined and unloaded.ie.,for example if 3 cache entries qualify for pre-emption, out of these, those entries that free up more cache memory is removed prior to others. Please refer [FAQs](./faq.md#how-to-check-lru-cache-memory-footprint) for checking LRU cache memory footprint. |
+| carbon.max.executor.lru.cache.size | -1 | Maximum memory **(in MB)** upto which the executor process can cache the data (BTree and reverse dictionary values).Default value of -1 means there is no memory limit for caching. Only integer values greater than 0 are accepted. **NOTE:** If this parameter is not configured, then the value of ***carbon.max.driver.lru.cache.size*** will be used. |
 | max.query.execution.time | 60 | Maximum time allowed for one query to be executed. The value is in minutes. |
-| carbon.enableMinMax | true | CarbonData maintains the metadata which enables to prune unnecessary files from being scanned as per the query conditions.To achieve pruning, Min,Max of each column is maintined.Based on the filter condition in the query, certain data can be skipped from scanning by matching the filter value against the min,max values of the column(s) present in that carbondata file.This pruing enhances query performance significantly. |
-| carbon.dynamicallocation.schedulertimeout | 5 | CarbonData has its own scheduling algorithm to suggest to Spark on how many tasks needs to be launched and how much work each task need to do in a Spark cluster for any query on CarbonData.To determine the number of tasks that can be scheduled, knowing the count of active executors is necessary.When dynamic allocation is enabled on a YARN based spark cluster,execuor processes are shutdown if no request is received for a particular amount of time.The executors are brought up when the requet is received again.This configuration specifies the maximum time (unit in seconds) the carbon scheduler can wait for executor to be active. Minimum value is 5 sec and maximum value is 15 sec.**NOTE: **Waiting for longer time leads to slow query response time.Moreover it might be possible that YARN is not able to start the executors and waiting is not beneficial. |
-| carbon.scheduler.minregisteredresourcesratio | 0.8 | Specifies the minimum resource (executor) ratio needed for starting the block distribution. The default value is 0.8, which indicates 80% of the requested resource is allocated for starting block distribution.  The minimum value is 0.1 min and the maximum value is 1.0. |
+| carbon.enableMinMax | true | CarbonData maintains the metadata which enables to prune unnecessary files from being scanned as per the query conditions. To achieve pruning, Min,Max of each column is maintined.Based on the filter condition in the query, certain data can be skipped from scanning by matching the filter value against the min,max values of the column(s) present in that carbondata file. This pruning enhances query performance significantly. |
+| carbon.dynamicallocation.schedulertimeout | 5 | CarbonData has its own scheduling algorithm to suggest to Spark on how many tasks needs to be launched and how much work each task need to do in a Spark cluster for any query on CarbonData. To determine the number of tasks that can be scheduled, knowing the count of active executors is necessary. When dynamic allocation is enabled on a YARN based spark cluster, executor processes are shutdown if no request is received for a particular amount of time. The executors are brought up when the requet is received again. This configuration specifies the maximum time (unit in seconds) the carbon scheduler can wait for executor to be active. Minimum value is 5 sec and maximum value is 15 sec.**NOTE: **Waiting for longer time leads to slow query response time.Moreover it might be possible that YARN is not able to start the executors and waiting is not beneficial. |
+| carbon.scheduler.minregisteredresourcesratio | 0.8 | Specifies the minimum resource (executor) ratio needed for starting the block distribution. The default value is 0.8, which indicates 80% of the requested resource is allocated for starting block distribution. The minimum value is 0.1 min and the maximum value is 1.0. |
 | carbon.search.enabled (Alpha Feature) | false | If set to true, it will use CarbonReader to do distributed scan directly instead of using compute framework like spark, thus avoiding limitation of compute framework like SQL optimizer and task scheduling overhead. |
-| carbon.search.query.timeout | 10s | Time within which the result is expected from the workers;beyond which the query is terminated |
+| carbon.search.query.timeout | 10s | Time within which the result is expected from the workers, beyond which the query is terminated |
 | carbon.search.scan.thread | num of cores available in worker node | Number of cores to be used in each worker for performing scan. |
 | carbon.search.master.port | 10020 | Port on which the search master listens for incoming query requests |
 | carbon.search.worker.port | 10021 | Port on which search master communicates with the workers. |
 | carbon.search.worker.workload.limit | 10 * *carbon.search.scan.thread* | Maximum number of active requests that can be sent to a worker.Beyond which the request needs to be rescheduled for later time or to a different worker. |
 | carbon.detail.batch.size | 100 | The buffer size to store records, returned from the block scan. In limit scenario this parameter is very important. For example your query limit is 1000. But if we set this value to 3000 that means we get 3000 records from scan but spark will only take 1000 rows. So the 2000 remaining are useless. In one Finance test case after we set it to 100, in the limit 1000 scenario the performance increase about 2 times in comparison to if we set this value to 12000. |
-| carbon.enable.vector.reader | true | Spark added vector processing to optimize cpu cache miss and there by increase the query performance.This configuration enables to fetch data as columnar batch of size 4*1024 rows instead of fetching data row by row and provide it to spark so that there is improvement in  select queries performance. |
+| carbon.enable.vector.reader | true | Spark added vector processing to optimize cpu cache miss and there by increase the query performance. This configuration enables to fetch data as columnar batch of size 4*1024 rows instead of fetching data row by row and provide it to spark so that there is improvement in  select queries performance. |
 | carbon.task.distribution | block | CarbonData has its own scheduling algorithm to suggest to Spark on how many tasks needs to be launched and how much work each task need to do in a Spark cluster for any query on CarbonData.Each of these task distribution suggestions has its own advantages and disadvantages.Based on the customer use case, appropriate task distribution can be configured.**block**: Setting this value will launch one task per block. This setting is suggested in case of concurrent queries and queries having big shuffling scenarios. **custom**: Setting this value will group the blocks and distribute it uniformly to the available resources in the cluster. This enhances the query performance but not suggested in case of concurrent queries and queries having big shuffling scenarios. **blocklet**: Setting this value will launch one task per blocklet. This setting is suggested in case of concurrent queries and queries having big shuffling scenarios. **merge_small_files**: S
 etting this value will merge all the small carbondata files upto a bigger size configured by ***spark.sql.files.maxPartitionBytes*** (128 MB is the default value,it is configurable) during querying. The small carbondata files are combined to a map task to reduce the number of read task. This enhances the performance. |
-| carbon.custom.block.distribution | false | CarbonData has its own scheduling algorithm to suggest to Spark on how many tasks needs to be launched and how much work each task need to do in a Spark cluster for any query on CarbonData.When this configuration is true, CarbonData would distribute the available blocks to be scanned among the available number of cores.For Example:If there are 10 blocks to be scanned and only 3 tasks can be run(only 3 executor cores available in the cluster), CarbonData would combine blocks as 4,3,3 and give it to 3 tasks to run.**NOTE:** When this configuration is false, as per the ***carbon.task.distribution*** configuration, each block/blocklet would be given to each task. |
-| enable.query.statistics | false | CarbonData has extensive logging which would be useful for debugging issues related to performance or hard to locate issues.This configuration when made ***true*** would log additional query statistics information to more accurately locate the issues being debugged.**NOTE:** Enabling this would log more debug information to log files, there by increasing the log files size significantly in short span of time.It is advised to configure the log files size, retention of log files parameters in log4j properties appropriately.Also extensive logging is an increased IO operation and hence over all query performance might get reduced.Therefore it is recommened to enable this configuration only for the duration of debugging. |
-| enable.unsafe.in.query.processing | true | CarbonData supports unsafe operations of Java to avoid GC overhead for certain operations.This configuration enables to use unsafe functions in CarbonData while scanning the  data during query. |
-| carbon.query.validate.directqueryondatamap | true | CarbonData supports creating pre-aggregate table datamaps as an independent tables.For some debugging purposes, it might be required to directly query from such datamap tables.This configuration allows to query on such datamaps. |
-| carbon.heap.memory.pooling.threshold.bytes | 1048576 | CarbonData supports unsafe operations of Java to avoid GC overhead for certain operations.Using unsafe, memory can be allocated on Java Heap or off heap.This configuration controlls the allocation mechanism on Java HEAP.If the heap memory allocations of the given size is greater or equal than this value,it should go through the pooling mechanism.But if set this size to -1, it should not go through the pooling mechanism.Default value is 1048576(1MB, the same as Spark).Value to be specified in bytes. |
+| carbon.custom.block.distribution | false | CarbonData has its own scheduling algorithm to suggest to Spark on how many tasks needs to be launched and how much work each task need to do in a Spark cluster for any query on CarbonData. When this configuration is true, CarbonData would distribute the available blocks to be scanned among the available number of cores.For Example:If there are 10 blocks to be scanned and only 3 tasks can be run(only 3 executor cores available in the cluster), CarbonData would combine blocks as 4,3,3 and give it to 3 tasks to run. **NOTE:** When this configuration is false, as per the ***carbon.task.distribution*** configuration, each block/blocklet would be given to each task. |
+| enable.query.statistics | false | CarbonData has extensive logging which would be useful for debugging issues related to performance or hard to locate issues. This configuration when made ***true*** would log additional query statistics information to more accurately locate the issues being debugged.**NOTE:** Enabling this would log more debug information to log files, there by increasing the log files size significantly in short span of time.It is advised to configure the log files size, retention of log files parameters in log4j properties appropriately. Also extensive logging is an increased IO operation and hence over all query performance might get reduced. Therefore it is recommended to enable this configuration only for the duration of debugging. |
+| enable.unsafe.in.query.processing | false | CarbonData supports unsafe operations of Java to avoid GC overhead for certain operations. This configuration enables to use unsafe functions in CarbonData while scanning the  data during query. |
+| carbon.query.validate.directqueryondatamap | true | CarbonData supports creating pre-aggregate table datamaps as an independent tables. For some debugging purposes, it might be required to directly query from such datamap tables. This configuration allows to query on such datamaps. |
+| carbon.heap.memory.pooling.threshold.bytes | 1048576 | CarbonData supports unsafe operations of Java to avoid GC overhead for certain operations. Using unsafe, memory can be allocated on Java Heap or off heap. This configuration controls the allocation mechanism on Java HEAP.If the heap memory allocations of the given size is greater or equal than this value,it should go through the pooling mechanism.But if set this size to -1, it should not go through the pooling mechanism.Default value is 1048576(1MB, the same as Spark).Value to be specified in bytes. |
 
 ## Data Mutation Configuration
 | Parameter | Default Value | Description |
 |--------------------------------------|---------------|---------------------------------------------------|
 | carbon.insert.persist.enable | false | CarbonData does loading in 2 major steps.1st step reads from the input source and generates the dictionary values.2nd step reads from the source again and encodes the data with the dictionary values, perform index calculations and writes in CarbonData format. Supp

<TRUNCATED>

[33/45] carbondata git commit: [CARBONDATA-2594] Do not add InvertedIndex in Encoding list for non-sort dimension column #2768

Posted by ra...@apache.org.
[CARBONDATA-2594] Do not add InvertedIndex in Encoding list for non-sort dimension column #2768

Not add InvertedIndex in Encoding list for non-sort dimension column

This closes #2768


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

Branch: refs/heads/branch-1.5
Commit: 18fbdfc409dc14812c9f384c437a793e9293b32b
Parents: 8fbd4a5
Author: Jacky Li <ja...@qq.com>
Authored: Wed Sep 26 21:31:35 2018 +0800
Committer: kumarvishal09 <ku...@gmail.com>
Committed: Thu Oct 4 16:57:57 2018 +0530

----------------------------------------------------------------------
 .../carbondata/core/metadata/schema/table/TableSchemaBuilder.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/18fbdfc4/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchemaBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchemaBuilder.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchemaBuilder.java
index f1be5ca..b5ce725 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchemaBuilder.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchemaBuilder.java
@@ -224,7 +224,7 @@ public class TableSchemaBuilder {
         }
       }
     }
-    if (newColumn.isDimensionColumn()) {
+    if (newColumn.isDimensionColumn() && newColumn.isSortColumn()) {
       newColumn.setUseInvertedIndex(true);
     }
     if (field.getDataType().isComplexType()) {


[34/45] carbondata git commit: [CARBONDATA-2975] DefaultValue choosing and removeNullValues on rangefilters is incorrect

Posted by ra...@apache.org.
[CARBONDATA-2975] DefaultValue choosing and removeNullValues on rangefilters is incorrect

This closes #2770


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

Branch: refs/heads/branch-1.5
Commit: 6aa2a90bf113132880694e305a264cdccc4c5693
Parents: 18fbdfc
Author: dhatchayani <dh...@gmail.com>
Authored: Wed Sep 26 20:49:06 2018 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Oct 4 17:22:14 2018 +0530

----------------------------------------------------------------------
 .../apache/carbondata/core/scan/filter/FilterUtil.java   | 11 ++++++-----
 .../filter/executer/RangeValueFilterExecuterImpl.java    |  2 +-
 .../executer/RowLevelRangeGrtThanFiterExecuterImpl.java  | 10 ++++++++--
 .../RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java   | 10 ++++++++--
 .../RowLevelRangeLessThanEqualFilterExecuterImpl.java    | 11 +++++++----
 .../RowLevelRangeLessThanFilterExecuterImpl.java         | 11 +++++++----
 .../spark/testsuite/sortcolumns/TestSortColumns.scala    |  6 ++++++
 7 files changed, 43 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/6aa2a90b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
index b4354d2..fe92c42 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
@@ -1947,11 +1947,12 @@ public final class FilterUtil {
   public static void removeNullValues(DimensionColumnPage dimensionColumnPage, BitSet bitSet,
       byte[] defaultValue) {
     if (!bitSet.isEmpty()) {
-      if (null != dimensionColumnPage.getNullBits() && !dimensionColumnPage.getNullBits().isEmpty()
-          && !dimensionColumnPage.isExplicitSorted() && !dimensionColumnPage.isAdaptiveEncoded()) {
-        for (int i = bitSet.nextSetBit(0); i >= 0; i = bitSet.nextSetBit(i + 1)) {
-          if (dimensionColumnPage.getNullBits().get(i)) {
-            bitSet.flip(i);
+      if (null != dimensionColumnPage.getNullBits()) {
+        if (!dimensionColumnPage.getNullBits().isEmpty()) {
+          for (int i = bitSet.nextSetBit(0); i >= 0; i = bitSet.nextSetBit(i + 1)) {
+            if (dimensionColumnPage.getNullBits().get(i)) {
+              bitSet.flip(i);
+            }
           }
         }
       } else {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6aa2a90b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java
index b9729db..886a13b 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java
@@ -632,7 +632,7 @@ public class RangeValueFilterExecuterImpl implements FilterExecuter {
       } else {
         if (dimColEvaluatorInfo.getDimension().getDataType() == DataTypes.STRING) {
           defaultValue = CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY;
-        } else {
+        } else if (!dimensionColumnPage.isAdaptiveEncoded()) {
           defaultValue = CarbonCommonConstants.EMPTY_BYTE_ARRAY;
         }
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6aa2a90b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
index 63a5976..c6835f8 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
@@ -28,6 +28,7 @@ import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
@@ -387,9 +388,14 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
     } else {
       bitSet = setFilterdIndexToBitSet(dimensionColumnPage, numerOfRows);
     }
+    byte[] defaultValue = null;
+    if (dimColEvaluatorInfoList.get(0).getDimension().getDataType() == DataTypes.STRING) {
+      defaultValue = CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY;
+    } else if (!dimensionColumnPage.isAdaptiveEncoded()) {
+      defaultValue = CarbonCommonConstants.EMPTY_BYTE_ARRAY;
+    }
     if (dimensionColumnPage.isNoDicitionaryColumn()) {
-      FilterUtil.removeNullValues(dimensionColumnPage, bitSet,
-          CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY);
+      FilterUtil.removeNullValues(dimensionColumnPage, bitSet, defaultValue);
     }
     return bitSet;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6aa2a90b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
index 0f9cfae..afb646a 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
@@ -28,6 +28,7 @@ import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
@@ -386,9 +387,14 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
     } else {
       bitSet = setFilterdIndexToBitSet(dimensionColumnPage, numerOfRows);
     }
+    byte[] defaultValue = null;
+    if (dimColEvaluatorInfoList.get(0).getDimension().getDataType() == DataTypes.STRING) {
+      defaultValue = CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY;
+    } else if (!dimensionColumnPage.isAdaptiveEncoded()) {
+      defaultValue = CarbonCommonConstants.EMPTY_BYTE_ARRAY;
+    }
     if (dimensionColumnPage.isNoDicitionaryColumn()) {
-      FilterUtil.removeNullValues(dimensionColumnPage, bitSet,
-          CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY);
+      FilterUtil.removeNullValues(dimensionColumnPage, bitSet, defaultValue);
     }
     return bitSet;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6aa2a90b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
index eff6509..647425d 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
@@ -383,13 +383,16 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
     BitSet bitSet = null;
     if (dimensionColumnPage.isExplicitSorted()) {
       bitSet = setFilterdIndexToBitSetWithColumnIndex(dimensionColumnPage, numerOfRows,
-          defaultValue);
+          dimensionColumnPage.isAdaptiveEncoded() ? null : defaultValue);
     } else {
-      bitSet = setFilterdIndexToBitSet(dimensionColumnPage, numerOfRows, defaultValue);
+      bitSet = setFilterdIndexToBitSet(dimensionColumnPage, numerOfRows,
+          dimensionColumnPage.isAdaptiveEncoded() ? null : defaultValue);
+    }
+    if (dimColEvaluatorInfoList.get(0).getDimension().getDataType() == DataTypes.STRING) {
+      defaultValue = CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY;
     }
     if (dimensionColumnPage.isNoDicitionaryColumn()) {
-      FilterUtil.removeNullValues(dimensionColumnPage, bitSet,
-          CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY);
+      FilterUtil.removeNullValues(dimensionColumnPage, bitSet, defaultValue);
     }
     return bitSet;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6aa2a90b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFilterExecuterImpl.java
index 7c48180..4ef9999 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFilterExecuterImpl.java
@@ -380,13 +380,16 @@ public class RowLevelRangeLessThanFilterExecuterImpl extends RowLevelFilterExecu
     BitSet bitSet = null;
     if (dimensionColumnPage.isExplicitSorted()) {
       bitSet = setFilterdIndexToBitSetWithColumnIndex(dimensionColumnPage, numerOfRows,
-          defaultValue);
+          dimensionColumnPage.isAdaptiveEncoded() ? null : defaultValue);
     } else {
-      bitSet = setFilterdIndexToBitSet(dimensionColumnPage, numerOfRows, defaultValue);
+      bitSet = setFilterdIndexToBitSet(dimensionColumnPage, numerOfRows,
+          dimensionColumnPage.isAdaptiveEncoded() ? null : defaultValue);
+    }
+    if (dimColEvaluatorInfoList.get(0).getDimension().getDataType() == DataTypes.STRING) {
+      defaultValue = CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY;
     }
     if (dimensionColumnPage.isNoDicitionaryColumn()) {
-      FilterUtil.removeNullValues(dimensionColumnPage, bitSet,
-          CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY);
+      FilterUtil.removeNullValues(dimensionColumnPage, bitSet, defaultValue);
     }
     return bitSet;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6aa2a90b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
index d9cb0e0..8d4dba3 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
@@ -343,6 +343,12 @@ class TestSortColumns extends QueryTest with BeforeAndAfterAll {
     sql(s"LOAD DATA local inpath '$resourcesPath/numeric_column_invalid_values.csv' INTO TABLE test_sort_col OPTIONS('FILEHEADER'='id,name,age')")
     // compare hive and carbon data
     checkAnswer(sql("select * from test_sort_col_hive"), sql("select * from test_sort_col"))
+    checkAnswer(sql("select * from test_sort_col_hive where age < 25"), sql("select * from test_sort_col where age < 25"))
+    checkAnswer(sql("select * from test_sort_col_hive where age <= 25"), sql("select * from test_sort_col where age <= 25"))
+    checkAnswer(sql("select * from test_sort_col_hive where age > 25"), sql("select * from test_sort_col where age > 25"))
+    checkAnswer(sql("select * from test_sort_col_hive where age >= 25"), sql("select * from test_sort_col where age >= 25"))
+    checkAnswer(sql("select * from test_sort_col_hive where age is null"), sql("select * from test_sort_col where age is null"))
+    checkAnswer(sql("select * from test_sort_col_hive where age is not null"), sql("select * from test_sort_col where age is not null"))
   }
 
   test("describe formatted for sort_columns") {