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 2017/04/18 12:03:37 UTC

[01/50] incubator-carbondata git commit: lazy rdd iterator [Forced Update!]

Repository: incubator-carbondata
Updated Branches:
  refs/heads/branch-1.1 8e50b878f -> dbf76485f (forced update)


lazy rdd iterator


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

Branch: refs/heads/branch-1.1
Commit: e52e641372e298511ba0135054c97855153356dc
Parents: d51387b
Author: QiangCai <qi...@qq.com>
Authored: Fri Apr 7 19:38:19 2017 +0530
Committer: Venkata Ramana G <ra...@huawei.com>
Committed: Tue Apr 11 14:08:53 2017 +0530

----------------------------------------------------------------------
 .../spark/rdd/NewCarbonDataLoadRDD.scala        | 100 +++++++++++++++----
 1 file changed, 81 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e52e6413/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 72ee90f..a6d231d 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
@@ -32,6 +32,7 @@ import org.apache.hadoop.mapreduce.{TaskAttemptID, TaskType}
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
 import org.apache.spark.{Partition, SerializableWritable, SparkContext, SparkEnv, TaskContext}
 import org.apache.spark.rdd.{DataLoadCoalescedRDD, DataLoadPartitionWrap, RDD}
+import org.apache.spark.serializer.SerializerInstance
 import org.apache.spark.sql.Row
 import org.apache.spark.util.SparkUtil
 
@@ -408,19 +409,14 @@ class NewDataFrameLoaderRDD[K, V](
         val recordReaders = mutable.Buffer[CarbonIterator[Array[AnyRef]]]()
         val partitionIterator = firstParent[DataLoadPartitionWrap[Row]].iterator(theSplit, context)
         val serializer = SparkEnv.get.closureSerializer.newInstance()
-        var serializeBuffer: ByteBuffer = null
+        var serializeBytes: Array[Byte] = null
         while(partitionIterator.hasNext) {
           val value = partitionIterator.next()
-          val newInstance = {
-            if (serializeBuffer == null) {
-              serializeBuffer = serializer.serialize[RDD[Row]](value.rdd)
-            }
-            serializeBuffer.rewind()
-            serializer.deserialize[RDD[Row]](serializeBuffer)
+          if (serializeBytes == null) {
+            serializeBytes = serializer.serialize[RDD[Row]](value.rdd).array()
           }
-          recordReaders += new NewRddIterator(newInstance.iterator(value.partition, context),
-              carbonLoadModel,
-              context)
+          recordReaders += new LazyRddIterator(serializer, serializeBytes, value.partition,
+              carbonLoadModel, context)
         }
 
         val loader = new SparkPartitionLoader(model,
@@ -477,15 +473,16 @@ class NewRddIterator(rddIter: Iterator[Row],
     carbonLoadModel: CarbonLoadModel,
     context: TaskContext) extends CarbonIterator[Array[AnyRef]] {
 
-  val timeStampformatString = CarbonProperties.getInstance().getProperty(CarbonCommonConstants
-    .CARBON_TIMESTAMP_FORMAT, CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
-  val timeStampFormat = new SimpleDateFormat(timeStampformatString)
-  val dateFormatString = CarbonProperties.getInstance().getProperty(CarbonCommonConstants
+  private val timeStampformatString = CarbonProperties.getInstance()
+    .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+      CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
+  private val timeStampFormat = new SimpleDateFormat(timeStampformatString)
+  private val dateFormatString = CarbonProperties.getInstance().getProperty(CarbonCommonConstants
     .CARBON_DATE_FORMAT, CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT)
-  val dateFormat = new SimpleDateFormat(dateFormatString)
-  val delimiterLevel1 = carbonLoadModel.getComplexDelimiterLevel1
-  val delimiterLevel2 = carbonLoadModel.getComplexDelimiterLevel2
-  val serializationNullFormat =
+  private val dateFormat = new SimpleDateFormat(dateFormatString)
+  private val delimiterLevel1 = carbonLoadModel.getComplexDelimiterLevel1
+  private val delimiterLevel2 = carbonLoadModel.getComplexDelimiterLevel2
+  private val serializationNullFormat =
     carbonLoadModel.getSerializationNullFormat.split(CarbonCommonConstants.COMMA, 2)(1)
   def hasNext: Boolean = rddIter.hasNext
 
@@ -499,8 +496,73 @@ class NewRddIterator(rddIter: Iterator[Row],
     columns
   }
 
-  override def initialize: Unit = {
+  override def initialize(): Unit = {
     SparkUtil.setTaskContext(context)
   }
 
 }
+
+/**
+ * LazyRddIterator invoke rdd.iterator method when invoking hasNext method.
+ * @param serializer
+ * @param serializeBytes
+ * @param partition
+ * @param carbonLoadModel
+ * @param context
+ */
+class LazyRddIterator(serializer: SerializerInstance,
+    serializeBytes: Array[Byte],
+    partition: Partition,
+    carbonLoadModel: CarbonLoadModel,
+    context: TaskContext) extends CarbonIterator[Array[AnyRef]] {
+
+  private val timeStampformatString = CarbonProperties.getInstance()
+    .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+      CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
+  private val timeStampFormat = new SimpleDateFormat(timeStampformatString)
+  private val dateFormatString = CarbonProperties.getInstance()
+    .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
+      CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT)
+  private val dateFormat = new SimpleDateFormat(dateFormatString)
+  private val delimiterLevel1 = carbonLoadModel.getComplexDelimiterLevel1
+  private val delimiterLevel2 = carbonLoadModel.getComplexDelimiterLevel2
+  private val serializationNullFormat =
+    carbonLoadModel.getSerializationNullFormat.split(CarbonCommonConstants.COMMA, 2)(1)
+
+  private var rddIter: Iterator[Row] = null
+  private var uninitialized = true
+  private var closed = false
+
+  def hasNext: Boolean = {
+    if (uninitialized) {
+      uninitialized = false
+      rddIter = serializer.deserialize[RDD[Row]](ByteBuffer.wrap(serializeBytes))
+        .iterator(partition, context)
+    }
+    if (closed) {
+      false
+    } else {
+      rddIter.hasNext
+    }
+  }
+
+  def next: Array[AnyRef] = {
+    val row = rddIter.next()
+    val columns = new Array[AnyRef](row.length)
+    for (i <- 0 until columns.length) {
+      columns(i) = CarbonScalaUtil.getString(row.get(i), serializationNullFormat,
+        delimiterLevel1, delimiterLevel2, timeStampFormat, dateFormat)
+    }
+    columns
+  }
+
+  override def initialize(): Unit = {
+    SparkUtil.setTaskContext(context)
+  }
+
+  override def close(): Unit = {
+    closed = true
+    rddIter = null
+  }
+
+}


[03/50] incubator-carbondata git commit: Removed unnecessary array copy and bitset checking

Posted by ra...@apache.org.
Removed unnecessary array copy and bitset checking

OPtimized code

Added table_blocksize option.

Removed unnecessary plan from optimized plan.

Fixed test

FIxed comment

Rebased


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

Branch: refs/heads/branch-1.1
Commit: 8f59a326ea4028ff7987e98826b00926df804ea7
Parents: 00535f4
Author: ravipesala <ra...@gmail.com>
Authored: Sun Mar 5 20:32:35 2017 +0530
Committer: jackylk <ja...@huawei.com>
Committed: Tue Apr 11 21:01:53 2017 +0800

----------------------------------------------------------------------
 .../carbondata/core/datastore/FileHolder.java   |   4 +-
 .../impl/VariableLengthDimensionDataChunk.java  |  16 +-
 ...mpressedDimensionChunkFileBasedReaderV1.java |  14 +-
 ...mpressedDimensionChunkFileBasedReaderV2.java |  19 +--
 ...mpressedDimensionChunkFileBasedReaderV3.java |  16 +-
 ...CompressedMeasureChunkFileBasedReaderV1.java |  16 +-
 ...CompressedMeasureChunkFileBasedReaderV2.java |  19 +--
 ...CompressedMeasureChunkFileBasedReaderV3.java |  34 ++--
 .../chunk/store/DimensionDataChunkStore.java    |   9 ++
 .../SafeAbsractDimensionDataChunkStore.java     |   4 +
 ...feVariableLengthDimensionDataChunkStore.java |  36 ++++-
 .../UnsafeAbstractDimensionDataChunkStore.java  |   4 +
 ...afeVariableLengthDimesionDataChunkStore.java |  12 ++
 .../core/datastore/impl/DFSFileHolderImpl.java  |   8 +-
 .../core/datastore/impl/FileHolderImpl.java     |   8 +-
 .../DictionaryBasedVectorResultCollector.java   |   1 -
 .../executor/impl/AbstractQueryExecutor.java    |   2 +-
 .../core/scan/executor/util/QueryUtil.java      |   2 +-
 .../vector/MeasureDataVectorProcessor.java      | 158 +++++++++++++------
 .../apache/carbondata/core/util/ByteUtil.java   |  29 ++++
 .../apache/carbondata/core/util/CarbonUtil.java |  12 +-
 .../carbondata/examples/CompareTest.scala       |   4 +
 .../apache/carbondata/spark/CarbonOption.scala  |   2 +
 .../VectorizedCarbonRecordReader.java           |   2 +-
 .../spark/sql/CarbonCatalystOperators.scala     |   2 +-
 .../spark/sql/CarbonDataFrameWriter.scala       |  13 +-
 .../spark/sql/CarbonDictionaryDecoder.scala     |  68 ++++----
 .../execution/CarbonLateDecodeStrategy.scala    |   5 +-
 28 files changed, 328 insertions(+), 191 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8f59a326/core/src/main/java/org/apache/carbondata/core/datastore/FileHolder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/FileHolder.java b/core/src/main/java/org/apache/carbondata/core/datastore/FileHolder.java
index b1eb1ee..1b972bc 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/FileHolder.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/FileHolder.java
@@ -27,12 +27,12 @@ public interface FileHolder {
    * and length(number of bytes) need to read
    *
    * @param filePath fully qualified file path
-   * @param byteBuffer
    * @param offset reading start position,
    * @param length number of bytes to be read
+   * @return ByteBuffer
    * @throws IOException
    */
-  void readByteBuffer(String filePath, ByteBuffer byteBuffer, long offset, int length)
+  ByteBuffer readByteBuffer(String filePath, long offset, int length)
       throws IOException;
   /**
    * This method will be used to read the byte array from file based on offset

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8f59a326/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/VariableLengthDimensionDataChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/VariableLengthDimensionDataChunk.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/VariableLengthDimensionDataChunk.java
index eac06f4..6c47bf5 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/VariableLengthDimensionDataChunk.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/VariableLengthDimensionDataChunk.java
@@ -16,8 +16,6 @@
  */
 package org.apache.carbondata.core.datastore.chunk.impl;
 
-import java.util.Arrays;
-
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.chunk.store.DimensionChunkStoreFactory;
 import org.apache.carbondata.core.datastore.chunk.store.DimensionChunkStoreFactory.DimensionStoreType;
@@ -109,14 +107,9 @@ public class VariableLengthDimensionDataChunk extends AbstractDimensionDataChunk
     int vectorOffset = columnVectorInfo.vectorOffset;
     int len = offset + columnVectorInfo.size;
     for (int i = offset; i < len; i++) {
-      byte[] value = dataChunkStore.getRow(i);
       // Considering only String case now as we support only
       // string in no dictionary case at present.
-      if (value == null || Arrays.equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY, value)) {
-        vector.putNull(vectorOffset++);
-      } else {
-        vector.putBytes(vectorOffset++, value);
-      }
+      dataChunkStore.fillRow(i, vector, vectorOffset++);
     }
     return column + 1;
   }
@@ -138,14 +131,9 @@ public class VariableLengthDimensionDataChunk extends AbstractDimensionDataChunk
     int vectorOffset = columnVectorInfo.vectorOffset;
     int len = offset + columnVectorInfo.size;
     for (int i = offset; i < len; i++) {
-      byte[] value = dataChunkStore.getRow(rowMapping[i]);
       // Considering only String case now as we support only
       // string in no dictionary case at present.
-      if (value == null || Arrays.equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY, value)) {
-        vector.putNull(vectorOffset++);
-      } else {
-        vector.putBytes(vectorOffset++, value);
-      }
+      dataChunkStore.fillRow(rowMapping[i], vector, vectorOffset++);
     }
     return column + 1;
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8f59a326/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java
index 00e6351..83e0c74 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java
@@ -85,12 +85,10 @@ public class CompressedDimensionChunkFileBasedReaderV1 extends AbstractChunkRead
   @Override public DimensionRawColumnChunk readRawDimensionChunk(FileHolder fileReader,
       int blockletIndex) throws IOException {
     DataChunk dataChunk = dimensionColumnChunk.get(blockletIndex);
-    ByteBuffer buffer =
-        ByteBuffer.allocateDirect(dataChunk.getDataPageLength());
+    ByteBuffer buffer = null;
     synchronized (fileReader) {
-      fileReader.readByteBuffer(filePath, buffer,
-          dataChunk.getDataPageOffset(),
-          dataChunk.getDataPageLength());
+      buffer = fileReader
+          .readByteBuffer(filePath, dataChunk.getDataPageOffset(), dataChunk.getDataPageLength());
     }
     DimensionRawColumnChunk rawColumnChunk = new DimensionRawColumnChunk(blockletIndex, buffer, 0,
         dataChunk.getDataPageLength(), this);
@@ -110,10 +108,8 @@ public class CompressedDimensionChunkFileBasedReaderV1 extends AbstractChunkRead
     FileHolder fileReader = dimensionRawColumnChunk.getFileReader();
 
     ByteBuffer rawData = dimensionRawColumnChunk.getRawData();
-    rawData.position(dimensionRawColumnChunk.getOffSet());
-    byte[] data = new byte[dimensionRawColumnChunk.getLength()];
-    rawData.get(data);
-    dataPage = COMPRESSOR.unCompressByte(data);
+    dataPage = COMPRESSOR.unCompressByte(rawData.array(), dimensionRawColumnChunk.getOffSet(),
+        dimensionRawColumnChunk.getLength());
 
     // if row id block is present then read the row id chunk and uncompress it
     DataChunk dataChunk = dimensionColumnChunk.get(blockIndex);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8f59a326/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java
index b2201cd..bd8de36 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java
@@ -66,9 +66,10 @@ public class CompressedDimensionChunkFileBasedReaderV2 extends AbstractChunkRead
       long currentDimensionOffset = dimensionChunksOffset.get(blockletIndex);
       length = (int) (dimensionChunksOffset.get(blockletIndex + 1) - currentDimensionOffset);
     }
-    ByteBuffer buffer = ByteBuffer.allocateDirect(length);
+    ByteBuffer buffer = null;
     synchronized (fileReader) {
-      fileReader.readByteBuffer(filePath, buffer, dimensionChunksOffset.get(blockletIndex), length);
+      buffer =
+          fileReader.readByteBuffer(filePath, dimensionChunksOffset.get(blockletIndex), length);
     }
     DimensionRawColumnChunk rawColumnChunk =
         new DimensionRawColumnChunk(blockletIndex, buffer, 0, length, this);
@@ -92,10 +93,9 @@ public class CompressedDimensionChunkFileBasedReaderV2 extends AbstractChunkRead
   protected DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileHolder fileReader,
       int startColumnBlockletIndex, int endColumnBlockletIndex) throws IOException {
     long currentDimensionOffset = dimensionChunksOffset.get(startColumnBlockletIndex);
-    ByteBuffer buffer = ByteBuffer.allocateDirect(
-        (int) (dimensionChunksOffset.get(endColumnBlockletIndex + 1) - currentDimensionOffset));
+    ByteBuffer buffer = null;
     synchronized (fileReader) {
-      fileReader.readByteBuffer(filePath, buffer, currentDimensionOffset,
+      buffer = fileReader.readByteBuffer(filePath, currentDimensionOffset,
           (int) (dimensionChunksOffset.get(endColumnBlockletIndex + 1) - currentDimensionOffset));
     }
     DimensionRawColumnChunk[] dataChunks =
@@ -132,8 +132,7 @@ public class CompressedDimensionChunkFileBasedReaderV2 extends AbstractChunkRead
           dimensionColumnChunk.data_page_length + dimensionColumnChunk.rle_page_length
               + dimensionColumnChunk.rowid_page_length;
       synchronized (dimensionRawColumnChunk.getFileReader()) {
-        rawData = ByteBuffer.allocateDirect(totalDimensionDataLength);
-        dimensionRawColumnChunk.getFileReader().readByteBuffer(filePath, rawData,
+        rawData = dimensionRawColumnChunk.getFileReader().readByteBuffer(filePath,
             dimensionChunksOffset.get(blockIndex) + dimensionChunksLength.get(blockIndex),
             totalDimensionDataLength);
       }
@@ -143,11 +142,9 @@ public class CompressedDimensionChunkFileBasedReaderV2 extends AbstractChunkRead
       copySourcePoint += dimensionChunksLength.get(blockIndex);
     }
 
-    byte[] data = new byte[dimensionColumnChunk.data_page_length];
-    rawData.position(copySourcePoint);
-    rawData.get(data);
     // first read the data and uncompressed it
-    dataPage = COMPRESSOR.unCompressByte(data, 0, dimensionColumnChunk.data_page_length);
+    dataPage = COMPRESSOR
+        .unCompressByte(rawData.array(), copySourcePoint, dimensionColumnChunk.data_page_length);
     copySourcePoint += dimensionColumnChunk.data_page_length;
     // if row id block is present then read the row id chunk and uncompress it
     if (hasEncoding(dimensionColumnChunk.encoders, Encoding.INVERTED_INDEX)) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8f59a326/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java
index 8bea132..9a14a85 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java
@@ -88,11 +88,10 @@ public class CompressedDimensionChunkFileBasedReaderV3 extends AbstractChunkRead
     } else {
       length = (int) (dimensionChunksOffset.get(blockletColumnIndex + 1) - currentDimensionOffset);
     }
-    // allocate the buffer
-    ByteBuffer buffer = ByteBuffer.allocateDirect(length);
+    ByteBuffer buffer = null;
     // read the data from carbon data file
     synchronized (fileReader) {
-      fileReader.readByteBuffer(filePath, buffer, currentDimensionOffset, length);
+      buffer = fileReader.readByteBuffer(filePath, currentDimensionOffset, length);
     }
     // get the data chunk which will have all the details about the data pages
     DataChunk3 dataChunk = CarbonUtil.readDataChunk3(buffer, 0, length);
@@ -148,11 +147,10 @@ public class CompressedDimensionChunkFileBasedReaderV3 extends AbstractChunkRead
     // column we can subtract the offset of start column offset with
     // end column+1 offset and get the total length.
     long currentDimensionOffset = dimensionChunksOffset.get(startBlockletColumnIndex);
-    ByteBuffer buffer = ByteBuffer.allocateDirect(
-        (int) (dimensionChunksOffset.get(endBlockletColumnIndex + 1) - currentDimensionOffset));
+    ByteBuffer buffer = null;
     // read the data from carbon data file
     synchronized (fileReader) {
-      fileReader.readByteBuffer(filePath, buffer, currentDimensionOffset,
+      buffer = fileReader.readByteBuffer(filePath, currentDimensionOffset,
           (int) (dimensionChunksOffset.get(endBlockletColumnIndex + 1) - currentDimensionOffset));
     }
     // create raw chunk for each dimension column
@@ -218,11 +216,9 @@ public class CompressedDimensionChunkFileBasedReaderV3 extends AbstractChunkRead
     // data chunk length + page offset
     int copySourcePoint = dimensionRawColumnChunk.getOffSet() + dimensionChunksLength
         .get(dimensionRawColumnChunk.getBlockletId()) + dataChunk3.getPage_offset().get(pageNumber);
-    byte[] data = new byte[dimensionColumnChunk.data_page_length];
-    rawData.position(copySourcePoint);
-    rawData.get(data);
     // first read the data and uncompressed it
-    dataPage = COMPRESSOR.unCompressByte(data, 0, dimensionColumnChunk.data_page_length);
+    dataPage = COMPRESSOR
+        .unCompressByte(rawData.array(), copySourcePoint, dimensionColumnChunk.data_page_length);
     copySourcePoint += dimensionColumnChunk.data_page_length;
     // if row id block is present then read the row id chunk and uncompress it
     if (hasEncoding(dimensionColumnChunk.encoders, Encoding.INVERTED_INDEX)) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8f59a326/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java
index 107c430..64e9b45 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java
@@ -82,11 +82,8 @@ public class CompressedMeasureChunkFileBasedReaderV1 extends AbstractMeasureChun
   @Override public MeasureRawColumnChunk readRawMeasureChunk(FileHolder fileReader, int blockIndex)
       throws IOException {
     DataChunk dataChunk = measureColumnChunks.get(blockIndex);
-    ByteBuffer buffer =
-        ByteBuffer.allocateDirect(dataChunk.getDataPageLength());
-    fileReader
-        .readByteBuffer(filePath, buffer, dataChunk.getDataPageOffset(),
-            dataChunk.getDataPageLength());
+    ByteBuffer buffer = fileReader
+        .readByteBuffer(filePath, dataChunk.getDataPageOffset(), dataChunk.getDataPageLength());
     MeasureRawColumnChunk rawColumnChunk = new MeasureRawColumnChunk(blockIndex, buffer, 0,
         dataChunk.getDataPageLength(), this);
     rawColumnChunk.setFileReader(fileReader);
@@ -104,15 +101,12 @@ public class CompressedMeasureChunkFileBasedReaderV1 extends AbstractMeasureChun
     ReaderCompressModel compressModel = ValueCompressionUtil.getReaderCompressModel(meta);
 
     ValueCompressionHolder values = compressModel.getValueCompressionHolder();
-    byte[] dataPage = new byte[measureRawColumnChunk.getLength()];
     ByteBuffer rawData = measureRawColumnChunk.getRawData();
-    rawData.position(measureRawColumnChunk.getOffSet());
-    rawData.get(dataPage);
 
     // unCompress data
-    values.uncompress(compressModel.getConvertedDataType(), dataPage, 0,
-        dataChunk.getDataPageLength(), compressModel.getMantissa(),
-        compressModel.getMaxValue(), numberOfRows);
+    values.uncompress(compressModel.getConvertedDataType(), rawData.array(),
+        measureRawColumnChunk.getOffSet(), dataChunk.getDataPageLength(),
+        compressModel.getMantissa(), compressModel.getMaxValue(), numberOfRows);
 
     CarbonReadDataHolder measureDataHolder = new CarbonReadDataHolder(values);
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8f59a326/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java
index 7b6acee..3ed1292 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java
@@ -58,10 +58,10 @@ public class CompressedMeasureChunkFileBasedReaderV2 extends AbstractMeasureChun
       long currentMeasureOffset = measureColumnChunkOffsets.get(blockIndex);
       dataLength = (int) (measureColumnChunkOffsets.get(blockIndex + 1) - currentMeasureOffset);
     }
-    ByteBuffer buffer = ByteBuffer.allocateDirect(dataLength);
+    ByteBuffer buffer = null;
     synchronized (fileReader) {
-      fileReader
-          .readByteBuffer(filePath, buffer, measureColumnChunkOffsets.get(blockIndex), dataLength);
+      buffer = fileReader
+          .readByteBuffer(filePath, measureColumnChunkOffsets.get(blockIndex), dataLength);
     }
     MeasureRawColumnChunk rawColumnChunk =
         new MeasureRawColumnChunk(blockIndex, buffer, 0, dataLength, this);
@@ -85,10 +85,9 @@ public class CompressedMeasureChunkFileBasedReaderV2 extends AbstractMeasureChun
   protected MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileHolder fileReader,
       int startColumnBlockletIndex, int endColumnBlockletIndex) throws IOException {
     long currentMeasureOffset = measureColumnChunkOffsets.get(startColumnBlockletIndex);
-    ByteBuffer buffer = ByteBuffer.allocateDirect(
-        (int) (measureColumnChunkOffsets.get(endColumnBlockletIndex + 1) - currentMeasureOffset));
+    ByteBuffer buffer = null;
     synchronized (fileReader) {
-      fileReader.readByteBuffer(filePath, buffer, currentMeasureOffset,
+      buffer = fileReader.readByteBuffer(filePath, currentMeasureOffset,
           (int) (measureColumnChunkOffsets.get(endColumnBlockletIndex + 1) - currentMeasureOffset));
     }
     MeasureRawColumnChunk[] dataChunks =
@@ -121,8 +120,7 @@ public class CompressedMeasureChunkFileBasedReaderV2 extends AbstractMeasureChun
       measureColumnChunk =
           CarbonUtil.readDataChunk(rawData, copyPoint, measureColumnChunkLength.get(blockIndex));
       synchronized (measureRawColumnChunk.getFileReader()) {
-        rawData = ByteBuffer.allocateDirect(measureColumnChunk.data_page_length);
-        measureRawColumnChunk.getFileReader().readByteBuffer(filePath, rawData,
+        rawData = measureRawColumnChunk.getFileReader().readByteBuffer(filePath,
             measureColumnChunkOffsets.get(blockIndex) + measureColumnChunkLength.get(blockIndex),
             measureColumnChunk.data_page_length);
       }
@@ -139,11 +137,8 @@ public class CompressedMeasureChunkFileBasedReaderV2 extends AbstractMeasureChun
     WriterCompressModel compressionModel = CarbonUtil.getValueCompressionModel(valueEncodeMeta);
 
     ValueCompressionHolder values = compressionModel.getValueCompressionHolder()[0];
-    byte[] data = new byte[measureColumnChunk.data_page_length];
-    rawData.position(copyPoint);
-    rawData.get(data);
     // uncompress
-    values.uncompress(compressionModel.getConvertedDataType()[0], data, 0,
+    values.uncompress(compressionModel.getConvertedDataType()[0], rawData.array(), copyPoint,
         measureColumnChunk.data_page_length, compressionModel.getMantissa()[0],
         compressionModel.getMaxValue()[0], numberOfRows);
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8f59a326/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMeasureChunkFileBasedReaderV3.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMeasureChunkFileBasedReaderV3.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMeasureChunkFileBasedReaderV3.java
index 570c0c8..36839fd 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMeasureChunkFileBasedReaderV3.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMeasureChunkFileBasedReaderV3.java
@@ -72,33 +72,33 @@ public class CompressedMeasureChunkFileBasedReaderV3 extends AbstractMeasureChun
    * @param blockIndex          blocklet index of the column in carbon data file
    * @return measure raw chunk
    */
-  @Override public MeasureRawColumnChunk readRawMeasureChunk(FileHolder fileReader, int blockIndex)
-      throws IOException {
+  @Override public MeasureRawColumnChunk readRawMeasureChunk(FileHolder fileReader,
+      int blockletColumnIndex) throws IOException {
     int dataLength = 0;
     // to calculate the length of the data to be read
     // column other than last column we can subtract the offset of current column with
     // next column and get the total length.
     // but for last column we need to use lastDimensionOffset which is the end position
     // of the last dimension, we can subtract current dimension offset from lastDimesionOffset
-    if (measureColumnChunkOffsets.size() - 1 == blockIndex) {
-      dataLength = (int) (measureOffsets - measureColumnChunkOffsets.get(blockIndex));
+    if (measureColumnChunkOffsets.size() - 1 == blockletColumnIndex) {
+      dataLength = (int) (measureOffsets - measureColumnChunkOffsets.get(blockletColumnIndex));
     } else {
-      dataLength = (int) (measureColumnChunkOffsets.get(blockIndex + 1) - measureColumnChunkOffsets
-          .get(blockIndex));
+      dataLength =
+          (int) (measureColumnChunkOffsets.get(blockletColumnIndex + 1) - measureColumnChunkOffsets
+              .get(blockletColumnIndex));
     }
-    // allocate the buffer
-    ByteBuffer buffer = ByteBuffer.allocateDirect(dataLength);
+    ByteBuffer buffer = null;
     // read the data from carbon data file
     synchronized (fileReader) {
-      fileReader
-          .readByteBuffer(filePath, buffer, measureColumnChunkOffsets.get(blockIndex), dataLength);
+      buffer = fileReader
+          .readByteBuffer(filePath, measureColumnChunkOffsets.get(blockletColumnIndex), dataLength);
     }
     // get the data chunk which will have all the details about the data pages
     DataChunk3 dataChunk =
-        CarbonUtil.readDataChunk3(buffer, 0, measureColumnChunkLength.get(blockIndex));
+        CarbonUtil.readDataChunk3(buffer, 0, measureColumnChunkLength.get(blockletColumnIndex));
     // creating a raw chunks instance and filling all the details
     MeasureRawColumnChunk rawColumnChunk =
-        new MeasureRawColumnChunk(blockIndex, buffer, 0, dataLength, this);
+        new MeasureRawColumnChunk(blockletColumnIndex, buffer, 0, dataLength, this);
     int numberOfPages = dataChunk.getPage_length().size();
     byte[][] maxValueOfEachPage = new byte[numberOfPages][];
     byte[][] minValueOfEachPage = new byte[numberOfPages][];
@@ -148,11 +148,10 @@ public class CompressedMeasureChunkFileBasedReaderV3 extends AbstractMeasureChun
     // column we can subtract the offset of start column offset with
     // end column+1 offset and get the total length.
     long currentMeasureOffset = measureColumnChunkOffsets.get(startColumnBlockletIndex);
-    ByteBuffer buffer = ByteBuffer.allocateDirect(
-        (int) (measureColumnChunkOffsets.get(endColumnBlockletIndex + 1) - currentMeasureOffset));
+    ByteBuffer buffer = null;
     // read the data from carbon data file
     synchronized (fileReader) {
-      fileReader.readByteBuffer(filePath, buffer, currentMeasureOffset,
+      buffer = fileReader.readByteBuffer(filePath, currentMeasureOffset,
           (int) (measureColumnChunkOffsets.get(endColumnBlockletIndex + 1) - currentMeasureOffset));
     }
     // create raw chunk for each measure column
@@ -224,11 +223,8 @@ public class CompressedMeasureChunkFileBasedReaderV3 extends AbstractMeasureChun
     WriterCompressModel compressionModel = CarbonUtil.getValueCompressionModel(valueEncodeMeta);
     ValueCompressionHolder values = compressionModel.getValueCompressionHolder()[0];
     // uncompress
-    byte[] data = new byte[measureColumnChunk.data_page_length];
     ByteBuffer rawData = measureRawColumnChunk.getRawData();
-    rawData.position(copyPoint);
-    rawData.get(data);
-    values.uncompress(compressionModel.getConvertedDataType()[0], data, 0,
+    values.uncompress(compressionModel.getConvertedDataType()[0], rawData.array(), copyPoint,
         measureColumnChunk.data_page_length, compressionModel.getMantissa()[0],
         compressionModel.getMaxValue()[0], measureRawColumnChunk.getRowCount()[pageNumber]);
     CarbonReadDataHolder measureDataHolder = new CarbonReadDataHolder(values);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8f59a326/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/DimensionDataChunkStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/DimensionDataChunkStore.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/DimensionDataChunkStore.java
index a3f7ac3..5301945 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/DimensionDataChunkStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/DimensionDataChunkStore.java
@@ -17,6 +17,8 @@
 
 package org.apache.carbondata.core.datastore.chunk.store;
 
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
+
 /**
  * Interface responsibility is to store dimension data in memory.
  * storage can be on heap or offheap.
@@ -42,6 +44,13 @@ public interface DimensionDataChunkStore {
   byte[] getRow(int rowId);
 
   /**
+   * Below method will be used to fill the row to vector
+   * based on row id passed
+   *
+   */
+  void fillRow(int rowId, CarbonColumnVector vector, int vectorRow);
+
+  /**
    * Below method will be used to fill the row values to buffer array
    *
    * @param rowId  row id of the data to be filled

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8f59a326/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeAbsractDimensionDataChunkStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeAbsractDimensionDataChunkStore.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeAbsractDimensionDataChunkStore.java
index cb10d3e..cb4c1a7 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeAbsractDimensionDataChunkStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeAbsractDimensionDataChunkStore.java
@@ -18,6 +18,7 @@
 package org.apache.carbondata.core.datastore.chunk.store.impl.safe;
 
 import org.apache.carbondata.core.datastore.chunk.store.DimensionDataChunkStore;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
 
 /**
  * Responsibility is to store dimension data
@@ -120,4 +121,7 @@ public abstract class SafeAbsractDimensionDataChunkStore implements DimensionDat
     throw new UnsupportedOperationException("Operation not supported");
   }
 
+  @Override public void fillRow(int rowId, CarbonColumnVector vector, int vectorRow) {
+    throw new UnsupportedOperationException("Operation not supported");
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8f59a326/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeVariableLengthDimensionDataChunkStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeVariableLengthDimensionDataChunkStore.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeVariableLengthDimensionDataChunkStore.java
index 28cd4c1..bb0edea 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeVariableLengthDimensionDataChunkStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeVariableLengthDimensionDataChunkStore.java
@@ -20,6 +20,7 @@ package org.apache.carbondata.core.datastore.chunk.store.impl.safe;
 import java.nio.ByteBuffer;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
 import org.apache.carbondata.core.util.ByteUtil;
 
 /**
@@ -73,16 +74,14 @@ public class SafeVariableLengthDimensionDataChunkStore extends SafeAbsractDimens
     // which we have to skip
     dataOffsets[0] = CarbonCommonConstants.SHORT_SIZE_IN_BYTE;
     // creating a byte buffer which will wrap the length of the row
-    ByteBuffer buffer = ByteBuffer.allocate(CarbonCommonConstants.SHORT_SIZE_IN_BYTE);
+    ByteBuffer buffer = ByteBuffer.wrap(data);
     for (int i = 1; i < numberOfRows; i++) {
-      buffer.put(data, startOffset, CarbonCommonConstants.SHORT_SIZE_IN_BYTE);
-      buffer.flip();
+      buffer.position(startOffset);
       // so current row position will be
       // previous row length + 2 bytes used for storing previous row data
       startOffset += buffer.getShort() + CarbonCommonConstants.SHORT_SIZE_IN_BYTE;
       // as same byte buffer is used to avoid creating many byte buffer for each row
       // we need to clear the byte buffer
-      buffer.clear();
       dataOffsets[i] = startOffset + CarbonCommonConstants.SHORT_SIZE_IN_BYTE;
     }
   }
@@ -113,6 +112,35 @@ public class SafeVariableLengthDimensionDataChunkStore extends SafeAbsractDimens
     return currentRowData;
   }
 
+  @Override public void fillRow(int rowId, CarbonColumnVector vector, int vectorRow) {
+    // if column was explicitly sorted we need to get the rowid based inverted index reverse
+    if (isExplictSorted) {
+      rowId = invertedIndexReverse[rowId];
+    }
+    // now to get the row from memory block we need to do following thing
+    // 1. first get the current offset
+    // 2. if it's not a last row- get the next row offset
+    // Subtract the current row offset + 2 bytes(to skip the data length) with next row offset
+    // else subtract the current row offset with complete data
+    // length get the offset of set of data
+    int currentDataOffset = dataOffsets[rowId];
+    short length = 0;
+    // calculating the length of data
+    if (rowId < numberOfRows - 1) {
+      length = (short) (dataOffsets[rowId + 1] - (currentDataOffset
+          + CarbonCommonConstants.SHORT_SIZE_IN_BYTE));
+    } else {
+      // for last record
+      length = (short) (this.data.length - currentDataOffset);
+    }
+    if (ByteUtil.UnsafeComparer.INSTANCE.equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY, 0,
+        CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY.length, data, currentDataOffset, length)) {
+      vector.putNull(vectorRow);
+    } else {
+      vector.putBytes(vectorRow, currentDataOffset, length, data);
+    }
+  }
+
   @Override public int compareTo(int index, byte[] compareValue) {
     // now to get the row from memory block we need to do following thing
     // 1. first get the current offset

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8f59a326/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 51ccc58..3752fb6 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
@@ -22,6 +22,7 @@ import org.apache.carbondata.core.datastore.chunk.store.DimensionDataChunkStore;
 import org.apache.carbondata.core.memory.CarbonUnsafe;
 import org.apache.carbondata.core.memory.MemoryAllocatorFactory;
 import org.apache.carbondata.core.memory.MemoryBlock;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
 
 /**
  * Responsibility is to store dimension data in memory. storage can be on heap
@@ -169,4 +170,7 @@ public abstract class UnsafeAbstractDimensionDataChunkStore implements Dimension
     throw new UnsupportedOperationException("Operation not supported");
   }
 
+  @Override public void fillRow(int rowId, CarbonColumnVector vector, int vectorRow) {
+    throw new UnsupportedOperationException("Operation not supported");
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8f59a326/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimesionDataChunkStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimesionDataChunkStore.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimesionDataChunkStore.java
index 82ec205..f5222fe 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimesionDataChunkStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimesionDataChunkStore.java
@@ -21,6 +21,8 @@ import java.nio.ByteBuffer;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.memory.CarbonUnsafe;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
+import org.apache.carbondata.core.util.ByteUtil;
 
 /**
  * Below class is responsible to store variable length dimension data chunk in
@@ -156,6 +158,16 @@ public class UnsafeVariableLengthDimesionDataChunkStore
     return data;
   }
 
+  @Override public void fillRow(int rowId, CarbonColumnVector vector, int vectorRow) {
+    byte[] value = getRow(rowId);
+    if (ByteUtil.UnsafeComparer.INSTANCE
+        .equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY, value)) {
+      vector.putNull(vectorRow);
+    } else {
+      vector.putBytes(vectorRow, value);
+    }
+  }
+
   /**
    * to compare the two byte array
    *

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8f59a326/core/src/main/java/org/apache/carbondata/core/datastore/impl/DFSFileHolderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/DFSFileHolderImpl.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/DFSFileHolderImpl.java
index dcd74c5..d14cff7 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/DFSFileHolderImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/DFSFileHolderImpl.java
@@ -131,11 +131,11 @@ public class DFSFileHolderImpl implements FileHolder {
     return fileChannel.readInt();
   }
 
-  @Override
-  public void readByteBuffer(String filePath, ByteBuffer byteBuffer,
-      long offset, int length) throws IOException {
+  @Override public ByteBuffer readByteBuffer(String filePath, long offset, int length)
+      throws IOException {
     byte[] readByteArray = readByteArray(filePath, offset, length);
-    byteBuffer.put(readByteArray);
+    ByteBuffer byteBuffer = ByteBuffer.wrap(readByteArray);
     byteBuffer.rewind();
+    return byteBuffer;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8f59a326/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileHolderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileHolderImpl.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileHolderImpl.java
index d78c28e..36b48f5 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileHolderImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileHolderImpl.java
@@ -192,13 +192,15 @@ public class FileHolderImpl implements FileHolder {
     ByteBuffer byteBffer = read(fileChannel, CarbonCommonConstants.LONG_SIZE_IN_BYTE, offset);
     return byteBffer.getLong();
   }
-  @Override
-  public void readByteBuffer(String filePath, ByteBuffer byteBuffer,
-      long offset, int length) throws IOException {
+
+  @Override public ByteBuffer readByteBuffer(String filePath, long offset, int length)
+      throws IOException {
+    ByteBuffer byteBuffer = ByteBuffer.allocate(length);
     FileChannel fileChannel = updateCache(filePath);
     fileChannel.position(offset);
     fileChannel.read(byteBuffer);
     byteBuffer.rewind();
+    return byteBuffer;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8f59a326/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedVectorResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedVectorResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedVectorResultCollector.java
index 082874d..af617be 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedVectorResultCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedVectorResultCollector.java
@@ -110,7 +110,6 @@ public class DictionaryBasedVectorResultCollector extends AbstractScannedResultC
     noDictionaryInfo = noDictInfoList.toArray(new ColumnVectorInfo[noDictInfoList.size()]);
     complexInfo = complexList.toArray(new ColumnVectorInfo[complexList.size()]);
     Arrays.sort(dictionaryInfo);
-    Arrays.sort(noDictionaryInfo);
     Arrays.sort(complexInfo);
   }
 

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

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

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8f59a326/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java
index 6f238c9..3c65198 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java
@@ -43,14 +43,22 @@ public class MeasureDataVectorProcessor {
       int vectorOffset = info.vectorOffset;
       CarbonColumnVector vector = info.vector;
       BitSet nullBitSet = dataChunk.getNullValueIndexHolder().getBitSet();
-      for (int i = offset; i < len; i++) {
-        if (nullBitSet.get(i)) {
-          vector.putNull(vectorOffset);
-        } else {
+      if (nullBitSet.isEmpty()) {
+        for (int i = offset; i < len; i++) {
           vector.putInt(vectorOffset,
               (int)dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(i));
+          vectorOffset++;
+        }
+      } else {
+        for (int i = offset; i < len; i++) {
+          if (nullBitSet.get(i)) {
+            vector.putNull(vectorOffset);
+          } else {
+            vector.putInt(vectorOffset,
+                (int)dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(i));
+          }
+          vectorOffset++;
         }
-        vectorOffset++;
       }
     }
 
@@ -62,15 +70,24 @@ public class MeasureDataVectorProcessor {
       int vectorOffset = info.vectorOffset;
       CarbonColumnVector vector = info.vector;
       BitSet nullBitSet = dataChunk.getNullValueIndexHolder().getBitSet();
-      for (int i = offset; i < len; i++) {
-        int currentRow = rowMapping[i];
-        if (nullBitSet.get(currentRow)) {
-          vector.putNull(vectorOffset);
-        } else {
+      if (nullBitSet.isEmpty()) {
+        for (int i = offset; i < len; i++) {
+          int currentRow = rowMapping[i];
           vector.putInt(vectorOffset,
               (int)dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(currentRow));
+          vectorOffset++;
+        }
+      } else {
+        for (int i = offset; i < len; i++) {
+          int currentRow = rowMapping[i];
+          if (nullBitSet.get(currentRow)) {
+            vector.putNull(vectorOffset);
+          } else {
+            vector.putInt(vectorOffset,
+                (int)dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(currentRow));
+          }
+          vectorOffset++;
         }
-        vectorOffset++;
       }
     }
   }
@@ -84,14 +101,22 @@ public class MeasureDataVectorProcessor {
       int vectorOffset = info.vectorOffset;
       CarbonColumnVector vector = info.vector;
       BitSet nullBitSet = dataChunk.getNullValueIndexHolder().getBitSet();
-      for (int i = offset; i < len; i++) {
-        if (nullBitSet.get(i)) {
-          vector.putNull(vectorOffset);
-        } else {
+      if (nullBitSet.isEmpty()) {
+        for (int i = offset; i < len; i++) {
           vector.putShort(vectorOffset,
               (short) dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(i));
+          vectorOffset++;
+        }
+      } else {
+        for (int i = offset; i < len; i++) {
+          if (nullBitSet.get(i)) {
+            vector.putNull(vectorOffset);
+          } else {
+            vector.putShort(vectorOffset,
+                (short) dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(i));
+          }
+          vectorOffset++;
         }
-        vectorOffset++;
       }
     }
 
@@ -103,15 +128,24 @@ public class MeasureDataVectorProcessor {
       int vectorOffset = info.vectorOffset;
       CarbonColumnVector vector = info.vector;
       BitSet nullBitSet = dataChunk.getNullValueIndexHolder().getBitSet();
-      for (int i = offset; i < len; i++) {
-        int currentRow = rowMapping[i];
-        if (nullBitSet.get(currentRow)) {
-          vector.putNull(vectorOffset);
-        } else {
+      if (nullBitSet.isEmpty()) {
+        for (int i = offset; i < len; i++) {
+          int currentRow = rowMapping[i];
           vector.putShort(vectorOffset,
               (short) dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(currentRow));
+          vectorOffset++;
+        }
+      } else {
+        for (int i = offset; i < len; i++) {
+          int currentRow = rowMapping[i];
+          if (nullBitSet.get(currentRow)) {
+            vector.putNull(vectorOffset);
+          } else {
+            vector.putShort(vectorOffset,
+                (short) dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(currentRow));
+          }
+          vectorOffset++;
         }
-        vectorOffset++;
       }
     }
   }
@@ -125,14 +159,22 @@ public class MeasureDataVectorProcessor {
       int vectorOffset = info.vectorOffset;
       CarbonColumnVector vector = info.vector;
       BitSet nullBitSet = dataChunk.getNullValueIndexHolder().getBitSet();
-      for (int i = offset; i < len; i++) {
-        if (nullBitSet.get(i)) {
-          vector.putNull(vectorOffset);
-        } else {
+      if (nullBitSet.isEmpty()) {
+        for (int i = offset; i < len; i++) {
           vector.putLong(vectorOffset,
               dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(i));
+          vectorOffset++;
+        }
+      } else {
+        for (int i = offset; i < len; i++) {
+          if (nullBitSet.get(i)) {
+            vector.putNull(vectorOffset);
+          } else {
+            vector.putLong(vectorOffset,
+                dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(i));
+          }
+          vectorOffset++;
         }
-        vectorOffset++;
       }
     }
 
@@ -144,15 +186,24 @@ public class MeasureDataVectorProcessor {
       int vectorOffset = info.vectorOffset;
       CarbonColumnVector vector = info.vector;
       BitSet nullBitSet = dataChunk.getNullValueIndexHolder().getBitSet();
-      for (int i = offset; i < len; i++) {
-        int currentRow = rowMapping[i];
-        if (nullBitSet.get(currentRow)) {
-          vector.putNull(vectorOffset);
-        } else {
+      if (nullBitSet.isEmpty()) {
+        for (int i = offset; i < len; i++) {
+          int currentRow = rowMapping[i];
           vector.putLong(vectorOffset,
               dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(currentRow));
+          vectorOffset++;
+        }
+      } else {
+        for (int i = offset; i < len; i++) {
+          int currentRow = rowMapping[i];
+          if (nullBitSet.get(currentRow)) {
+            vector.putNull(vectorOffset);
+          } else {
+            vector.putLong(vectorOffset,
+                dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(currentRow));
+          }
+          vectorOffset++;
         }
-        vectorOffset++;
       }
     }
   }
@@ -200,7 +251,7 @@ public class MeasureDataVectorProcessor {
         } else {
           BigDecimal decimal =
               dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(currentRow);
-          Decimal toDecimal = org.apache.spark.sql.types.Decimal.apply(decimal);
+          Decimal toDecimal = Decimal.apply(decimal);
           vector.putDecimal(vectorOffset, toDecimal, precision);
         }
         vectorOffset++;
@@ -217,14 +268,22 @@ public class MeasureDataVectorProcessor {
       int vectorOffset = info.vectorOffset;
       CarbonColumnVector vector = info.vector;
       BitSet nullBitSet = dataChunk.getNullValueIndexHolder().getBitSet();
-      for (int i = offset; i < len; i++) {
-        if (nullBitSet.get(i)) {
-          vector.putNull(vectorOffset);
-        } else {
+      if (nullBitSet.isEmpty()) {
+        for (int i = offset; i < len; i++) {
           vector.putDouble(vectorOffset,
               dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(i));
+          vectorOffset++;
+        }
+      } else {
+        for (int i = offset; i < len; i++) {
+          if (nullBitSet.get(i)) {
+            vector.putNull(vectorOffset);
+          } else {
+            vector.putDouble(vectorOffset,
+                dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(i));
+          }
+          vectorOffset++;
         }
-        vectorOffset++;
       }
     }
 
@@ -236,15 +295,24 @@ public class MeasureDataVectorProcessor {
       int vectorOffset = info.vectorOffset;
       CarbonColumnVector vector = info.vector;
       BitSet nullBitSet = dataChunk.getNullValueIndexHolder().getBitSet();
-      for (int i = offset; i < len; i++) {
-        int currentRow = rowMapping[i];
-        if (nullBitSet.get(currentRow)) {
-          vector.putNull(vectorOffset);
-        } else {
+      if (nullBitSet.isEmpty()) {
+        for (int i = offset; i < len; i++) {
+          int currentRow = rowMapping[i];
           vector.putDouble(vectorOffset,
               dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(currentRow));
+          vectorOffset++;
+        }
+      } else {
+        for (int i = offset; i < len; i++) {
+          int currentRow = rowMapping[i];
+          if (nullBitSet.get(currentRow)) {
+            vector.putNull(vectorOffset);
+          } else {
+            vector.putDouble(vectorOffset,
+                dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(currentRow));
+          }
+          vectorOffset++;
         }
-        vectorOffset++;
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8f59a326/core/src/main/java/org/apache/carbondata/core/util/ByteUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/ByteUtil.java b/core/src/main/java/org/apache/carbondata/core/util/ByteUtil.java
index 9b2c2ed..8f83f3d 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/ByteUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/ByteUtil.java
@@ -339,6 +339,35 @@ public final class ByteUtil {
       return true;
     }
 
+    public boolean equals(byte[] buffer1, int offset1, int length1, byte[] buffer2, int offset2,
+        int length2) {
+      if (length1 != length2) {
+        return false;
+      }
+      int len = length1 / 8;
+      long currentOffset = CarbonUnsafe.BYTE_ARRAY_OFFSET;
+      for (int i = 0; i < len; i++) {
+        long lw = CarbonUnsafe.unsafe.getLong(buffer1, currentOffset + offset1);
+        long rw = CarbonUnsafe.unsafe.getLong(buffer2, currentOffset + offset2);
+        if (lw != rw) {
+          return false;
+        }
+        currentOffset += 8;
+      }
+      len = buffer1.length % 8;
+      if (len > 0) {
+        for (int i = 0; i < len; i += 1) {
+          long lw = CarbonUnsafe.unsafe.getByte(buffer1, currentOffset + offset1);
+          long rw = CarbonUnsafe.unsafe.getByte(buffer2, currentOffset + offset2);
+          if (lw != rw) {
+            return false;
+          }
+          currentOffset += 1;
+        }
+      }
+      return true;
+    }
+
     /**
      * Comparing the 2 byte buffers. This is used in case of data load sorting step.
      *

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8f59a326/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 e10bd02..7d57fb8 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
@@ -1351,26 +1351,22 @@ public final class CarbonUtil {
 
   public static DataChunk3 readDataChunk3(ByteBuffer dataChunkBuffer, int offset, int length)
       throws IOException {
-    byte[] data = new byte[length];
-    dataChunkBuffer.position(offset);
-    dataChunkBuffer.get(data);
+    byte[] data = dataChunkBuffer.array();
     return (DataChunk3) read(data, new ThriftReader.TBaseCreator() {
       @Override public TBase create() {
         return new DataChunk3();
       }
-    }, 0, length);
+    }, offset, length);
   }
 
   public static DataChunk2 readDataChunk(ByteBuffer dataChunkBuffer, int offset, int length)
       throws IOException {
-    byte[] data = new byte[length];
-    dataChunkBuffer.position(offset);
-    dataChunkBuffer.get(data);
+    byte[] data = dataChunkBuffer.array();
     return (DataChunk2) read(data, new ThriftReader.TBaseCreator() {
       @Override public TBase create() {
         return new DataChunk2();
       }
-    }, 0, length);
+    }, offset, length);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8f59a326/examples/spark2/src/main/scala/org/apache/carbondata/examples/CompareTest.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CompareTest.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CompareTest.scala
index 82bd02a..41a7850 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CompareTest.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CompareTest.scala
@@ -266,6 +266,7 @@ object CompareTest {
           .option("tempCSV", "false")
           .option("single_pass", "true")
           .option("dictionary_exclude", "id") // id is high cardinality column
+          .option("table_blocksize", "32")
           .mode(SaveMode.Overwrite)
           .save()
     }
@@ -306,6 +307,8 @@ object CompareTest {
     // do GC and sleep for some time before running next table
     System.gc()
     Thread.sleep(1000)
+    System.gc()
+    Thread.sleep(1000)
     val carbonResult: Array[(Double, Int)] = runQueries(spark, carbonTableName("3"))
     // check result by comparing output from parquet and carbon
     parquetResult.zipWithIndex.foreach { case (result, index) =>
@@ -334,6 +337,7 @@ object CompareTest {
     CarbonProperties.getInstance()
         .addProperty("carbon.enable.vector.reader", "true")
         .addProperty("enable.unsafe.sort", "true")
+        .addProperty("carbon.blockletgroup.size.in.mb", "32")
     import org.apache.spark.sql.CarbonSession._
     val rootPath = new File(this.getClass.getResource("/").getPath
         + "../../../..").getCanonicalPath

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8f59a326/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonOption.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonOption.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonOption.scala
index c29c1a2..6ad10f3 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonOption.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonOption.scala
@@ -50,6 +50,8 @@ class CarbonOption(options: Map[String, String]) {
 
   def dictionaryExclude: Option[String] = options.get("dictionary_exclude")
 
+  def tableBlockSize: Option[String] = options.get("table_blocksize")
+
   def bucketNumber: Int = options.getOrElse("bucketnumber", "0").toInt
 
   def bucketColumns: String = options.getOrElse("bucketcolumns", "")

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8f59a326/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java b/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
index 9db5ace..ffff956 100644
--- a/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
+++ b/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
@@ -73,7 +73,7 @@ class VectorizedCarbonRecordReader extends RecordReader<Void, Object> {
   /**
    * The default config on whether columnarBatch should be offheap.
    */
-  private static final MemoryMode DEFAULT_MEMORY_MODE = MemoryMode.ON_HEAP;
+  private static final MemoryMode DEFAULT_MEMORY_MODE = MemoryMode.OFF_HEAP;
 
   private QueryModel queryModel;
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8f59a326/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
index 9b1533e..5917369 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
@@ -21,7 +21,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical.{UnaryNode, _}
 import org.apache.spark.sql.execution.datasources.LogicalRelation
-import org.apache.spark.sql.hive.{HiveContext, HiveSessionCatalog}
+import org.apache.spark.sql.hive.HiveSessionCatalog
 import org.apache.spark.sql.optimizer.CarbonDecoderRelation
 import org.apache.spark.sql.types.{StringType, TimestampType}
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8f59a326/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala
index e6efeaa..9ad9504 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala
@@ -162,7 +162,7 @@ class CarbonDataFrameWriter(sqlContext: SQLContext, val dataFrame: DataFrame) {
     val carbonSchema = schema.map { field =>
       s"${ field.name } ${ convertToCarbonType(field.dataType) }"
     }
-    val property = new StringBuilder
+    var property = new StringBuilder
     property.append(
       if (options.dictionaryInclude.isDefined) {
         s"'DICTIONARY_INCLUDE' = '${options.dictionaryInclude.get}' ,"
@@ -171,11 +171,20 @@ class CarbonDataFrameWriter(sqlContext: SQLContext, val dataFrame: DataFrame) {
       }
     ).append(
       if (options.dictionaryExclude.isDefined) {
-        s"'DICTIONARY_EXCLUDE' = '${options.dictionaryExclude.get}'"
+        s"'DICTIONARY_EXCLUDE' = '${options.dictionaryExclude.get}' ,"
+      } else {
+        ""
+      }
+    ).append(
+      if (options.tableBlockSize.isDefined) {
+        s"'table_blocksize' = '${options.tableBlockSize.get}'"
       } else {
         ""
       }
     )
+    if (property.nonEmpty && property.charAt(property.length-1) == ',') {
+      property = property.replace(property.length - 1, property.length, "")
+    }
 
     s"""
        | CREATE TABLE IF NOT EXISTS ${options.dbName}.${options.tableName}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8f59a326/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 d450b69..d47ff1a 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
@@ -57,28 +57,8 @@ case class CarbonDictionaryDecoder(
     child.outputPartitioning
   }
 
-  val getDictionaryColumnIds: Array[(String, ColumnIdentifier, CarbonDimension)] = {
-    child.output.map { attribute =>
-      val attr = aliasMap.getOrElse(attribute, attribute)
-      val relation = relations.find(p => p.contains(attr))
-      if (relation.isDefined && CarbonDictionaryDecoder.canBeDecoded(attr, profile)) {
-        val carbonTable = relation.get.carbonRelation.carbonRelation.metaData.carbonTable
-        val carbonDimension =
-          carbonTable.getDimensionByName(carbonTable.getFactTableName, attr.name)
-        if (carbonDimension != null &&
-            carbonDimension.hasEncoding(Encoding.DICTIONARY) &&
-            !carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY) &&
-            !carbonDimension.isComplex) {
-          (carbonTable.getFactTableName, carbonDimension.getColumnIdentifier,
-            carbonDimension)
-        } else {
-          (null, null, null)
-        }
-      } else {
-        (null, null, null)
-      }
-    }.toArray
-  }
+  val getDictionaryColumnIds: Array[(String, ColumnIdentifier, CarbonDimension)] =
+    CarbonDictionaryDecoder.getDictionaryColumnMapping(child.output, relations, profile, aliasMap)
 
   override def doExecute(): RDD[InternalRow] = {
     attachTree(this, "execute") {
@@ -88,7 +68,7 @@ case class CarbonDictionaryDecoder(
         (carbonTable.getFactTableName, carbonTable.getAbsoluteTableIdentifier)
       }.toMap
 
-      if (isRequiredToDecode) {
+      if (CarbonDictionaryDecoder.isRequiredToDecode(getDictionaryColumnIds)) {
         val dataTypes = child.output.map { attr => attr.dataType }
         child.execute().mapPartitions { iter =>
           val cacheProvider: CacheProvider = CacheProvider.getInstance
@@ -142,7 +122,7 @@ case class CarbonDictionaryDecoder(
       (carbonTable.getFactTableName, carbonTable.getAbsoluteTableIdentifier)
     }.toMap
 
-    if (isRequiredToDecode) {
+    if (CarbonDictionaryDecoder.isRequiredToDecode(getDictionaryColumnIds)) {
       val cacheProvider: CacheProvider = CacheProvider.getInstance
       val forwardDictionaryCache: Cache[DictionaryColumnUniqueIdentifier, Dictionary] =
         cacheProvider.createCache(CacheType.FORWARD_DICTIONARY, storePath)
@@ -255,13 +235,6 @@ case class CarbonDictionaryDecoder(
     child.asInstanceOf[CodegenSupport].produce(ctx, this)
   }
 
-  private def isRequiredToDecode = {
-    getDictionaryColumnIds.find(p => p._1 != null) match {
-      case Some(value) => true
-      case _ => false
-    }
-  }
-
   private def getDictionary(atiMap: Map[String, AbsoluteTableIdentifier],
       cache: Cache[DictionaryColumnUniqueIdentifier, Dictionary]) = {
     val dicts: Seq[Dictionary] = getDictionaryColumnIds.map { f =>
@@ -422,6 +395,39 @@ object CarbonDictionaryDecoder {
           .toDataType(s"array<${ relation.getArrayChildren(carbonDimension.getColName) }>")
     }
   }
+
+  def getDictionaryColumnMapping(output: Seq[Attribute],
+      relations: Seq[CarbonDecoderRelation],
+      profile: CarbonProfile,
+      aliasMap: CarbonAliasDecoderRelation): Array[(String, ColumnIdentifier, CarbonDimension)] = {
+    output.map { attribute =>
+      val attr = aliasMap.getOrElse(attribute, attribute)
+      val relation = relations.find(p => p.contains(attr))
+      if (relation.isDefined && CarbonDictionaryDecoder.canBeDecoded(attr, profile)) {
+        val carbonTable = relation.get.carbonRelation.carbonRelation.metaData.carbonTable
+        val carbonDimension =
+          carbonTable.getDimensionByName(carbonTable.getFactTableName, attr.name)
+        if (carbonDimension != null &&
+            carbonDimension.hasEncoding(Encoding.DICTIONARY) &&
+            !carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY) &&
+            !carbonDimension.isComplex) {
+          (carbonTable.getFactTableName, carbonDimension.getColumnIdentifier,
+            carbonDimension)
+        } else {
+          (null, null, null)
+        }
+      } else {
+        (null, null, null)
+      }
+    }.toArray
+  }
+
+  def isRequiredToDecode(colIdents: Array[(String, ColumnIdentifier, CarbonDimension)]): Boolean = {
+    colIdents.find(p => p._1 != null) match {
+      case Some(value) => true
+      case _ => false
+    }
+  }
 }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8f59a326/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CarbonLateDecodeStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CarbonLateDecodeStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CarbonLateDecodeStrategy.scala
index ed5d362..976759a 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CarbonLateDecodeStrategy.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CarbonLateDecodeStrategy.scala
@@ -61,7 +61,10 @@ private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
             a.map(_.name).toArray, f), needDecoder)) ::
             Nil
       case CarbonDictionaryCatalystDecoder(relations, profile, aliasMap, _, child) =>
-        if (profile.isInstanceOf[IncludeProfile] && profile.isEmpty) {
+        if ((profile.isInstanceOf[IncludeProfile] && profile.isEmpty) ||
+            !CarbonDictionaryDecoder.
+              isRequiredToDecode(CarbonDictionaryDecoder.
+                getDictionaryColumnMapping(child.output, relations, profile, aliasMap))) {
           planLater(child) :: Nil
         } else {
           CarbonDictionaryDecoder(relations,


[20/50] incubator-carbondata git commit: [CARBONDATA-903] data load is not failing even though bad records exists in the data in case of unsafe sort or batch sort. This closes #783

Posted by ra...@apache.org.
[CARBONDATA-903] data load is not failing even though bad records exists in the data in case of unsafe sort or batch sort. This closes #783


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

Branch: refs/heads/branch-1.1
Commit: f4fc651990d5f3865ef93dd6fb3ac49920cff0e5
Parents: 9efcacd 53accb3
Author: ravipesala <ra...@gmail.com>
Authored: Thu Apr 13 16:08:43 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Apr 13 16:08:43 2017 +0530

----------------------------------------------------------------------
 .../DataLoadFailAllTypeSortTest.scala           | 218 +++++++++++++++++++
 .../newflow/sort/AbstractMergeSorter.java       |  43 ++++
 .../sort/impl/ParallelReadMergeSorterImpl.java  |  18 +-
 ...arallelReadMergeSorterWithBucketingImpl.java |  16 +-
 .../UnsafeBatchParallelReadMergeSorterImpl.java |  43 +++-
 .../impl/UnsafeParallelReadMergeSorterImpl.java |  19 +-
 .../UnsafeSingleThreadFinalSortFilesMerger.java |  10 +
 7 files changed, 333 insertions(+), 34 deletions(-)
----------------------------------------------------------------------



[27/50] incubator-carbondata git commit: Problem: After drop table dictionary and BTree instances are not getting cleared from driver memory. Due to this memory will keep growing and after some time GC problems will occur. In real case scenarios usually

Posted by ra...@apache.org.
Problem: After drop table dictionary and BTree instances are not getting cleared from driver memory. Due to this memory will keep growing and after some time GC problems will occur. In real case scenarios usually driver memory is on lower side hence it is more prone to GC problems.

Solution:
1. When a table is being clear BTree and dictionary instances from LRU cache.
2. Clear the access count for each segment immediately after block pruning rather then loading all the segments first and at lats clearing access count for all the segments together.


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

Branch: refs/heads/branch-1.1
Commit: 673f8c2263c94e3475552de4da06327029f06eb3
Parents: 144620d
Author: manishgupta88 <to...@gmail.com>
Authored: Tue Apr 11 12:21:35 2017 +0530
Committer: manishgupta88 <to...@gmail.com>
Committed: Thu Apr 13 21:13:08 2017 +0530

----------------------------------------------------------------------
 .../core/cache/dictionary/ManageDictionary.java | 108 -------------
 .../dictionary/ManageDictionaryAndBTree.java    | 158 +++++++++++++++++++
 .../carbondata/hadoop/CacheAccessClient.java    |  23 ++-
 .../carbondata/hadoop/CarbonInputFormat.java    | 153 +++++++++---------
 .../spark/rdd/AlterTableDropColumnRDD.scala     |   4 +-
 .../execution/command/carbonTableSchema.scala   |   8 +-
 .../carbondata/CarbonDataSourceSuite.scala      |   2 +-
 7 files changed, 269 insertions(+), 187 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/673f8c22/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionary.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionary.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionary.java
deleted file mode 100644
index 0a38890..0000000
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionary.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.cache.dictionary;
-
-import java.io.IOException;
-
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.cache.Cache;
-import org.apache.carbondata.core.cache.CacheProvider;
-import org.apache.carbondata.core.cache.CacheType;
-import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
-import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
-import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.metadata.ColumnIdentifier;
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
-
-/**
- * This class is aimed at managing dictionary files for any new addition and deletion
- * and calling of clear cache for the non existing dictionary files
- */
-public class ManageDictionary {
-
-  /**
-   * Attribute for Carbon LOGGER
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(ManageDictionary.class.getName());
-
-  /**
-   * This method will delete the dictionary files for the given column IDs and
-   * clear the dictionary cache
-   *
-   * @param columnSchema
-   * @param carbonTableIdentifier
-   * @param storePath
-   */
-  public static void deleteDictionaryFileAndCache(final ColumnSchema columnSchema,
-      CarbonTableIdentifier carbonTableIdentifier, String storePath) {
-    CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier);
-    String metadataDirectoryPath = carbonTablePath.getMetadataDirectoryPath();
-    CarbonFile metadataDir = FileFactory
-        .getCarbonFile(metadataDirectoryPath, FileFactory.getFileType(metadataDirectoryPath));
-    // sort index file is created with dictionary size appended to it. So all the files
-    // with a given column ID need to be listed
-    CarbonFile[] listFiles = metadataDir.listFiles(new CarbonFileFilter() {
-      @Override public boolean accept(CarbonFile path) {
-        if (path.getName().startsWith(columnSchema.getColumnUniqueId())) {
-          return true;
-        }
-        return false;
-      }
-    });
-    for (CarbonFile file : listFiles) {
-      // try catch is inside for loop because even if one deletion fails, other files
-      // still need to be deleted
-      try {
-        FileFactory
-            .deleteFile(file.getCanonicalPath(), FileFactory.getFileType(file.getCanonicalPath()));
-      } catch (IOException e) {
-        LOGGER.error("Failed to delete dictionary or sortIndex file for column " + columnSchema
-            .getColumnName() + "with column ID " + columnSchema.getColumnUniqueId());
-      }
-    }
-    // remove dictionary cache
-    removeDictionaryColumnFromCache(carbonTableIdentifier, storePath,
-        columnSchema.getColumnUniqueId());
-  }
-
-  /**
-   * This method will remove dictionary cache from driver for both reverse and forward dictionary
-   *
-   * @param carbonTableIdentifier
-   * @param storePath
-   * @param columnId
-   */
-  public static void removeDictionaryColumnFromCache(CarbonTableIdentifier carbonTableIdentifier,
-      String storePath, String columnId) {
-    Cache<DictionaryColumnUniqueIdentifier, Dictionary> dictCache =
-        CacheProvider.getInstance().createCache(CacheType.REVERSE_DICTIONARY, storePath);
-    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        new DictionaryColumnUniqueIdentifier(carbonTableIdentifier,
-            new ColumnIdentifier(columnId, null, null));
-    dictCache.invalidate(dictionaryColumnUniqueIdentifier);
-    dictCache = CacheProvider.getInstance().createCache(CacheType.FORWARD_DICTIONARY, storePath);
-    dictCache.invalidate(dictionaryColumnUniqueIdentifier);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/673f8c22/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionaryAndBTree.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionaryAndBTree.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionaryAndBTree.java
new file mode 100644
index 0000000..a50bf15
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionaryAndBTree.java
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.core.cache.dictionary;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.cache.Cache;
+import org.apache.carbondata.core.cache.CacheProvider;
+import org.apache.carbondata.core.cache.CacheType;
+import org.apache.carbondata.core.datastore.TableSegmentUniqueIdentifier;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.ColumnIdentifier;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
+import org.apache.carbondata.core.util.path.CarbonStorePath;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+
+/**
+ * This class is aimed at managing dictionary files for any new addition and deletion
+ * and calling of clear cache for BTree and dictionary instances from LRU cache
+ */
+public class ManageDictionaryAndBTree {
+
+  /**
+   * Attribute for Carbon LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(ManageDictionaryAndBTree.class.getName());
+
+  /**
+   * This method will delete the dictionary files for the given column IDs and
+   * clear the dictionary cache
+   *
+   * @param columnSchema
+   * @param carbonTableIdentifier
+   * @param storePath
+   */
+  public static void deleteDictionaryFileAndCache(final ColumnSchema columnSchema,
+      CarbonTableIdentifier carbonTableIdentifier, String storePath) {
+    CarbonTablePath carbonTablePath =
+        CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier);
+    String metadataDirectoryPath = carbonTablePath.getMetadataDirectoryPath();
+    CarbonFile metadataDir = FileFactory
+        .getCarbonFile(metadataDirectoryPath, FileFactory.getFileType(metadataDirectoryPath));
+    // sort index file is created with dictionary size appended to it. So all the files
+    // with a given column ID need to be listed
+    CarbonFile[] listFiles = metadataDir.listFiles(new CarbonFileFilter() {
+      @Override public boolean accept(CarbonFile path) {
+        if (path.getName().startsWith(columnSchema.getColumnUniqueId())) {
+          return true;
+        }
+        return false;
+      }
+    });
+    for (CarbonFile file : listFiles) {
+      // try catch is inside for loop because even if one deletion fails, other files
+      // still need to be deleted
+      try {
+        FileFactory
+            .deleteFile(file.getCanonicalPath(), FileFactory.getFileType(file.getCanonicalPath()));
+      } catch (IOException e) {
+        LOGGER.error("Failed to delete dictionary or sortIndex file for column " + columnSchema
+            .getColumnName() + "with column ID " + columnSchema.getColumnUniqueId());
+      }
+    }
+    // remove dictionary cache
+    removeDictionaryColumnFromCache(carbonTableIdentifier, storePath,
+        columnSchema.getColumnUniqueId());
+  }
+
+  /**
+   * This method will remove dictionary cache from driver for both reverse and forward dictionary
+   *
+   * @param carbonTableIdentifier
+   * @param storePath
+   * @param columnId
+   */
+  public static void removeDictionaryColumnFromCache(CarbonTableIdentifier carbonTableIdentifier,
+      String storePath, String columnId) {
+    Cache<DictionaryColumnUniqueIdentifier, Dictionary> dictCache =
+        CacheProvider.getInstance().createCache(CacheType.REVERSE_DICTIONARY, storePath);
+    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
+        new DictionaryColumnUniqueIdentifier(carbonTableIdentifier,
+            new ColumnIdentifier(columnId, null, null));
+    dictCache.invalidate(dictionaryColumnUniqueIdentifier);
+    dictCache = CacheProvider.getInstance().createCache(CacheType.FORWARD_DICTIONARY, storePath);
+    dictCache.invalidate(dictionaryColumnUniqueIdentifier);
+  }
+
+  /**
+   * This mwthod will invalidate both BTree and dictionary instances from LRU cache
+   *
+   * @param carbonTable
+   */
+  public static void clearBTreeAndDictionaryLRUCache(CarbonTable carbonTable) {
+    // clear Btree cache from LRU cache
+    LoadMetadataDetails[] loadMetadataDetails =
+        SegmentStatusManager.readLoadMetadata(carbonTable.getMetaDataFilepath());
+    if (null != loadMetadataDetails) {
+      String[] segments = new String[loadMetadataDetails.length];
+      int i = 0;
+      for (LoadMetadataDetails loadMetadataDetail : loadMetadataDetails) {
+        segments[i++] = loadMetadataDetail.getLoadName();
+      }
+      invalidateBTreeCache(carbonTable.getAbsoluteTableIdentifier(), segments);
+    }
+    // clear dictionary cache from LRU cache
+    List<CarbonDimension> dimensions =
+        carbonTable.getDimensionByTableName(carbonTable.getFactTableName());
+    for (CarbonDimension dimension : dimensions) {
+      removeDictionaryColumnFromCache(carbonTable.getCarbonTableIdentifier(),
+          carbonTable.getStorePath(), dimension.getColumnId());
+    }
+  }
+
+  /**
+   * This method will remove the BTree instances from LRU cache
+   *
+   * @param absoluteTableIdentifier
+   * @param segments
+   */
+  public static void invalidateBTreeCache(AbsoluteTableIdentifier absoluteTableIdentifier,
+      String[] segments) {
+    Cache<Object, Object> driverBTreeCache = CacheProvider.getInstance()
+        .createCache(CacheType.DRIVER_BTREE, absoluteTableIdentifier.getStorePath());
+    for (String segmentNo : segments) {
+      TableSegmentUniqueIdentifier tableSegmentUniqueIdentifier =
+          new TableSegmentUniqueIdentifier(absoluteTableIdentifier, segmentNo);
+      driverBTreeCache.invalidate(tableSegmentUniqueIdentifier);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/673f8c22/hadoop/src/main/java/org/apache/carbondata/hadoop/CacheAccessClient.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/CacheAccessClient.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/CacheAccessClient.java
index 85ecfb8..e8d292d 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/CacheAccessClient.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/CacheAccessClient.java
@@ -18,7 +18,9 @@ package org.apache.carbondata.hadoop;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.carbondata.core.cache.Cache;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
@@ -30,7 +32,7 @@ public class CacheAccessClient<K, V> {
   /**
    * List of segments
    */
-  private List<K> segmentList = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+  private Set<K> segmentSet = new HashSet<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
 
   private Cache<K, V> cache;
 
@@ -48,7 +50,7 @@ public class CacheAccessClient<K, V> {
   public V getIfPresent(K key) {
     V value = cache.getIfPresent(key);
     if (value != null) {
-      segmentList.add(key);
+      segmentSet.add(key);
     }
     return value;
   }
@@ -64,7 +66,7 @@ public class CacheAccessClient<K, V> {
   public V get(K key) throws IOException {
     V value = cache.get(key);
     if (value != null) {
-      segmentList.add(key);
+      segmentSet.add(key);
     }
     return value;
   }
@@ -73,7 +75,9 @@ public class CacheAccessClient<K, V> {
    * the method is used to clear access count of the unused segments cacheable object
    */
   public void close() {
-    cache.clearAccessCount(segmentList);
+    List<K> segmentArrayList = new ArrayList<>(segmentSet.size());
+    segmentArrayList.addAll(segmentSet);
+    cache.clearAccessCount(segmentArrayList);
     cache = null;
   }
 
@@ -88,4 +92,15 @@ public class CacheAccessClient<K, V> {
     }
   }
 
+  /**
+   * This method will clear the access count for a given list of segments
+   *
+   * @param segmentList
+   */
+  public void clearAccessCount(List<K> segmentList) {
+    cache.clearAccessCount(segmentList);
+    // remove from segment set so that access count is not decremented again during close operation
+    segmentSet.removeAll(segmentList);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/673f8c22/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
index b330f12..b1a383e 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
@@ -223,56 +223,60 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
   @Override public List<InputSplit> getSplits(JobContext job) throws IOException {
     AbsoluteTableIdentifier identifier = getAbsoluteTableIdentifier(job.getConfiguration());
     CacheClient cacheClient = new CacheClient(identifier.getStorePath());
-    List<String> invalidSegments = new ArrayList<>();
-    List<UpdateVO> invalidTimestampsList = new ArrayList<>();
-
-    // get all valid segments and set them into the configuration
-    if (getSegmentsToAccess(job).length == 0) {
-      SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier);
-      SegmentStatusManager.ValidAndInvalidSegmentsInfo segments =
-              segmentStatusManager.getValidAndInvalidSegments();
-      SegmentUpdateStatusManager updateStatusManager = new SegmentUpdateStatusManager(identifier);
-      setSegmentsToAccess(job.getConfiguration(), segments.getValidSegments());
-      if (segments.getValidSegments().size() == 0) {
-        return new ArrayList<>(0);
+    try {
+      List<String> invalidSegments = new ArrayList<>();
+      List<UpdateVO> invalidTimestampsList = new ArrayList<>();
+
+      // get all valid segments and set them into the configuration
+      if (getSegmentsToAccess(job).length == 0) {
+        SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier);
+        SegmentStatusManager.ValidAndInvalidSegmentsInfo segments =
+            segmentStatusManager.getValidAndInvalidSegments();
+        SegmentUpdateStatusManager updateStatusManager = new SegmentUpdateStatusManager(identifier);
+        setSegmentsToAccess(job.getConfiguration(), segments.getValidSegments());
+        if (segments.getValidSegments().size() == 0) {
+          return new ArrayList<>(0);
+        }
+
+        // remove entry in the segment index if there are invalid segments
+        invalidSegments.addAll(segments.getInvalidSegments());
+        for (String invalidSegmentId : invalidSegments) {
+          invalidTimestampsList.add(updateStatusManager.getInvalidTimestampRange(invalidSegmentId));
+        }
+        if (invalidSegments.size() > 0) {
+          List<TableSegmentUniqueIdentifier> invalidSegmentsIds =
+              new ArrayList<>(invalidSegments.size());
+          for (String segId : invalidSegments) {
+            invalidSegmentsIds.add(new TableSegmentUniqueIdentifier(identifier, segId));
+          }
+          cacheClient.getSegmentAccessClient().invalidateAll(invalidSegmentsIds);
+        }
       }
 
-      // remove entry in the segment index if there are invalid segments
-      invalidSegments.addAll(segments.getInvalidSegments());
-      for (String invalidSegmentId : invalidSegments) {
-        invalidTimestampsList.add(updateStatusManager.getInvalidTimestampRange(invalidSegmentId));
+      // process and resolve the expression
+      Expression filter = getFilterPredicates(job.getConfiguration());
+      CarbonTable carbonTable = getCarbonTable(job.getConfiguration());
+      // this will be null in case of corrupt schema file.
+      if (null == carbonTable) {
+        throw new IOException("Missing/Corrupt schema file for table.");
       }
+      CarbonInputFormatUtil.processFilterExpression(filter, carbonTable);
+      FilterResolverIntf filterInterface = CarbonInputFormatUtil.resolveFilter(filter, identifier);
+
+      // do block filtering and get split
+      List<InputSplit> splits = getSplits(job, filterInterface, cacheClient);
+      // pass the invalid segment to task side in order to remove index entry in task side
       if (invalidSegments.size() > 0) {
-        List<TableSegmentUniqueIdentifier> invalidSegmentsIds
-            = new ArrayList<>(invalidSegments.size());
-        for (String segId : invalidSegments) {
-          invalidSegmentsIds.add(new TableSegmentUniqueIdentifier(identifier, segId));
+        for (InputSplit split : splits) {
+          ((CarbonInputSplit) split).setInvalidSegments(invalidSegments);
+          ((CarbonInputSplit) split).setInvalidTimestampRange(invalidTimestampsList);
         }
-        cacheClient.getSegmentAccessClient().invalidateAll(invalidSegmentsIds);
-      }
-    }
-
-    // process and resolve the expression
-    Expression filter = getFilterPredicates(job.getConfiguration());
-    CarbonTable carbonTable = getCarbonTable(job.getConfiguration());
-    // this will be null in case of corrupt schema file.
-    if (null == carbonTable) {
-      throw new IOException("Missing/Corrupt schema file for table.");
-    }
-    CarbonInputFormatUtil.processFilterExpression(filter, carbonTable);
-    FilterResolverIntf filterInterface = CarbonInputFormatUtil.resolveFilter(filter, identifier);
-
-    // do block filtering and get split
-    List<InputSplit> splits = getSplits(job, filterInterface, cacheClient);
-    cacheClient.close();
-    // pass the invalid segment to task side in order to remove index entry in task side
-    if (invalidSegments.size() > 0) {
-      for (InputSplit split : splits) {
-        ((CarbonInputSplit) split).setInvalidSegments(invalidSegments);
-        ((CarbonInputSplit) split).setInvalidTimestampRange(invalidTimestampsList);
       }
+      return splits;
+    } finally {
+      // close the cache cache client to clear LRU cache memory
+      cacheClient.close();
     }
-    return splits;
   }
 
   private List<InputSplit> getSplitsInternal(JobContext job) throws IOException {
@@ -354,37 +358,44 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
       AbsoluteTableIdentifier absoluteTableIdentifier, FilterResolverIntf resolver,
       String segmentId, CacheClient cacheClient, SegmentUpdateStatusManager updateStatusManager)
       throws IOException {
-    QueryStatisticsRecorder recorder = CarbonTimeStatisticsFactory.createDriverRecorder();
-    QueryStatistic statistic = new QueryStatistic();
-    Map<SegmentTaskIndexStore.TaskBucketHolder, AbstractIndex> segmentIndexMap =
-        getSegmentAbstractIndexs(job, absoluteTableIdentifier, segmentId, cacheClient,
-            updateStatusManager);
-
-    List<DataRefNode> resultFilterredBlocks = new LinkedList<DataRefNode>();
-
-    if (null != segmentIndexMap) {
-      // build result
-      for (AbstractIndex abstractIndex : segmentIndexMap.values()) {
-        List<DataRefNode> filterredBlocks;
-        // if no filter is given get all blocks from Btree Index
-        if (null == resolver) {
-          filterredBlocks = getDataBlocksOfIndex(abstractIndex);
-        } else {
-          // apply filter and get matching blocks
-          filterredBlocks = filterExpressionProcessor.getFilterredBlocks(
-                  abstractIndex.getDataRefNode(),
-                  resolver,
-                  abstractIndex,
-                  absoluteTableIdentifier
-          );
+    Map<SegmentTaskIndexStore.TaskBucketHolder, AbstractIndex> segmentIndexMap = null;
+    try {
+      QueryStatisticsRecorder recorder = CarbonTimeStatisticsFactory.createDriverRecorder();
+      QueryStatistic statistic = new QueryStatistic();
+      segmentIndexMap =
+          getSegmentAbstractIndexs(job, absoluteTableIdentifier, segmentId, cacheClient,
+              updateStatusManager);
+      List<DataRefNode> resultFilterredBlocks = new LinkedList<DataRefNode>();
+      if (null != segmentIndexMap) {
+        // build result
+        for (AbstractIndex abstractIndex : segmentIndexMap.values()) {
+          List<DataRefNode> filterredBlocks;
+          // if no filter is given get all blocks from Btree Index
+          if (null == resolver) {
+            filterredBlocks = getDataBlocksOfIndex(abstractIndex);
+          } else {
+            // apply filter and get matching blocks
+            filterredBlocks = filterExpressionProcessor
+                .getFilterredBlocks(abstractIndex.getDataRefNode(), resolver, abstractIndex,
+                    absoluteTableIdentifier);
+          }
+          resultFilterredBlocks.addAll(filterredBlocks);
         }
-        resultFilterredBlocks.addAll(filterredBlocks);
+      }
+      statistic
+          .addStatistics(QueryStatisticsConstants.LOAD_BLOCKS_DRIVER, System.currentTimeMillis());
+      recorder.recordStatisticsForDriver(statistic, job.getConfiguration().get("query.id"));
+      return resultFilterredBlocks;
+    } finally {
+      // clean up the access count for a segment as soon as its usage is complete so that in
+      // low memory systems the same memory can be utilized efficiently
+      if (null != segmentIndexMap) {
+        List<TableSegmentUniqueIdentifier> tableSegmentUniqueIdentifiers = new ArrayList<>(1);
+        tableSegmentUniqueIdentifiers
+            .add(new TableSegmentUniqueIdentifier(absoluteTableIdentifier, segmentId));
+        cacheClient.getSegmentAccessClient().clearAccessCount(tableSegmentUniqueIdentifiers);
       }
     }
-    statistic
-        .addStatistics(QueryStatisticsConstants.LOAD_BLOCKS_DRIVER, System.currentTimeMillis());
-    recorder.recordStatisticsForDriver(statistic, job.getConfiguration().get("query.id"));
-    return resultFilterredBlocks;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/673f8c22/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableDropColumnRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableDropColumnRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableDropColumnRDD.scala
index 49dadd3..53796bb 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableDropColumnRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableDropColumnRDD.scala
@@ -21,7 +21,7 @@ import org.apache.spark.{Partition, SparkContext, TaskContext}
 import org.apache.spark.rdd.RDD
 
 import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.cache.dictionary.ManageDictionary
+import org.apache.carbondata.core.cache.dictionary.ManageDictionaryAndBTree
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier
 import org.apache.carbondata.core.metadata.encoder.Encoding
@@ -65,7 +65,7 @@ class AlterTableDropColumnRDD[K, V](sc: SparkContext,
         val columnSchema = split.asInstanceOf[DropColumnPartition].columnSchema
         if (columnSchema.hasEncoding(Encoding.DICTIONARY) &&
             !columnSchema.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-          ManageDictionary
+          ManageDictionaryAndBTree
             .deleteDictionaryFileAndCache(columnSchema, carbonTableIdentifier, carbonStorePath)
         }
       } catch {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/673f8c22/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
index 77a0d90..1451247 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
@@ -37,7 +37,7 @@ import org.codehaus.jackson.map.ObjectMapper
 
 import org.apache.carbondata.api.CarbonStore
 import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.cache.dictionary.ManageDictionary
+import org.apache.carbondata.core.cache.dictionary.ManageDictionaryAndBTree
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.dictionary.server.DictionaryServer
@@ -723,6 +723,12 @@ case class CarbonDropTableCommand(ifExistsSet: Boolean,
         sys.error("Table is locked for deletion. Please try after some time")
       }
       LOGGER.audit(s"Deleting table [$tableName] under database [$dbName]")
+      val carbonTable = CarbonEnv.get.carbonMetastore.getTableFromMetadata(dbName, tableName)
+        .map(_.carbonTable).getOrElse(null)
+      if (null != carbonTable) {
+        // clear driver B-tree and dictionary cache
+        ManageDictionaryAndBTree.clearBTreeAndDictionaryLRUCache(carbonTable)
+      }
       CarbonEnv.get.carbonMetastore.dropTable(storePath, identifier)(sparkSession)
       LOGGER.audit(s"Deleted table [$tableName] under database [$dbName]")
     } finally {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/673f8c22/integration/spark2/src/test/scala/org/apache/spark/carbondata/CarbonDataSourceSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/CarbonDataSourceSuite.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/CarbonDataSourceSuite.scala
index 30634d1..ac7dce3 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/CarbonDataSourceSuite.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/CarbonDataSourceSuite.scala
@@ -171,7 +171,7 @@ class CarbonDataSourceSuite extends QueryTest with BeforeAndAfterAll {
       case e =>
         println(e.getMessage)
     }
-    checkAnswer(sql("select * from testdb.test1"), Seq(Row("xx", 1), Row("xx", 11)))
+    checkAnswer(sql("select count(*) from testdb.test1"), Seq(Row(2)))
     sql("drop table testdb.test1")
     sql("drop database testdb")
   }


[47/50] incubator-carbondata git commit: changed max columns from static value to configurable

Posted by ra...@apache.org.
changed max columns from static value to configurable


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

Branch: refs/heads/branch-1.1
Commit: 0a09472a4e710caa3960188b7ab0a405abdc9abc
Parents: a8ed450
Author: kunal642 <ku...@knoldus.in>
Authored: Sat Apr 15 13:48:17 2017 +0530
Committer: Venkata Ramana G <ra...@huawei.com>
Committed: Mon Apr 17 13:32:06 2017 +0530

----------------------------------------------------------------------
 .../hadoop/test/util/StoreCreator.java          |  3 ++
 .../TestDataLoadWithColumnsMoreThanSchema.scala | 36 ++++++-------
 .../dataload/TestLoadDataWithHiveSyntax.scala   |  2 +-
 .../carbondata/spark/rdd/CarbonMergerRDD.scala  |  3 ++
 .../carbondata/spark/util/CommonUtil.scala      | 57 +++++++++++++++++++-
 .../execution/command/carbonTableSchema.scala   |  4 +-
 .../dataload/SparkDatasourceSuite.scala         |  1 -
 .../util/ExternalColumnDictionaryTestCase.scala |  1 +
 ...GlobalDictionaryUtilConcurrentTestCase.scala | 12 +++--
 .../util/GlobalDictionaryUtilTestCase.scala     |  1 +
 .../execution/command/carbonTableSchema.scala   |  5 +-
 .../processing/csvload/CSVInputFormat.java      | 25 ++++++++-
 .../processing/model/CarbonLoadModel.java       |  1 +
 .../newflow/CarbonDataLoadConfiguration.java    | 12 +++++
 .../carbondata/processing/StoreCreator.java     |  3 ++
 .../processing/csvload/CSVInputFormatTest.java  |  2 +
 16 files changed, 136 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/0a09472a/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
----------------------------------------------------------------------
diff --git a/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java b/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
index 51ce2c5..2997e94 100644
--- a/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
+++ b/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
@@ -166,6 +166,7 @@ public class StoreCreator {
       loadModel.setSegmentId("0");
       loadModel.setPartitionId("0");
       loadModel.setFactTimeStamp(System.currentTimeMillis());
+      loadModel.setMaxColumns("10");
 
       executeGraph(loadModel, absoluteTableIdentifier.getStorePath());
 
@@ -399,6 +400,8 @@ public class StoreCreator {
     CSVInputFormat.setReadBufferSize(configuration, CarbonProperties.getInstance()
         .getProperty(CarbonCommonConstants.CSV_READ_BUFFER_SIZE,
             CarbonCommonConstants.CSV_READ_BUFFER_SIZE_DEFAULT));
+    CSVInputFormat.setNumberOfColumns(configuration, String.valueOf(loadModel.getCsvHeaderColumns().length));
+    CSVInputFormat.setMaxColumns(configuration, "10");
 
     TaskAttemptContextImpl hadoopAttemptContext = new TaskAttemptContextImpl(configuration, new TaskAttemptID("", 1, TaskType.MAP, 0, 0));
     CSVInputFormat format = new CSVInputFormat();

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/0a09472a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithColumnsMoreThanSchema.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithColumnsMoreThanSchema.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithColumnsMoreThanSchema.scala
index 9711051..c25e520 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithColumnsMoreThanSchema.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithColumnsMoreThanSchema.scala
@@ -84,46 +84,42 @@ class TestDataLoadWithColumnsMoreThanSchema extends QueryTest with BeforeAndAfte
   }
 
   test("test for maxcolumns option value greater than threshold value for maxcolumns") {
-    sql("DROP TABLE IF EXISTS valid_max_columns_test")
-    sql("CREATE TABLE valid_max_columns_test (imei string,age int,task bigint,num double,level decimal(10,3),productdate timestamp,mark int,name string)STORED BY 'org.apache.carbondata.format'")
-    try {
+    intercept[Exception] {
+      sql("DROP TABLE IF EXISTS valid_max_columns_test")
+      sql(
+        "CREATE TABLE valid_max_columns_test (imei string,age int,task bigint,num double,level decimal(10,3),productdate timestamp,mark int,name string)STORED BY 'org.apache.carbondata.format'")
       sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/character_carbon.csv' into table valid_max_columns_test options('MAXCOLUMNS'='22000')")
-      checkAnswer(sql("select count(*) from valid_max_columns_test"),
-        sql("select count(*) from hive_char_test"))
-    } catch {
-      case _: Throwable => assert(false)
     }
   }
 
   test("test for boundary value for maxcolumns") {
-    sql("DROP TABLE IF EXISTS boundary_max_columns_test")
-    sql("CREATE TABLE boundary_max_columns_test (empno string, empname String, designation String, doj String, " +
+    intercept[Exception] {
+      sql("DROP TABLE IF EXISTS boundary_max_columns_test")
+      sql(
+        "CREATE TABLE boundary_max_columns_test (empno string, empname String, designation " +
+        "String, doj String, " +
         "workgroupcategory string, workgroupcategoryname String, deptno string, deptname String, " +
         "projectcode string, projectjoindate String, projectenddate String,attendance double," +
         "utilization double,salary double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES" +
         "('DICTIONARY_EXCLUDE'='empno,empname,designation,doj,workgroupcategory," +
         "workgroupcategoryname,deptno,deptname,projectcode,projectjoindate,projectenddate')")
-    try {
-      sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/data.csv' into table boundary_max_columns_test options('MAXCOLUMNS'='14')")
-      assert(true)
-    } catch {
-      case _: Throwable => assert(false)
+      sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/data.csv' into table boundary_max_columns_test" +
+          s" options('MAXCOLUMNS'='14')")
+
     }
   }
 
   test("test for maxcolumns value less than columns in 1st line of csv file") {
-    sql("DROP TABLE IF EXISTS boundary_max_columns_test")
-    sql("CREATE TABLE boundary_max_columns_test (empno string, empname String, designation String, doj String, " +
+    intercept[Exception] {
+      sql("DROP TABLE IF EXISTS boundary_max_columns_test")
+      sql(
+        "CREATE TABLE boundary_max_columns_test (empno string, empname String, designation String, doj String, " +
         "workgroupcategory string, workgroupcategoryname String, deptno string, deptname String, " +
         "projectcode string, projectjoindate String, projectenddate String,attendance double," +
         "utilization double,salary double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES" +
         "('DICTIONARY_EXCLUDE'='empno,empname,designation,doj,workgroupcategory," +
         "workgroupcategoryname,deptno,deptname,projectcode,projectjoindate,projectenddate')")
-    try {
       sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/data.csv' into table boundary_max_columns_test options('MAXCOLUMNS'='13')")
-      assert(true)
-    } catch {
-      case _: Throwable => assert(false)
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/0a09472a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala
index 353db9e..561b0d1 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala
@@ -651,7 +651,7 @@ class TestLoadDataWithHiveSyntax extends QueryTest with BeforeAndAfterAll {
     )
     sql(
       s"LOAD DATA local inpath '$resourcesPath/comment.csv' INTO TABLE comment_test " +
-        "options('DELIMITER' = ',', 'QUOTECHAR' = '.', 'COMMENTCHAR' = '?','FILEHEADER'='imei,age,task,num,level,productdate,mark,name')"
+        "options('DELIMITER' = ',', 'QUOTECHAR' = '.', 'COMMENTCHAR' = '?','FILEHEADER'='imei,age,task,num,level,productdate,mark,name', 'maxcolumns'='180')"
     )
     checkAnswer(sql("select imei from comment_test"),Seq(Row("\".carbon"),Row("#?carbon"), Row(""),
       Row("~carbon,")))

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/0a09472a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
index 350a2ec..49984c9 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
@@ -64,6 +64,7 @@ class CarbonMergerRDD[K, V](
   sc.setLocalProperty("spark.scheduler.pool", "DDL")
   sc.setLocalProperty("spark.job.interruptOnCancel", "true")
 
+  private val queryId = sparkContext.getConf.get("queryId", System.nanoTime() + "")
   var storeLocation: String = null
   var mergeResult: String = null
   val hdfsStoreLocation = carbonMergerMapping.hdfsStoreLocation
@@ -260,6 +261,8 @@ class CarbonMergerRDD[K, V](
     val jobConf: JobConf = new JobConf(new Configuration)
     val job: Job = new Job(jobConf)
     val format = CarbonInputFormatUtil.createCarbonInputFormat(absoluteTableIdentifier, job)
+    // initialise query_id for job
+    job.getConfiguration.set("query.id", queryId)
     var defaultParallelism = sparkContext.defaultParallelism
     val result = new java.util.ArrayList[Partition](defaultParallelism)
     var partitionNo = 0

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/0a09472a/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 7592e4e..679a4e7 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
@@ -273,6 +273,9 @@ object CommonUtil {
     CSVInputFormat.setCommentCharacter(configuration, carbonLoadModel.getCommentChar)
     CSVInputFormat.setCSVDelimiter(configuration, carbonLoadModel.getCsvDelimiter)
     CSVInputFormat.setEscapeCharacter(configuration, carbonLoadModel.getEscapeChar)
+    CSVInputFormat.setMaxColumns(configuration, carbonLoadModel.getMaxColumns)
+    CSVInputFormat.setNumberOfColumns(configuration, carbonLoadModel.getCsvHeaderColumns.length
+      .toString)
     CSVInputFormat.setHeaderExtractionEnabled(configuration,
       carbonLoadModel.getCsvHeader == null || carbonLoadModel.getCsvHeader.isEmpty)
     CSVInputFormat.setQuoteCharacter(configuration, carbonLoadModel.getQuoteChar)
@@ -342,7 +345,59 @@ object CommonUtil {
           + "the same. Input file : " + csvFile)
       }
     }
-
     csvColumns
   }
+
+  def validateMaxColumns(csvHeaders: Array[String], maxColumns: String): Int = {
+    /*
+    User configures both csvheadercolumns, maxcolumns,
+      if csvheadercolumns >= maxcolumns, give error
+      if maxcolumns > threashold, give error
+    User configures csvheadercolumns
+      if csvheadercolumns >= maxcolumns(default) then maxcolumns = csvheadercolumns+1
+      if csvheadercolumns >= threashold, give error
+    User configures nothing
+      if csvheadercolumns >= maxcolumns(default) then maxcolumns = csvheadercolumns+1
+      if csvheadercolumns >= threashold, give error
+     */
+    val columnCountInSchema = csvHeaders.length
+    var maxNumberOfColumnsForParsing = 0
+    val maxColumnsInt = getMaxColumnValue(maxColumns)
+    if (maxColumnsInt != null) {
+      if (columnCountInSchema >= maxColumnsInt) {
+        sys.error(s"csv headers should be less than the max columns: $maxColumnsInt")
+      } else if (maxColumnsInt > CSVInputFormat.THRESHOLD_MAX_NUMBER_OF_COLUMNS_FOR_PARSING) {
+        sys.error(s"max columns cannot be greater than the threshold value: ${
+          CSVInputFormat.THRESHOLD_MAX_NUMBER_OF_COLUMNS_FOR_PARSING
+        }")
+      } else {
+        maxNumberOfColumnsForParsing = maxColumnsInt
+      }
+    } else if (columnCountInSchema >= CSVInputFormat.THRESHOLD_MAX_NUMBER_OF_COLUMNS_FOR_PARSING) {
+      sys.error(s"csv header columns should be less than max threashold: ${
+        CSVInputFormat
+          .THRESHOLD_MAX_NUMBER_OF_COLUMNS_FOR_PARSING
+      }")
+    } else if (columnCountInSchema >= CSVInputFormat.DEFAULT_MAX_NUMBER_OF_COLUMNS_FOR_PARSING) {
+      maxNumberOfColumnsForParsing = columnCountInSchema + 1
+    } else {
+      maxNumberOfColumnsForParsing = CSVInputFormat.DEFAULT_MAX_NUMBER_OF_COLUMNS_FOR_PARSING
+    }
+    maxNumberOfColumnsForParsing
+  }
+
+  private def getMaxColumnValue(maxColumn: String): Integer = {
+    if (maxColumn != null) {
+      try {
+        maxColumn.toInt
+      } catch {
+        case e: Exception =>
+          LOGGER.error(s"Invalid value for max column in load options ${ e.getMessage }")
+          null
+      }
+    } else {
+      null
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/0a09472a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
index 5a22e9c..15472e5 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
@@ -404,7 +404,6 @@ case class LoadTable(
       validateDateFormat(dateFormat, table)
       val maxColumns = options.getOrElse("maxcolumns", null)
 
-      carbonLoadModel.setMaxColumns(checkDefaultValue(maxColumns, null))
       carbonLoadModel.setEscapeChar(checkDefaultValue(escapeChar, "\\"))
       carbonLoadModel.setQuoteChar(checkDefaultValue(quoteChar, "\""))
       carbonLoadModel.setCommentChar(checkDefaultValue(commentchar, "#"))
@@ -474,6 +473,9 @@ case class LoadTable(
         carbonLoadModel.setColDictFilePath(columnDict)
         carbonLoadModel.setDirectLoad(true)
         carbonLoadModel.setCsvHeaderColumns(CommonUtil.getCsvHeaderColumns(carbonLoadModel))
+        val validatedMaxColumns = CommonUtil.validateMaxColumns(carbonLoadModel.getCsvHeaderColumns,
+          maxColumns)
+        carbonLoadModel.setMaxColumns(validatedMaxColumns.toString)
         GlobalDictionaryUtil.updateTableMetadataFunc = updateTableMetadata
 
         if (carbonLoadModel.getUseOnePass) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/0a09472a/integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/SparkDatasourceSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/SparkDatasourceSuite.scala b/integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/SparkDatasourceSuite.scala
index 4c5b241..0b64759 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/SparkDatasourceSuite.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/SparkDatasourceSuite.scala
@@ -17,7 +17,6 @@
 
 package org.apache.carbondata.integration.spark.testsuite.dataload
 
-import java.io.File
 
 import org.apache.spark.sql.common.util.QueryTest
 import org.apache.spark.sql.{DataFrame, Row, SQLContext, SaveMode}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/0a09472a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
index 5a986b7..05b94ee 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
@@ -148,6 +148,7 @@ class ExternalColumnDictionaryTestCase extends QueryTest with BeforeAndAfterAll
       CarbonCommonConstants.CARBON_DATE_FORMAT,
       CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT))  
     carbonLoadModel.setCsvHeaderColumns(CommonUtil.getCsvHeaderColumns(carbonLoadModel))
+    carbonLoadModel.setMaxColumns("100")
     carbonLoadModel
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/0a09472a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilConcurrentTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilConcurrentTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilConcurrentTestCase.scala
index 377bbaa..9e0f851 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilConcurrentTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilConcurrentTestCase.scala
@@ -16,7 +16,6 @@
  */
 package org.apache.carbondata.spark.util
 
-import java.io.File
 import java.util.concurrent.{Callable, Executors}
 
 import scala.collection.mutable.ListBuffer
@@ -64,6 +63,7 @@ class GlobalDictionaryUtilConcurrentTestCase extends QueryTest with BeforeAndAft
       CarbonCommonConstants.CARBON_DATE_FORMAT,
       CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT))  
     carbonLoadModel.setCsvHeaderColumns(CommonUtil.getCsvHeaderColumns(carbonLoadModel))
+    carbonLoadModel.setMaxColumns("2000")
     carbonLoadModel
   }
 
@@ -88,6 +88,7 @@ class GlobalDictionaryUtilConcurrentTestCase extends QueryTest with BeforeAndAft
       "employee")(sqlContext)
       .asInstanceOf[CarbonRelation]
   }
+
   def writedummydata(filePath: String, recCount: Int) = {
     var a: Int = 0
     var records: StringBuilder = StringBuilder.newBuilder
@@ -98,6 +99,7 @@ class GlobalDictionaryUtilConcurrentTestCase extends QueryTest with BeforeAndAft
     dis.writeBytes(records.toString())
     dis.close()
   }
+
   test("concurrent dictionary generation") {
     CarbonProperties.getInstance.addProperty(CarbonCommonConstants.MAX_QUERY_EXECUTION_TIME, "-1")
     val noOfFiles = 5
@@ -114,8 +116,8 @@ class GlobalDictionaryUtilConcurrentTestCase extends QueryTest with BeforeAndAft
       for (i <- 0 until noOfFiles) {
         dictGenerators.add(new DictGenerator(loadModels(i)))
       }
-      val executorService = Executors.newFixedThreadPool(10);
-      val results = executorService.invokeAll(dictGenerators);
+      val executorService = Executors.newFixedThreadPool(10)
+      val results = executorService.invokeAll(dictGenerators)
       for (i <- 0 until noOfFiles) {
         val res = results.get(i).get
         assert("Pass".equals(res))
@@ -128,7 +130,7 @@ class GlobalDictionaryUtilConcurrentTestCase extends QueryTest with BeforeAndAft
     val carbonTableIdentifier = sampleRelation.tableMeta.carbonTable.getCarbonTableIdentifier
     val columnIdentifier = sampleRelation.tableMeta.carbonTable.getDimensionByName("employee", "empid").getColumnIdentifier
     val carbonTablePath = PathFactory.getInstance()
-        .getCarbonTablePath(sampleRelation.tableMeta.storePath, carbonTableIdentifier);
+        .getCarbonTablePath(sampleRelation.tableMeta.storePath, carbonTableIdentifier)
     val dictPath = carbonTablePath.getDictionaryFilePath(columnIdentifier.getColumnId)
     val dictFile = FileFactory.getCarbonFile(dictPath, FileFactory.getFileType(dictPath))
     val offSet = dictFile.getSize
@@ -146,11 +148,13 @@ class GlobalDictionaryUtilConcurrentTestCase extends QueryTest with BeforeAndAft
       file.delete()
     }
   }
+
   override def afterAll {
     sql("drop table if exists employee")
     CarbonProperties.getInstance.addProperty(CarbonCommonConstants.MAX_QUERY_EXECUTION_TIME,
         Integer.toString(CarbonCommonConstants.DEFAULT_MAX_QUERY_EXECUTION_TIME))
   }
+
   class DictGenerator(loadModel: CarbonLoadModel) extends Callable[String] {
    override def call:String = {
      var result = "Pass"

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/0a09472a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilTestCase.scala
index 189e694..c4b213f 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilTestCase.scala
@@ -70,6 +70,7 @@ class GlobalDictionaryUtilTestCase extends QueryTest with BeforeAndAfterAll {
       CarbonCommonConstants.CARBON_DATE_FORMAT,
       CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT))  
     carbonLoadModel.setCsvHeaderColumns(CommonUtil.getCsvHeaderColumns(carbonLoadModel))
+    carbonLoadModel.setMaxColumns("2000")
     carbonLoadModel
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/0a09472a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
index 1451247..c8e0436 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
@@ -405,7 +405,6 @@ case class LoadTable(
       val dateFormat = options.getOrElse("dateformat", null)
       validateDateFormat(dateFormat, table)
       val maxColumns = options.getOrElse("maxcolumns", null)
-      carbonLoadModel.setMaxColumns(checkDefaultValue(maxColumns, null))
       carbonLoadModel.setEscapeChar(checkDefaultValue(escapeChar, "\\"))
       carbonLoadModel.setQuoteChar(checkDefaultValue(quoteChar, "\""))
       carbonLoadModel.setCommentChar(checkDefaultValue(commentchar, "#"))
@@ -472,8 +471,10 @@ case class LoadTable(
         carbonLoadModel.setColDictFilePath(columnDict)
         carbonLoadModel.setDirectLoad(true)
         carbonLoadModel.setCsvHeaderColumns(CommonUtil.getCsvHeaderColumns(carbonLoadModel))
+        val validatedMaxColumns = CommonUtil.validateMaxColumns(carbonLoadModel.getCsvHeaderColumns,
+          maxColumns)
+        carbonLoadModel.setMaxColumns(validatedMaxColumns.toString)
         GlobalDictionaryUtil.updateTableMetadataFunc = LoadTable.updateTableMetadata
-
         if (carbonLoadModel.getUseOnePass) {
           val colDictFilePath = carbonLoadModel.getColDictFilePath
           if (colDictFilePath != null) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/0a09472a/processing/src/main/java/org/apache/carbondata/processing/csvload/CSVInputFormat.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvload/CSVInputFormat.java b/processing/src/main/java/org/apache/carbondata/processing/csvload/CSVInputFormat.java
index 1f7d403..e252e7f 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/csvload/CSVInputFormat.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/csvload/CSVInputFormat.java
@@ -21,6 +21,9 @@ import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.Reader;
 
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+
 import com.univocity.parsers.csv.CsvParser;
 import com.univocity.parsers.csv.CsvParserSettings;
 import org.apache.hadoop.conf.Configuration;
@@ -63,6 +66,14 @@ public class CSVInputFormat extends FileInputFormat<NullWritable, StringArrayWri
   public static final boolean HEADER_PRESENT_DEFAULT = false;
   public static final String READ_BUFFER_SIZE = "carbon.csvinputformat.read.buffer.size";
   public static final String READ_BUFFER_SIZE_DEFAULT = "65536";
+  public static final String MAX_COLUMNS = "carbon.csvinputformat.max.columns";
+  public static final String NUMBER_OF_COLUMNS = "carbon.csvinputformat.number.of.columns";
+  public static final int DEFAULT_MAX_NUMBER_OF_COLUMNS_FOR_PARSING = 2000;
+  public static final int THRESHOLD_MAX_NUMBER_OF_COLUMNS_FOR_PARSING = 20000;
+
+  private static LogService LOGGER =
+      LogServiceFactory.getLogService(CSVInputFormat.class.toString());
+
 
   @Override
   public RecordReader<NullWritable, StringArrayWritable> createRecordReader(InputSplit inputSplit,
@@ -145,6 +156,16 @@ public class CSVInputFormat extends FileInputFormat<NullWritable, StringArrayWri
     }
   }
 
+  public static void setMaxColumns(Configuration configuration, String maxColumns) {
+    if (maxColumns != null) {
+      configuration.set(MAX_COLUMNS, maxColumns);
+    }
+  }
+
+  public static void setNumberOfColumns(Configuration configuration, String numberOfColumns) {
+    configuration.set(NUMBER_OF_COLUMNS, numberOfColumns);
+  }
+
   /**
    * Treats value as line in file. Key is null.
    */
@@ -220,8 +241,8 @@ public class CSVInputFormat extends FileInputFormat<NullWritable, StringArrayWri
       parserSettings.setIgnoreTrailingWhitespaces(false);
       parserSettings.setSkipEmptyLines(false);
       parserSettings.setMaxCharsPerColumn(100000);
-      // TODO get from csv file.
-      parserSettings.setMaxColumns(1000);
+      String maxColumns = job.get(MAX_COLUMNS);
+      parserSettings.setMaxColumns(Integer.parseInt(maxColumns));
       parserSettings.getFormat().setQuote(job.get(QUOTE, QUOTE_DEFAULT).charAt(0));
       parserSettings.getFormat().setQuoteEscape(job.get(ESCAPE, ESCAPE_DEFAULT).charAt(0));
       if (start == 0) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/0a09472a/processing/src/main/java/org/apache/carbondata/processing/model/CarbonLoadModel.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/model/CarbonLoadModel.java b/processing/src/main/java/org/apache/carbondata/processing/model/CarbonLoadModel.java
index 525874f..d8f84bf 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/model/CarbonLoadModel.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/model/CarbonLoadModel.java
@@ -171,6 +171,7 @@ public class CarbonLoadModel implements Serializable {
    */
   private boolean preFetch;
 
+  private String numberOfcolumns;
   /**
    * get escape char
    *

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/0a09472a/processing/src/main/java/org/apache/carbondata/processing/newflow/CarbonDataLoadConfiguration.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/CarbonDataLoadConfiguration.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/CarbonDataLoadConfiguration.java
index 0bd3e45..12be777 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/CarbonDataLoadConfiguration.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/newflow/CarbonDataLoadConfiguration.java
@@ -46,6 +46,10 @@ public class CarbonDataLoadConfiguration {
 
   private BucketingInfo bucketingInfo;
 
+  private String numberOfColumns;
+
+  private String maxColumns;
+
   private Map<String, Object> dataLoadProperties = new HashMap<>();
 
   /**
@@ -185,6 +189,14 @@ public class CarbonDataLoadConfiguration {
     this.taskNo = taskNo;
   }
 
+  public void setMaxColumns(String maxColumns) {
+    this.maxColumns = maxColumns;
+  }
+
+  public void setNumberOfColumns(int numberOfColumns) {
+    this.numberOfColumns = String.valueOf(numberOfColumns);
+  }
+
   public void setDataLoadProperty(String key, Object value) {
     dataLoadProperties.put(key, value);
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/0a09472a/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java b/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java
index 85f8470..87f1190 100644
--- a/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java
+++ b/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java
@@ -166,6 +166,7 @@ public class StoreCreator {
       loadModel.setSegmentId("0");
       loadModel.setPartitionId("0");
       loadModel.setFactTimeStamp(System.currentTimeMillis());
+      loadModel.setMaxColumns("10");
 
       executeGraph(loadModel, absoluteTableIdentifier.getStorePath());
 
@@ -399,6 +400,8 @@ public class StoreCreator {
     CSVInputFormat.setReadBufferSize(configuration, CarbonProperties.getInstance()
         .getProperty(CarbonCommonConstants.CSV_READ_BUFFER_SIZE,
             CarbonCommonConstants.CSV_READ_BUFFER_SIZE_DEFAULT));
+    CSVInputFormat.setMaxColumns(configuration, "10");
+    CSVInputFormat.setNumberOfColumns(configuration, "7");
 
     TaskAttemptContextImpl hadoopAttemptContext = new TaskAttemptContextImpl(configuration, new TaskAttemptID("", 1, TaskType.MAP, 0, 0));
     CSVInputFormat format = new CSVInputFormat();

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/0a09472a/processing/src/test/java/org/apache/carbondata/processing/csvload/CSVInputFormatTest.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/apache/carbondata/processing/csvload/CSVInputFormatTest.java b/processing/src/test/java/org/apache/carbondata/processing/csvload/CSVInputFormatTest.java
index 66aedb6..676838d 100644
--- a/processing/src/test/java/org/apache/carbondata/processing/csvload/CSVInputFormatTest.java
+++ b/processing/src/test/java/org/apache/carbondata/processing/csvload/CSVInputFormatTest.java
@@ -150,6 +150,8 @@ public class CSVInputFormatTest extends TestCase {
 
   private void prepareConf(Configuration conf) {
     conf.setBoolean(CSVInputFormat.HEADER_PRESENT, true);
+    conf.set(CSVInputFormat.MAX_COLUMNS, "10");
+    conf.set(CSVInputFormat.NUMBER_OF_COLUMNS, "7");
   }
 
   private void deleteOutput(File output) {


[28/50] incubator-carbondata git commit: [CARBONDATA-914] Clear BTree and Dictionary instances from LRU cache on table drop. This closes #788

Posted by ra...@apache.org.
[CARBONDATA-914] Clear BTree and Dictionary instances from LRU cache on table drop. This closes #788


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

Branch: refs/heads/branch-1.1
Commit: 51f96299ad4b14baf689e05cb9d74d39d2089402
Parents: 144620d 673f8c2
Author: Venkata Ramana G <ra...@huawei.com>
Authored: Thu Apr 13 22:29:30 2017 +0530
Committer: Venkata Ramana G <ra...@huawei.com>
Committed: Thu Apr 13 22:29:30 2017 +0530

----------------------------------------------------------------------
 .../core/cache/dictionary/ManageDictionary.java | 108 -------------
 .../dictionary/ManageDictionaryAndBTree.java    | 158 +++++++++++++++++++
 .../carbondata/hadoop/CacheAccessClient.java    |  23 ++-
 .../carbondata/hadoop/CarbonInputFormat.java    | 153 +++++++++---------
 .../spark/rdd/AlterTableDropColumnRDD.scala     |   4 +-
 .../execution/command/carbonTableSchema.scala   |   8 +-
 .../carbondata/CarbonDataSourceSuite.scala      |   2 +-
 7 files changed, 269 insertions(+), 187 deletions(-)
----------------------------------------------------------------------



[34/50] incubator-carbondata git commit: [CARBONDATA-863] Fixed column clean up during altertable add column failure. This closes #777

Posted by ra...@apache.org.
[CARBONDATA-863] Fixed column clean up during altertable add column failure. This closes #777


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

Branch: refs/heads/branch-1.1
Commit: 19b9223eb08b7673144ba549d0820adf243c01cc
Parents: f27b491 36112fa
Author: Venkata Ramana G <ra...@huawei.com>
Authored: Fri Apr 14 16:23:58 2017 +0530
Committer: Venkata Ramana G <ra...@huawei.com>
Committed: Fri Apr 14 16:23:58 2017 +0530

----------------------------------------------------------------------
 .../spark/rdd/AlterTableAddColumnRDD.scala      |  6 +-
 .../execution/command/carbonTableSchema.scala   |  8 +--
 .../execution/command/AlterTableCommands.scala  | 62 ++++++++++++--------
 .../org/apache/spark/util/AlterTableUtil.scala  | 16 +++--
 .../restructure/AlterTableRevertTestCase.scala  | 19 +++++-
 5 files changed, 68 insertions(+), 43 deletions(-)
----------------------------------------------------------------------



[39/50] incubator-carbondata git commit: Show segment link in data management doc is corrected show segment command output is reformatted

Posted by ra...@apache.org.
Show segment link in data management doc is corrected
show segment command output is reformatted

Fixed formatting


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

Branch: refs/heads/branch-1.1
Commit: f2f757d6b267284fa458eb2ebe49b1741e46d363
Parents: afa7887
Author: Sanoj MG <sa...@gmail.com>
Authored: Fri Apr 14 22:05:27 2017 +0400
Committer: chenliang613 <ch...@huawei.com>
Committed: Sat Apr 15 16:36:21 2017 +0800

----------------------------------------------------------------------
 docs/data-management.md | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/f2f757d6/docs/data-management.md
----------------------------------------------------------------------
diff --git a/docs/data-management.md b/docs/data-management.md
index 2663aff..297251d 100644
--- a/docs/data-management.md
+++ b/docs/data-management.md
@@ -47,12 +47,12 @@ This tutorial is going to introduce you to the conceptual details of data manage
 | Success | All the data is loaded into table and no bad records found. |
 | Partial Success | Data is loaded into table and bad records are found. Bad records are stored at carbon.badrecords.location. |
    
-   In case of failure, the error will be logged in error log. Details of loads can be seen with [SHOW SEGMENTS](dml-operation-on-carbondata.md) command. The show segment command output consists of :
+   In case of failure, the error will be logged in error log. Details of loads can be seen with [SHOW SEGMENTS](dml-operation-on-carbondata.md#show-segments) command. The show segment command output consists of :
    
-   - SegmentSequenceID
-   - START_TIME OF LOAD
-   - END_TIME OF LOAD 
-   - LOAD STATUS
+   - SegmentSequenceId
+   - Status
+   - Load Start Time
+   - Load End Time 
  
    The latest load will be displayed first in the output.
    


[09/50] incubator-carbondata git commit: CARBONDATA-880 Path is getting printed in the explain extended DDL. this should not get printed. so overriding the toString method.

Posted by ra...@apache.org.
CARBONDATA-880
Path is getting printed in the explain extended DDL. this should not get printed. so overriding the toString method.


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

Branch: refs/heads/branch-1.1
Commit: 591f9f692cd2e7561e315dae60bf0c482ae46a01
Parents: aea9ed6
Author: ravikiran <ra...@gmail.com>
Authored: Thu Apr 6 18:30:39 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Wed Apr 12 13:55:01 2017 +0530

----------------------------------------------------------------------
 .../org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala   | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/591f9f69/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 b9e8682..322e1ae 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
@@ -74,4 +74,9 @@ case class CarbonDatasourceHadoopRelation(
   }
   override def unhandledFilters(filters: Array[Filter]): Array[Filter] = new Array[Filter](0)
 
+  override def toString: String = {
+    "CarbonDatasourceHadoopRelation [ " + "Database name :" + carbonTable.getDatabaseName +
+    ", " + "Table name :" + carbonTable.getFactTableName + ", Schema :" + tableSchema + " ]"
+  }
+
 }


[25/50] incubator-carbondata git commit: fixed cast exception for integer data types in RestructureBasedVectorResultCollector

Posted by ra...@apache.org.
fixed cast exception for integer data types in RestructureBasedVectorResultCollector

fixed cast exception for double datatype


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

Branch: refs/heads/branch-1.1
Commit: 8c5c00c0c169d74adb72f13b8a781c893a6ad791
Parents: 8478939
Author: kunal642 <ku...@knoldus.in>
Authored: Fri Apr 7 18:56:16 2017 +0530
Committer: Venkata Ramana G <ra...@huawei.com>
Committed: Thu Apr 13 19:15:54 2017 +0530

----------------------------------------------------------------------
 .../RestructureBasedVectorResultCollector.java  | 28 ++++++++---
 .../scan/executor/util/RestructureUtil.java     | 50 +++++++++++++++++++-
 .../scan/executor/util/RestructureUtilTest.java |  2 +-
 .../carbondata/spark/load/CarbonLoaderUtil.java |  2 +-
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala |  6 +--
 .../execution/command/AlterTableCommands.scala  |  4 --
 .../apache/spark/sql/hive/CarbonMetastore.scala |  8 ++--
 .../vectorreader/AddColumnTestCases.scala       | 44 +++++++++++++++++
 8 files changed, 123 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8c5c00c0/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java
index 99b7f01..c09ec12 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java
@@ -16,7 +16,6 @@
  */
 package org.apache.carbondata.core.scan.collector.impl;
 
-import java.math.BigDecimal;
 import java.util.List;
 
 import org.apache.carbondata.core.metadata.encoder.Encoding;
@@ -36,10 +35,13 @@ import org.apache.spark.sql.types.Decimal;
  */
 public class RestructureBasedVectorResultCollector extends DictionaryBasedVectorResultCollector {
 
+  private Object[] measureDefaultValues = null;
+
   public RestructureBasedVectorResultCollector(BlockExecutionInfo blockExecutionInfos) {
     super(blockExecutionInfos);
     queryDimensions = tableBlockExecutionInfos.getActualQueryDimensions();
     queryMeasures = tableBlockExecutionInfos.getActualQueryMeasures();
+    measureDefaultValues = new Object[queryMeasures.length];
     allColumnInfo = new ColumnVectorInfo[queryDimensions.length + queryMeasures.length];
     createVectorForNewlyAddedDimensions();
     createVectorForNewlyAddedMeasures();
@@ -68,10 +70,23 @@ public class RestructureBasedVectorResultCollector extends DictionaryBasedVector
         // add a dummy column vector result collector object
         ColumnVectorInfo columnVectorInfo = new ColumnVectorInfo();
         allColumnInfo[queryMeasures[i].getQueryOrder()] = columnVectorInfo;
+        measureDefaultValues[i] = getMeasureDefaultValue(queryMeasures[i].getMeasure());
       }
     }
   }
 
+  /**
+   * Gets the default value for each CarbonMeasure
+   * @param carbonMeasure
+   * @return
+   */
+  private Object getMeasureDefaultValue(CarbonMeasure carbonMeasure) {
+    return RestructureUtil.getMeasureDefaultValueByType(carbonMeasure.getColumnSchema(),
+        carbonMeasure.getDefaultValue());
+  }
+
+
+
   @Override public List<Object[]> collectData(AbstractScannedResult scannedResult, int batchSize) {
     throw new UnsupportedOperationException("collectData is not supported here");
   }
@@ -177,11 +192,11 @@ public class RestructureBasedVectorResultCollector extends DictionaryBasedVector
   private void fillDataForNonExistingMeasures() {
     for (int i = 0; i < tableBlockExecutionInfos.getActualQueryMeasures().length; i++) {
       if (!measureInfo.getMeasureExists()[i]) {
+        int queryOrder = tableBlockExecutionInfos.getActualQueryMeasures()[i].getQueryOrder();
         CarbonMeasure measure = tableBlockExecutionInfos.getActualQueryMeasures()[i].getMeasure();
-        ColumnVectorInfo columnVectorInfo = allColumnInfo[i];
-        CarbonColumnVector vector = allColumnInfo[i].vector;
-        Object defaultValue = RestructureUtil
-            .getMeasureDefaultValue(measure.getColumnSchema(), measure.getDefaultValue());
+        ColumnVectorInfo columnVectorInfo = allColumnInfo[queryOrder];
+        CarbonColumnVector vector = columnVectorInfo.vector;
+        Object defaultValue = measureDefaultValues[i];
         if (null == defaultValue) {
           vector.putNulls(columnVectorInfo.vectorOffset, columnVectorInfo.size);
         } else {
@@ -199,9 +214,8 @@ public class RestructureBasedVectorResultCollector extends DictionaryBasedVector
                   (long) defaultValue);
               break;
             case DECIMAL:
-              Decimal convertToSparkType = Decimal.apply((BigDecimal) defaultValue);
               vector.putDecimals(columnVectorInfo.vectorOffset, columnVectorInfo.size,
-                  convertToSparkType, measure.getPrecision());
+                  (Decimal) defaultValue, measure.getPrecision());
               break;
             default:
               vector.putDoubles(columnVectorInfo.vectorOffset, columnVectorInfo.size,

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8c5c00c0/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java
index 955f1f1..ab0ed55 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java
@@ -39,6 +39,7 @@ import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.DataTypeUtil;
 
 import org.apache.commons.lang3.ArrayUtils;
+import org.apache.spark.sql.types.Decimal;
 import org.apache.spark.unsafe.types.UTF8String;
 
 /**
@@ -295,7 +296,54 @@ public class RestructureUtil {
               new String(defaultValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
           Double parsedValue = Double.valueOf(value);
           if (!Double.isInfinite(parsedValue) && !Double.isNaN(parsedValue)) {
-            measureDefaultValue = value;
+            measureDefaultValue = parsedValue;
+          }
+      }
+    }
+    return measureDefaultValue;
+  }
+
+  /**
+   * Gets the default value based on the column data type.
+   *
+   * @param columnSchema
+   * @param defaultValue
+   * @return
+   */
+  public static Object getMeasureDefaultValueByType(ColumnSchema columnSchema,
+      byte[] defaultValue) {
+    Object measureDefaultValue = null;
+    if (!isDefaultValueNull(defaultValue)) {
+      String value = null;
+      switch (columnSchema.getDataType()) {
+        case SHORT:
+          value =
+              new String(defaultValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+          measureDefaultValue = Short.parseShort(value);
+          break;
+        case INT:
+          value =
+              new String(defaultValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+          measureDefaultValue = Integer.parseInt(value);
+          break;
+        case LONG:
+          value =
+              new String(defaultValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+          measureDefaultValue = Long.parseLong(value);
+          break;
+        case DECIMAL:
+          BigDecimal decimal = DataTypeUtil.byteToBigDecimal(defaultValue);
+          if (columnSchema.getScale() > decimal.scale()) {
+            decimal = decimal.setScale(columnSchema.getScale(), RoundingMode.HALF_UP);
+          }
+          measureDefaultValue = Decimal.apply(decimal);
+          break;
+        default:
+          value =
+              new String(defaultValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+          Double parsedValue = Double.valueOf(value);
+          if (!Double.isInfinite(parsedValue) && !Double.isNaN(parsedValue)) {
+            measureDefaultValue = parsedValue;
           }
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8c5c00c0/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java b/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java
index 5387823..df14381 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java
@@ -129,7 +129,7 @@ public class RestructureUtilTest {
     MeasureInfo measureInfo = blockExecutionInfo.getMeasureInfo();
     boolean[] measuresExist = { true, true, false };
     assertThat(measureInfo.getMeasureExists(), is(equalTo(measuresExist)));
-    Object[] defaultValues = { null, null, "3" };
+    Object[] defaultValues = { null, null, 3.0 };
     assertThat(measureInfo.getDefaultValues(), is(equalTo(defaultValues)));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8c5c00c0/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java
index 036e574..27a8e3a 100644
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java
+++ b/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java
@@ -184,7 +184,7 @@ public final class CarbonLoaderUtil {
           try {
             long startTime = System.currentTimeMillis();
             File file = new File(localStoreLocation);
-            CarbonUtil.deleteFoldersAndFiles(file.getParentFile());
+            CarbonUtil.deleteFoldersAndFiles(file);
             LOGGER.info(
                 "Deleted the local store location" + localStoreLocation + " : TIme taken: " + (
                     System.currentTimeMillis() - startTime));

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8c5c00c0/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
index 9d2c245..9ca9163 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
@@ -150,12 +150,12 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
   protected val DECIMAL = carbonKeyWord("DECIMAL")
   protected val DOUBLE = carbonKeyWord("DOUBLE")
   protected val FLOAT = carbonKeyWord("FLOAT")
-  protected val SHORT = carbonKeyWord("SMALLINT")
+  protected val SHORT = carbonKeyWord("SHORT")
   protected val INT = carbonKeyWord("INT")
   protected val BIGINT = carbonKeyWord("BIGINT")
   protected val ARRAY = carbonKeyWord("ARRAY")
   protected val STRUCT = carbonKeyWord("STRUCT")
-
+  protected val SMALLINT = carbonKeyWord("SMALLINT")
   protected val CHANGE = carbonKeyWord("CHANGE")
   protected val TBLPROPERTIES = carbonKeyWord("TBLPROPERTIES")
 
@@ -856,7 +856,7 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
   protected lazy val primitiveTypes =
     STRING ^^^ "string" | INTEGER ^^^ "integer" |
     TIMESTAMP ^^^ "timestamp" | NUMERIC ^^^ "numeric" |
-    BIGINT ^^^ "bigint" | SHORT ^^^ "smallint" |
+    BIGINT ^^^ "bigint" | (SHORT | SMALLINT) ^^^ "smallint" |
     INT ^^^ "int" | DOUBLE ^^^ "double" | FLOAT ^^^ "double" | decimalType |
     DATE ^^^ "date" | charType
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8c5c00c0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommands.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommands.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommands.scala
index 0be0bdf..e380217 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommands.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommands.scala
@@ -82,8 +82,6 @@ private[sql] case class AlterTableAddColumns(
       schemaEvolutionEntry.setAdded(newCols.toList.asJava)
       val thriftTable = schemaConverter
         .fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
-      thriftTable.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
-        .setTime_stamp(System.currentTimeMillis)
       AlterTableUtil
         .updateSchemaInfo(carbonTable,
           schemaConverter.fromWrapperToExternalSchemaEvolutionEntry(schemaEvolutionEntry),
@@ -301,8 +299,6 @@ private[sql] case class AlterTableDropColumns(
         }
       }
       // add deleted columns to schema evolution history and update the schema
-      tableInfo.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
-        .setTime_stamp(System.currentTimeMillis)
       val schemaEvolutionEntry = new SchemaEvolutionEntry(System.currentTimeMillis)
       schemaEvolutionEntry.setRemoved(deletedColumnSchema.toList.asJava)
       AlterTableUtil

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8c5c00c0/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetastore.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetastore.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetastore.scala
index 6f74960..1f5736e 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetastore.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetastore.scala
@@ -315,12 +315,12 @@ class CarbonMetastore(conf: RuntimeConfig, val storePath: String) {
       carbonStorePath: String)
     (sparkSession: SparkSession): String = {
     val schemaConverter = new ThriftWrapperSchemaConverterImpl
+    thriftTableInfo.fact_table.schema_evolution.schema_evolution_history.add(schemaEvolutionEntry)
     val wrapperTableInfo = schemaConverter
       .fromExternalToWrapperTableInfo(thriftTableInfo,
-        carbonTableIdentifier.getDatabaseName,
-        carbonTableIdentifier.getTableName,
-        carbonStorePath)
-    thriftTableInfo.fact_table.schema_evolution.schema_evolution_history.add(schemaEvolutionEntry)
+          carbonTableIdentifier.getDatabaseName,
+          carbonTableIdentifier.getTableName,
+          carbonStorePath)
     createSchemaThriftFile(wrapperTableInfo,
       thriftTableInfo,
       carbonTableIdentifier.getDatabaseName,

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8c5c00c0/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala
index 747af05..3b4a25c 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala
@@ -187,6 +187,50 @@ class AddColumnTestCases extends QueryTest with BeforeAndAfterAll {
     checkAnswer(sql("select distinct(CUST_NAME) from carbon_new"),Row("testuser"))
   }
 
+  test("test to check if intField returns correct result") {
+    sql("DROP TABLE IF EXISTS carbon_table")
+    sql("CREATE TABLE carbon_table(intField int,stringField string,charField string,timestampField timestamp, decimalField decimal(6,2)) STORED BY 'carbondata'")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data1.csv' INTO TABLE carbon_table options('FILEHEADER'='intField,stringField,charField,timestampField,decimalField')")
+    sql(
+      "Alter table carbon_table add columns(newField int) TBLPROPERTIES" +
+      "('DEFAULT.VALUE.newField'='67890')")
+    checkAnswer(sql("select distinct(newField) from carbon_table"), Row(67890))
+    sql("DROP TABLE IF EXISTS carbon_table")
+  }
+
+  test("test to check if shortField returns correct result") {
+    sql("DROP TABLE IF EXISTS carbon_table")
+    sql("CREATE TABLE carbon_table(intField int,stringField string,charField string,timestampField timestamp, decimalField decimal(6,2)) STORED BY 'carbondata'")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data1.csv' INTO TABLE carbon_table options('FILEHEADER'='intField,stringField,charField,timestampField,decimalField')")
+    sql(
+      "Alter table carbon_table add columns(newField short) TBLPROPERTIES" +
+      "('DEFAULT.VALUE.newField'='1')")
+    checkAnswer(sql("select distinct(newField) from carbon_table"), Row(1))
+    sql("DROP TABLE IF EXISTS carbon_table")
+  }
+
+  test("test to check if doubleField returns correct result") {
+    sql("DROP TABLE IF EXISTS carbon_table")
+    sql("CREATE TABLE carbon_table(intField int,stringField string,charField string,timestampField timestamp, decimalField decimal(6,2)) STORED BY 'carbondata'")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data1.csv' INTO TABLE carbon_table options('FILEHEADER'='intField,stringField,charField,timestampField,decimalField')")
+    sql(
+      "Alter table carbon_table add columns(newField double) TBLPROPERTIES" +
+      "('DEFAULT.VALUE.newField'='1457567.87')")
+    checkAnswer(sql("select distinct(newField) from carbon_table"), Row(1457567.87))
+    sql("DROP TABLE IF EXISTS carbon_table")
+  }
+
+  test("test to check if decimalField returns correct result") {
+    sql("DROP TABLE IF EXISTS carbon_table")
+    sql("CREATE TABLE carbon_table(intField int,stringField string,charField string,timestampField timestamp, decimalField decimal(6,2)) STORED BY 'carbondata'")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data1.csv' INTO TABLE carbon_table options('FILEHEADER'='intField,stringField,charField,timestampField,decimalField')")
+    sql(
+      "Alter table carbon_table add columns(newField decimal(5,2)) TBLPROPERTIES" +
+      "('DEFAULT.VALUE.newField'='21.87')")
+    checkAnswer(sql("select distinct(newField) from carbon_table"), Row(21.87))
+    sql("DROP TABLE IF EXISTS carbon_table")
+  }
+
 
   override def afterAll {
     sql("DROP TABLE IF EXISTS addcolumntest")


[41/50] incubator-carbondata git commit: Drop table IS is throwing exception

Posted by ra...@apache.org.
Drop table IS is throwing exception


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

Branch: refs/heads/branch-1.1
Commit: 3d21ccba668d3c42c9bd2d276d45c55d23e0b7fe
Parents: ab4556d
Author: Manohar <ma...@gmail.com>
Authored: Sat Apr 15 13:42:29 2017 +0530
Committer: Manohar <ma...@gmail.com>
Committed: Sat Apr 15 19:38:56 2017 +0530

----------------------------------------------------------------------
 .../core/metadata/AbsoluteTableIdentifier.java  |  5 +-
 .../spark/sql/hive/CarbonHiveMetadataUtil.scala |  5 +-
 .../deletetable/DeleteTableTestCase.scala       | 65 ++++++++++++++++++++
 3 files changed, 71 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3d21ccba/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java b/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java
index 3791150..3c39145 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java
@@ -90,8 +90,9 @@ public class AbsoluteTableIdentifier implements Serializable {
 
     String tableName = names[names.length - 1];
     String dbName = names[names.length - 2];
-    String storePath = formattedTablePath.substring(0, formattedTablePath.lastIndexOf(dbName +
-            CarbonCommonConstants.FILE_SEPARATOR + tableName));
+    String storePath = formattedTablePath.substring(0,
+        formattedTablePath.lastIndexOf(dbName + CarbonCommonConstants.FILE_SEPARATOR + tableName)
+            - 1);
 
     CarbonTableIdentifier identifier =
         new CarbonTableIdentifier(dbName, tableName, Long.toString(System.currentTimeMillis()));

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3d21ccba/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetadataUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetadataUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetadataUtil.scala
index 657d7de..d67ae6b 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetadataUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetadataUtil.scala
@@ -17,6 +17,7 @@
 package org.apache.spark.sql.hive
 
 import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.TableIdentifier
 
 import org.apache.carbondata.common.logging.LogServiceFactory
 
@@ -41,8 +42,8 @@ object CarbonHiveMetadataUtil {
       tableName: String,
       sparkSession: SparkSession): Unit = {
     try {
-      sparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client.
-        runSqlHive(s"DROP TABLE IF EXISTS $databaseName.$tableName")
+      val tabelIdentifier = TableIdentifier(tableName, Some(databaseName))
+      sparkSession.sessionState.catalog.dropTable(tabelIdentifier, true, false)
     } catch {
       case e: Exception =>
         LOGGER.audit(

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3d21ccba/integration/spark2/src/test/scala/org/apache/spark/carbondata/deletetable/DeleteTableTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/deletetable/DeleteTableTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/deletetable/DeleteTableTestCase.scala
new file mode 100644
index 0000000..1ab9409
--- /dev/null
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/deletetable/DeleteTableTestCase.scala
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.carbondata.deletetable
+
+import org.apache.spark.sql.common.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+/**
+ * Test cases for drop table
+ */
+class DeleteTableTestCase extends QueryTest with BeforeAndAfterAll {
+
+  override def beforeAll {
+    sql("drop table if exists IS")
+  }
+
+  test("drop table IS with load") {
+    sql("drop table if exists IS")
+    sql(
+      "CREATE TABLE IS (imei string,age int,task bigint,name string,country string," +
+      "city string,sale int,num double,level decimal(10,3),quest bigint,productdate timestamp," +
+      "enddate timestamp,PointId double,score decimal(10,3))STORED BY 'org.apache.carbondata" +
+      ".format'")
+    sql(
+      s"LOAD DATA INPATH '$resourcesPath/big_int_Decimal.csv'  INTO TABLE IS " +
+      "options ('DELIMITER'=',', 'QUOTECHAR'='\"', 'COMPLEX_DELIMITER_LEVEL_1'='$'," +
+      "'COMPLEX_DELIMITER_LEVEL_2'=':', 'FILEHEADER'= '')")
+    sql("drop table IS")
+    sql("drop table if exists IS")
+    sql(
+      "CREATE TABLE IS (imei string,age int,task bigint,name string,country string," +
+      "city string,sale int,num double,level decimal(10,3),quest bigint,productdate timestamp," +
+      "enddate timestamp,PointId double,score decimal(10,3))STORED BY 'org.apache.carbondata" +
+      ".format'")
+    sql("drop table if exists IS")
+  }
+
+  test("drop table IS without load") {
+    sql("drop table if exists IS")
+    sql(
+      "CREATE TABLE IS (imei string,age int,task bigint,name string,country string," +
+      "city string,sale int,num double,level decimal(10,3),quest bigint,productdate timestamp," +
+      "enddate timestamp,PointId double,score decimal(10,3))STORED BY 'org.apache.carbondata" +
+      ".format'")
+    sql("drop table if exists IS")
+  }
+  override def afterAll {
+    sql("drop table if exists IS")
+  }
+}


[12/50] incubator-carbondata git commit: [CARBONDATA-890] For Spark 2.1 LRU cache size at driver is getting configured with the executor lru cache size. This closes #774

Posted by ra...@apache.org.
[CARBONDATA-890] For Spark 2.1 LRU cache size at driver is getting configured with the executor lru cache size. This closes #774


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

Branch: refs/heads/branch-1.1
Commit: ff182b769d556743ed2c95bf7807ed703dc4d6c6
Parents: 3966f99 99582a7
Author: ravipesala <ra...@gmail.com>
Authored: Wed Apr 12 14:11:10 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Wed Apr 12 14:11:10 2017 +0530

----------------------------------------------------------------------
 .../core/cache/CacheProviderTest.java           | 53 ++++++++++++++++++++
 .../scala/org/apache/spark/sql/CarbonEnv.scala  |  1 +
 2 files changed, 54 insertions(+)
----------------------------------------------------------------------



[24/50] incubator-carbondata git commit: [CARBONDATA-898]NullPointerException is getting thrown when rename table and select query is run concurrently. This closes #730

Posted by ra...@apache.org.
[CARBONDATA-898]NullPointerException is getting thrown when rename table and select query is run concurrently. This closes #730


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

Branch: refs/heads/branch-1.1
Commit: 8478939119462b7c071144d4bd2e0d239722992e
Parents: f7d7e41 bcd2839
Author: ravipesala <ra...@gmail.com>
Authored: Thu Apr 13 16:21:29 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Apr 13 16:21:29 2017 +0530

----------------------------------------------------------------------
 .../cache/dictionary/AbstractDictionaryCache.java   | 16 ++++++++++------
 .../apache/carbondata/hadoop/util/SchemaReader.java |  2 +-
 2 files changed, 11 insertions(+), 7 deletions(-)
----------------------------------------------------------------------



[07/50] incubator-carbondata git commit: fix some spelling mistakes in SegmentProperties.java

Posted by ra...@apache.org.
fix some spelling mistakes in SegmentProperties.java

fix some spelling mistakes in SegmentProperties.java


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

Branch: refs/heads/branch-1.1
Commit: 81ccec2ab75c7504d7b5689c7098cf925acb54b9
Parents: 58dc3aa
Author: CGF <cg...@foxmail.com>
Authored: Tue Apr 11 19:42:30 2017 +0800
Committer: chenliang613 <ch...@huawei.com>
Committed: Wed Apr 12 00:42:56 2017 +0530

----------------------------------------------------------------------
 .../core/datastore/block/SegmentProperties.java | 24 ++++++++++----------
 1 file changed, 12 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/81ccec2a/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java b/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java
index 6c7d014..3bc208d 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java
@@ -372,7 +372,7 @@ public class SegmentProperties {
           isComplexDimensionStarted = true;
           int previouseOrdinal = dimensonOrdinal;
           dimensonOrdinal =
-              readAllComplexTypeChildrens(dimensonOrdinal, columnSchema.getNumberOfChild(),
+              readAllComplexTypeChildren(dimensonOrdinal, columnSchema.getNumberOfChild(),
                   columnsInTable, carbonDimension, complexTypeOrdinal);
           int numberOfChildrenDimensionAdded = dimensonOrdinal - previouseOrdinal;
           for (int i = 0; i < numberOfChildrenDimensionAdded; i++) {
@@ -418,25 +418,25 @@ public class SegmentProperties {
    * @param parentDimension
    * @return
    */
-  private int readAllComplexTypeChildrens(int dimensionOrdinal, int childCount,
+  private int readAllComplexTypeChildren(int dimensionOrdinal, int childCount,
       List<ColumnSchema> listOfColumns, CarbonDimension parentDimension,
-      int complexDimensionOrdianl) {
+      int complexDimensionOrdinal) {
     for (int i = 0; i < childCount; i++) {
       ColumnSchema columnSchema = listOfColumns.get(dimensionOrdinal);
       if (columnSchema.isDimensionColumn()) {
         if (columnSchema.getNumberOfChild() > 0) {
           CarbonDimension complexDimension =
               new CarbonDimension(columnSchema, dimensionOrdinal++, -1, -1,
-                  complexDimensionOrdianl++);
+                  complexDimensionOrdinal++);
           complexDimension.initializeChildDimensionsList(columnSchema.getNumberOfChild());
           parentDimension.getListOfChildDimensions().add(complexDimension);
           dimensionOrdinal =
-              readAllComplexTypeChildrens(dimensionOrdinal, columnSchema.getNumberOfChild(),
-                  listOfColumns, complexDimension, complexDimensionOrdianl);
+              readAllComplexTypeChildren(dimensionOrdinal, columnSchema.getNumberOfChild(),
+                  listOfColumns, complexDimension, complexDimensionOrdinal);
         } else {
           parentDimension.getListOfChildDimensions().add(
               new CarbonDimension(columnSchema, dimensionOrdinal++, -1, -1,
-                  complexDimensionOrdianl++));
+                  complexDimensionOrdinal++));
         }
       }
     }
@@ -447,18 +447,18 @@ public class SegmentProperties {
    * Read all primitive/complex children and set it as list of child carbon dimension to parent
    * dimension
    */
-  private int assignComplexOrdinal(CarbonDimension parentDimension, int complexDimensionOrdianl) {
+  private int assignComplexOrdinal(CarbonDimension parentDimension, int complexDimensionOrdinal) {
     for (int i = 0; i < parentDimension.getNumberOfChild(); i++) {
       CarbonDimension dimension = parentDimension.getListOfChildDimensions().get(i);
       if (dimension.getNumberOfChild() > 0) {
-        dimension.setComplexTypeOridnal(++complexDimensionOrdianl);
-        complexDimensionOrdianl = assignComplexOrdinal(dimension, complexDimensionOrdianl);
+        dimension.setComplexTypeOridnal(++complexDimensionOrdinal);
+        complexDimensionOrdinal = assignComplexOrdinal(dimension, complexDimensionOrdinal);
       } else {
         parentDimension.getListOfChildDimensions().get(i)
-            .setComplexTypeOridnal(++complexDimensionOrdianl);
+            .setComplexTypeOridnal(++complexDimensionOrdinal);
       }
     }
-    return complexDimensionOrdianl;
+    return complexDimensionOrdinal;
   }
 
   /**


[43/50] incubator-carbondata git commit: BigDecimal and VariableLength Dimension fixes

Posted by ra...@apache.org.
BigDecimal and VariableLength Dimension fixes


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

Branch: refs/heads/branch-1.1
Commit: 92352f3a695e99702564287921b57e72aa7f3a07
Parents: f58853d
Author: vpp9380 <vp...@gmail.com>
Authored: Sat Apr 15 17:01:48 2017 +0530
Committer: Venkata Ramana G <ra...@huawei.com>
Committed: Sat Apr 15 22:04:01 2017 +0530

----------------------------------------------------------------------
 .../store/impl/unsafe/UnsafeBigDecimalMeasureChunkStore.java    | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/92352f3a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeBigDecimalMeasureChunkStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeBigDecimalMeasureChunkStore.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeBigDecimalMeasureChunkStore.java
index 52bd74b..4082689 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeBigDecimalMeasureChunkStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeBigDecimalMeasureChunkStore.java
@@ -124,11 +124,10 @@ public class UnsafeBigDecimalMeasureChunkStore extends UnsafeAbstractMeasureData
       int OffsetOfNextdata = CarbonUnsafe.unsafe.getInt(dataPageMemoryBlock.getBaseObject(),
           dataPageMemoryBlock.getBaseOffset() + this.offsetStartPosition + ((index + 1)
               * CarbonCommonConstants.INT_SIZE_IN_BYTE));
-      length =
-          (short) (OffsetOfNextdata - (currentDataOffset + CarbonCommonConstants.INT_SIZE_IN_BYTE));
+      length = OffsetOfNextdata - (currentDataOffset + CarbonCommonConstants.INT_SIZE_IN_BYTE);
     } else {
       // for last record we need to subtract with data length
-      length = (short) (this.offsetStartPosition - currentDataOffset);
+      length = (int) this.offsetStartPosition - currentDataOffset;
     }
     byte[] row = new byte[length];
     CarbonUnsafe.unsafe.copyMemory(dataPageMemoryBlock.getBaseObject(),


[18/50] incubator-carbondata git commit: [CARBONDATA-916] Major Compaction : Updating proper schema update time stamp. This closes #789

Posted by ra...@apache.org.
[CARBONDATA-916] Major Compaction : Updating proper schema update time stamp. This closes #789


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

Branch: refs/heads/branch-1.1
Commit: 9efcacdac2ff9c07107e7a6b6fcf299c0bc817a6
Parents: 11df451 ebc5ee9
Author: ravipesala <ra...@gmail.com>
Authored: Thu Apr 13 15:37:28 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Apr 13 15:37:28 2017 +0530

----------------------------------------------------------------------
 .../MajorCompactionIgnoreInMinorTest.scala      | 53 +++++++++++++++++---
 .../spark/rdd/CarbonDataRDDFactory.scala        | 11 ++--
 .../store/CarbonFactDataHandlerModel.java       |  1 +
 3 files changed, 54 insertions(+), 11 deletions(-)
----------------------------------------------------------------------



[16/50] incubator-carbondata git commit: [CARBONDATA-897] Removed redundant fields from table in configuration-parameters.md This closes #776

Posted by ra...@apache.org.
[CARBONDATA-897] Removed redundant fields from table in configuration-parameters.md This closes #776


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

Branch: refs/heads/branch-1.1
Commit: 11df4513bd88d053ea0d5d41ab9919d1f47ebbfd
Parents: a5eb323 d8590b1
Author: chenliang613 <ch...@huawei.com>
Authored: Wed Apr 12 18:04:01 2017 +0530
Committer: chenliang613 <ch...@huawei.com>
Committed: Wed Apr 12 18:04:01 2017 +0530

----------------------------------------------------------------------
 docs/configuration-parameters.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------



[49/50] incubator-carbondata git commit: null value handled during compaction

Posted by ra...@apache.org.
null value handled during compaction


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

Branch: refs/heads/branch-1.1
Commit: 8ed7931bae6858a47c4989dde7223b20eecc7803
Parents: 4397d05
Author: rahulforallp <ra...@knoldus.in>
Authored: Tue Apr 18 12:26:51 2017 +0530
Committer: rahulforallp <ra...@knoldus.in>
Committed: Tue Apr 18 12:26:51 2017 +0530

----------------------------------------------------------------------
 .../rowreader/AddColumnTestCases.scala          | 22 ++++++++++++++++++++
 .../merger/CompactionResultSortProcessor.java   |  7 ++++---
 2 files changed, 26 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8ed7931b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/rowreader/AddColumnTestCases.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/rowreader/AddColumnTestCases.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/rowreader/AddColumnTestCases.scala
index e6fb265..06f480b 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/rowreader/AddColumnTestCases.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/rowreader/AddColumnTestCases.scala
@@ -131,6 +131,28 @@ class AddColumnTestCases extends QueryTest with BeforeAndAfterAll {
     sql("DROP TABLE IF EXISTS carbon_table")
   }
 
+
+  test("test add column compaction") {
+    sql("DROP TABLE IF EXISTS carbon_table")
+    sql(
+      "CREATE TABLE carbon_table(intField int,stringField string,charField string,timestampField " +
+      "timestamp)STORED BY 'carbondata' TBLPROPERTIES" +
+      "('DICTIONARY_EXCLUDE'='charField')")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data1.csv' INTO TABLE carbon_table " +
+        s"options('FILEHEADER'='intField,stringField,charField,timestampField,decimalField')")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data1.csv' INTO TABLE carbon_table " +
+        s"options('FILEHEADER'='intField,stringField,charField,timestampField,decimalField')")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data1.csv' INTO TABLE carbon_table " +
+        s"options('FILEHEADER'='intField,stringField,charField,timestampField,decimalField')")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data1.csv' INTO TABLE carbon_table " +
+        s"options('FILEHEADER'='intField,stringField,charField,timestampField,decimalField')")
+    sql("Alter table carbon_table add columns(decimalField decimal(6,2))")
+
+    sql("Alter table carbon_table compact 'minor'")
+
+    sql("DROP TABLE IF EXISTS carbon_table")
+  }
+
   override def afterAll {
     sql("DROP TABLE IF EXISTS addcolumntest")
     sql("drop table if exists hivetable")

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8ed7931b/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
index ebf3683..8c1f577 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
@@ -184,8 +184,7 @@ public class CompactionResultSortProcessor extends AbstractResultProcessor {
    *
    * @param resultIteratorList
    */
-  private void processResult(List<RawResultIterator> resultIteratorList)
-      throws Exception {
+  private void processResult(List<RawResultIterator> resultIteratorList) throws Exception {
     for (RawResultIterator resultIterator : resultIteratorList) {
       while (resultIterator.hasNext()) {
         addRowForSorting(prepareRowObjectForSorting(resultIterator.next()));
@@ -250,7 +249,9 @@ public class CompactionResultSortProcessor extends AbstractResultProcessor {
   private Object getConvertedMeasureValue(Object value, char aggType) {
     switch (aggType) {
       case CarbonCommonConstants.BIG_DECIMAL_MEASURE:
-        value = ((org.apache.spark.sql.types.Decimal) value).toJavaBigDecimal();
+        if (value != null) {
+          value = ((org.apache.spark.sql.types.Decimal) value).toJavaBigDecimal();
+        }
         return value;
       default:
         return value;


[21/50] incubator-carbondata git commit: result_size in query statistics is not giving valid row count if vector reader is enabled.

Posted by ra...@apache.org.
result_size in query statistics is not giving valid row count if vector reader is enabled.


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

Branch: refs/heads/branch-1.1
Commit: ec2d742f2e479e40883b92df014a8b260d50e526
Parents: f4fc651
Author: nareshpr <pr...@gmail.com>
Authored: Wed Apr 12 19:53:21 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Apr 13 16:14:09 2017 +0530

----------------------------------------------------------------------
 .../carbondata/hadoop/AbstractRecordReader.java | 45 ++++++++++++++++++++
 .../carbondata/hadoop/CarbonRecordReader.java   |  5 ++-
 .../carbondata/spark/rdd/CarbonScanRDD.scala    | 13 +-----
 .../VectorizedCarbonRecordReader.java           | 11 +++--
 4 files changed, 58 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ec2d742f/hadoop/src/main/java/org/apache/carbondata/hadoop/AbstractRecordReader.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/AbstractRecordReader.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/AbstractRecordReader.java
new file mode 100644
index 0000000..e571ccf
--- /dev/null
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/AbstractRecordReader.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.hadoop;
+
+import org.apache.carbondata.core.stats.QueryStatistic;
+import org.apache.carbondata.core.stats.QueryStatisticsConstants;
+import org.apache.carbondata.core.stats.QueryStatisticsRecorder;
+
+import org.apache.hadoop.mapreduce.RecordReader;
+
+/**
+ * This class will have all the common methods for vector and row based reader
+ */
+public abstract class AbstractRecordReader<T> extends RecordReader<Void, T> {
+
+  protected int rowCount = 0;
+
+  /**
+   * This method will log query result count and querytime
+   * @param recordCount
+   * @param recorder
+   */
+  public void logStatistics(int recordCount, QueryStatisticsRecorder recorder) {
+    // result size
+    QueryStatistic queryStatistic = new QueryStatistic();
+    queryStatistic.addCountStatistic(QueryStatisticsConstants.RESULT_SIZE, recordCount);
+    recorder.recordStatistics(queryStatistic);
+    // print executor query statistics for each task_id
+    recorder.logStatisticsAsTableExecutor();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ec2d742f/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonRecordReader.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonRecordReader.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonRecordReader.java
index 27c8b2f..26b269a 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonRecordReader.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonRecordReader.java
@@ -33,13 +33,12 @@ import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport;
 
 import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
 /**
  * Reads the data from Carbon store.
  */
-public class CarbonRecordReader<T> extends RecordReader<Void, T> {
+public class CarbonRecordReader<T> extends AbstractRecordReader<T> {
 
   private QueryModel queryModel;
 
@@ -92,6 +91,7 @@ public class CarbonRecordReader<T> extends RecordReader<Void, T> {
   }
 
   @Override public T getCurrentValue() throws IOException, InterruptedException {
+    rowCount += 1;
     return readSupport.readRow(carbonIterator.next());
   }
 
@@ -101,6 +101,7 @@ public class CarbonRecordReader<T> extends RecordReader<Void, T> {
   }
 
   @Override public void close() throws IOException {
+    logStatistics(rowCount, queryModel.getStatisticsRecorder());
     // clear dictionary cache
     Map<String, Dictionary> columnToDictionaryMapping = queryModel.getColumnToDictionaryMapping();
     if (null != columnToDictionaryMapping) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ec2d742f/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 ab0d603..4807b90 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
@@ -207,10 +207,9 @@ class CarbonScanRDD(
       new Iterator[Any] {
         private var havePair = false
         private var finished = false
-        private var count = 0
 
         context.addTaskCompletionListener { context =>
-          logStatistics(queryStartTime, count, model.getStatisticsRecorder)
+          logStatistics(queryStartTime, model.getStatisticsRecorder)
           reader.close()
         }
 
@@ -231,7 +230,6 @@ class CarbonScanRDD(
           }
           havePair = false
           val value = reader.getCurrentValue
-          count += 1
           value
         }
       }
@@ -265,18 +263,11 @@ class CarbonScanRDD(
     format
   }
 
-  def logStatistics(queryStartTime: Long, recordCount: Int,
-      recorder: QueryStatisticsRecorder): Unit = {
+  def logStatistics(queryStartTime: Long, recorder: QueryStatisticsRecorder): Unit = {
     var queryStatistic = new QueryStatistic()
     queryStatistic.addFixedTimeStatistic(QueryStatisticsConstants.EXECUTOR_PART,
       System.currentTimeMillis - queryStartTime)
     recorder.recordStatistics(queryStatistic)
-    // result size
-    queryStatistic = new QueryStatistic()
-    queryStatistic.addCountStatistic(QueryStatisticsConstants.RESULT_SIZE, recordCount)
-    recorder.recordStatistics(queryStatistic)
-    // print executor query statistics for each task_id
-    recorder.logStatisticsAsTableExecutor()
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ec2d742f/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java b/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
index ffff956..3fdf9af 100644
--- a/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
+++ b/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
@@ -38,12 +38,12 @@ import org.apache.carbondata.core.scan.result.iterator.AbstractDetailQueryResult
 import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
 import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
 import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.hadoop.AbstractRecordReader;
 import org.apache.carbondata.hadoop.CarbonInputSplit;
 import org.apache.carbondata.hadoop.CarbonMultiBlockSplit;
 import org.apache.carbondata.spark.util.CarbonScalaUtil;
 
 import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.spark.memory.MemoryMode;
 import org.apache.spark.sql.execution.vectorized.ColumnarBatch;
@@ -55,7 +55,7 @@ import org.apache.spark.sql.types.StructType;
  * A specialized RecordReader that reads into InternalRows or ColumnarBatches directly using the
  * carbondata column APIs and fills the data directly into columns.
  */
-class VectorizedCarbonRecordReader extends RecordReader<Void, Object> {
+class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
 
   private int batchIdx = 0;
 
@@ -116,6 +116,7 @@ class VectorizedCarbonRecordReader extends RecordReader<Void, Object> {
   }
 
   @Override public void close() throws IOException {
+    logStatistics(rowCount, queryModel.getStatisticsRecorder());
     if (columnarBatch != null) {
       columnarBatch.close();
       columnarBatch = null;
@@ -147,7 +148,11 @@ class VectorizedCarbonRecordReader extends RecordReader<Void, Object> {
   }
 
   @Override public Object getCurrentValue() throws IOException, InterruptedException {
-    if (returnColumnarBatch) return columnarBatch;
+    if (returnColumnarBatch) {
+      rowCount += columnarBatch.numValidRows();
+      return columnarBatch;
+    }
+    rowCount += 1;
     return columnarBatch.getRow(batchIdx - 1);
   }
 


[48/50] incubator-carbondata git commit: [CARBONDATA-926] Changed max columns from static value to configurable value in Load DDL.This closes #802

Posted by ra...@apache.org.
[CARBONDATA-926] Changed max columns from static value to configurable value in Load DDL.This closes #802


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

Branch: refs/heads/branch-1.1
Commit: 4397d05999393a43676b5cc56621268c85304e48
Parents: a8ed450 0a09472
Author: Venkata Ramana G <ra...@huawei.com>
Authored: Mon Apr 17 13:35:49 2017 +0530
Committer: Venkata Ramana G <ra...@huawei.com>
Committed: Mon Apr 17 13:35:49 2017 +0530

----------------------------------------------------------------------
 .../hadoop/test/util/StoreCreator.java          |  3 ++
 .../TestDataLoadWithColumnsMoreThanSchema.scala | 36 ++++++-------
 .../dataload/TestLoadDataWithHiveSyntax.scala   |  2 +-
 .../carbondata/spark/rdd/CarbonMergerRDD.scala  |  3 ++
 .../carbondata/spark/util/CommonUtil.scala      | 57 +++++++++++++++++++-
 .../execution/command/carbonTableSchema.scala   |  4 +-
 .../dataload/SparkDatasourceSuite.scala         |  1 -
 .../util/ExternalColumnDictionaryTestCase.scala |  1 +
 ...GlobalDictionaryUtilConcurrentTestCase.scala | 12 +++--
 .../util/GlobalDictionaryUtilTestCase.scala     |  1 +
 .../execution/command/carbonTableSchema.scala   |  5 +-
 .../processing/csvload/CSVInputFormat.java      | 25 ++++++++-
 .../processing/model/CarbonLoadModel.java       |  1 +
 .../newflow/CarbonDataLoadConfiguration.java    | 12 +++++
 .../carbondata/processing/StoreCreator.java     |  3 ++
 .../processing/csvload/CSVInputFormatTest.java  |  2 +
 16 files changed, 136 insertions(+), 32 deletions(-)
----------------------------------------------------------------------



[29/50] incubator-carbondata git commit: Problem: Is null query on a newly added measure column is not returning proper results.

Posted by ra...@apache.org.
Problem: Is null query on a newly added measure column is not returning proper results.

Analysis: When is null query is executed on newly added measure column, control goes to RowLevelFilterExecuterImpl class, where measure existence is checked. In case the measure is not found, bitset group is not getting populated with default values due to which that block is not returning any result.

Solution: When query is on a restructured block where newly added column does not exist, create the default bitset group so that based on default value existence default bitset group is created and results are returned based on that.


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

Branch: refs/heads/branch-1.1
Commit: 1e8d26c29d6025d083bde62a535b415993606219
Parents: 51f9629
Author: manishgupta88 <to...@gmail.com>
Authored: Tue Apr 11 16:54:03 2017 +0530
Committer: manishgupta88 <to...@gmail.com>
Committed: Thu Apr 13 23:26:52 2017 +0530

----------------------------------------------------------------------
 .../carbondata/core/scan/filter/FilterUtil.java | 10 +++--
 .../executer/RowLevelFilterExecuterImpl.java    |  8 ++++
 .../core/scan/filter/FilterUtilTest.java        | 12 +++++
 .../src/test/resources/restructure/data6.csv    |  3 ++
 .../vectorreader/AddColumnTestCases.scala       | 46 ++++++++++++++++++++
 5 files changed, 75 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1e8d26c2/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 770aa7e..7799b6a 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
@@ -1447,10 +1447,12 @@ public final class FilterUtil {
       bitSet.set(0, numberOfRows, defaultValue);
       bitSetGroup.setBitSet(bitSet, i);
     }
-    // create and fill bitset for the last page
-    BitSet bitSet = new BitSet(rowCountForLastPage);
-    bitSet.set(0, rowCountForLastPage, defaultValue);
-    bitSetGroup.setBitSet(bitSet, pagesTobeFullFilled);
+    // create and fill bitset for the last page if any records are left
+    if (rowCountForLastPage > 0) {
+      BitSet bitSet = new BitSet(rowCountForLastPage);
+      bitSet.set(0, rowCountForLastPage, defaultValue);
+      bitSetGroup.setBitSet(bitSet, pagesTobeFullFilled);
+    }
     return bitSetGroup;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1e8d26c2/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
index 10902ea..470de89 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
@@ -167,6 +167,10 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
             blockChunkHolder.getDimensionRawDataChunk()[dimensionBlocksIndex[0]].getPagesCount();
         numberOfRows =
             blockChunkHolder.getDimensionRawDataChunk()[dimensionBlocksIndex[0]].getRowCount();
+      } else {
+        // specific for restructure case where default values need to be filled
+        pageNumbers = blockChunkHolder.getDataBlock().numberOfPages();
+        numberOfRows = new int[] { blockChunkHolder.getDataBlock().nodeSize() };
       }
     }
     if (msrColEvalutorInfoList.size() > 0) {
@@ -175,6 +179,10 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
             blockChunkHolder.getMeasureRawDataChunk()[measureBlocksIndex[0]].getPagesCount();
         numberOfRows =
             blockChunkHolder.getMeasureRawDataChunk()[measureBlocksIndex[0]].getRowCount();
+      } else {
+        // specific for restructure case where default values need to be filled
+        pageNumbers = blockChunkHolder.getDataBlock().numberOfPages();
+        numberOfRows = new int[] { blockChunkHolder.getDataBlock().nodeSize() };
       }
     }
     BitSetGroup bitSetGroup = new BitSetGroup(pageNumbers);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1e8d26c2/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java b/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
index 06063a4..fee3d3d 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
@@ -38,6 +38,7 @@ import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.expression.LiteralExpression;
 import org.apache.carbondata.core.scan.expression.conditional.ListExpression;
 import org.apache.carbondata.core.scan.filter.intf.RowImpl;
+import org.apache.carbondata.core.util.BitSetGroup;
 
 import mockit.Mock;
 import mockit.MockUp;
@@ -387,4 +388,15 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
     SegmentProperties segmentProperties = new SegmentProperties(columnsInTable, columnCardinality);
     assertTrue(FilterUtil.prepareDefaultStartIndexKey(segmentProperties) instanceof IndexKey);
   }
+
+  @Test public void testCreateBitSetGroupWithDefaultValue() {
+    // test for exactly divisible values
+    BitSetGroup bitSetGroupWithDefaultValue =
+        FilterUtil.createBitSetGroupWithDefaultValue(14, 448000, true);
+    assertTrue(bitSetGroupWithDefaultValue.getNumberOfPages() == 14);
+    // test for remainder values
+    bitSetGroupWithDefaultValue =
+        FilterUtil.createBitSetGroupWithDefaultValue(15, 448200, true);
+    assertTrue(bitSetGroupWithDefaultValue.getNumberOfPages() == 15);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1e8d26c2/integration/spark-common-test/src/test/resources/restructure/data6.csv
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/resources/restructure/data6.csv b/integration/spark-common-test/src/test/resources/restructure/data6.csv
new file mode 100644
index 0000000..c7b4df2
--- /dev/null
+++ b/integration/spark-common-test/src/test/resources/restructure/data6.csv
@@ -0,0 +1,3 @@
+7,hello1
+8,welcome1
+bye,11
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1e8d26c2/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala
index 3b4a25c..1c7cc2a 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala
@@ -187,6 +187,25 @@ class AddColumnTestCases extends QueryTest with BeforeAndAfterAll {
     checkAnswer(sql("select distinct(CUST_NAME) from carbon_new"),Row("testuser"))
   }
 
+  test("test for checking newly added measure column for is null condition") {
+    sql("DROP TABLE IF EXISTS carbon_measure_is_null")
+    sql("CREATE TABLE carbon_measure_is_null (CUST_ID int,CUST_NAME String) STORED BY 'carbondata'")
+    sql(
+      s"LOAD DATA INPATH '$resourcesPath/restructure/data6.csv' into table carbon_measure_is_null" +
+      s" OPTIONS" +
+      s"('BAD_RECORDS_LOGGER_ENABLE'='TRUE', " +
+      s"'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME')")
+    sql("ALTER TABLE carbon_measure_is_null ADD COLUMNS (a6 int)")
+    sql(
+      s"LOAD DATA INPATH '$resourcesPath/restructure/data6.csv' into table carbon_measure_is_null" +
+      s" OPTIONS" +
+      s"('BAD_RECORDS_LOGGER_ENABLE'='TRUE', " +
+      s"'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,a6')")
+    checkAnswer(sql("select * from carbon_measure_is_null"),
+      sql("select * from carbon_measure_is_null where a6 is null"))
+    checkAnswer(sql("select count(*) from carbon_measure_is_null where a6 is not null"), Row(0))
+    sql("DROP TABLE IF EXISTS carbon_measure_is_null")
+  }
   test("test to check if intField returns correct result") {
     sql("DROP TABLE IF EXISTS carbon_table")
     sql("CREATE TABLE carbon_table(intField int,stringField string,charField string,timestampField timestamp, decimalField decimal(6,2)) STORED BY 'carbondata'")
@@ -232,6 +251,33 @@ class AddColumnTestCases extends QueryTest with BeforeAndAfterAll {
   }
 
 
+  test("test for checking newly added dictionary column for is null condition") {
+    sql("DROP TABLE IF EXISTS carbon_dictionary_is_null")
+    sql(
+      "CREATE TABLE carbon_dictionary_is_null (CUST_ID int,CUST_NAME String) STORED BY " +
+      "'carbondata'")
+    sql(
+      s"LOAD DATA INPATH '$resourcesPath/restructure/data6.csv' into table " +
+      s"carbon_dictionary_is_null" +
+      s" OPTIONS" +
+      s"('BAD_RECORDS_LOGGER_ENABLE'='TRUE', " +
+      s"'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME')")
+    sql(
+      "ALTER TABLE carbon_dictionary_is_null ADD COLUMNS (a6 int) tblproperties" +
+      "('dictionary_include'='a6')")
+    sql(
+      s"LOAD DATA INPATH '$resourcesPath/restructure/data6.csv' into table " +
+      s"carbon_dictionary_is_null" +
+      s" OPTIONS" +
+      s"('BAD_RECORDS_LOGGER_ENABLE'='TRUE', " +
+      s"'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,a6')")
+    checkAnswer(sql("select * from carbon_dictionary_is_null"),
+      sql("select * from carbon_dictionary_is_null where a6 is null"))
+    checkAnswer(sql("select count(*) from carbon_dictionary_is_null where a6 is not null"), Row(0))
+    sql("DROP TABLE IF EXISTS carbon_dictionary_is_null")
+  }
+
+
   override def afterAll {
     sql("DROP TABLE IF EXISTS addcolumntest")
     sql("drop table if exists hivetable")


[08/50] incubator-carbondata git commit: [CARBONDATA-901] Fix some spelling mistakes This closes #782

Posted by ra...@apache.org.
[CARBONDATA-901] Fix some spelling mistakes This closes #782


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

Branch: refs/heads/branch-1.1
Commit: aea9ed608f75454267561df30324cd4b850fd32b
Parents: 58dc3aa 81ccec2
Author: chenliang613 <ch...@huawei.com>
Authored: Wed Apr 12 00:44:28 2017 +0530
Committer: chenliang613 <ch...@huawei.com>
Committed: Wed Apr 12 00:44:28 2017 +0530

----------------------------------------------------------------------
 .../core/datastore/block/SegmentProperties.java | 24 ++++++++++----------
 1 file changed, 12 insertions(+), 12 deletions(-)
----------------------------------------------------------------------



[15/50] incubator-carbondata git commit: Removed redundant fields from table in configuration-parameters.md

Posted by ra...@apache.org.
Removed redundant fields from table in configuration-parameters.md


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

Branch: refs/heads/branch-1.1
Commit: d8590b16a094ea1afef84c85d61fb334e8b5730e
Parents: a5eb323
Author: jatin <ja...@knoldus.in>
Authored: Tue Apr 11 13:38:10 2017 +0530
Committer: chenliang613 <ch...@huawei.com>
Committed: Wed Apr 12 18:03:35 2017 +0530

----------------------------------------------------------------------
 docs/configuration-parameters.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d8590b16/docs/configuration-parameters.md
----------------------------------------------------------------------
diff --git a/docs/configuration-parameters.md b/docs/configuration-parameters.md
index 4f454e6..e4f8f33 100644
--- a/docs/configuration-parameters.md
+++ b/docs/configuration-parameters.md
@@ -131,7 +131,7 @@ This section provides the details of all the configurations required for CarbonD
 | Parameter | Default Value | Description |
 |---------------------------------------|---------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
 | high.cardinality.identify.enable | true | If the parameter is true, the high cardinality columns of the dictionary code are automatically recognized and these columns will not be used as global dictionary encoding. If the parameter is false, all dictionary encoding columns are used as dictionary encoding. The high cardinality column must meet the following requirements: value of cardinality > configured value of high.cardinalityEqually, the value of cardinality is higher than the threshold.value of cardinality/ row number x 100 > configured value of high.cardinality.row.count.percentageEqually, the ratio of the cardinality value to data row number is higher than the configured percentage. |
-| high.cardinality.threshold | 1000000 | high.cardinality.threshold | 1000000 | It is a threshold to identify high cardinality of the columns.If the value of columns' cardinality > the configured value, then the columns are excluded from dictionary encoding. |
+| high.cardinality.threshold | 1000000 | It is a threshold to identify high cardinality of the columns.If the value of columns' cardinality > the configured value, then the columns are excluded from dictionary encoding. |
 | high.cardinality.row.count.percentage | 80 | Percentage to identify whether column cardinality is more than configured percent of total row count.Configuration value formula:Value of cardinality/ row number x 100 > configured value of high.cardinality.row.count.percentageThe value of the parameter must be larger than 0. |
 | carbon.cutOffTimestamp | 1970-01-01 05:30:00 | 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". The date must be in the form "carbon.timestamp.format". NOTE: The CarbonData supports data store up to 68 years from the cut-off time defined. For example, if the cut-off time is 1970-01-01 05:30:00, then the data can be stored up to 2038-01-01 05:30:00. |
 | carbon.timegranularity | SECOND | The property used to set the data granularity level DAY, HOUR, MINUTE, or SECOND. |


[31/50] incubator-carbondata git commit: fixed NullPointerException for 2nd level subquery

Posted by ra...@apache.org.
fixed NullPointerException for 2nd level subquery


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

Branch: refs/heads/branch-1.1
Commit: fc0fc7d906c9f6579bc3fc67b5672cac4de55068
Parents: 27a1384
Author: kunal642 <ku...@knoldus.in>
Authored: Tue Apr 11 16:43:39 2017 +0530
Committer: Venkata Ramana G <ra...@huawei.com>
Committed: Fri Apr 14 12:54:09 2017 +0530

----------------------------------------------------------------------
 .../src/test/resources/temp/data1.csv           |  4 ++
 .../sql/optimizer/CarbonLateDecodeRule.scala    |  6 ++-
 .../carbondata/query/SubQueryTestSuite.scala    | 44 ++++++++++++++++++++
 3 files changed, 53 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/fc0fc7d9/integration/spark-common-test/src/test/resources/temp/data1.csv
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/resources/temp/data1.csv b/integration/spark-common-test/src/test/resources/temp/data1.csv
new file mode 100644
index 0000000..44e9c10
--- /dev/null
+++ b/integration/spark-common-test/src/test/resources/temp/data1.csv
@@ -0,0 +1,4 @@
+id,name,rating
+1,xyz,5
+2,ghj,2
+3,ghj,3

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/fc0fc7d9/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala
index 181328d..45cc330 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala
@@ -60,7 +60,11 @@ class CarbonLateDecodeRule extends Rule[LogicalPlan] with PredicateHelper {
     if (relations.nonEmpty && !isOptimized(plan)) {
       // In case scalar subquery skip the transformation and update the flag.
       if (relations.exists(_.carbonRelation.isSubquery.nonEmpty)) {
-        relations.foreach(p => p.carbonRelation.isSubquery.remove(0))
+        relations.foreach{carbonDecoderRelation =>
+          if (carbonDecoderRelation.carbonRelation.isSubquery.nonEmpty) {
+            carbonDecoderRelation.carbonRelation.isSubquery.remove(0)
+          }
+        }
         LOGGER.info("Skip CarbonOptimizer for scalar/predicate sub query")
         return plan
       }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/fc0fc7d9/integration/spark2/src/test/scala/org/apache/spark/carbondata/query/SubQueryTestSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/query/SubQueryTestSuite.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/query/SubQueryTestSuite.scala
new file mode 100644
index 0000000..fbc859f
--- /dev/null
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/query/SubQueryTestSuite.scala
@@ -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.carbondata.query
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.common.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+class SubQueryTestSuite extends QueryTest with BeforeAndAfterAll {
+
+  val tempDirPath = s"$resourcesPath/temp"
+
+  override def beforeAll(){
+    sql("drop table if exists subquery")
+    sql("create table subquery(id int, name string, rating float) stored by 'carbondata'")
+    sql(s"load data local inpath '$tempDirPath/data1.csv' into table subquery")
+  }
+
+  test("test to check if 2nd level subquery gives correct result") {
+    checkAnswer(sql(
+      "select * from subquery where id in(select id from subquery where name in(select name from" +
+      " subquery where rating=2.0))"),
+      Seq(Row(2,"ghj",2.0), Row(3,"ghj",3.0)))
+  }
+
+  override def afterAll() {
+    sql("drop table if exists subquery")
+  }
+}


[42/50] incubator-carbondata git commit: [CARBONDATA-930] Drop table IS throwing exception. This closes #801

Posted by ra...@apache.org.
[CARBONDATA-930] Drop table IS throwing exception. This closes #801


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

Branch: refs/heads/branch-1.1
Commit: f58853d2f7fe8a6f665d64c2ffb4d65d6e3e7f39
Parents: ab4556d 3d21ccb
Author: Venkata Ramana G <ra...@huawei.com>
Authored: Sat Apr 15 20:06:02 2017 +0530
Committer: Venkata Ramana G <ra...@huawei.com>
Committed: Sat Apr 15 20:06:02 2017 +0530

----------------------------------------------------------------------
 .../core/metadata/AbsoluteTableIdentifier.java  |  5 +-
 .../spark/sql/hive/CarbonHiveMetadataUtil.scala |  5 +-
 .../deletetable/DeleteTableTestCase.scala       | 65 ++++++++++++++++++++
 3 files changed, 71 insertions(+), 4 deletions(-)
----------------------------------------------------------------------



[50/50] incubator-carbondata git commit: [CARBONDATA-949] null value handled during compaction This closes #811

Posted by ra...@apache.org.
[CARBONDATA-949] null value handled during compaction This closes #811


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

Branch: refs/heads/branch-1.1
Commit: dbf76485f9964cce38118bc5a81d41050f9592d1
Parents: 4397d05 8ed7931
Author: ravipesala <ra...@gmail.com>
Authored: Tue Apr 18 13:43:03 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Tue Apr 18 13:43:03 2017 +0530

----------------------------------------------------------------------
 .../rowreader/AddColumnTestCases.scala          | 22 ++++++++++++++++++++
 .../merger/CompactionResultSortProcessor.java   |  7 ++++---
 2 files changed, 26 insertions(+), 3 deletions(-)
----------------------------------------------------------------------



[14/50] incubator-carbondata git commit: [CARBONDATA-907] The grammar for DELETE SEGMENT FOR DATE in website is not correct This closes #785

Posted by ra...@apache.org.
[CARBONDATA-907] The grammar for DELETE SEGMENT FOR DATE in website is not correct This closes #785


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

Branch: refs/heads/branch-1.1
Commit: a5eb3230fc4769f4cdaa0f3a985cef5dde2b9416
Parents: ff182b7 b5c9985
Author: chenliang613 <ch...@huawei.com>
Authored: Wed Apr 12 17:36:45 2017 +0530
Committer: chenliang613 <ch...@huawei.com>
Committed: Wed Apr 12 17:36:45 2017 +0530

----------------------------------------------------------------------
 docs/dml-operation-on-carbondata.md | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[13/50] incubator-carbondata git commit: Update dml-operation-on-carbondata.md

Posted by ra...@apache.org.
Update dml-operation-on-carbondata.md

Update dml-operation-on-carbondata.md


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

Branch: refs/heads/branch-1.1
Commit: b5c9985ec3e210b64c28eff1b7070be7f387fd7a
Parents: ff182b7
Author: chenerlu <ch...@huawei.com>
Authored: Wed Apr 12 10:03:29 2017 +0800
Committer: chenliang613 <ch...@huawei.com>
Committed: Wed Apr 12 17:36:08 2017 +0530

----------------------------------------------------------------------
 docs/dml-operation-on-carbondata.md | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/b5c9985e/docs/dml-operation-on-carbondata.md
----------------------------------------------------------------------
diff --git a/docs/dml-operation-on-carbondata.md b/docs/dml-operation-on-carbondata.md
index 6f67543..579b9cb 100644
--- a/docs/dml-operation-on-carbondata.md
+++ b/docs/dml-operation-on-carbondata.md
@@ -290,8 +290,8 @@ This command will allow to delete the CarbonData segment(s) from the store based
 The segment created before the particular date will be removed from the specific stores.
 
 ```
-DELETE FROM TABLE [schema_name.]table_name 
-WHERE[DATE_FIELD]BEFORE [DATE_VALUE]
+DELETE SEGMENTS FROM TABLE [db_name.]table_name 
+WHERE STARTTIME BEFORE DATE_VALUE
 ```
 
 ### Parameter Description


[23/50] incubator-carbondata git commit: NullPointer is getting thrown when rename table and select query is fired concurrently

Posted by ra...@apache.org.
NullPointer is getting thrown when rename table and select query is fired concurrently


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

Branch: refs/heads/branch-1.1
Commit: bcd28391a6052954994513b0583397bc9c50ae8d
Parents: f7d7e41
Author: nareshpr <pr...@gmail.com>
Authored: Wed Apr 5 12:36:45 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Apr 13 16:21:01 2017 +0530

----------------------------------------------------------------------
 .../cache/dictionary/AbstractDictionaryCache.java   | 16 ++++++++++------
 .../apache/carbondata/hadoop/util/SchemaReader.java |  2 +-
 2 files changed, 11 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/bcd28391/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
index cc4fc14..d7c25f1 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
@@ -126,15 +126,19 @@ public abstract class AbstractDictionaryCache<K extends DictionaryColumnUniqueId
    * @return
    */
   private CarbonFile getDictionaryMetaCarbonFile(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
+      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) throws IOException {
     PathService pathService = CarbonCommonFactory.getPathService();
     CarbonTablePath carbonTablePath = pathService.getCarbonTablePath(carbonStorePath,
-            dictionaryColumnUniqueIdentifier.getCarbonTableIdentifier());
-    String dictionaryFilePath =
-        carbonTablePath.getDictionaryMetaFilePath(dictionaryColumnUniqueIdentifier
-            .getColumnIdentifier().getColumnId());
+        dictionaryColumnUniqueIdentifier.getCarbonTableIdentifier());
+    String dictionaryFilePath = carbonTablePath.getDictionaryMetaFilePath(
+        dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId());
     FileFactory.FileType fileType = FileFactory.getFileType(dictionaryFilePath);
-    return FileFactory.getCarbonFile(dictionaryFilePath, fileType);
+    CarbonFile dictFile = FileFactory.getCarbonFile(dictionaryFilePath, fileType);
+    // When rename table triggered parallely with select query, dictionary files may not exist
+    if (!dictFile.exists()) {
+      throw new IOException("Dictionary file does not exist: " + dictionaryFilePath);
+    }
+    return dictFile;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/bcd28391/hadoop/src/main/java/org/apache/carbondata/hadoop/util/SchemaReader.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/SchemaReader.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/SchemaReader.java
index 022b3bb..f6c9e59 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/SchemaReader.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/SchemaReader.java
@@ -67,7 +67,7 @@ public class SchemaReader {
       return CarbonMetadata.getInstance().getCarbonTable(
           identifier.getCarbonTableIdentifier().getTableUniqueName());
     } else {
-      return null;
+      throw new IOException("File does not exist: " + schemaFilePath);
     }
   }
 }


[05/50] incubator-carbondata git commit: [CARBONDATA-660] Bad Records Logs and Raw CSVs should get display under segment id instead of Tasks id

Posted by ra...@apache.org.
[CARBONDATA-660] Bad Records Logs and Raw CSVs should get display under segment id instead of Tasks id


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

Branch: refs/heads/branch-1.1
Commit: 8d733f5a4ac92ae2ba0c15b15e73033670945578
Parents: 4cdb7a2
Author: mohammadshahidkhan <mo...@gmail.com>
Authored: Tue Apr 11 10:18:24 2017 +0530
Committer: jackylk <ja...@huawei.com>
Committed: Tue Apr 11 21:17:41 2017 +0800

----------------------------------------------------------------------
 .../steps/DataConverterProcessorWithBucketingStepImpl.java    | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8d733f5a/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataConverterProcessorWithBucketingStepImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataConverterProcessorWithBucketingStepImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataConverterProcessorWithBucketingStepImpl.java
index e7b3876..86971c3 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataConverterProcessorWithBucketingStepImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataConverterProcessorWithBucketingStepImpl.java
@@ -179,9 +179,10 @@ public class DataConverterProcessorWithBucketingStepImpl extends AbstractDataLoa
         configuration.getTableIdentifier().getCarbonTableIdentifier();
     BadRecordsLogger badRecordsLogger = new BadRecordsLogger(identifier.getBadRecordLoggerKey(),
         identifier.getTableName() + '_' + System.currentTimeMillis(), getBadLogStoreLocation(
-        identifier.getDatabaseName() + File.separator + identifier.getTableName() + File.separator
-            + configuration.getTaskNo()), badRecordsLogRedirect, badRecordsLoggerEnable,
-        badRecordConvertNullDisable, isDataLoadFail);
+        identifier.getDatabaseName() + CarbonCommonConstants.FILE_SEPARATOR + identifier
+            .getTableName() + CarbonCommonConstants.FILE_SEPARATOR + configuration.getSegmentId()
+            + CarbonCommonConstants.FILE_SEPARATOR + configuration.getTaskNo()),
+        badRecordsLogRedirect, badRecordsLoggerEnable, badRecordConvertNullDisable, isDataLoadFail);
     return badRecordsLogger;
   }
 


[06/50] incubator-carbondata git commit: [CARBONDATA-660] Bad Records Logs and Raw CSVs should get display under segment id instead of Tasks id This closes #775

Posted by ra...@apache.org.
[CARBONDATA-660] Bad Records Logs and Raw CSVs should get display under segment id instead of Tasks id  This closes #775


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

Branch: refs/heads/branch-1.1
Commit: 58dc3aa646c2ba470828a7cdd3f101f2f63239e4
Parents: 4cdb7a2 8d733f5
Author: jackylk <ja...@huawei.com>
Authored: Tue Apr 11 21:18:06 2017 +0800
Committer: jackylk <ja...@huawei.com>
Committed: Tue Apr 11 21:18:06 2017 +0800

----------------------------------------------------------------------
 .../steps/DataConverterProcessorWithBucketingStepImpl.java    | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------



[36/50] incubator-carbondata git commit: [CARBONDATA-928] Added a link to configuration parameters in doc This closes #798

Posted by ra...@apache.org.
[CARBONDATA-928] Added a link to configuration parameters in doc This closes #798


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

Branch: refs/heads/branch-1.1
Commit: 69ba22370eb46915bb2857793954a75827f68288
Parents: 19b9223 e1b1713
Author: chenliang613 <ch...@huawei.com>
Authored: Sat Apr 15 12:04:29 2017 +0800
Committer: chenliang613 <ch...@huawei.com>
Committed: Sat Apr 15 12:04:29 2017 +0800

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



[19/50] incubator-carbondata git commit: [CARBONDATA-903] data load is not failing even though bad records exists in the data in case of unsafe sort or batch sort

Posted by ra...@apache.org.
[CARBONDATA-903] data load is not failing even though bad records exists in the data in case of unsafe sort or batch sort


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

Branch: refs/heads/branch-1.1
Commit: 53accb35685fa959b5262a46518b6e9b0480439f
Parents: 9efcacd
Author: mohammadshahidkhan <mo...@gmail.com>
Authored: Tue Apr 11 18:26:51 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Apr 13 16:07:58 2017 +0530

----------------------------------------------------------------------
 .../DataLoadFailAllTypeSortTest.scala           | 218 +++++++++++++++++++
 .../newflow/sort/AbstractMergeSorter.java       |  43 ++++
 .../sort/impl/ParallelReadMergeSorterImpl.java  |  18 +-
 ...arallelReadMergeSorterWithBucketingImpl.java |  16 +-
 .../UnsafeBatchParallelReadMergeSorterImpl.java |  43 +++-
 .../impl/UnsafeParallelReadMergeSorterImpl.java |  19 +-
 .../UnsafeSingleThreadFinalSortFilesMerger.java |  10 +
 7 files changed, 333 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/53accb35/integration/spark2/src/test/scala/org/apache/spark/carbondata/DataLoadFailAllTypeSortTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/DataLoadFailAllTypeSortTest.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/DataLoadFailAllTypeSortTest.scala
new file mode 100644
index 0000000..478b4d3
--- /dev/null
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/DataLoadFailAllTypeSortTest.scala
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.carbondata
+
+import java.io.File
+
+import org.apache.spark.sql.common.util.QueryTest
+import org.apache.spark.sql.hive.HiveContext
+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 DataLoadFailAllTypeSortTest extends QueryTest with BeforeAndAfterAll {
+  var hiveContext: HiveContext = _
+
+  override def beforeAll: Unit = {
+    sql("drop table IF EXISTS data_pm")
+    sql("drop table IF EXISTS data_um")
+    sql("drop table IF EXISTS data_bm")
+    sql("drop table IF EXISTS data_bmf")
+    sql("drop table IF EXISTS data_tbm")
+  }
+
+  test("dataload with parallel merge with bad_records_action='FAIL'") {
+    try {
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC,
+          new File("./target/test/badRecords")
+            .getCanonicalPath)
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION, "FAIL");
+      sql("create table data_pm(name String, dob long, weight int) " +
+          "STORED BY 'org.apache.carbondata.format'")
+      val testData = s"$resourcesPath/badrecords/dummy.csv"
+      sql(s"""LOAD DATA LOCAL INPATH '$testData' INTO table data_pm""")
+
+
+    } catch {
+      case x: Throwable => {
+        assert(x.getMessage.contains("Data load failed due to bad record"))
+        CarbonProperties.getInstance()
+          .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
+      }
+    }
+    finally {
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION,
+          CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION_DEFAULT);
+    }
+  }
+
+  test("dataload with ENABLE_UNSAFE_SORT='true' with bad_records_action='FAIL'") {
+    try {
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC,
+          new File("./target/test/badRecords")
+            .getCanonicalPath)
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.ENABLE_UNSAFE_SORT, "true");
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION, "FAIL");
+      sql("create table data_um(name String, dob long, weight int) " +
+          "STORED BY 'org.apache.carbondata.format'")
+      val testData = s"$resourcesPath/badrecords/dummy.csv"
+      sql(s"""LOAD DATA LOCAL INPATH '$testData' INTO table data_um""")
+
+
+    } catch {
+      case x: Throwable => {
+        assert(x.getMessage.contains("Data load failed due to bad record"))
+        CarbonProperties.getInstance()
+          .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
+      }
+    }
+    finally {
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.ENABLE_UNSAFE_SORT, "false");
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION,
+          CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION_DEFAULT);
+    }
+  }
+
+  test("dataload with LOAD_USE_BATCH_SORT='true' with bad_records_action='FAIL'") {
+    try {
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC,
+          new File("./target/test/badRecords")
+            .getCanonicalPath)
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.LOAD_USE_BATCH_SORT, "true");
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION, "FAIL");
+      sql("create table data_bm(name String, dob long, weight int) " +
+          "STORED BY 'org.apache.carbondata.format'")
+      val testData = s"$resourcesPath/badrecords/dummy.csv"
+      sql(s"""LOAD DATA LOCAL INPATH '$testData' INTO table data_bm""")
+
+
+    } catch {
+      case x: Throwable => {
+        assert(x.getMessage.contains("Data load failed due to bad record"))
+        CarbonProperties.getInstance()
+          .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
+      }
+    }
+    finally {
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.LOAD_USE_BATCH_SORT, "false");
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION,
+          CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION_DEFAULT);
+    }
+  }
+
+  test("dataload with LOAD_USE_BATCH_SORT='true' with bad_records_action='FORCE'") {
+    try {
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC,
+          new File("./target/test/badRecords")
+            .getCanonicalPath)
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.LOAD_USE_BATCH_SORT, "true");
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION, "FORCE");
+      sql("create table data_bmf(name String, dob long, weight int) " +
+          "STORED BY 'org.apache.carbondata.format'")
+      val testData = s"$resourcesPath/badrecords/dummy.csv"
+      sql(s"""LOAD DATA LOCAL INPATH '$testData' INTO table data_bmf""")
+
+
+    } catch {
+      case x: Throwable => {
+        assert(false)
+        CarbonProperties.getInstance()
+          .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
+      }
+    }
+    finally {
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.LOAD_USE_BATCH_SORT, "false");
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION,
+          CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION_DEFAULT);
+    }
+  }
+
+  test("dataload with table bucketing with bad_records_action='FAIL'") {
+    try {
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC,
+          new File("./target/test/badRecords")
+            .getCanonicalPath)
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION, "FAIL");
+      sql("create table data_tbm(name String, dob long, weight int) " +
+          "USING org.apache.spark.sql.CarbonSource OPTIONS('bucketnumber'='4', " +
+          "'bucketcolumns'='name', 'tableName'='data_tbm')")
+      val testData = s"$resourcesPath/badrecords/dummy.csv"
+      sql(s"""LOAD DATA LOCAL INPATH '$testData' INTO table data_tbm""")
+
+
+    } catch {
+      case x: Throwable => {
+        assert(x.getMessage.contains("Data load failed due to bad record"))
+        CarbonProperties.getInstance()
+          .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
+      }
+    }
+    finally {
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION,
+          CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION_DEFAULT);
+    }
+  }
+
+  //
+  override def afterAll {
+    sql("drop table IF EXISTS data_pm")
+    sql("drop table IF EXISTS data_um")
+    sql("drop table IF EXISTS data_bm")
+    sql("drop table IF EXISTS data_bmf")
+    sql("drop table IF EXISTS data_tbm")
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
+  }
+}
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/53accb35/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ParallelReadMergeSorterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ParallelReadMergeSorterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ParallelReadMergeSorterImpl.java
index ad96578..856b6ac 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ParallelReadMergeSorterImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ParallelReadMergeSorterImpl.java
@@ -33,7 +33,7 @@ import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
 import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException;
 import org.apache.carbondata.processing.newflow.row.CarbonRow;
 import org.apache.carbondata.processing.newflow.row.CarbonRowBatch;
-import org.apache.carbondata.processing.newflow.sort.Sorter;
+import org.apache.carbondata.processing.newflow.sort.AbstractMergeSorter;
 import org.apache.carbondata.processing.sortandgroupby.exception.CarbonSortKeyAndGroupByException;
 import org.apache.carbondata.processing.sortandgroupby.sortdata.SortDataRows;
 import org.apache.carbondata.processing.sortandgroupby.sortdata.SortIntermediateFileMerger;
@@ -47,7 +47,7 @@ import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
  * First it sorts the data and write to temp files. These temp files will be merge sorted to get
  * final merge sort result.
  */
-public class ParallelReadMergeSorterImpl implements Sorter {
+public class ParallelReadMergeSorterImpl extends AbstractMergeSorter {
 
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(ParallelReadMergeSorterImpl.class.getName());
@@ -58,8 +58,6 @@ public class ParallelReadMergeSorterImpl implements Sorter {
 
   private ExecutorService executorService;
 
-  private ThreadStatusObserver threadStatusObserver;
-
   private SingleThreadFinalSortFilesMerger finalMerger;
 
   private AtomicLong rowCounter;
@@ -154,18 +152,6 @@ public class ParallelReadMergeSorterImpl implements Sorter {
   }
 
   /**
-   * Below method will be used to check error in exception
-   */
-  private void checkError() {
-    if (threadStatusObserver.getThrowable() != null) {
-      if (threadStatusObserver.getThrowable() instanceof CarbonDataLoadingException) {
-        throw (CarbonDataLoadingException) threadStatusObserver.getThrowable();
-      } else {
-        throw new CarbonDataLoadingException(threadStatusObserver.getThrowable());
-      }
-    }
-  }
-  /**
    * Below method will be used to process data to next step
    */
   private boolean processRowToNextStep(SortDataRows sortDataRows, SortParameters parameters)

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/53accb35/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ParallelReadMergeSorterWithBucketingImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ParallelReadMergeSorterWithBucketingImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ParallelReadMergeSorterWithBucketingImpl.java
index e3049d2..e5af1c6 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ParallelReadMergeSorterWithBucketingImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ParallelReadMergeSorterWithBucketingImpl.java
@@ -34,7 +34,7 @@ import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
 import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException;
 import org.apache.carbondata.processing.newflow.row.CarbonRow;
 import org.apache.carbondata.processing.newflow.row.CarbonRowBatch;
-import org.apache.carbondata.processing.newflow.sort.Sorter;
+import org.apache.carbondata.processing.newflow.sort.AbstractMergeSorter;
 import org.apache.carbondata.processing.sortandgroupby.exception.CarbonSortKeyAndGroupByException;
 import org.apache.carbondata.processing.sortandgroupby.sortdata.SortDataRows;
 import org.apache.carbondata.processing.sortandgroupby.sortdata.SortIntermediateFileMerger;
@@ -50,7 +50,7 @@ import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
  * This step is specifically for bucketing, it sorts each bucket data separately and write to
  * temp files.
  */
-public class ParallelReadMergeSorterWithBucketingImpl implements Sorter {
+public class ParallelReadMergeSorterWithBucketingImpl extends AbstractMergeSorter {
 
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(ParallelReadMergeSorterImpl.class.getName());
@@ -100,17 +100,21 @@ public class ParallelReadMergeSorterWithBucketingImpl implements Sorter {
       throw new CarbonDataLoadingException(e);
     }
     this.executorService = Executors.newFixedThreadPool(iterators.length);
+    this.threadStatusObserver = new ThreadStatusObserver(this.executorService);
     final int batchSize = CarbonProperties.getInstance().getBatchSize();
     try {
       for (int i = 0; i < iterators.length; i++) {
-        executorService.submit(new SortIteratorThread(iterators[i], sortDataRows, rowCounter));
+        executorService.submit(new SortIteratorThread(iterators[i], sortDataRows, rowCounter,
+            this.threadStatusObserver));
       }
       executorService.shutdown();
       executorService.awaitTermination(2, TimeUnit.DAYS);
       processRowToNextStep(sortDataRows, sortParameters);
     } catch (Exception e) {
+      checkError();
       throw new CarbonDataLoadingException("Problem while shutdown the server ", e);
     }
+    checkError();
     try {
       intermediateFileMerger.finish();
     } catch (CarbonDataWriterException e) {
@@ -197,11 +201,14 @@ public class ParallelReadMergeSorterWithBucketingImpl implements Sorter {
 
     private AtomicLong rowCounter;
 
+    private ThreadStatusObserver threadStatusObserver;
+
     public SortIteratorThread(Iterator<CarbonRowBatch> iterator, SortDataRows[] sortDataRows,
-        AtomicLong rowCounter) {
+        AtomicLong rowCounter, ThreadStatusObserver observer) {
       this.iterator = iterator;
       this.sortDataRows = sortDataRows;
       this.rowCounter = rowCounter;
+      this.threadStatusObserver = observer;
     }
 
     @Override public Void call() throws CarbonDataLoadingException {
@@ -222,6 +229,7 @@ public class ParallelReadMergeSorterWithBucketingImpl implements Sorter {
         }
       } catch (Exception e) {
         LOGGER.error(e);
+        this.threadStatusObserver.notifyFailed(e);
         throw new CarbonDataLoadingException(e);
       }
       return null;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/53accb35/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/UnsafeBatchParallelReadMergeSorterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/UnsafeBatchParallelReadMergeSorterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/UnsafeBatchParallelReadMergeSorterImpl.java
index f3a60fc..a54410c 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/UnsafeBatchParallelReadMergeSorterImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/UnsafeBatchParallelReadMergeSorterImpl.java
@@ -36,7 +36,7 @@ import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingExcep
 import org.apache.carbondata.processing.newflow.row.CarbonRow;
 import org.apache.carbondata.processing.newflow.row.CarbonRowBatch;
 import org.apache.carbondata.processing.newflow.row.CarbonSortBatch;
-import org.apache.carbondata.processing.newflow.sort.Sorter;
+import org.apache.carbondata.processing.newflow.sort.AbstractMergeSorter;
 import org.apache.carbondata.processing.newflow.sort.unsafe.UnsafeCarbonRowPage;
 import org.apache.carbondata.processing.newflow.sort.unsafe.UnsafeSortDataRows;
 import org.apache.carbondata.processing.newflow.sort.unsafe.merger.UnsafeIntermediateMerger;
@@ -49,7 +49,7 @@ import org.apache.carbondata.processing.store.writer.exception.CarbonDataWriterE
  * It parallely reads data from array of iterates and do merge sort.
  * It sorts data in batches and send to the next step.
  */
-public class UnsafeBatchParallelReadMergeSorterImpl implements Sorter {
+public class UnsafeBatchParallelReadMergeSorterImpl extends AbstractMergeSorter {
 
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(UnsafeBatchParallelReadMergeSorterImpl.class.getName());
@@ -72,18 +72,22 @@ public class UnsafeBatchParallelReadMergeSorterImpl implements Sorter {
   @Override public Iterator<CarbonRowBatch>[] sort(Iterator<CarbonRowBatch>[] iterators)
       throws CarbonDataLoadingException {
     this.executorService = Executors.newFixedThreadPool(iterators.length);
+    this.threadStatusObserver = new ThreadStatusObserver(this.executorService);
     int batchSize = CarbonProperties.getInstance().getBatchSize();
-    final SortBatchHolder sortBatchHolder = new SortBatchHolder(sortParameters, iterators.length);
+    final SortBatchHolder sortBatchHolder = new SortBatchHolder(sortParameters, iterators.length,
+        this.threadStatusObserver);
 
     try {
       for (int i = 0; i < iterators.length; i++) {
-        executorService
-            .submit(new SortIteratorThread(iterators[i], sortBatchHolder, batchSize, rowCounter));
+        executorService.submit(
+            new SortIteratorThread(iterators[i], sortBatchHolder, batchSize, rowCounter,
+                this.threadStatusObserver));
       }
     } catch (Exception e) {
+      checkError();
       throw new CarbonDataLoadingException("Problem while shutdown the server ", e);
     }
-
+    checkError();
     // Creates the iterator to read from merge sorter.
     Iterator<CarbonSortBatch> batchIterator = new CarbonIterator<CarbonSortBatch>() {
 
@@ -120,12 +124,15 @@ public class UnsafeBatchParallelReadMergeSorterImpl implements Sorter {
 
     private AtomicLong rowCounter;
 
+    private ThreadStatusObserver threadStatusObserver;
+
     public SortIteratorThread(Iterator<CarbonRowBatch> iterator, SortBatchHolder sortDataRows,
-        int batchSize, AtomicLong rowCounter) {
+        int batchSize, AtomicLong rowCounter, ThreadStatusObserver threadStatusObserver) {
       this.iterator = iterator;
       this.sortDataRows = sortDataRows;
       this.buffer = new Object[batchSize][];
       this.rowCounter = rowCounter;
+      this.threadStatusObserver = threadStatusObserver;
     }
 
     @Override public Void call() throws CarbonDataLoadingException {
@@ -152,6 +159,7 @@ public class UnsafeBatchParallelReadMergeSorterImpl implements Sorter {
         }
       } catch (Exception e) {
         LOGGER.error(e);
+        this.threadStatusObserver.notifyFailed(e);
         throw new CarbonDataLoadingException(e);
       } finally {
         sortDataRows.finishThread();
@@ -176,10 +184,14 @@ public class UnsafeBatchParallelReadMergeSorterImpl implements Sorter {
 
     private AtomicInteger iteratorCount;
 
-    public SortBatchHolder(SortParameters sortParameters, int numberOfThreads) {
+    private ThreadStatusObserver threadStatusObserver;
+
+    public SortBatchHolder(SortParameters sortParameters, int numberOfThreads,
+        ThreadStatusObserver threadStatusObserver) {
       this.sortParameters = sortParameters;
       this.iteratorCount = new AtomicInteger(numberOfThreads);
       this.mergerQueue = new LinkedBlockingQueue<>();
+      this.threadStatusObserver = threadStatusObserver;
       createSortDataRows();
     }
 
@@ -197,7 +209,12 @@ public class UnsafeBatchParallelReadMergeSorterImpl implements Sorter {
 
     @Override public UnsafeSingleThreadFinalSortFilesMerger next() {
       try {
-        return mergerQueue.take();
+        UnsafeSingleThreadFinalSortFilesMerger unsafeSingleThreadFinalSortFilesMerger =
+            mergerQueue.take();
+        if (unsafeSingleThreadFinalSortFilesMerger.isStopProcess()) {
+          throw new RuntimeException(threadStatusObserver.getThrowable());
+        }
+        return unsafeSingleThreadFinalSortFilesMerger;
       } catch (InterruptedException e) {
         throw new RuntimeException(e);
       }
@@ -209,6 +226,14 @@ public class UnsafeBatchParallelReadMergeSorterImpl implements Sorter {
 
     public void finish() {
       try {
+        // if the mergerQue is empty and some CarbonDataLoadingException exception has occurred
+        // then set stop process to true in the finalmerger instance
+        if (mergerQueue.isEmpty() && threadStatusObserver != null
+            && threadStatusObserver.getThrowable() != null && threadStatusObserver
+            .getThrowable() instanceof CarbonDataLoadingException) {
+          finalMerger.setStopProcess(true);
+          mergerQueue.offer(finalMerger);
+        }
         processRowToNextStep(sortDataRow, sortParameters);
         unsafeIntermediateFileMerger.finish();
         List<UnsafeCarbonRowPage> rowPages = unsafeIntermediateFileMerger.getRowPages();

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/53accb35/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/UnsafeParallelReadMergeSorterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/UnsafeParallelReadMergeSorterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/UnsafeParallelReadMergeSorterImpl.java
index 18cf314..0caafec 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/UnsafeParallelReadMergeSorterImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/UnsafeParallelReadMergeSorterImpl.java
@@ -34,7 +34,7 @@ import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
 import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException;
 import org.apache.carbondata.processing.newflow.row.CarbonRow;
 import org.apache.carbondata.processing.newflow.row.CarbonRowBatch;
-import org.apache.carbondata.processing.newflow.sort.Sorter;
+import org.apache.carbondata.processing.newflow.sort.AbstractMergeSorter;
 import org.apache.carbondata.processing.newflow.sort.unsafe.UnsafeCarbonRowPage;
 import org.apache.carbondata.processing.newflow.sort.unsafe.UnsafeSortDataRows;
 import org.apache.carbondata.processing.newflow.sort.unsafe.merger.UnsafeIntermediateMerger;
@@ -49,7 +49,7 @@ import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
  * First it sorts the data and write to temp files. These temp files will be merge sorted to get
  * final merge sort result.
  */
-public class UnsafeParallelReadMergeSorterImpl implements Sorter {
+public class UnsafeParallelReadMergeSorterImpl extends AbstractMergeSorter {
 
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(UnsafeParallelReadMergeSorterImpl.class.getName());
@@ -92,18 +92,22 @@ public class UnsafeParallelReadMergeSorterImpl implements Sorter {
       throw new CarbonDataLoadingException(e);
     }
     this.executorService = Executors.newFixedThreadPool(iterators.length);
+    this.threadStatusObserver = new ThreadStatusObserver(this.executorService);
 
     try {
       for (int i = 0; i < iterators.length; i++) {
-        executorService
-            .submit(new SortIteratorThread(iterators[i], sortDataRow, batchSize, rowCounter));
+        executorService.submit(
+            new SortIteratorThread(iterators[i], sortDataRow, batchSize, rowCounter,
+                this.threadStatusObserver));
       }
       executorService.shutdown();
       executorService.awaitTermination(2, TimeUnit.DAYS);
       processRowToNextStep(sortDataRow, sortParameters);
     } catch (Exception e) {
+      checkError();
       throw new CarbonDataLoadingException("Problem while shutdown the server ", e);
     }
+    checkError();
     try {
       unsafeIntermediateFileMerger.finish();
       List<UnsafeCarbonRowPage> rowPages = unsafeIntermediateFileMerger.getRowPages();
@@ -182,12 +186,16 @@ public class UnsafeParallelReadMergeSorterImpl implements Sorter {
 
     private AtomicLong rowCounter;
 
+    private ThreadStatusObserver threadStatusObserver;
+
     public SortIteratorThread(Iterator<CarbonRowBatch> iterator,
-        UnsafeSortDataRows sortDataRows, int batchSize, AtomicLong rowCounter) {
+        UnsafeSortDataRows sortDataRows, int batchSize, AtomicLong rowCounter,
+        ThreadStatusObserver threadStatusObserver) {
       this.iterator = iterator;
       this.sortDataRows = sortDataRows;
       this.buffer = new Object[batchSize][];
       this.rowCounter = rowCounter;
+      this.threadStatusObserver = threadStatusObserver;
     }
 
     @Override public Void call() throws CarbonDataLoadingException {
@@ -208,6 +216,7 @@ public class UnsafeParallelReadMergeSorterImpl implements Sorter {
         }
       } catch (Exception e) {
         LOGGER.error(e);
+        this.threadStatusObserver.notifyFailed(e);
         throw new CarbonDataLoadingException(e);
       }
       return null;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/53accb35/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
index b98a072..10c5191 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
@@ -80,6 +80,8 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
 
   private String tableName;
 
+  private boolean isStopProcess;
+
   public UnsafeSingleThreadFinalSortFilesMerger(SortParameters parameters) {
     this.parameters = parameters;
     // set measure and dimension count
@@ -305,4 +307,12 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
       recordHolderHeapLocal = null;
     }
   }
+
+  public boolean isStopProcess() {
+    return isStopProcess;
+  }
+
+  public void setStopProcess(boolean stopProcess) {
+    isStopProcess = stopProcess;
+  }
 }


[44/50] incubator-carbondata git commit: [CARBONDATA-931] BigDecimal and VariableLength Dimension fixes. This closes #803

Posted by ra...@apache.org.
[CARBONDATA-931] BigDecimal and VariableLength Dimension fixes. This closes #803


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

Branch: refs/heads/branch-1.1
Commit: c73e02f203403746a0cb8bbb943ca2e8e20dc1a9
Parents: f58853d 92352f3
Author: Venkata Ramana G <ra...@huawei.com>
Authored: Sat Apr 15 22:14:33 2017 +0530
Committer: Venkata Ramana G <ra...@huawei.com>
Committed: Sat Apr 15 22:14:33 2017 +0530

----------------------------------------------------------------------
 .../store/impl/unsafe/UnsafeBigDecimalMeasureChunkStore.java    | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)
----------------------------------------------------------------------



[10/50] incubator-carbondata git commit: [CARBONDATA-880] Path should not get printed in the explain extended DDL This closes #754

Posted by ra...@apache.org.
[CARBONDATA-880] Path should not get printed in the explain extended DDL This closes #754


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

Branch: refs/heads/branch-1.1
Commit: 3966f99276d9ba68b91ea16db65bb86d602c0b52
Parents: aea9ed6 591f9f6
Author: ravipesala <ra...@gmail.com>
Authored: Wed Apr 12 13:55:40 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Wed Apr 12 13:55:40 2017 +0530

----------------------------------------------------------------------
 .../org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala   | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------



[35/50] incubator-carbondata git commit: Added link to configuration parameters in main page of doc

Posted by ra...@apache.org.
Added link to configuration parameters in main page of doc


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

Branch: refs/heads/branch-1.1
Commit: e1b1713f9c08395e8818b4569bff95633fe8d841
Parents: 19b9223
Author: Sanoj MG <sa...@gmail.com>
Authored: Fri Apr 14 22:35:23 2017 +0400
Committer: Sanoj MG <sa...@gmail.com>
Committed: Fri Apr 14 22:35:23 2017 +0400

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


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e1b1713f/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index db1a1e2..fafbf7a 100644
--- a/README.md
+++ b/README.md
@@ -46,6 +46,7 @@ CarbonData is built using Apache Maven, to [build CarbonData](https://github.com
 * [DDL Operations on CarbonData](https://github.com/apache/incubator-carbondata/blob/master/docs/ddl-operation-on-carbondata.md) 
 * [DML Operations on CarbonData](https://github.com/apache/incubator-carbondata/blob/master/docs/dml-operation-on-carbondata.md)  
 * [Cluster Installation and Deployment](https://github.com/apache/incubator-carbondata/blob/master/docs/installation-guide.md)
+* [Configuring Carbondata](https://github.com/apache/incubator-carbondata/blob/master/docs/configuration-parameters.md)
 * [FAQ](https://github.com/apache/incubator-carbondata/blob/master/docs/faq.md)
 * [Trouble Shooting](https://github.com/apache/incubator-carbondata/blob/master/docs/troubleshooting.md)
 * [Useful Tips](https://github.com/apache/incubator-carbondata/blob/master/docs/useful-tips-on-carbondata.md)


[11/50] incubator-carbondata git commit: [CARBONDATA-890] For Spark 2.1 LRU cache size at driver is getting configured with the executor lru cache size.

Posted by ra...@apache.org.
[CARBONDATA-890] For Spark 2.1 LRU cache size at driver is getting configured with the executor lru cache size.


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

Branch: refs/heads/branch-1.1
Commit: 99582a762871c8a2c6cc9f5db43718d21e13de0c
Parents: 3966f99
Author: mohammadshahidkhan <mo...@gmail.com>
Authored: Tue Apr 11 09:59:45 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Wed Apr 12 14:10:34 2017 +0530

----------------------------------------------------------------------
 .../core/cache/CacheProviderTest.java           | 53 ++++++++++++++++++++
 .../scala/org/apache/spark/sql/CarbonEnv.scala  |  1 +
 2 files changed, 54 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/99582a76/core/src/test/java/org/apache/carbondata/core/cache/CacheProviderTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/CacheProviderTest.java b/core/src/test/java/org/apache/carbondata/core/cache/CacheProviderTest.java
index 1be5cb2..4c12259 100644
--- a/core/src/test/java/org/apache/carbondata/core/cache/CacheProviderTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/cache/CacheProviderTest.java
@@ -17,16 +17,26 @@
 
 package org.apache.carbondata.core.cache;
 
+import java.io.IOException;
+import java.lang.reflect.Field;
+
 import org.apache.carbondata.core.cache.dictionary.Dictionary;
 import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
 import org.apache.carbondata.core.cache.dictionary.ForwardDictionaryCache;
 import org.apache.carbondata.core.cache.dictionary.ReverseDictionaryCache;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.BlockIndexStore;
+import org.apache.carbondata.core.datastore.SegmentTaskIndexStore;
+import org.apache.carbondata.core.datastore.TableSegmentUniqueIdentifier;
+import org.apache.carbondata.core.datastore.block.SegmentTaskIndexWrapper;
+import org.apache.carbondata.core.datastore.block.TableBlockUniqueIdentifier;
 import org.apache.carbondata.core.util.CarbonProperties;
 
+import org.apache.avro.Schema;
 import org.junit.Before;
 import org.junit.Test;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
@@ -39,6 +49,9 @@ public class CacheProviderTest {
     // enable lru cache by setting cache size
     CarbonProperties.getInstance()
         .addProperty(CarbonCommonConstants.CARBON_MAX_DRIVER_LRU_CACHE_SIZE, "10");
+    // enable lru cache by setting cache size
+    CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_MAX_EXECUTOR_LRU_CACHE_SIZE, "20");
   }
 
   @Test public void getInstance() throws Exception {
@@ -62,4 +75,44 @@ public class CacheProviderTest {
     assertTrue(reverseDictionaryCache instanceof ReverseDictionaryCache);
     assertFalse(reverseDictionaryCache instanceof ForwardDictionaryCache);
   }
+
+  /**
+   * to test the driver and executor lru memory configuration
+   *
+   * @throws IOException
+   * @throws NoSuchFieldException
+   * @throws IllegalAccessException
+   */
+  @Test public void driverExecutorCacheConfTest()
+      throws IOException, NoSuchFieldException, IllegalAccessException {
+    // get cache provider instance
+    CacheProvider cacheProvider = CacheProvider.getInstance();
+    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.IS_DRIVER_INSTANCE, "true");
+    Cache<TableSegmentUniqueIdentifier, SegmentTaskIndexStore> driverCache =
+        cacheProvider.createCache(CacheType.DRIVER_BTREE, "carbonStore");
+    Field carbonLRUCacheField = SegmentTaskIndexStore.class.getDeclaredField("lruCache");
+    carbonLRUCacheField.setAccessible(true);
+    CarbonLRUCache carbonLRUCache = (CarbonLRUCache) carbonLRUCacheField.get(driverCache);
+    Field lruCacheMemorySizeField = CarbonLRUCache.class.getDeclaredField("lruCacheMemorySize");
+    lruCacheMemorySizeField.setAccessible(true);
+    long lruCacheMemorySize = (long) lruCacheMemorySizeField.get(carbonLRUCache);
+    String driverCacheSize = CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.CARBON_MAX_DRIVER_LRU_CACHE_SIZE);
+    assertEquals(1024 * 1024 * Integer.parseInt(driverCacheSize), lruCacheMemorySize);
+    // drop cache
+    cacheProvider.dropAllCache();
+    // validation test for the executor memory.
+    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.IS_DRIVER_INSTANCE, "false");
+    Cache<TableBlockUniqueIdentifier, BlockIndexStore> executorCache =
+        cacheProvider.createCache(CacheType.EXECUTOR_BTREE, "carbonStore");
+    carbonLRUCacheField = BlockIndexStore.class.getSuperclass().getDeclaredField("lruCache");
+    carbonLRUCacheField.setAccessible(true);
+    carbonLRUCache = (CarbonLRUCache) carbonLRUCacheField.get(executorCache);
+    lruCacheMemorySizeField = CarbonLRUCache.class.getDeclaredField("lruCacheMemorySize");
+    lruCacheMemorySizeField.setAccessible(true);
+    lruCacheMemorySize = (long) lruCacheMemorySizeField.get(carbonLRUCache);
+    String executorCacheSize = CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.CARBON_MAX_EXECUTOR_LRU_CACHE_SIZE);
+    assertEquals(1024 * 1024 * Integer.parseInt(executorCacheSize), lruCacheMemorySize);
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/99582a76/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
index 976a1b1..d426348 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
@@ -49,6 +49,7 @@ object CarbonEnv {
         LOGGER.info(s"carbon env initial: $storePath")
         new CarbonMetastore(sparkSession.conf, storePath)
       }
+      CarbonProperties.getInstance.addProperty(CarbonCommonConstants.IS_DRIVER_INSTANCE, "true")
       carbonEnv = CarbonEnv(catalog)
       initialized = true
     }


[26/50] incubator-carbondata git commit: [CARBONDATA-883] Fixed cast exception for integer data types in RestructureBasedVectorResultCollector. This closes #771

Posted by ra...@apache.org.
[CARBONDATA-883] Fixed cast exception for integer data types in RestructureBasedVectorResultCollector. This closes #771


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

Branch: refs/heads/branch-1.1
Commit: 144620d94845179169b54f587bf653db15ff8d60
Parents: 8478939 8c5c00c
Author: Venkata Ramana G <ra...@huawei.com>
Authored: Thu Apr 13 20:19:51 2017 +0530
Committer: Venkata Ramana G <ra...@huawei.com>
Committed: Thu Apr 13 20:19:51 2017 +0530

----------------------------------------------------------------------
 .../RestructureBasedVectorResultCollector.java  | 28 ++++++++---
 .../scan/executor/util/RestructureUtil.java     | 50 +++++++++++++++++++-
 .../scan/executor/util/RestructureUtilTest.java |  2 +-
 .../carbondata/spark/load/CarbonLoaderUtil.java |  2 +-
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala |  6 +--
 .../execution/command/AlterTableCommands.scala  |  4 --
 .../apache/spark/sql/hive/CarbonMetastore.scala |  8 ++--
 .../vectorreader/AddColumnTestCases.scala       | 44 +++++++++++++++++
 8 files changed, 123 insertions(+), 21 deletions(-)
----------------------------------------------------------------------



[22/50] incubator-carbondata git commit: [CARBONDATA-919]result_size in query statistics is not giving valid row count if vector reader is enabled. This closes #790

Posted by ra...@apache.org.
[CARBONDATA-919]result_size in query statistics is not giving valid row count if vector reader is enabled. This closes #790


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

Branch: refs/heads/branch-1.1
Commit: f7d7e41efd987d341b5a40085ec550d3ee265e23
Parents: f4fc651 ec2d742
Author: ravipesala <ra...@gmail.com>
Authored: Thu Apr 13 16:15:09 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Apr 13 16:15:09 2017 +0530

----------------------------------------------------------------------
 .../carbondata/hadoop/AbstractRecordReader.java | 45 ++++++++++++++++++++
 .../carbondata/hadoop/CarbonRecordReader.java   |  5 ++-
 .../carbondata/spark/rdd/CarbonScanRDD.scala    | 13 +-----
 .../VectorizedCarbonRecordReader.java           | 11 +++--
 4 files changed, 58 insertions(+), 16 deletions(-)
----------------------------------------------------------------------



[37/50] incubator-carbondata git commit: updation in file ddl-operation-on-carbondata.md

Posted by ra...@apache.org.
updation in file ddl-operation-on-carbondata.md


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

Branch: refs/heads/branch-1.1
Commit: ad738544b68ce2bbfce995c80ad1cfbbf705d6b3
Parents: 69ba223
Author: shivangi <sh...@knoldus.in>
Authored: Wed Apr 12 13:11:09 2017 +0530
Committer: chenliang613 <ch...@huawei.com>
Committed: Sat Apr 15 16:28:50 2017 +0800

----------------------------------------------------------------------
 docs/ddl-operation-on-carbondata.md | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ad738544/docs/ddl-operation-on-carbondata.md
----------------------------------------------------------------------
diff --git a/docs/ddl-operation-on-carbondata.md b/docs/ddl-operation-on-carbondata.md
index 2744691..d9d5718 100644
--- a/docs/ddl-operation-on-carbondata.md
+++ b/docs/ddl-operation-on-carbondata.md
@@ -68,8 +68,8 @@ The following DDL operations are supported in CarbonData :
 
        Column groups with more than one column are stored in row format, instead of columnar format. By default, each column is a separate column group.
 ```
-TBLPROPERTIES ('COLUMN_GROUPS'='(column1, column3),
-(Column4,Column5,Column6)')
+TBLPROPERTIES ('COLUMN_GROUPS'='(column1, column2),
+(Column3,Column4,Column5)')
 ```
 
    - **Table Block Size Configuration**
@@ -214,17 +214,17 @@ of columns is used.
 ```
  CREATE TABLE IF NOT EXISTS productSchema.productSalesTable (
                                 productNumber Int,
+                                saleQuantity Int,
                                 productName String,
                                 storeCity String,
                                 storeProvince String,
                                 productCategory String,
                                 productBatch String,
-                                saleQuantity Int,
                                 revenue Int)
    STORED BY 'carbondata'
-   TBLPROPERTIES ('COLUMN_GROUPS'='(productName,productNumber)',
+   TBLPROPERTIES ('COLUMN_GROUPS'='(productNumber,saleQuantity)',
                   'DICTIONARY_EXCLUDE'='productName',
-                  'DICTIONARY_INCLUDE'='productNumber',
+                  'DICTIONARY_INCLUDE'='productNumber,saleQuantity',
                   'NO_INVERTED_INDEX'='productBatch',
                   'BUCKETNUMBER'='4',
                   'BUCKETCOLUMNS'='productName')


[02/50] incubator-carbondata git commit: [CARBONDATA-887] lazy rdd iterator for InsertInto. This closes #765

Posted by ra...@apache.org.
[CARBONDATA-887] lazy rdd iterator for InsertInto. This closes #765


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

Branch: refs/heads/branch-1.1
Commit: 00535f4a9473d2879f4852d52fbcd5e07b695f52
Parents: d51387b e52e641
Author: Venkata Ramana G <ra...@huawei.com>
Authored: Tue Apr 11 14:11:31 2017 +0530
Committer: Venkata Ramana G <ra...@huawei.com>
Committed: Tue Apr 11 14:11:31 2017 +0530

----------------------------------------------------------------------
 .../spark/rdd/NewCarbonDataLoadRDD.scala        | 100 +++++++++++++++----
 1 file changed, 81 insertions(+), 19 deletions(-)
----------------------------------------------------------------------



[46/50] incubator-carbondata git commit: [CARBONDATA-932] Fixed variable length filter query with empty data. This closes #804

Posted by ra...@apache.org.
[CARBONDATA-932] Fixed variable length filter query with empty data. This closes #804


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

Branch: refs/heads/branch-1.1
Commit: a8ed450bf1546dd69891c26de4661e18008ac11c
Parents: c73e02f 0729379
Author: Venkata Ramana G <ra...@huawei.com>
Authored: Sat Apr 15 22:22:40 2017 +0530
Committer: Venkata Ramana G <ra...@huawei.com>
Committed: Sat Apr 15 22:22:40 2017 +0530

----------------------------------------------------------------------
 .../UnsafeVariableLengthDimesionDataChunkStore.java       | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------



[04/50] incubator-carbondata git commit: [CARBONDATA-861] Improvements in query This closes #709

Posted by ra...@apache.org.
[CARBONDATA-861] Improvements in query This closes #709


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

Branch: refs/heads/branch-1.1
Commit: 4cdb7a290a2ac5a53fdd66fab635803f281b3f35
Parents: 00535f4 8f59a32
Author: jackylk <ja...@huawei.com>
Authored: Tue Apr 11 21:02:25 2017 +0800
Committer: jackylk <ja...@huawei.com>
Committed: Tue Apr 11 21:02:25 2017 +0800

----------------------------------------------------------------------
 .../carbondata/core/datastore/FileHolder.java   |   4 +-
 .../impl/VariableLengthDimensionDataChunk.java  |  16 +-
 ...mpressedDimensionChunkFileBasedReaderV1.java |  14 +-
 ...mpressedDimensionChunkFileBasedReaderV2.java |  19 +--
 ...mpressedDimensionChunkFileBasedReaderV3.java |  16 +-
 ...CompressedMeasureChunkFileBasedReaderV1.java |  16 +-
 ...CompressedMeasureChunkFileBasedReaderV2.java |  19 +--
 ...CompressedMeasureChunkFileBasedReaderV3.java |  34 ++--
 .../chunk/store/DimensionDataChunkStore.java    |   9 ++
 .../SafeAbsractDimensionDataChunkStore.java     |   4 +
 ...feVariableLengthDimensionDataChunkStore.java |  36 ++++-
 .../UnsafeAbstractDimensionDataChunkStore.java  |   4 +
 ...afeVariableLengthDimesionDataChunkStore.java |  12 ++
 .../core/datastore/impl/DFSFileHolderImpl.java  |   8 +-
 .../core/datastore/impl/FileHolderImpl.java     |   8 +-
 .../DictionaryBasedVectorResultCollector.java   |   1 -
 .../executor/impl/AbstractQueryExecutor.java    |   2 +-
 .../core/scan/executor/util/QueryUtil.java      |   2 +-
 .../vector/MeasureDataVectorProcessor.java      | 158 +++++++++++++------
 .../apache/carbondata/core/util/ByteUtil.java   |  29 ++++
 .../apache/carbondata/core/util/CarbonUtil.java |  12 +-
 .../carbondata/examples/CompareTest.scala       |   4 +
 .../apache/carbondata/spark/CarbonOption.scala  |   2 +
 .../VectorizedCarbonRecordReader.java           |   2 +-
 .../spark/sql/CarbonCatalystOperators.scala     |   2 +-
 .../spark/sql/CarbonDataFrameWriter.scala       |  13 +-
 .../spark/sql/CarbonDictionaryDecoder.scala     |  68 ++++----
 .../execution/CarbonLateDecodeStrategy.scala    |   5 +-
 28 files changed, 328 insertions(+), 191 deletions(-)
----------------------------------------------------------------------



[30/50] incubator-carbondata git commit: [CARBONDATA-900] Is null query on a newly added measure column is not returning proper results. This closes #781

Posted by ra...@apache.org.
[CARBONDATA-900] Is null query on a newly added measure column is not returning proper results. This closes #781


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

Branch: refs/heads/branch-1.1
Commit: 27a1384385f5e857677bbe0b7ef0663851125efc
Parents: 51f9629 1e8d26c
Author: Venkata Ramana G <ra...@huawei.com>
Authored: Fri Apr 14 11:45:42 2017 +0530
Committer: Venkata Ramana G <ra...@huawei.com>
Committed: Fri Apr 14 11:45:42 2017 +0530

----------------------------------------------------------------------
 .../carbondata/core/scan/filter/FilterUtil.java | 10 +++--
 .../executer/RowLevelFilterExecuterImpl.java    |  8 ++++
 .../core/scan/filter/FilterUtilTest.java        | 12 +++++
 .../src/test/resources/restructure/data6.csv    |  3 ++
 .../vectorreader/AddColumnTestCases.scala       | 46 ++++++++++++++++++++
 5 files changed, 75 insertions(+), 4 deletions(-)
----------------------------------------------------------------------



[45/50] incubator-carbondata git commit: fixed variable length filter query with empty data

Posted by ra...@apache.org.
fixed variable length filter query with empty data


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

Branch: refs/heads/branch-1.1
Commit: 0729379a43d72d0e2abd2e15ab3ad5d224faedc6
Parents: c73e02f
Author: vpp9380 <vp...@gmail.com>
Authored: Sat Apr 15 17:01:48 2017 +0530
Committer: Venkata Ramana G <ra...@huawei.com>
Committed: Sat Apr 15 22:18:42 2017 +0530

----------------------------------------------------------------------
 .../UnsafeVariableLengthDimesionDataChunkStore.java       | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/0729379a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimesionDataChunkStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimesionDataChunkStore.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimesionDataChunkStore.java
index f5222fe..2b84449 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimesionDataChunkStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimesionDataChunkStore.java
@@ -171,7 +171,7 @@ public class UnsafeVariableLengthDimesionDataChunkStore
   /**
    * to compare the two byte array
    *
-   * @param index        index of first byte array
+   * @param index index of first byte array
    * @param compareValue value of to be compared
    * @return compare result
    */
@@ -199,20 +199,20 @@ public class UnsafeVariableLengthDimesionDataChunkStore
     }
     // as this class handles this variable length data, so filter value can be
     // smaller or bigger than than actual data, so we need to take the smaller length
-    int compareResult = 0;
-    int compareLength = length < compareValue.length ? length : compareValue.length;
+    int compareResult;
+    int compareLength = Math.min(length , compareValue.length);
     for (int i = 0; i < compareLength; i++) {
       compareResult = (CarbonUnsafe.unsafe.getByte(dataPageMemoryBlock.getBaseObject(),
           dataPageMemoryBlock.getBaseOffset() + currentDataOffset) & 0xff) - (compareValue[i]
           & 0xff);
       // if compare result is not equal we can break
       if (compareResult != 0) {
-        break;
+        return compareResult;
       }
       // increment the offset by one as comparison is done byte by byte
       currentDataOffset++;
     }
-    return compareResult;
+    return length - compareValue.length;
   }
 
 }


[33/50] incubator-carbondata git commit: added AlterTableAddColumnRDD to AlterTableCommands

Posted by ra...@apache.org.
added AlterTableAddColumnRDD to AlterTableCommands


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

Branch: refs/heads/branch-1.1
Commit: 36112fa46d2a15e6403af97a7d8677231bedc8d0
Parents: f27b491
Author: kunal642 <ku...@knoldus.in>
Authored: Tue Apr 11 14:15:10 2017 +0530
Committer: Venkata Ramana G <ra...@huawei.com>
Committed: Fri Apr 14 16:16:52 2017 +0530

----------------------------------------------------------------------
 .../spark/rdd/AlterTableAddColumnRDD.scala      |  6 +-
 .../execution/command/carbonTableSchema.scala   |  8 +--
 .../execution/command/AlterTableCommands.scala  | 62 ++++++++++++--------
 .../org/apache/spark/util/AlterTableUtil.scala  | 16 +++--
 .../restructure/AlterTableRevertTestCase.scala  | 19 +++++-
 5 files changed, 68 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/36112fa4/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala
index bb65b0b..ab1fd9c 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala
@@ -19,7 +19,6 @@ package org.apache.carbondata.spark.rdd
 
 import org.apache.spark.{Partition, SparkContext, TaskContext}
 import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.execution.command.AlterTableAddColumnsModel
 
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
@@ -49,13 +48,12 @@ class AddColumnPartition(rddId: Int, idx: Int, schema: ColumnSchema) extends Par
  */
 class AlterTableAddColumnRDD[K, V](sc: SparkContext,
     @transient newColumns: Seq[ColumnSchema],
-    alterTableModel: AlterTableAddColumnsModel,
     carbonTableIdentifier: CarbonTableIdentifier,
     carbonStorePath: String) extends RDD[(Int, String)](sc, Nil) {
 
   override def getPartitions: Array[Partition] = {
     newColumns.zipWithIndex.map { column =>
-      new DropColumnPartition(id, column._2, column._1)
+      new AddColumnPartition(id, column._2, column._1)
     }.toArray
   }
 
@@ -65,7 +63,7 @@ class AlterTableAddColumnRDD[K, V](sc: SparkContext,
     val status = CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS
     val iter = new Iterator[(Int, String)] {
       try {
-        val columnSchema = split.asInstanceOf[DropColumnPartition].columnSchema
+        val columnSchema = split.asInstanceOf[AddColumnPartition].columnSchema
         // create dictionary file if it is a dictionary column
         if (columnSchema.hasEncoding(Encoding.DICTIONARY) &&
             !columnSchema.hasEncoding(Encoding.DIRECT_DICTIONARY)) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/36112fa4/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
index 117b365..5108df8 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
@@ -152,7 +152,7 @@ case class AlterTableDropColumnModel(databaseName: Option[String],
     tableName: String,
     columns: List[String])
 
-class AlterTableProcessor(
+class AlterTableColumnSchemaGenerator(
     alterTableModel: AlterTableAddColumnsModel,
     dbName: String,
     tableInfo: TableInfo,
@@ -253,12 +253,6 @@ class AlterTableProcessor(
         }
       }
     }
-    // generate dictionary files for the newly added columns
-    new AlterTableAddColumnRDD(sc,
-      newCols,
-      alterTableModel,
-      tableIdentifier,
-      storePath).collect()
     tableSchema.setListOfColumns(allColumns.asJava)
     tableInfo.setLastUpdatedTime(System.currentTimeMillis())
     tableInfo.setFactTable(tableSchema)

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/36112fa4/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommands.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommands.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommands.scala
index e380217..8b194da 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommands.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommands.scala
@@ -28,17 +28,15 @@ import org.apache.spark.util.AlterTableUtil
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.locks.{CarbonLockFactory, LockUsage}
+import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
 import org.apache.carbondata.core.metadata.{CarbonMetadata, CarbonTableIdentifier}
 import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
 import org.apache.carbondata.core.metadata.encoder.Encoding
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn
-import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
 import org.apache.carbondata.core.util.CarbonUtil
 import org.apache.carbondata.core.util.path.CarbonStorePath
 import org.apache.carbondata.format.{ColumnSchema, SchemaEvolutionEntry, TableInfo}
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
-import org.apache.carbondata.spark.rdd.AlterTableDropColumnRDD
+import org.apache.carbondata.spark.rdd.{AlterTableAddColumnRDD, AlterTableDropColumnRDD}
 import org.apache.carbondata.spark.util.{CarbonScalaUtil, DataTypeConverterUtil}
 
 private[sql] case class AlterTableAddColumns(
@@ -52,13 +50,15 @@ private[sql] case class AlterTableAddColumns(
       .getOrElse(sparkSession.catalog.currentDatabase)
     LOGGER.audit(s"Alter table add columns request has been received for $dbName.$tableName")
     val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.COMPACTION_LOCK)
-    val locks = AlterTableUtil
-      .validateTableAndAcquireLock(dbName, tableName, locksToBeAcquired, LOGGER)(sparkSession)
-    // get the latest carbon table and check for column existence
-    val carbonTable = CarbonMetadata.getInstance.getCarbonTable(dbName + "_" + tableName)
+    var locks = List.empty[ICarbonLock]
+    var lastUpdatedTime = 0L
     var newCols = Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]()
-    val lastUpdatedTime = carbonTable.getTableLastUpdatedTime
+    val carbonTable = CarbonMetadata.getInstance.getCarbonTable(dbName + "_" + tableName)
     try {
+      locks = AlterTableUtil
+        .validateTableAndAcquireLock(dbName, tableName, locksToBeAcquired, LOGGER)(sparkSession)
+      // get the latest carbon table and check for column existence
+      lastUpdatedTime = carbonTable.getTableLastUpdatedTime
       // read the latest schema file
       val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
         carbonTable.getCarbonTableIdentifier)
@@ -71,12 +71,17 @@ private[sql] case class AlterTableAddColumns(
           dbName,
           tableName,
           carbonTable.getStorePath)
-      newCols = new AlterTableProcessor(alterTableAddColumnsModel,
+      newCols = new AlterTableColumnSchemaGenerator(alterTableAddColumnsModel,
         dbName,
         wrapperTableInfo,
         carbonTablePath,
         carbonTable.getCarbonTableIdentifier,
         carbonTable.getStorePath, sparkSession.sparkContext).process
+      // generate dictionary files for the newly added columns
+      new AlterTableAddColumnRDD(sparkSession.sparkContext,
+        newCols,
+        carbonTable.getCarbonTableIdentifier,
+        carbonTable.getStorePath).collect()
       val schemaEvolutionEntry = new org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry
       schemaEvolutionEntry.setTimeStamp(System.currentTimeMillis)
       schemaEvolutionEntry.setAdded(newCols.toList.asJava)
@@ -92,7 +97,7 @@ private[sql] case class AlterTableAddColumns(
     } catch {
       case e: Exception => LOGGER
         .error("Alter table add columns failed :" + e.getMessage)
-        if (!newCols.isEmpty) {
+        if (newCols.nonEmpty) {
           LOGGER.info("Cleaning up the dictionary files as alter table add operation failed")
           new AlterTableDropColumnRDD(sparkSession.sparkContext,
             newCols,
@@ -100,7 +105,7 @@ private[sql] case class AlterTableAddColumns(
             carbonTable.getStorePath).collect()
         }
         AlterTableUtil.revertAddColumnChanges(dbName, tableName, lastUpdatedTime)(sparkSession)
-        sys.error("Alter table add column operation failed. Please check the logs")
+        sys.error(s"Alter table add operation failed: ${e.getMessage}")
     } finally {
       // release lock after command execution completion
       AlterTableUtil.releaseLocks(locks, LOGGER)
@@ -147,12 +152,14 @@ private[sql] case class AlterTableRenameTable(alterTableRenameModel: AlterTableR
       LockUsage.DELETE_SEGMENT_LOCK,
       LockUsage.CLEAN_FILES_LOCK,
       LockUsage.DROP_TABLE_LOCK)
-    val locks = AlterTableUtil
-      .validateTableAndAcquireLock(oldDatabaseName, oldTableName, locksToBeAcquired, LOGGER)(
-        sparkSession)
+    var locks = List.empty[ICarbonLock]
+    var lastUpdatedTime = 0L
     val carbonTable = relation.tableMeta.carbonTable
-    val lastUpdatedTime = carbonTable.getTableLastUpdatedTime
     try {
+      locks = AlterTableUtil
+        .validateTableAndAcquireLock(oldDatabaseName, oldTableName, locksToBeAcquired, LOGGER)(
+          sparkSession)
+      lastUpdatedTime = carbonTable.getTableLastUpdatedTime
       // get the latest carbon table and check for column existence
       val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
         carbonTable.getCarbonTableIdentifier)
@@ -197,7 +204,7 @@ private[sql] case class AlterTableRenameTable(alterTableRenameModel: AlterTableR
         AlterTableUtil.revertRenameTableChanges(oldTableIdentifier, newTableName, lastUpdatedTime)(
             sparkSession)
         renameBadRecords(newTableName, oldTableName, oldDatabaseName)
-        sys.error("Alter table rename table operation failed. Please check the logs")
+        sys.error(s"Alter table rename table operation failed: ${e.getMessage}")
     } finally {
       // release lock after command execution completion
       AlterTableUtil.releaseLocks(locks, LOGGER)
@@ -242,13 +249,15 @@ private[sql] case class AlterTableDropColumns(
     val dbName = alterTableDropColumnModel.databaseName
       .getOrElse(sparkSession.catalog.currentDatabase)
     LOGGER.audit(s"Alter table drop columns request has been received for $dbName.$tableName")
+    var locks = List.empty[ICarbonLock]
+    var lastUpdatedTime = 0L
     val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.COMPACTION_LOCK)
-    val locks = AlterTableUtil
-      .validateTableAndAcquireLock(dbName, tableName, locksToBeAcquired, LOGGER)(sparkSession)
     // get the latest carbon table and check for column existence
     val carbonTable = CarbonMetadata.getInstance.getCarbonTable(dbName + "_" + tableName)
-    val lastUpdatedTime = carbonTable.getTableLastUpdatedTime
     try {
+      locks = AlterTableUtil
+        .validateTableAndAcquireLock(dbName, tableName, locksToBeAcquired, LOGGER)(sparkSession)
+      lastUpdatedTime = carbonTable.getTableLastUpdatedTime
       // check each column existence in the table
       val tableColumns = carbonTable.getCreateOrderColumn(tableName).asScala
       var dictionaryColumns = Seq[org.apache.carbondata.core.metadata.schema.table.column
@@ -318,7 +327,7 @@ private[sql] case class AlterTableDropColumns(
       case e: Exception => LOGGER
         .error("Alter table drop columns failed : " + e.getMessage)
         AlterTableUtil.revertDropColumnChanges(dbName, tableName, lastUpdatedTime)(sparkSession)
-        sys.error("Alter table drop column operation failed. Please check the logs")
+        sys.error(s"Alter table drop column operation failed: ${e.getMessage}")
     } finally {
       // release lock after command execution completion
       AlterTableUtil.releaseLocks(locks, LOGGER)
@@ -338,15 +347,16 @@ private[sql] case class AlterTableDataTypeChange(
       .getOrElse(sparkSession.catalog.currentDatabase)
     LOGGER.audit(s"Alter table change data type request has been received for $dbName.$tableName")
     val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.COMPACTION_LOCK)
-    val locks = AlterTableUtil
-      .validateTableAndAcquireLock(dbName, tableName, locksToBeAcquired, LOGGER)(sparkSession)
+    var locks = List.empty[ICarbonLock]
     // get the latest carbon table and check for column existence
     val carbonTable = CarbonMetadata.getInstance.getCarbonTable(dbName + "_" + tableName)
-    val lastUpdatedTime = carbonTable.getTableLastUpdatedTime
+    var lastUpdatedTime = 0L
     try {
+      locks = AlterTableUtil
+        .validateTableAndAcquireLock(dbName, tableName, locksToBeAcquired, LOGGER)(sparkSession)
+      lastUpdatedTime = carbonTable.getTableLastUpdatedTime
       val columnName = alterTableDataTypeChangeModel.columnName
       val carbonColumns = carbonTable.getCreateOrderColumn(tableName).asScala.filter(!_.isInvisible)
-
       if (!carbonColumns.exists(_.getColName.equalsIgnoreCase(columnName))) {
         LOGGER.audit(s"Alter table change data type request has failed. " +
                      s"Column $columnName does not exist")
@@ -397,7 +407,7 @@ private[sql] case class AlterTableDataTypeChange(
       case e: Exception => LOGGER
         .error("Alter table change datatype failed : " + e.getMessage)
         AlterTableUtil.revertDataTypeChanges(dbName, tableName, lastUpdatedTime)(sparkSession)
-        sys.error("Alter table data type change operation failed. Please check the logs")
+        sys.error(s"Alter table data type change operation failed: ${e.getMessage}")
     } finally {
       // release lock after command execution completion
       AlterTableUtil.releaseLocks(locks, LOGGER)

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/36112fa4/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
index 5057d75..f5248f5 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
@@ -64,11 +64,17 @@ object AlterTableUtil {
     }
     // acquire the lock first
     val table = relation.tableMeta.carbonTable
-    var acquiredLocks = ListBuffer[ICarbonLock]()
-    locksToBeAcquired.foreach { lock =>
-      acquiredLocks += getLockObject(table, lock, LOGGER)
+    val acquiredLocks = ListBuffer[ICarbonLock]()
+    try {
+      locksToBeAcquired.foreach { lock =>
+        acquiredLocks += getLockObject(table, lock, LOGGER)
+      }
+      acquiredLocks.toList
+    } catch {
+      case e: Exception =>
+        releaseLocks(acquiredLocks.toList, LOGGER)
+        throw e
     }
-    acquiredLocks.toList
   }
 
   /**
@@ -249,7 +255,7 @@ object AlterTableUtil {
     val evolutionEntryList = thriftTable.fact_table.schema_evolution.schema_evolution_history
     val updatedTime = evolutionEntryList.get(evolutionEntryList.size() - 1).time_stamp
     if (updatedTime > lastUpdatedTime) {
-      LOGGER.error(s"Reverting changes for $dbName.$tableName")
+      LOGGER.info(s"Reverting changes for $dbName.$tableName")
       val addedSchemas = evolutionEntryList.get(evolutionEntryList.size() - 1).added
       thriftTable.fact_table.table_columns.removeAll(addedSchemas)
       CarbonEnv.get.carbonMetastore.revertTableSchema(carbonTable.getCarbonTableIdentifier,

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/36112fa4/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala
index 05b79a8..c9244bc 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala
@@ -24,6 +24,7 @@ import org.apache.spark.sql.common.util.QueryTest
 import org.apache.spark.sql.test.TestQueryExecutor
 import org.scalatest.BeforeAndAfterAll
 
+import org.apache.carbondata.core.metadata.CarbonMetadata
 
 class AlterTableRevertTestCase extends QueryTest with BeforeAndAfterAll {
 
@@ -41,7 +42,7 @@ class AlterTableRevertTestCase extends QueryTest with BeforeAndAfterAll {
       hiveClient.runSqlHive("set hive.security.authorization.enabled=true")
       sql(
         "Alter table reverttest add columns(newField string) TBLPROPERTIES" +
-        "('DICTIONARY_EXCLUDE'='newField','DEFAULT.VALUE.charfield'='def')")
+        "('DEFAULT.VALUE.newField'='def')")
       hiveClient.runSqlHive("set hive.security.authorization.enabled=false")
       intercept[AnalysisException] {
         sql("select newField from reverttest")
@@ -78,6 +79,22 @@ class AlterTableRevertTestCase extends QueryTest with BeforeAndAfterAll {
       sql("select intfield from reverttest").schema.fields.apply(0).dataType.simpleString == "int")
   }
 
+  test("test to check if dictionary files are deleted for new column if query fails") {
+    intercept[RuntimeException] {
+      hiveClient.runSqlHive("set hive.security.authorization.enabled=true")
+      sql(
+        "Alter table reverttest add columns(newField string) TBLPROPERTIES" +
+        "('DEFAULT.VALUE.newField'='def')")
+      hiveClient.runSqlHive("set hive.security.authorization.enabled=false")
+      intercept[AnalysisException] {
+        sql("select newField from reverttest")
+      }
+      val carbonTable = CarbonMetadata.getInstance.getCarbonTable("default_reverttest")
+
+      assert(new File(carbonTable.getMetaDataFilepath).listFiles().length < 6)
+    }
+  }
+
   override def afterAll() {
     hiveClient.runSqlHive("set hive.security.authorization.enabled=false")
     sql("drop table if exists reverttest")


[40/50] incubator-carbondata git commit: [CARBONDATA-927] show segment in data management doc This closes #797

Posted by ra...@apache.org.
[CARBONDATA-927] show segment in data management doc This closes #797


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

Branch: refs/heads/branch-1.1
Commit: ab4556d7102fa5c09adb14869fe75c15f5b44925
Parents: afa7887 f2f757d
Author: chenliang613 <ch...@huawei.com>
Authored: Sat Apr 15 16:37:09 2017 +0800
Committer: chenliang613 <ch...@huawei.com>
Committed: Sat Apr 15 16:37:09 2017 +0800

----------------------------------------------------------------------
 docs/data-management.md | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------



[17/50] incubator-carbondata git commit: Updating proper schema update time stamp

Posted by ra...@apache.org.
Updating proper schema update time stamp


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

Branch: refs/heads/branch-1.1
Commit: ebc5ee973cf6e55194d9804a7a70b2011505339a
Parents: 11df451
Author: Manohar <ma...@gmail.com>
Authored: Wed Apr 12 16:22:26 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Apr 13 15:36:19 2017 +0530

----------------------------------------------------------------------
 .../MajorCompactionIgnoreInMinorTest.scala      | 53 +++++++++++++++++---
 .../spark/rdd/CarbonDataRDDFactory.scala        | 11 ++--
 .../store/CarbonFactDataHandlerModel.java       |  1 +
 3 files changed, 54 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ebc5ee97/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionIgnoreInMinorTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionIgnoreInMinorTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionIgnoreInMinorTest.scala
index a879598..9d2cf96 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionIgnoreInMinorTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionIgnoreInMinorTest.scala
@@ -35,6 +35,10 @@ import org.apache.carbondata.hadoop.CacheClient
   */
 class MajorCompactionIgnoreInMinorTest extends QueryTest with BeforeAndAfterAll {
 
+  val csvFilePath1 = s"$resourcesPath/compaction/compaction1.csv"
+  val csvFilePath2 = s"$resourcesPath/compaction/compaction2.csv"
+  val csvFilePath3 = s"$resourcesPath/compaction/compaction3.csv"
+
   override def beforeAll {
     CarbonProperties.getInstance().addProperty("carbon.compaction.level.threshold", "2,2")
     sql("drop table if exists  ignoremajor")
@@ -46,12 +50,6 @@ class MajorCompactionIgnoreInMinorTest extends QueryTest with BeforeAndAfterAll
         "phonetype String, serialname String, salary Int) STORED BY 'org.apache.carbondata" +
         ".format'"
     )
-
-
-    val csvFilePath1 = s"$resourcesPath/compaction/compaction1.csv"
-    val csvFilePath2 = s"$resourcesPath/compaction/compaction2.csv"
-    val csvFilePath3 = s"$resourcesPath/compaction/compaction3.csv"
-
     sql("LOAD DATA LOCAL INPATH '" + csvFilePath1 + "' INTO TABLE ignoremajor OPTIONS" +
       "('DELIMITER'= ',', 'QUOTECHAR'= '\"')"
     )
@@ -149,8 +147,51 @@ class MajorCompactionIgnoreInMinorTest extends QueryTest with BeforeAndAfterAll
 
   }
 
+  /**
+   * Execute two major compactions sequentially
+   */
+  test("Execute two major compactions sequentially") {
+    sql("drop table if exists testmajor")
+    sql(
+      "CREATE TABLE IF NOT EXISTS testmajor (country String, ID Int, date Timestamp, name " +
+      "String, " +
+      "phonetype String, serialname String, salary Int) STORED BY 'org.apache.carbondata" +
+      ".format'"
+    )
+    sql("LOAD DATA LOCAL INPATH '" + csvFilePath1 + "' INTO TABLE testmajor OPTIONS" +
+        "('DELIMITER'= ',', 'QUOTECHAR'= '\"')"
+    )
+    sql("LOAD DATA LOCAL INPATH '" + csvFilePath2 + "' INTO TABLE testmajor  OPTIONS" +
+        "('DELIMITER'= ',', 'QUOTECHAR'= '\"')"
+    )
+    // compaction will happen here.
+    sql("alter table testmajor compact 'major'")
+    sql("LOAD DATA LOCAL INPATH '" + csvFilePath1 + "' INTO TABLE testmajor OPTIONS" +
+        "('DELIMITER'= ',', 'QUOTECHAR'= '\"')"
+    )
+    sql("LOAD DATA LOCAL INPATH '" + csvFilePath2 + "' INTO TABLE testmajor  OPTIONS" +
+        "('DELIMITER'= ',', 'QUOTECHAR'= '\"')"
+    )
+    sql("alter table testmajor compact 'major'")
+    val identifier = new AbsoluteTableIdentifier(
+      CarbonProperties.getInstance.getProperty(CarbonCommonConstants.STORE_LOCATION),
+      new CarbonTableIdentifier(
+        CarbonCommonConstants.DATABASE_DEFAULT_NAME, "testmajor", "ttt")
+    )
+    val segmentStatusManager: SegmentStatusManager = new SegmentStatusManager(identifier)
+
+    // merged segment should not be there
+    val segments = segmentStatusManager.getValidAndInvalidSegments.getValidSegments.asScala.toList
+    assert(!segments.contains("0.1"))
+    assert(segments.contains("0.2"))
+    assert(!segments.contains("2"))
+    assert(!segments.contains("3"))
+
+  }
+
   override def afterAll {
     sql("drop table if exists  ignoremajor")
+    sql("drop table if exists  testmajor")
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ebc5ee97/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 4f33043..ca96a17 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
@@ -162,6 +162,7 @@ object CarbonDataRDDFactory {
           case e: Exception =>
             LOGGER.error(s"Exception in start compaction thread. ${ e.getMessage }")
             lock.unlock()
+            throw e
         }
       } else {
         LOGGER.audit("Not able to acquire the compaction lock for table " +
@@ -334,11 +335,11 @@ object CarbonDataRDDFactory {
                       .tablesMeta.toArray, skipCompactionTables.asJava
                   )
             }
-            // giving the user his error for telling in the beeline if his triggered table
-            // compaction is failed.
-            if (!triggeredCompactionStatus) {
-              throw new Exception("Exception in compaction " + exception.getMessage)
-            }
+          }
+          // giving the user his error for telling in the beeline if his triggered table
+          // compaction is failed.
+          if (!triggeredCompactionStatus) {
+            throw new Exception("Exception in compaction " + exception.getMessage)
           }
         } finally {
           executor.shutdownNow()

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ebc5ee97/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
index e64caea..15c75c1 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
@@ -308,6 +308,7 @@ public class CarbonFactDataHandlerModel {
       CarbonTable carbonTable, SegmentProperties segmentProperties, String tableName,
       String tempStoreLocation) {
     CarbonFactDataHandlerModel carbonFactDataHandlerModel = new CarbonFactDataHandlerModel();
+    carbonFactDataHandlerModel.setSchemaUpdatedTimeStamp(carbonTable.getTableLastUpdatedTime());
     carbonFactDataHandlerModel.setDatabaseName(loadModel.getDatabaseName());
     carbonFactDataHandlerModel.setTableName(tableName);
     carbonFactDataHandlerModel.setMeasureCount(segmentProperties.getMeasures().size());


[32/50] incubator-carbondata git commit: [CARBONDATA-892] Fixed NullPointerException for 2nd level subquery. This closes #780

Posted by ra...@apache.org.
[CARBONDATA-892] Fixed NullPointerException for 2nd level subquery. This closes #780


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

Branch: refs/heads/branch-1.1
Commit: f27b4918c8b9979aa69277cdc3cf73ab8b766b85
Parents: 27a1384 fc0fc7d
Author: Venkata Ramana G <ra...@huawei.com>
Authored: Fri Apr 14 12:58:29 2017 +0530
Committer: Venkata Ramana G <ra...@huawei.com>
Committed: Fri Apr 14 12:58:29 2017 +0530

----------------------------------------------------------------------
 .../src/test/resources/temp/data1.csv           |  4 ++
 .../sql/optimizer/CarbonLateDecodeRule.scala    |  6 ++-
 .../carbondata/query/SubQueryTestSuite.scala    | 44 ++++++++++++++++++++
 3 files changed, 53 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[38/50] incubator-carbondata git commit: [CARBONDATA-911] Updated ddl-operation-on-carbondata.md This closes #787

Posted by ra...@apache.org.
[CARBONDATA-911] Updated ddl-operation-on-carbondata.md This closes #787


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

Branch: refs/heads/branch-1.1
Commit: afa7887b853f150e81d75acca446a859feffc8d5
Parents: 69ba223 ad73854
Author: chenliang613 <ch...@huawei.com>
Authored: Sat Apr 15 16:29:40 2017 +0800
Committer: chenliang613 <ch...@huawei.com>
Committed: Sat Apr 15 16:29:40 2017 +0800

----------------------------------------------------------------------
 docs/ddl-operation-on-carbondata.md | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------