[21/50] [abbrv] carbondata git commit: [CARBONDATA-2025] Unify all path construction through CarbonTablePath static method

2018-03-02 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/9daad358/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/AggregateDataMapCompactor.scala
--
diff --git 
a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/AggregateDataMapCompactor.scala
 
b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/AggregateDataMapCompactor.scala
index c8a6b1d..184bf1b 100644
--- 
a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/AggregateDataMapCompactor.scala
+++ 
b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/AggregateDataMapCompactor.scala
@@ -28,7 +28,7 @@ import 
org.apache.spark.sql.execution.command.preaaggregate.PreAggregateUtil
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.statusmanager.{SegmentStatus, 
SegmentStatusManager}
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.OperationContext
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.processing.merger.{CarbonDataMergerUtil, 
CompactionType}
@@ -74,7 +74,7 @@ class AggregateDataMapCompactor(carbonLoadModel: 
CarbonLoadModel,
   "true")
 loadCommand.processData(sqlContext.sparkSession)
 val newLoadMetaDataDetails = SegmentStatusManager.readLoadMetadata(
-  carbonTable.getMetaDataFilepath, uuid)
+  carbonTable.getMetadataPath, uuid)
 val updatedLoadMetaDataDetails = newLoadMetaDataDetails collect {
   case load if loadMetaDataDetails.contains(load) =>
 load.setMergedLoadName(mergedLoadName)
@@ -83,11 +83,8 @@ class AggregateDataMapCompactor(carbonLoadModel: 
CarbonLoadModel,
 load
   case other => other
 }
-val carbonTablePath = CarbonStorePath
-  
.getCarbonTablePath(carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-.getAbsoluteTableIdentifier)
-SegmentStatusManager
-  
.writeLoadDetailsIntoFile(carbonTablePath.getTableStatusFilePathWithUUID(uuid),
+SegmentStatusManager.writeLoadDetailsIntoFile(
+  CarbonTablePath.getTableStatusFilePathWithUUID(uuid),
 updatedLoadMetaDataDetails)
 
carbonLoadModel.setLoadMetadataDetails(updatedLoadMetaDataDetails.toList.asJava)
   } finally {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/9daad358/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 6f08154..8d3110a 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
@@ -59,7 +59,7 @@ import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.scan.partition.PartitionUtil
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, 
SegmentStatus, SegmentStatusManager}
 import org.apache.carbondata.core.util.{ByteUtil, CarbonProperties, CarbonUtil}
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.{OperationContext, OperationListenerBus}
 import org.apache.carbondata.processing.exception.DataLoadingException
 import org.apache.carbondata.processing.loading.FailureCauses
@@ -72,7 +72,7 @@ import 
org.apache.carbondata.processing.merger.{CarbonCompactionUtil, CarbonData
 import org.apache.carbondata.processing.util.{CarbonDataProcessorUtil, 
CarbonLoaderUtil}
 import org.apache.carbondata.spark.{DataLoadResultImpl, PartitionFactory, _}
 import org.apache.carbondata.spark.load._
-import org.apache.carbondata.spark.util.{CarbonScalaUtil, CommonUtil, 
DataLoadingUtil, Util}
+import org.apache.carbondata.spark.util.{CarbonScalaUtil, CommonUtil, Util}
 
 /**
  * This is the factory class which can create different RDD depends on user 
needs.
@@ -127,7 +127,7 @@ object CarbonDataRDDFactory {
   LOGGER.error("Not able to acquire the compaction lock for table " +
   s"${ carbonLoadModel.getDatabaseName }.${ 
carbonLoadModel.getTableName }")
   CarbonCompactionUtil
-  .createCompactionRequiredFile(carbonTable.getMetaDataFilepath, 
compactionType)
+  .createCompactionRequiredFile(carbonTable.getMetadataPath, 
compactionType)
   // throw exception only in case of DDL trigger.
   if (compactionModel.i

[12/50] [abbrv] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

2018-03-02 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/636eb799/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
 
b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
index 447ab46..547ecaa 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
@@ -22,7 +22,7 @@ import java.util.List;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
@@ -35,12 +35,11 @@ import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.scan.expression.Expression;
-import 
org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import 
org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import 
org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -73,7 +72,7 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends 
RowLevelFilterExecut
   comparator = 
Comparator.getComparatorByDataTypeForMeasure(measure.getDataType());
 }
 ifDefaultValueMatchesFilter();
-if (isDimensionPresentInCurrentBlock[0] == true) {
+if (isDimensionPresentInCurrentBlock[0]) {
   isNaturalSorted = 
dimColEvaluatorInfoList.get(0).getDimension().isUseInvertedIndex()
   && dimColEvaluatorInfoList.get(0).getDimension().isSortColumn();
 }
@@ -120,11 +119,11 @@ public class RowLevelRangeLessThanFiterExecuterImpl 
extends RowLevelFilterExecut
 boolean isScanRequired = false;
 if (isMeasurePresentInCurrentBlock[0] || 
isDimensionPresentInCurrentBlock[0]) {
   if (isMeasurePresentInCurrentBlock[0]) {
-minValue = blockMinValue[measureBlocksIndex[0] + 
lastDimensionColOrdinal];
+minValue = blockMinValue[measureChunkIndex[0] + 
lastDimensionColOrdinal];
 isScanRequired =
 isScanRequired(minValue, msrFilterRangeValues, 
msrColEvalutorInfoList.get(0).getType());
   } else {
-minValue = blockMinValue[dimensionBlocksIndex[0]];
+minValue = blockMinValue[dimensionChunkIndex[0]];
 isScanRequired = isScanRequired(minValue, filterRangeValues);
   }
 } else {
@@ -170,67 +169,69 @@ public class RowLevelRangeLessThanFiterExecuterImpl 
extends RowLevelFilterExecut
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean 
useBitsetPipeLine)
-  throws FilterUnsupportedException, IOException {
+  public BitSetGroup applyFilter(RawBlockletColumnChunks 
rawBlockletColumnChunks,
+  boolean useBitsetPipeLine) throws IOException {
 // select all rows if dimension does not exists in the current block
 if (!isDimensionPresentInCurrentBlock[0] && 
!isMeasurePresentInCurrentBlock[0]) {
-  int numberOfRows = blockChunkHolder.getDataBlock().nodeSize();
+  int numberOfRows = rawBlockletColumnChunks.getDataBlock().numRows();
   return FilterUtil
-  
.createBitSetGroupWithDefaultValue(blockChunkHolder.getDataBlock().numberOfPages(),
+  
.createBitSetGroupWithDefaultValue(rawBlockletColumnChunks.getDataBlock().numberOfPages(),
   numberOfRows, true);
 }
 if (isDimensionPresentInCurrentBlock[0]) {
-  int blockIndex =
-  
segmentProperties.getDimensionOrdinalToBlockMapping().get(dimensionBlocksIndex[0]);
-  if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = 
blockChunkHolder.getDataBlock()
-.getDimensionC

[33/50] [abbrv] carbondata git commit: Revert "[CARBONDATA-2018][DataLoad] Optimization in reading/writing for sort temp row"

2018-03-02 Thread jackylk
Revert "[CARBONDATA-2018][DataLoad] Optimization in reading/writing for sort 
temp row"

This reverts commit de92ea9a123b17d903f2d1d4662299315c792954.


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

Branch: refs/heads/carbonstore-rebase5
Commit: 78aa2cc3d1e52bf940138538a1f8c0bdfac666bc
Parents: 22bb333
Author: Jacky Li 
Authored: Sat Feb 10 20:11:25 2018 +0800
Committer: Jacky Li 
Committed: Fri Mar 2 15:52:35 2018 +0800

--
 .../carbondata/core/util/NonDictionaryUtil.java |  67 ++-
 .../presto/util/CarbonDataStoreCreator.scala|   1 +
 .../load/DataLoadProcessorStepOnSpark.scala |   6 +-
 .../loading/row/IntermediateSortTempRow.java| 117 -
 .../loading/sort/SortStepRowHandler.java| 466 ---
 .../loading/sort/SortStepRowUtil.java   | 103 
 .../sort/unsafe/UnsafeCarbonRowPage.java| 331 +++--
 .../loading/sort/unsafe/UnsafeSortDataRows.java |  57 ++-
 .../unsafe/comparator/UnsafeRowComparator.java  |  95 ++--
 .../UnsafeRowComparatorForNormalDIms.java   |  59 +++
 .../UnsafeRowComparatorForNormalDims.java   |  59 ---
 .../sort/unsafe/holder/SortTempChunkHolder.java |   3 +-
 .../holder/UnsafeFinalMergePageHolder.java  |  19 +-
 .../unsafe/holder/UnsafeInmemoryHolder.java |  21 +-
 .../holder/UnsafeSortTempFileChunkHolder.java   | 138 --
 .../merger/UnsafeIntermediateFileMerger.java| 118 -
 .../UnsafeSingleThreadFinalSortFilesMerger.java |  27 +-
 .../merger/CompactionResultSortProcessor.java   |   1 +
 .../sort/sortdata/IntermediateFileMerger.java   |  95 +++-
 .../IntermediateSortTempRowComparator.java  |  73 ---
 .../sort/sortdata/NewRowComparator.java |   5 +-
 .../sortdata/NewRowComparatorForNormalDims.java |   3 +-
 .../processing/sort/sortdata/RowComparator.java |  94 
 .../sortdata/RowComparatorForNormalDims.java|  62 +++
 .../SingleThreadFinalSortFilesMerger.java   |  25 +-
 .../processing/sort/sortdata/SortDataRows.java  |  85 +++-
 .../sort/sortdata/SortTempFileChunkHolder.java  | 174 +--
 .../sort/sortdata/TableFieldStat.java   | 176 ---
 28 files changed, 1294 insertions(+), 1186 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/78aa2cc3/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java 
b/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
index fca1244..d6ecfbc 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
@@ -82,26 +82,18 @@ public class NonDictionaryUtil {
   }
 
   /**
-   * Method to get the required dictionary Dimension from obj []
+   * Method to get the required Dimension from obj []
*
* @param index
* @param row
* @return
*/
-  public static int getDictDimension(int index, Object[] row) {
-int[] dimensions = (int[]) row[WriteStepRowUtil.DICTIONARY_DIMENSION];
+  public static Integer getDimension(int index, Object[] row) {
+
+Integer[] dimensions = (Integer[]) 
row[WriteStepRowUtil.DICTIONARY_DIMENSION];
+
 return dimensions[index];
-  }
 
-  /**
-   * Method to get the required non-dictionary & complex from 3-parted row
-   * @param index
-   * @param row
-   * @return
-   */
-  public static byte[] getNoDictOrComplex(int index, Object[] row) {
-byte[][] nonDictArray = (byte[][]) 
row[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX];
-return nonDictArray[index];
   }
 
   /**
@@ -116,11 +108,60 @@ public class NonDictionaryUtil {
 return measures[index];
   }
 
+  public static byte[] getByteArrayForNoDictionaryCols(Object[] row) {
+
+return (byte[]) row[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX];
+  }
+
   public static void prepareOutObj(Object[] out, int[] dimArray, byte[][] 
byteBufferArr,
   Object[] measureArray) {
+
 out[WriteStepRowUtil.DICTIONARY_DIMENSION] = dimArray;
 out[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX] = byteBufferArr;
 out[WriteStepRowUtil.MEASURE] = measureArray;
 
   }
+
+  /**
+   * This method will extract the single dimension from the complete high card 
dims byte[].+ *
+   * The format of the byte [] will be,  Totallength,CompleteStartOffsets,Dat
+   *
+   * @param highCardArr
+   * @param index
+   * @param highCardinalityCount
+   * @param outBuffer
+   */
+  public static void extractSingleHighCardDims(byte[] highCardArr, int index,
+  int highCardin

[36/50] [abbrv] carbondata git commit: [HotFix][CheckStyle] Fix import related checkstyle

2018-03-02 Thread jackylk
[HotFix][CheckStyle] Fix import related checkstyle

This closes #1952


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

Branch: refs/heads/carbonstore-rebase5
Commit: 67ca605eaaf5705aebdb611c71f59bf77508ffb7
Parents: d099053
Author: xuchuanyin 
Authored: Thu Feb 8 15:39:45 2018 +0800
Committer: Jacky Li 
Committed: Fri Mar 2 15:52:35 2018 +0800

--
 .../core/indexstore/blockletindex/BlockletDataRefNode.java | 2 +-
 .../org/apache/carbondata/core/memory/HeapMemoryAllocator.java | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/67ca605e/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
 
b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
index b8fd6ff..50862a7 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
@@ -33,8 +33,8 @@ import 
org.apache.carbondata.core.datastore.chunk.reader.MeasureColumnChunkReade
 import org.apache.carbondata.core.indexstore.BlockletDetailInfo;
 import org.apache.carbondata.core.indexstore.FineGrainBlocklet;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
-import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.metadata.blocklet.index.BlockletIndex;
+import org.apache.carbondata.core.util.BitSetGroup;
 
 /**
  * wrapper for blocklet data map data

http://git-wip-us.apache.org/repos/asf/carbondata/blob/67ca605e/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java 
b/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
index 53cbb1d..242995b 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
@@ -17,11 +17,11 @@
 
 package org.apache.carbondata.core.memory;
 
-import javax.annotation.concurrent.GuardedBy;
 import java.lang.ref.WeakReference;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.Map;
+import javax.annotation.concurrent.GuardedBy;
 
 import org.apache.carbondata.core.util.CarbonProperties;
 



[35/50] [abbrv] carbondata git commit: [CARBONDATA-2018][DataLoad] Optimization in reading/writing for sort temp row

2018-03-02 Thread jackylk
[CARBONDATA-2018][DataLoad] Optimization in reading/writing for sort temp row

Pick up the no-sort fields in the row and pack them as bytes array and skip 
parsing them during merge sort to reduce CPU consumption

This closes #1792


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

Branch: refs/heads/carbonstore-rebase5
Commit: d115c479a6b4b859c834bd7a633da644e21e4abc
Parents: 67ca605
Author: xuchuanyin 
Authored: Thu Feb 8 14:35:14 2018 +0800
Committer: Jacky Li 
Committed: Fri Mar 2 15:52:35 2018 +0800

--
 .../carbondata/core/util/NonDictionaryUtil.java |  67 +--
 .../presto/util/CarbonDataStoreCreator.scala|   1 -
 .../load/DataLoadProcessorStepOnSpark.scala |   6 +-
 .../loading/row/IntermediateSortTempRow.java| 117 +
 .../loading/sort/SortStepRowHandler.java| 466 +++
 .../loading/sort/SortStepRowUtil.java   | 103 
 .../sort/unsafe/UnsafeCarbonRowPage.java| 331 ++---
 .../loading/sort/unsafe/UnsafeSortDataRows.java |  57 +--
 .../unsafe/comparator/UnsafeRowComparator.java  |  95 ++--
 .../UnsafeRowComparatorForNormalDIms.java   |  59 ---
 .../UnsafeRowComparatorForNormalDims.java   |  59 +++
 .../sort/unsafe/holder/SortTempChunkHolder.java |   3 +-
 .../holder/UnsafeFinalMergePageHolder.java  |  19 +-
 .../unsafe/holder/UnsafeInmemoryHolder.java |  21 +-
 .../holder/UnsafeSortTempFileChunkHolder.java   | 138 ++
 .../merger/UnsafeIntermediateFileMerger.java| 118 +
 .../UnsafeSingleThreadFinalSortFilesMerger.java |  27 +-
 .../merger/CompactionResultSortProcessor.java   |   1 -
 .../sort/sortdata/IntermediateFileMerger.java   |  95 +---
 .../IntermediateSortTempRowComparator.java  |  73 +++
 .../sort/sortdata/NewRowComparator.java |   5 +-
 .../sortdata/NewRowComparatorForNormalDims.java |   3 +-
 .../processing/sort/sortdata/RowComparator.java |  94 
 .../sortdata/RowComparatorForNormalDims.java|  62 ---
 .../SingleThreadFinalSortFilesMerger.java   |  25 +-
 .../processing/sort/sortdata/SortDataRows.java  |  85 +---
 .../sort/sortdata/SortTempFileChunkHolder.java  | 174 ++-
 .../sort/sortdata/TableFieldStat.java   | 176 +++
 28 files changed, 1186 insertions(+), 1294 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/d115c479/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java 
b/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
index d6ecfbc..fca1244 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
@@ -82,18 +82,26 @@ public class NonDictionaryUtil {
   }
 
   /**
-   * Method to get the required Dimension from obj []
+   * Method to get the required dictionary Dimension from obj []
*
* @param index
* @param row
* @return
*/
-  public static Integer getDimension(int index, Object[] row) {
-
-Integer[] dimensions = (Integer[]) 
row[WriteStepRowUtil.DICTIONARY_DIMENSION];
-
+  public static int getDictDimension(int index, Object[] row) {
+int[] dimensions = (int[]) row[WriteStepRowUtil.DICTIONARY_DIMENSION];
 return dimensions[index];
+  }
 
+  /**
+   * Method to get the required non-dictionary & complex from 3-parted row
+   * @param index
+   * @param row
+   * @return
+   */
+  public static byte[] getNoDictOrComplex(int index, Object[] row) {
+byte[][] nonDictArray = (byte[][]) 
row[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX];
+return nonDictArray[index];
   }
 
   /**
@@ -108,60 +116,11 @@ public class NonDictionaryUtil {
 return measures[index];
   }
 
-  public static byte[] getByteArrayForNoDictionaryCols(Object[] row) {
-
-return (byte[]) row[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX];
-  }
-
   public static void prepareOutObj(Object[] out, int[] dimArray, byte[][] 
byteBufferArr,
   Object[] measureArray) {
-
 out[WriteStepRowUtil.DICTIONARY_DIMENSION] = dimArray;
 out[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX] = byteBufferArr;
 out[WriteStepRowUtil.MEASURE] = measureArray;
 
   }
-
-  /**
-   * This method will extract the single dimension from the complete high card 
dims byte[].+ *
-   * The format of the byte [] will be,  Totallength,CompleteStartOffsets,Dat
-   *
-   * @param highCardArr
-   * @param index
-   * @param highCardinalityCount
-   * @param outBuffer
-   */
-  public static v

[19/50] [abbrv] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

2018-03-02 Thread jackylk
[CARBONDATA-2099] Refactor query scan process to improve readability

Unified concepts in scan process flow:

1.QueryModel contains all parameter for scan, it is created by API in 
CarbonTable. (In future, CarbonTable will be the entry point for various table 
operations)
2.Use term ColumnChunk to represent one column in one blocklet, and use 
ChunkIndex in reader to read specified column chunk
3.Use term ColumnPage to represent one page in one ColumnChunk
4.QueryColumn => ProjectionColumn, indicating it is for projection

This closes #1874


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

Branch: refs/heads/carbonstore-rebase5
Commit: 636eb799441ad977194574ad9a4d8ef552f28a38
Parents: 6944dd4
Author: Jacky Li 
Authored: Tue Jan 30 21:24:04 2018 +0800
Committer: Jacky Li 
Committed: Fri Mar 2 15:51:54 2018 +0800

--
 .../dictionary/AbstractDictionaryCache.java |   3 +-
 .../cache/dictionary/DictionaryCacheLoader.java |   7 +-
 .../dictionary/DictionaryCacheLoaderImpl.java   |  11 +-
 .../core/datastore/BTreeBuilderInfo.java|   6 -
 .../carbondata/core/datastore/DataRefNode.java  |  81 +--
 .../carbondata/core/datastore/FileHolder.java   | 118 
 .../carbondata/core/datastore/FileReader.java   | 114 +++
 .../core/datastore/block/SegmentProperties.java |  50 +-
 .../chunk/DimensionColumnDataChunk.java | 116 ---
 .../datastore/chunk/DimensionColumnPage.java| 111 +++
 .../chunk/impl/AbstractDimensionColumnPage.java |  89 +++
 .../chunk/impl/AbstractDimensionDataChunk.java  |  95 ---
 .../impl/ColumnGroupDimensionColumnPage.java| 194 ++
 .../impl/ColumnGroupDimensionDataChunk.java | 194 --
 .../chunk/impl/DimensionRawColumnChunk.java |  46 +-
 .../impl/FixedLengthDimensionColumnPage.java| 163 +
 .../impl/FixedLengthDimensionDataChunk.java | 163 -
 .../chunk/impl/MeasureRawColumnChunk.java   |  26 +-
 .../impl/VariableLengthDimensionColumnPage.java | 133 
 .../impl/VariableLengthDimensionDataChunk.java  | 140 
 .../reader/DimensionColumnChunkReader.java  |  14 +-
 .../chunk/reader/MeasureColumnChunkReader.java  |  12 +-
 .../AbstractChunkReaderV2V3Format.java  |  34 +-
 ...mpressedDimensionChunkFileBasedReaderV1.java |  38 +-
 ...mpressedDimensionChunkFileBasedReaderV2.java |  30 +-
 ...essedDimChunkFileBasedPageLevelReaderV3.java |  11 +-
 ...mpressedDimensionChunkFileBasedReaderV3.java |  49 +-
 .../AbstractMeasureChunkReaderV2V3Format.java   |  42 +-
 ...CompressedMeasureChunkFileBasedReaderV1.java |  16 +-
 ...CompressedMeasureChunkFileBasedReaderV2.java |  24 +-
 ...CompressedMeasureChunkFileBasedReaderV3.java |  45 +-
 ...essedMsrChunkFileBasedPageLevelReaderV3.java |   8 +-
 .../chunk/store/ColumnPageWrapper.java  |  30 +-
 .../chunk/store/DimensionDataChunkStore.java|   8 +-
 .../SafeFixedLengthDimensionDataChunkStore.java |   6 +-
 ...feVariableLengthDimensionDataChunkStore.java |   8 +-
 ...nsafeFixedLengthDimensionDataChunkStore.java |  10 +-
 ...afeVariableLengthDimesionDataChunkStore.java |  10 +-
 .../datastore/columnar/ColumnGroupModel.java|  26 -
 .../core/datastore/impl/DFSFileHolderImpl.java  | 166 -
 .../core/datastore/impl/DFSFileReaderImpl.java  | 155 
 .../datastore/impl/DefaultFileTypeProvider.java |  16 +-
 .../core/datastore/impl/FileFactory.java|   4 +-
 .../core/datastore/impl/FileHolderImpl.java | 224 --
 .../core/datastore/impl/FileReaderImpl.java | 215 ++
 .../core/datastore/impl/FileTypeInerface.java   |   4 +-
 .../impl/btree/AbstractBTreeLeafNode.java   |  60 +-
 .../impl/btree/BTreeDataRefNodeFinder.java  |   6 +-
 .../datastore/impl/btree/BTreeNonLeafNode.java  |  52 +-
 .../impl/btree/BlockBTreeLeafNode.java  |   6 +-
 .../impl/btree/BlockletBTreeLeafNode.java   |  46 +-
 .../page/encoding/EncodingFactory.java  |   8 +-
 .../server/NonSecureDictionaryServer.java   |   1 -
 .../core/indexstore/BlockletDetailInfo.java |   4 -
 .../blockletindex/BlockletDataRefNode.java  | 228 ++
 .../BlockletDataRefNodeWrapper.java | 241 ---
 .../indexstore/blockletindex/IndexWrapper.java  |   2 +-
 .../blockletindex/SegmentIndexFileStore.java|   7 +-
 .../core/memory/HeapMemoryAllocator.java|   2 +-
 .../core/metadata/blocklet/SegmentInfo.java |  19 -
 .../core/metadata/schema/table/CarbonTable.java | 130 +++-
 .../schema/table/RelationIdentifier.java|  16 -
 .../core/metadata/schema/table/TableInfo.java   |   6 +-
 .../schema/table/column/CarbonColumn.java   |   2 +-
 .../schema/table/column/CarbonDimension.java|  12 -
 .../core/mutate/CarbonUpdateUtil.java   |  20 +-
 .

[04/50] [abbrv] carbondata git commit: [CARBONDATA-1968] Add external table support

2018-03-02 Thread jackylk
[CARBONDATA-1968] Add external table support

This PR adds support for creating external table with existing carbondata 
files, using Hive syntax.
CREATE EXTERNAL TABLE tableName STORED BY 'carbondata' LOCATION 'path'

This closes #1749


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

Branch: refs/heads/carbonstore-rebase5
Commit: 2529e477032c623563087494060ecfdd34df5579
Parents: 2e28c15
Author: Jacky Li 
Authored: Tue Jan 2 23:46:14 2018 +0800
Committer: Jacky Li 
Committed: Fri Mar 2 15:51:53 2018 +0800

--
 .../core/metadata/schema/table/CarbonTable.java |  9 ++
 .../createTable/TestCreateExternalTable.scala   | 91 
 .../TestDataWithDicExcludeAndInclude.scala  | 10 ---
 .../command/table/CarbonDropTableCommand.scala  |  5 +-
 .../spark/sql/parser/CarbonSparkSqlParser.scala | 64 +-
 5 files changed, 147 insertions(+), 32 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/2529e477/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
 
b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
index 09ff440..6036569 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
@@ -785,6 +785,15 @@ public class CarbonTable implements Serializable {
 && !tableInfo.getParentRelationIdentifiers().isEmpty();
   }
 
+  /**
+   * Return true if this is an external table (table with property 
"_external"="true", this is
+   * an internal table property set during table creation)
+   */
+  public boolean isExternalTable() {
+String external = 
tableInfo.getFactTable().getTableProperties().get("_external");
+return external != null && external.equalsIgnoreCase("true");
+  }
+
   public long size() throws IOException {
 Map dataIndexSize = CarbonUtil.calculateDataIndexSize(this);
 Long dataSize = 
dataIndexSize.get(CarbonCommonConstants.CARBON_TOTAL_DATA_SIZE);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2529e477/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateExternalTable.scala
--
diff --git 
a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateExternalTable.scala
 
b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateExternalTable.scala
new file mode 100644
index 000..67370eb
--- /dev/null
+++ 
b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateExternalTable.scala
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.spark.testsuite.createTable
+
+import java.io.File
+
+import org.apache.spark.sql.{AnalysisException, CarbonEnv}
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+class TestCreateExternalTable extends QueryTest with BeforeAndAfterAll {
+
+  var originDataPath: String = _
+
+  override def beforeAll(): Unit = {
+sql("DROP TABLE IF EXISTS origin")
+// create carbon table and insert data
+sql("CREATE TABLE origin(key INT, value STRING) STORED BY 'carbondata'")
+sql("INSERT INTO origin select 100,'spark'")
+sql("INSERT INTO origin select 200,'hive'")
+originDataPath = s"$storeLocation/origin"
+  }
+
+  override def afterAll(): Unit = {
+sql("DROP TABLE IF EXISTS origin")
+  }
+
+  test("create external table with existing files") {
+assert(new

[06/50] [abbrv] carbondata git commit: [CARBONDATA-1992] Remove partitionId in CarbonTablePath

2018-03-02 Thread jackylk
[CARBONDATA-1992] Remove partitionId in CarbonTablePath

In CarbonTablePath, there is a deprecated partition id which is always 0, it 
should be removed to avoid confusion.

This closes #1765


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

Branch: refs/heads/carbonstore-rebase5
Commit: 2e28c15681066bad85dab0104307f927fb777a63
Parents: 7f7ea4d
Author: Jacky Li 
Authored: Sat Jan 6 20:28:44 2018 +0800
Committer: Jacky Li 
Committed: Fri Mar 2 15:51:53 2018 +0800

--
 .../core/metadata/PartitionMapFileStore.java|   0
 .../core/mutate/CarbonUpdateUtil.java   |   8 +-
 .../core/statusmanager/LoadMetadataDetails.java |   1 +
 .../SegmentUpdateStatusManager.java |   6 +-
 .../apache/carbondata/core/util/CarbonUtil.java |   6 +-
 .../core/util/path/CarbonTablePath.java |  55 ---
 .../CarbonFormatDirectoryStructureTest.java |   4 +-
 .../hadoop/api/CarbonTableInputFormat.java  |   2 +-
 .../streaming/CarbonStreamRecordWriter.java |   2 +-
 .../hadoop/test/util/StoreCreator.java  |   1 -
 .../presto/util/CarbonDataStoreCreator.scala|   1 -
 .../dataload/TestLoadDataGeneral.scala  |   2 +-
 .../InsertIntoCarbonTableTestCase.scala |   4 +-
 .../dataload/TestBatchSortDataLoad.scala|   3 +-
 .../dataload/TestDataLoadWithFileName.scala |   2 +-
 .../dataload/TestGlobalSortDataLoad.scala   |   4 +-
 .../testsuite/datamap/TestDataMapCommand.scala  |  34 ++--
 .../TestDataLoadingForPartitionTable.scala  |   3 +-
 .../load/DataLoadProcessBuilderOnSpark.scala|   1 -
 .../load/DataLoadProcessorStepOnSpark.scala |   2 +-
 .../spark/rdd/AlterTableLoadPartitionRDD.scala  | 154 +++
 .../carbondata/spark/rdd/CarbonMergerRDD.scala  |  11 +-
 .../spark/rdd/NewCarbonDataLoadRDD.scala|  25 ++-
 .../org/apache/spark/util/PartitionUtils.scala  |   5 +-
 .../spark/rdd/CarbonDataRDDFactory.scala|   5 +-
 .../datasources/CarbonFileFormat.scala  |   1 -
 .../partition/TestAlterPartitionTable.scala |   2 +-
 .../bucketing/TableBucketingTestCase.scala  |   2 +
 .../loading/CarbonDataLoadConfiguration.java|  10 --
 .../loading/DataLoadProcessBuilder.java |   1 -
 .../loading/TableProcessingOperations.java  |   3 +-
 .../sort/impl/ParallelReadMergeSorterImpl.java  |   4 +-
 ...arallelReadMergeSorterWithBucketingImpl.java |  15 +-
 .../UnsafeBatchParallelReadMergeSorterImpl.java |   7 +-
 ...arallelReadMergeSorterWithBucketingImpl.java |  21 ++-
 .../CarbonRowDataWriterProcessorStepImpl.java   |  33 ++--
 .../steps/DataWriterBatchProcessorStepImpl.java |  25 +--
 .../steps/DataWriterProcessorStepImpl.java  |  22 +--
 .../processing/merger/CarbonDataMergerUtil.java |   6 +-
 .../merger/CompactionResultSortProcessor.java   |   4 +-
 .../sort/sortdata/SortParameters.java   |  16 +-
 .../store/CarbonFactDataHandlerModel.java   |   3 +-
 .../util/CarbonDataProcessorUtil.java   |   9 +-
 .../processing/util/CarbonLoaderUtil.java   |  12 +-
 .../processing/util/DeleteLoadFolders.java  |   7 +-
 .../carbondata/processing/StoreCreator.java |   1 -
 .../carbondata/streaming/StreamHandoffRDD.scala |   1 -
 .../streaming/StreamSinkFactory.scala   |   2 +-
 .../streaming/CarbonAppendableStreamSink.scala  |   8 +-
 49 files changed, 274 insertions(+), 282 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/2e28c156/core/src/main/java/org/apache/carbondata/core/metadata/PartitionMapFileStore.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/metadata/PartitionMapFileStore.java
 
b/core/src/main/java/org/apache/carbondata/core/metadata/PartitionMapFileStore.java
new file mode 100644
index 000..e69de29

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2e28c156/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java 
b/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
index de98fa8..18eae11 100644
--- a/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
@@ -319,9 +319,7 @@ public class CarbonUpdateUtil {
 CarbonTablePath carbonTablePath = CarbonStorePath
 .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
 absoluteTableIdentifier.get

[44/50] [abbrv] carbondata git commit: [CARBONDATA-2159] Remove carbon-spark dependency in store-sdk module

2018-03-02 Thread jackylk
[CARBONDATA-2159] Remove carbon-spark dependency in store-sdk module

To make assembling JAR of store-sdk module, it should not depend on 
carbon-spark module

This closes #1970


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

Branch: refs/heads/carbonstore-rebase5
Commit: c5740b19d0e5e190e89b4ccf2e6275fd2ee0b812
Parents: bf4973c
Author: Jacky Li 
Authored: Sun Feb 11 21:37:04 2018 +0800
Committer: Jacky Li 
Committed: Fri Mar 2 15:52:36 2018 +0800

--
 .../java/org/apache/carbondata/common/Maps.java |  39 ++
 .../org/apache/carbondata/common/Strings.java   |   3 +
 .../exceptions/TableStatusLockException.java|  34 ++
 .../sql/InvalidLoadOptionException.java |  33 +
 .../sql/MalformedCarbonCommandException.java|  75 +++
 .../sql/MalformedDataMapCommandException.java   |  37 ++
 .../exceptions/sql/NoSuchDataMapException.java  |  39 ++
 .../carbondata/core/datamap/TableDataMap.java   |   5 +-
 .../exception/ConcurrentOperationException.java |  50 ++
 .../statusmanager/SegmentStatusManager.java | 124 
 .../carbondata/core/util/DeleteLoadFolders.java | 210 +++
 .../preaggregate/TestPreAggCreateCommand.scala  |   2 +-
 .../preaggregate/TestPreAggregateDrop.scala |   2 +-
 .../timeseries/TestTimeSeriesCreateTable.scala  |   2 +-
 .../timeseries/TestTimeSeriesDropSuite.scala|   2 +-
 .../TestTimeseriesTableSelection.scala  |   2 +-
 .../TestDataLoadWithColumnsMoreThanSchema.scala |   3 +-
 .../dataload/TestGlobalSortDataLoad.scala   |   2 +-
 .../TestLoadDataWithDiffTimestampFormat.scala   |   2 +-
 .../TestLoadDataWithFileHeaderException.scala   |  11 +-
 ...ataWithMalformedCarbonCommandException.scala |   3 +-
 .../testsuite/dataload/TestLoadOptions.scala|   2 +-
 .../dataload/TestTableLevelBlockSize.scala  |   4 +-
 .../testsuite/datamap/TestDataMapCommand.scala  |   2 +-
 .../dataretention/DataRetentionTestCase.scala   |   2 +-
 .../spark/testsuite/datetype/DateTypeTest.scala |   2 +-
 .../testsuite/sortcolumns/TestSortColumns.scala |   3 +-
 integration/spark-common/pom.xml|   5 -
 .../exception/ConcurrentOperationException.java |  38 --
 .../MalformedCarbonCommandException.java|  69 ---
 .../MalformedDataMapCommandException.java   |  32 -
 .../spark/exception/NoSuchDataMapException.java |  33 -
 .../org/apache/carbondata/api/CarbonStore.scala |   3 +-
 .../spark/CarbonColumnValidator.scala   |   8 +-
 .../carbondata/spark/load/ValidateUtil.scala|  72 ---
 .../carbondata/spark/rdd/CarbonMergerRDD.scala  |   6 +-
 .../carbondata/spark/util/CommonUtil.scala  |  70 +--
 .../carbondata/spark/util/DataLoadingUtil.scala | 610 ---
 .../spark/util/GlobalDictionaryUtil.scala   |   2 +-
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala |   2 +-
 .../spark/rdd/CarbonDataRDDFactory.scala|   4 +-
 .../spark/rdd/CarbonTableCompactor.scala|   2 +-
 .../org/apache/spark/sql/CarbonSource.scala |   2 +-
 .../datamap/CarbonCreateDataMapCommand.scala|   2 +-
 .../datamap/CarbonDropDataMapCommand.scala  |   2 +-
 .../CarbonAlterTableCompactionCommand.scala |  13 +-
 .../management/CarbonLoadDataCommand.scala  |  17 +-
 .../CarbonProjectForDeleteCommand.scala |   2 +-
 .../CarbonProjectForUpdateCommand.scala |   2 +-
 .../command/mutation/IUDCommonUtil.scala|   2 +-
 .../CreatePreAggregateTableCommand.scala|   7 +-
 .../preaaggregate/PreAggregateUtil.scala|   2 +-
 .../schema/CarbonAlterTableRenameCommand.scala  |   3 +-
 .../command/timeseries/TimeSeriesUtil.scala |   2 +-
 .../datasources/CarbonFileFormat.scala  |  14 +-
 .../sql/execution/strategy/DDLStrategy.scala|   2 +-
 .../strategy/StreamingTableStrategy.scala   |   2 +-
 .../execution/command/CarbonHiveCommands.scala  |   2 +-
 .../sql/parser/CarbonSpark2SqlParser.scala  |   2 +-
 .../spark/sql/parser/CarbonSparkSqlParser.scala |   2 +-
 .../org/apache/spark/util/AlterTableUtil.scala  |   2 +-
 .../org/apache/spark/util/TableAPIUtil.scala|   2 +-
 .../spark/sql/hive/CarbonSessionState.scala |   3 +-
 .../segmentreading/TestSegmentReading.scala |   2 +-
 .../spark/util/AllDictionaryTestCase.scala  |   4 +-
 .../util/ExternalColumnDictionaryTestCase.scala |   6 +-
 .../TestStreamingTableOperation.scala   |   4 +-
 .../bucketing/TableBucketingTestCase.scala  |   2 +-
 .../vectorreader/AddColumnTestCases.scala   |   2 +-
 .../loading/model/CarbonLoadModel.java  |  14 +-
 .../loading/model/CarbonLoadModelBuilder.java   | 322 ++
 .../processing/loading/model/LoadOption.java| 251 
 .../processing/util/CarbonLo

[45/50] [abbrv] carbondata git commit: Support generating assembling JAR for store-sdk module

2018-03-02 Thread jackylk
Support generating assembling JAR for store-sdk module

Support generating assembling JAR for store-sdk module and remove junit 
dependency

This closes #1976


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

Branch: refs/heads/carbonstore-rebase5
Commit: d85215a134a2be2048668cc4a736871b73a6487d
Parents: fbe9778
Author: Jacky Li 
Authored: Tue Feb 13 09:12:09 2018 +0800
Committer: Jacky Li 
Committed: Fri Mar 2 15:52:36 2018 +0800

--
 common/pom.xml|  2 +
 core/pom.xml  |  2 +
 hadoop/pom.xml|  1 +
 integration/presto/pom.xml|  3 +-
 integration/spark-common-cluster-test/pom.xml |  2 +-
 integration/spark-common-test/pom.xml |  3 +-
 integration/spark-common/pom.xml  |  2 +-
 integration/spark2/pom.xml|  2 +-
 pom.xml   |  5 +++
 processing/pom.xml|  1 +
 store/sdk/pom.xml | 50 +-
 streaming/pom.xml |  1 -
 12 files changed, 66 insertions(+), 8 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/d85215a1/common/pom.xml
--
diff --git a/common/pom.xml b/common/pom.xml
index 5550129..433d575 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -42,10 +42,12 @@
 
   junit
   junit
+  test
 
 
   org.jmockit
   jmockit
+  test
 
 
   org.apache.hadoop

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d85215a1/core/pom.xml
--
diff --git a/core/pom.xml b/core/pom.xml
index 92c9607..824de0d 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -70,10 +70,12 @@
 
   org.jmockit
   jmockit
+  test
 
 
   junit
   junit
+  test
 
 
   org.apache.spark

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d85215a1/hadoop/pom.xml
--
diff --git a/hadoop/pom.xml b/hadoop/pom.xml
index 2aaac99..c3964c5 100644
--- a/hadoop/pom.xml
+++ b/hadoop/pom.xml
@@ -42,6 +42,7 @@
 
   junit
   junit
+  test
 
   
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d85215a1/integration/presto/pom.xml
--
diff --git a/integration/presto/pom.xml b/integration/presto/pom.xml
index aaaf175..0abcf38 100644
--- a/integration/presto/pom.xml
+++ b/integration/presto/pom.xml
@@ -193,7 +193,7 @@
 
 
   org.scalatest
-  scalatest_2.11
+  scalatest_${scala.binary.version}
 
 
   org.apache.zookeeper
@@ -330,7 +330,6 @@
 
   org.scalatest
   scalatest_${scala.binary.version}
-  2.2.1
   test
 
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d85215a1/integration/spark-common-cluster-test/pom.xml
--
diff --git a/integration/spark-common-cluster-test/pom.xml 
b/integration/spark-common-cluster-test/pom.xml
index fd907a3..028da11 100644
--- a/integration/spark-common-cluster-test/pom.xml
+++ b/integration/spark-common-cluster-test/pom.xml
@@ -49,11 +49,11 @@
 
   junit
   junit
+  test
 
 
   org.scalatest
   scalatest_${scala.binary.version}
-  2.2.1
   test
 
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d85215a1/integration/spark-common-test/pom.xml
--
diff --git a/integration/spark-common-test/pom.xml 
b/integration/spark-common-test/pom.xml
index 67a2317..d1c04ae 100644
--- a/integration/spark-common-test/pom.xml
+++ b/integration/spark-common-test/pom.xml
@@ -106,16 +106,17 @@
 
   junit
   junit
+  test
 
 
   org.scalatest
   scalatest_${scala.binary.version}
-  2.2.1
   test
 
 
   org.jmockit
   jmockit
+  test
 
   
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d85215a1/integration/spark-common/pom.xml
--
diff --git a/integration/spark-common/pom.xml b/integration/spark-common/pom.xml
index 295d62b..16f327d 100644
--- a/integration/spark-common/pom.xml
+++ b/integration/spark-common/pom.xml
@@ -58,11 +58,11 @@
 
   junit
   junit
+  test

[11/50] [abbrv] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

2018-03-02 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/636eb799/core/src/main/java/org/apache/carbondata/core/scan/processor/DataBlockIterator.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/scan/processor/DataBlockIterator.java
 
b/core/src/main/java/org/apache/carbondata/core/scan/processor/DataBlockIterator.java
new file mode 100644
index 000..fde4e55
--- /dev/null
+++ 
b/core/src/main/java/org/apache/carbondata/core/scan/processor/DataBlockIterator.java
@@ -0,0 +1,269 @@
+/*
+ * 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.scan.processor;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.carbondata.common.CarbonIterator;
+import org.apache.carbondata.core.datastore.DataRefNode;
+import org.apache.carbondata.core.datastore.FileReader;
+import org.apache.carbondata.core.scan.collector.ResultCollectorFactory;
+import org.apache.carbondata.core.scan.collector.ScannedResultCollector;
+import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
+import org.apache.carbondata.core.scan.scanner.BlockletScanner;
+import org.apache.carbondata.core.scan.scanner.impl.BlockletFilterScanner;
+import org.apache.carbondata.core.scan.scanner.impl.BlockletFullScanner;
+import org.apache.carbondata.core.stats.QueryStatisticsModel;
+import org.apache.carbondata.core.util.TaskMetricsMap;
+
+/**
+ * This abstract class provides a skeletal implementation of the
+ * Block iterator.
+ */
+public class DataBlockIterator extends CarbonIterator> {
+
+  /**
+   * iterator which will be used to iterate over blocklets
+   */
+  private BlockletIterator blockletIterator;
+
+  /**
+   * result collector which will be used to aggregate the scanned result
+   */
+  private ScannedResultCollector scannerResultAggregator;
+
+  /**
+   * processor which will be used to process the block processing can be
+   * filter processing or non filter processing
+   */
+  private BlockletScanner blockletScanner;
+
+  /**
+   * batch size of result
+   */
+  private int batchSize;
+
+  private ExecutorService executorService;
+
+  private Future future;
+
+  private Future futureIo;
+
+  private BlockletScannedResult scannedResult;
+
+  private BlockExecutionInfo blockExecutionInfo;
+
+  private FileReader fileReader;
+
+  private AtomicBoolean nextBlock;
+
+  private AtomicBoolean nextRead;
+
+  public DataBlockIterator(BlockExecutionInfo blockExecutionInfo, FileReader 
fileReader,
+  int batchSize, QueryStatisticsModel queryStatisticsModel, 
ExecutorService executorService) {
+this.blockExecutionInfo = blockExecutionInfo;
+this.fileReader = fileReader;
+blockletIterator = new 
BlockletIterator(blockExecutionInfo.getFirstDataBlock(),
+blockExecutionInfo.getNumberOfBlockToScan());
+if (blockExecutionInfo.getFilterExecuterTree() != null) {
+  blockletScanner = new BlockletFilterScanner(blockExecutionInfo, 
queryStatisticsModel);
+} else {
+  blockletScanner = new BlockletFullScanner(blockExecutionInfo, 
queryStatisticsModel);
+}
+this.scannerResultAggregator =
+ResultCollectorFactory.getScannedResultCollector(blockExecutionInfo);
+this.batchSize = batchSize;
+this.executorService = executorService;
+this.nextBlock = new AtomicBoolean(false);
+this.nextRead = new AtomicBoolean(false);
+  }
+
+  @Override
+  public List next() {
+List collectedResult = null;
+if (updateScanner()) {
+  collectedResult = 
this.scannerResultAggregator.collectResultInRow(scannedResult, batchSize);
+  while (collectedResult.size() < batchSize && updateScanner()) {
+List data = this.scannerResultAggregator
+.collectResultInRow(scannedResult, batchSize - 
collectedResul

[15/50] [abbrv] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

2018-03-02 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/636eb799/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 69f5ceb..22d1df1 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
@@ -43,10 +43,9 @@ import 
org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.datastore.block.TableBlockUniqueIdentifier;
 import org.apache.carbondata.core.indexstore.BlockletDetailInfo;
-import 
org.apache.carbondata.core.indexstore.blockletindex.BlockletDataRefNodeWrapper;
+import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataRefNode;
 import org.apache.carbondata.core.indexstore.blockletindex.IndexWrapper;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
-import org.apache.carbondata.core.keygenerator.KeyGenerator;
 import org.apache.carbondata.core.memory.UnsafeMemoryManager;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.blocklet.BlockletInfo;
@@ -64,8 +63,8 @@ import 
org.apache.carbondata.core.scan.executor.util.RestructureUtil;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.SingleTableProvider;
 import org.apache.carbondata.core.scan.filter.TableProvider;
-import org.apache.carbondata.core.scan.model.QueryDimension;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
+import org.apache.carbondata.core.scan.model.ProjectionMeasure;
 import org.apache.carbondata.core.scan.model.QueryModel;
 import org.apache.carbondata.core.stats.QueryStatistic;
 import org.apache.carbondata.core.stats.QueryStatisticsConstants;
@@ -121,7 +120,6 @@ public abstract class AbstractQueryExecutor implements 
QueryExecutor {
 queryProperties.queryStatisticsRecorder =
 
CarbonTimeStatisticsFactory.createExecutorRecorder(queryModel.getQueryId());
 queryModel.setStatisticsRecorder(queryProperties.queryStatisticsRecorder);
-QueryUtil.resolveQueryModel(queryModel);
 QueryStatistic queryStatistic = new QueryStatistic();
 // sort the block info
 // so block will be loaded in sorted order this will be required for
@@ -168,12 +166,12 @@ public abstract class AbstractQueryExecutor implements 
QueryExecutor {
 .addStatistics(QueryStatisticsConstants.LOAD_BLOCKS_EXECUTOR, 
System.currentTimeMillis());
 queryProperties.queryStatisticsRecorder.recordStatistics(queryStatistic);
 // calculating the total number of aggeragted columns
-int measureCount = queryModel.getQueryMeasures().size();
+int measureCount = queryModel.getProjectionMeasures().size();
 
 int currentIndex = 0;
 DataType[] dataTypes = new DataType[measureCount];
 
-for (QueryMeasure carbonMeasure : queryModel.getQueryMeasures()) {
+for (ProjectionMeasure carbonMeasure : queryModel.getProjectionMeasures()) 
{
   // adding the data type and aggregation type of all the measure this
   // can be used
   // to select the aggregator
@@ -198,9 +196,11 @@ public abstract class AbstractQueryExecutor implements 
QueryExecutor {
 queryStatistic = new QueryStatistic();
 // dictionary column unique column id to dictionary mapping
 // which will be used to get column actual data
-queryProperties.columnToDictionayMapping = QueryUtil
-.getDimensionDictionaryDetail(queryModel.getQueryDimension(),
-queryProperties.complexFilterDimension, 
queryModel.getAbsoluteTableIdentifier(),
+queryProperties.columnToDictionayMapping =
+QueryUtil.getDimensionDictionaryDetail(
+queryModel.getProjectionDimensions(),
+queryProperties.complexFilterDimension,
+queryModel.getAbsoluteTableIdentifier(),
 tableProvider);
 queryStatistic
 .addStatistics(QueryStatisticsConstants.LOAD_DICTIONARY, 
System.currentTimeMillis());
@@ -263,8 +263,8 @@ public abstract class AbstractQueryExecutor implements 
QueryExecutor {
 // and query will be executed based on that infos
 for (int i = 0; i < queryProperties.dataBlocks.size(); i++) {
   AbstractIndex abstractIndex = queryProperties.dataBlocks.get(i);
-  BlockletDataRefNodeWrapper dataRefNode =
-  (BlockletDataRefNodeWrapper) abstractIndex.getDataRefNode();
+  BlockletDataRefNode dataRefNode =
+  (BlockletDataRefNode) abstractIndex.getDataRefNode();
   blockExec

[49/50] [abbrv] carbondata git commit: [CARBONDATA-2186] Add InterfaceAudience.Internal to annotate internal interface

2018-03-02 Thread jackylk
[CARBONDATA-2186] Add InterfaceAudience.Internal to annotate internal interface

This closes #1986


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

Branch: refs/heads/carbonstore-rebase5
Commit: b1bc9c79a02d70cdd9bb3c7de076929f63ced3b7
Parents: d85215a
Author: Jacky Li 
Authored: Tue Feb 20 11:16:53 2018 +0800
Committer: Jacky Li 
Committed: Fri Mar 2 15:52:37 2018 +0800

--
 .../java/org/apache/carbondata/common/Maps.java  |  2 +-
 .../org/apache/carbondata/common/Strings.java|  2 +-
 .../common/annotations/InterfaceAudience.java| 19 ++-
 .../common/annotations/InterfaceStability.java   |  2 +-
 .../loading/model/CarbonLoadModelBuilder.java|  2 +-
 .../processing/loading/model/LoadOption.java |  2 +-
 .../carbondata/sdk/file/CSVCarbonWriter.java |  4 +---
 7 files changed, 20 insertions(+), 13 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/b1bc9c79/common/src/main/java/org/apache/carbondata/common/Maps.java
--
diff --git a/common/src/main/java/org/apache/carbondata/common/Maps.java 
b/common/src/main/java/org/apache/carbondata/common/Maps.java
index 14fc329..4e76192 100644
--- a/common/src/main/java/org/apache/carbondata/common/Maps.java
+++ b/common/src/main/java/org/apache/carbondata/common/Maps.java
@@ -21,7 +21,7 @@ import java.util.Map;
 
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 
-@InterfaceAudience.Developer
+@InterfaceAudience.Internal
 public class Maps {
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/b1bc9c79/common/src/main/java/org/apache/carbondata/common/Strings.java
--
diff --git a/common/src/main/java/org/apache/carbondata/common/Strings.java 
b/common/src/main/java/org/apache/carbondata/common/Strings.java
index 08fdc3c..23c7f9f 100644
--- a/common/src/main/java/org/apache/carbondata/common/Strings.java
+++ b/common/src/main/java/org/apache/carbondata/common/Strings.java
@@ -21,7 +21,7 @@ import java.util.Objects;
 
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 
-@InterfaceAudience.Developer
+@InterfaceAudience.Internal
 public class Strings {
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/b1bc9c79/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceAudience.java
--
diff --git 
a/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceAudience.java
 
b/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceAudience.java
index fa9729d..8d214ff 100644
--- 
a/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceAudience.java
+++ 
b/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceAudience.java
@@ -25,10 +25,10 @@ import java.lang.annotation.RetentionPolicy;
  * This annotation is ported and modified from Apache Hadoop project.
  *
  * Annotation to inform users of a package, class or method's intended 
audience.
- * Currently the audience can be {@link User}, {@link Developer}
+ * Currently the audience can be {@link User}, {@link Developer}, {@link 
Internal}
  *
  * Public classes that are not marked with this annotation must be
- * considered by default as {@link Developer}.
+ * considered by default as {@link Internal}.
  *
  * External applications must only use classes that are marked {@link User}.
  *
@@ -47,12 +47,21 @@ public class InterfaceAudience {
   public @interface User { }
 
   /**
-   * Intended only for developers to extend interface for CarbonData project
-   * For example, new Datamap implementations.
+   * Intended for developers to develop extension for Apache CarbonData project
+   * For example, "Index DataMap" to add a new index implementation, etc
*/
   @Documented
   @Retention(RetentionPolicy.RUNTIME)
-  public @interface Developer { }
+  public @interface Developer {
+String[] value();
+  }
+
+  /**
+   * Intended only for internal usage within Apache CarbonData project.
+   */
+  @Documented
+  @Retention(RetentionPolicy.RUNTIME)
+  public @interface Internal { }
 
   private InterfaceAudience() { } // Audience can't exist on its own
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/b1bc9c79/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceStability.java
--
diff --git 
a/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceStability.java
 
b/co

[34/50] [abbrv] carbondata git commit: [CARBONDATA-2018][DataLoad] Optimization in reading/writing for sort temp row

2018-03-02 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/d115c479/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
--
diff --git 
a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
 
b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
index 11b3d43..527452a 100644
--- 
a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
+++ 
b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
@@ -31,15 +31,14 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.DataTypeUtil;
-import 
org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
+import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
+import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
 import 
org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
-import org.apache.carbondata.processing.sort.sortdata.NewRowComparator;
+import 
org.apache.carbondata.processing.sort.sortdata.IntermediateSortTempRowComparator;
 import org.apache.carbondata.processing.sort.sortdata.SortParameters;
+import org.apache.carbondata.processing.sort.sortdata.TableFieldStat;
 
 public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
 
@@ -63,21 +62,15 @@ public class UnsafeSortTempFileChunkHolder implements 
SortTempChunkHolder {
* entry count
*/
   private int entryCount;
-
   /**
* return row
*/
-  private Object[] returnRow;
-  private int dimCnt;
-  private int complexCnt;
-  private int measureCnt;
-  private boolean[] isNoDictionaryDimensionColumn;
-  private DataType[] measureDataTypes;
+  private IntermediateSortTempRow returnRow;
   private int readBufferSize;
   private String compressorName;
-  private Object[][] currentBuffer;
+  private IntermediateSortTempRow[] currentBuffer;
 
-  private Object[][] backupBuffer;
+  private IntermediateSortTempRow[] backupBuffer;
 
   private boolean isBackupFilled;
 
@@ -100,27 +93,21 @@ public class UnsafeSortTempFileChunkHolder implements 
SortTempChunkHolder {
 
   private int numberOfObjectRead;
 
-  private int nullSetWordsLength;
-
-  private Comparator comparator;
-
+  private TableFieldStat tableFieldStat;
+  private SortStepRowHandler sortStepRowHandler;
+  private Comparator comparator;
   /**
* Constructor to initialize
*/
   public UnsafeSortTempFileChunkHolder(File tempFile, SortParameters 
parameters) {
 // set temp file
 this.tempFile = tempFile;
-this.dimCnt = parameters.getDimColCount();
-this.complexCnt = parameters.getComplexDimColCount();
-this.measureCnt = parameters.getMeasureColCount();
-this.isNoDictionaryDimensionColumn = 
parameters.getNoDictionaryDimnesionColumn();
-this.measureDataTypes = parameters.getMeasureDataType();
 this.readBufferSize = parameters.getBufferSize();
 this.compressorName = parameters.getSortTempCompressorName();
-
+this.tableFieldStat = new TableFieldStat(parameters);
+this.sortStepRowHandler = new SortStepRowHandler(tableFieldStat);
 this.executorService = Executors.newFixedThreadPool(1);
-this.nullSetWordsLength = ((parameters.getMeasureColCount() - 1) >> 6) + 1;
-comparator = new NewRowComparator(parameters.getNoDictionarySortColumn());
+comparator = new 
IntermediateSortTempRowComparator(parameters.getNoDictionarySortColumn());
 initialize();
   }
 
@@ -169,11 +156,17 @@ public class UnsafeSortTempFileChunkHolder implements 
SortTempChunkHolder {
*
* @throws CarbonSortKeyAndGroupByException problem while reading
*/
+  @Override
   public void readRow() throws CarbonSortKeyAndGroupByException {
 if (prefetch) {
   fillDataForPrefetch();
 } else {
-  this.returnRow = getRowFromStream();
+  try {
+this.returnRow = 
sortStepRowHandler.readIntermediateSortTempRowFromInputStream(stream);
+this.numberOfObjectRead++;
+  } catch (IOException e) {
+throw new CarbonSortKeyAndGroupByException("Problems while reading 
row", e);
+  }
 }
   }
 
@@ -207,63 +200,22 @@ public class UnsafeSortTempFileChunkHolder implements 
SortTempChunkHolder {
   }
 
   /**
-   * @retu

[16/50] [abbrv] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

2018-03-02 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/636eb799/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/SegmentInfo.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/SegmentInfo.java
 
b/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/SegmentInfo.java
index 0cb2918..099fffd 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/SegmentInfo.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/SegmentInfo.java
@@ -29,31 +29,12 @@ public class SegmentInfo implements Serializable {
   private static final long serialVersionUID = -174987462709431L;
 
   /**
-   * number of column in the segment
-   */
-  private int numberOfColumns;
-
-  /**
* cardinality of each columns
* column which is not participating in the multidimensional key cardinality 
will be -1;
*/
   private int[] columnCardinality;
 
   /**
-   * @return the numberOfColumns
-   */
-  public int getNumberOfColumns() {
-return numberOfColumns;
-  }
-
-  /**
-   * @param numberOfColumns the numberOfColumns to set
-   */
-  public void setNumberOfColumns(int numberOfColumns) {
-this.numberOfColumns = numberOfColumns;
-  }
-
-  /**
* @return the columnCardinality
*/
   public int[] getColumnCardinality() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/636eb799/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
 
b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
index 6036569..d17d865 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
@@ -19,7 +19,13 @@ package org.apache.carbondata.core.metadata.schema.table;
 
 import java.io.IOException;
 import java.io.Serializable;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
@@ -33,7 +39,10 @@ import 
org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import 
org.apache.carbondata.core.metadata.schema.table.column.CarbonImplicitDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.scan.model.QueryModel;
+import org.apache.carbondata.core.scan.model.QueryProjection;
 import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.DataTypeConverter;
 import org.apache.carbondata.core.util.DataTypeUtil;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 
@@ -136,10 +145,7 @@ public class CarbonTable implements Serializable {
   /**
* During creation of TableInfo from hivemetastore the DataMapSchemas and 
the columns
* DataTypes are not converted to the appropriate child classes.
-   *
* This method will cast the same to the appropriate classes
-   *
-   * @param tableInfo
*/
   public static void updateTableInfo(TableInfo tableInfo) {
 List dataMapSchemas = new ArrayList<>();
@@ -153,8 +159,9 @@ public class CarbonTable implements Serializable {
 }
 tableInfo.setDataMapSchemaList(dataMapSchemas);
 for (ColumnSchema columnSchema : 
tableInfo.getFactTable().getListOfColumns()) {
-  columnSchema.setDataType(DataTypeUtil.valueOf(columnSchema.getDataType(),
-  columnSchema.getPrecision(), columnSchema.getScale()));
+  columnSchema.setDataType(
+  DataTypeUtil.valueOf(
+  columnSchema.getDataType(), columnSchema.getPrecision(), 
columnSchema.getScale()));
 }
 List childSchema = tableInfo.getDataMapSchemaList();
 for (DataMapSchema dataMapSchema : childSchema) {
@@ -168,10 +175,11 @@ public class CarbonTable implements Serializable {
   }
 }
 if (tableInfo.getFactTable().getBucketingInfo() != null) {
-  for (ColumnSchema columnSchema : tableInfo.getFactTable()
-  .getBucketingInfo().getListOfColumns()) {
-
columnSchema.setDataType(DataTypeUtil.valueOf(columnSchema.getDataType(),
-columnSchema.getPrecision(), columnSchema.getScale()));
+  for (ColumnSchema columnSchema :
+  tableInfo.getFactTable().getBucketingInfo().getListOfColumns()) {
+columnSchema.setDataType(
+DataTypeUtil.valueOf(
+columnSchema.getDataType(), columnSchema.getPrecision(), 

[47/50] [abbrv] carbondata git commit: [CARBONDATA-2091][DataLoad] Support specifying sort column bounds in data loading

2018-03-02 Thread jackylk
[CARBONDATA-2091][DataLoad] Support specifying sort column bounds in data 
loading

Enhance data loading performance by specifying sort column bounds
1. Add row range number during convert-process-step
2. Dispatch rows to each sorter by range number
3. Sort/Write process step can be done concurrently in each range
4. Since all sorttemp files will be written in one folders, we add range
number to the file name to distingush them

Tests added and docs updated

This closes #1953


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

Branch: refs/heads/carbonstore-rebase5
Commit: b1c962855ebdcdf57a1f2531183d14f2ade2c94c
Parents: b1bc9c7
Author: xuchuanyin 
Authored: Tue Feb 13 10:58:06 2018 +0800
Committer: Jacky Li 
Committed: Fri Mar 2 15:52:37 2018 +0800

--
 .../constants/CarbonLoadOptionConstants.java|  10 +
 .../core/datastore/row/CarbonRow.java   |  10 +-
 .../ThriftWrapperSchemaConverterImpl.java   |   2 +-
 .../core/metadata/schema/BucketingInfo.java |  24 +-
 .../core/metadata/schema/ColumnRangeInfo.java   |  29 ++
 .../metadata/schema/SortColumnRangeInfo.java|  83 +
 docs/data-management-on-carbondata.md   |  11 +
 .../TestLoadDataWithSortColumnBounds.scala  | 348 +++
 .../carbondata/spark/rdd/CarbonScanRDD.scala|   2 +-
 .../carbondata/spark/rdd/PartitionDropper.scala |   2 +-
 .../spark/rdd/PartitionSplitter.scala   |   2 +-
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala |   3 +-
 .../strategy/CarbonLateDecodeStrategy.scala |   2 +-
 .../loading/CarbonDataLoadConfiguration.java|  11 +
 .../loading/DataLoadProcessBuilder.java |  77 +++-
 .../loading/converter/RowConverter.java |   2 +-
 .../converter/impl/RowConverterImpl.java|   5 +
 .../loading/model/CarbonLoadModel.java  |  14 +
 .../loading/model/CarbonLoadModelBuilder.java   |   1 +
 .../processing/loading/model/LoadOption.java|   1 +
 .../partition/impl/HashPartitionerImpl.java |  10 +-
 .../partition/impl/RangePartitionerImpl.java|  71 
 .../partition/impl/RawRowComparator.java|  63 
 .../processing/loading/sort/SorterFactory.java  |  16 +-
 ...arallelReadMergeSorterWithBucketingImpl.java | 272 ---
 ...allelReadMergeSorterWithColumnRangeImpl.java | 289 +++
 ...arallelReadMergeSorterWithBucketingImpl.java | 263 --
 ...allelReadMergeSorterWithColumnRangeImpl.java | 293 
 .../loading/sort/unsafe/UnsafeSortDataRows.java |   6 +-
 .../unsafe/merger/UnsafeIntermediateMerger.java |   6 +-
 .../UnsafeSingleThreadFinalSortFilesMerger.java |  11 +-
 .../steps/DataConverterProcessorStepImpl.java   | 102 +-
 ...ConverterProcessorWithBucketingStepImpl.java | 161 -
 .../steps/DataWriterProcessorStepImpl.java  |  70 +++-
 .../SingleThreadFinalSortFilesMerger.java   |   3 +-
 .../processing/sort/sortdata/SortDataRows.java  |  11 +-
 .../sortdata/SortIntermediateFileMerger.java|   6 +-
 .../sort/sortdata/SortParameters.java   |  10 +
 .../store/CarbonFactDataHandlerColumnar.java|   6 +-
 39 files changed, 1558 insertions(+), 750 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/b1c96285/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
 
b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
index a6bf60f..8ff8dc4 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
@@ -124,4 +124,14 @@ public final class CarbonLoadOptionConstants {
   public static final String ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION
   = "carbon.load.skewedDataOptimization.enabled";
   public static final String 
ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION_DEFAULT = "false";
+
+  /**
+   * field delimiter for each field in one bound
+   */
+  public static final String SORT_COLUMN_BOUNDS_FIELD_DELIMITER = ",";
+
+  /**
+   * row delimiter for each sort column bounds
+   */
+  public static final String SORT_COLUMN_BOUNDS_ROW_DELIMITER = ";";
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/b1c96285/core/src/main/java/org/apache/carbondata/core/datastore/row/CarbonRow.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata

[50/50] [abbrv] carbondata git commit: [REBASE] Solve conflict after merging master

2018-03-02 Thread jackylk
[REBASE] Solve conflict after merging master


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

Branch: refs/heads/carbonstore-rebase5
Commit: 7540cc9cab404c06eaf5c92da77ef7e36f9182b5
Parents: 8a9dd8b
Author: Jacky Li 
Authored: Tue Feb 27 11:26:30 2018 +0800
Committer: Jacky Li 
Committed: Fri Mar 2 16:00:28 2018 +0800

--
 .../carbondata/core/datamap/dev/DataMap.java|   9 +-
 .../core/datamap/dev/DataMapFactory.java|   2 +-
 .../exception/ConcurrentOperationException.java |  16 +-
 .../core/indexstore/BlockletDetailsFetcher.java |   3 +-
 .../blockletindex/BlockletDataMap.java  |   3 +-
 .../blockletindex/SegmentIndexFileStore.java|   2 -
 .../core/metadata/PartitionMapFileStore.java|   0
 .../scan/executor/util/RestructureUtil.java |   6 +-
 .../statusmanager/SegmentStatusManager.java |  10 +-
 .../SegmentUpdateStatusManager.java |   7 +-
 .../CarbonStreamSparkStreamingExample.scala |  14 +-
 ...CarbonStructuredStreamingWithRowParser.scala |   8 +-
 .../hadoop/api/CarbonTableInputFormat.java  |   5 +-
 .../preaggregate/TestPreAggCreateCommand.scala  |   2 +-
 .../TestInsertAndOtherCommandConcurrent.scala   |   2 +-
 .../StandardPartitionGlobalSortTestCase.scala   |   2 +-
 .../exception/ProcessMetaDataException.java |   2 +
 .../org/apache/carbondata/api/CarbonStore.scala |   6 +-
 .../carbondata/spark/load/CsvRDDHelper.scala| 157 +++
 .../load/DataLoadProcessBuilderOnSpark.scala|   3 +-
 .../carbondata/spark/util/CarbonScalaUtil.scala |   2 +-
 .../carbondata/spark/util/CommonUtil.scala  |   2 -
 .../command/carbonTableSchemaCommon.scala   |   6 +-
 .../CarbonAlterTableCompactionCommand.scala |   3 +-
 .../management/CarbonCleanFilesCommand.scala|   2 +-
 .../CarbonDeleteLoadByIdCommand.scala   |   2 +-
 .../CarbonDeleteLoadByLoadDateCommand.scala |   2 +-
 .../management/CarbonLoadDataCommand.scala  |  28 ++--
 .../CarbonProjectForDeleteCommand.scala |   2 +-
 .../CarbonProjectForUpdateCommand.scala |   2 +-
 .../schema/CarbonAlterTableRenameCommand.scala  |   2 +-
 .../command/table/CarbonDropTableCommand.scala  |   2 +-
 .../datasources/CarbonFileFormat.scala  |   3 -
 .../TestStreamingTableWithRowParser.scala   |   9 +-
 .../vectorreader/AddColumnTestCases.scala   |   1 +
 .../datamap/DataMapWriterListener.java  |   3 +-
 .../loading/model/CarbonLoadModelBuilder.java   |  34 +++-
 .../processing/loading/model/LoadOption.java|  15 +-
 .../processing/merger/CarbonDataMergerUtil.java |  19 ++-
 .../merger/CompactionResultSortProcessor.java   |   4 +-
 .../merger/RowResultMergerProcessor.java|   4 +-
 .../partition/spliter/RowResultProcessor.java   |   4 +-
 .../util/CarbonDataProcessorUtil.java   |   3 +-
 store/sdk/pom.xml   |   2 +-
 .../carbondata/sdk/file/CSVCarbonWriter.java|   8 +-
 45 files changed, 305 insertions(+), 118 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/7540cc9c/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java 
b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
index 4a68286..fdeacff 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
@@ -38,18 +38,13 @@ public interface DataMap {
   /**
* Prune the datamap with filter expression and partition information. It 
returns the list of
* blocklets where these filters can exist.
-   *
-   * @param filterExp
-   * @return
*/
-  List prune(FilterResolverIntf filterExp, SegmentProperties 
segmentProperties, List partitions);
+  List prune(FilterResolverIntf filterExp, SegmentProperties 
segmentProperties,
+  List partitions);
 
   // TODO Move this method to Abstract class
   /**
* Validate whether the current segment needs to be fetching the required 
data
-   *
-   * @param filterExp
-   * @return
*/
   boolean isScanRequired(FilterResolverIntf filterExp);
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7540cc9c/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java 
b/core/src/main/java/org/apache/carbondata/core/datamap/dev/Data

[08/50] [abbrv] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

2018-03-02 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/636eb799/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
--
diff --git 
a/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java 
b/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
index f4450e3..5f8d199 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
@@ -31,7 +31,7 @@ import java.util.Map;
 
 import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
-import 
org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionDataChunk;
+import 
org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionColumnPage;
 import org.apache.carbondata.core.datastore.columnar.ColumnGroupModel;
 import org.apache.carbondata.core.datastore.filesystem.LocalCarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
@@ -45,7 +45,7 @@ import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.scan.model.QueryDimension;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
 
 import mockit.Mock;
 import mockit.MockUp;
@@ -267,8 +267,8 @@ public class CarbonUtilTest {
   @Test public void testToGetNextLesserValue() {
 byte[] dataChunks = { 5, 6, 7, 8, 9 };
 byte[] compareValues = { 7 };
-FixedLengthDimensionDataChunk fixedLengthDataChunk =
-new FixedLengthDimensionDataChunk(dataChunks, null, null, 5, 1);
+FixedLengthDimensionColumnPage fixedLengthDataChunk =
+new FixedLengthDimensionColumnPage(dataChunks, null, null, 5, 1);
 int result = CarbonUtil.nextLesserValueToTarget(2, fixedLengthDataChunk, 
compareValues);
 assertEquals(result, 1);
   }
@@ -276,8 +276,8 @@ public class CarbonUtilTest {
   @Test public void testToGetNextLesserValueToTarget() {
 byte[] dataChunks = { 7, 7, 7, 8, 9 };
 byte[] compareValues = { 7 };
-FixedLengthDimensionDataChunk fixedLengthDataChunk =
-new FixedLengthDimensionDataChunk(dataChunks, null, null, 5, 1);
+FixedLengthDimensionColumnPage fixedLengthDataChunk =
+new FixedLengthDimensionColumnPage(dataChunks, null, null, 5, 1);
 int result = CarbonUtil.nextLesserValueToTarget(2, fixedLengthDataChunk, 
compareValues);
 assertEquals(result, -1);
   }
@@ -285,8 +285,8 @@ public class CarbonUtilTest {
   @Test public void testToGetnextGreaterValue() {
 byte[] dataChunks = { 5, 6, 7, 8, 9 };
 byte[] compareValues = { 7 };
-FixedLengthDimensionDataChunk fixedLengthDataChunk =
-new FixedLengthDimensionDataChunk(dataChunks, null, null, 5, 1);
+FixedLengthDimensionColumnPage fixedLengthDataChunk =
+new FixedLengthDimensionColumnPage(dataChunks, null, null, 5, 1);
 int result = CarbonUtil.nextGreaterValueToTarget(2, fixedLengthDataChunk, 
compareValues, 5);
 assertEquals(result, 3);
   }
@@ -302,8 +302,8 @@ public class CarbonUtilTest {
   @Test public void testToGetnextGreaterValueToTarget() {
 byte[] dataChunks = { 5, 6, 7, 7, 7 };
 byte[] compareValues = { 7 };
-FixedLengthDimensionDataChunk fixedLengthDataChunk =
-new FixedLengthDimensionDataChunk(dataChunks, null, null, 5, 1);
+FixedLengthDimensionColumnPage fixedLengthDataChunk =
+new FixedLengthDimensionColumnPage(dataChunks, null, null, 5, 1);
 int result = CarbonUtil.nextGreaterValueToTarget(2, fixedLengthDataChunk, 
compareValues, 5);
 assertEquals(result, 5);
   }
@@ -525,23 +525,23 @@ public class CarbonUtilTest {
   }
 
   @Test public void testToGetDictionaryEncodingArray() {
-QueryDimension column1 = new QueryDimension("Column1");
-QueryDimension column2 = new QueryDimension("Column2");
 ColumnSchema column1Schema = new ColumnSchema();
 ColumnSchema column2Schema = new ColumnSchema();
 column1Schema.setColumnName("Column1");
 List encoding = new ArrayList<>();
 encoding.add(Encoding.DICTIONARY);
 column1Schema.setEncodingList(encoding);
-column1.setDimension(new CarbonDimension(column1Schema, 1, 1, 1, 1));
+ProjectionDimension
+column1 = new ProjectionDimension(new CarbonDimension(column1Schema, 
1, 1, 1, 1));
 
 column2Schema.setColumnName("Column2");
 List encoding2 = new ArrayList<>();
 encoding2.add(Encoding.DELTA);
 column2Schema.setEncodingList(encoding2);
-column2.setDimension(new CarbonDimension(column2Schema, 1, 1, 1, 1));
+ProjectionDimension
+column2 = new ProjectionDimension(new CarbonDimension(column2Schema, 
1, 1, 1, 1));
 
-Que

[24/50] [abbrv] carbondata git commit: [CARBONDATA-1480]Min Max Index Example for DataMap

2018-03-02 Thread jackylk
[CARBONDATA-1480]Min Max Index Example for DataMap

Datamap Example. Implementation of Min Max Index through Datamap. And Using the 
Index while prunning.

This closes #1359


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

Branch: refs/heads/carbonstore-rebase5
Commit: 6882f737ebafb8a6470be638e7f4bb273e6c7292
Parents: 8da481a
Author: sounakr 
Authored: Thu Sep 28 16:21:05 2017 +0530
Committer: Jacky Li 
Committed: Fri Mar 2 15:52:34 2018 +0800

--
 .../core/datamap/DataMapStoreManager.java   |  16 +-
 .../carbondata/core/datamap/TableDataMap.java   |  18 +-
 .../carbondata/core/datamap/dev/DataMap.java|  11 +-
 .../core/datamap/dev/DataMapWriter.java |   3 +-
 .../indexstore/SegmentPropertiesFetcher.java|  36 +++
 .../blockletindex/BlockletDataMap.java  |   5 +-
 .../blockletindex/BlockletDataMapFactory.java   |  32 ++-
 datamap/examples/pom.xml| 111 ++
 .../datamap/examples/BlockletMinMax.java|  41 
 .../datamap/examples/MinMaxDataMap.java | 143 
 .../datamap/examples/MinMaxDataMapFactory.java  | 114 ++
 .../datamap/examples/MinMaxDataWriter.java  | 221 +++
 .../examples/MinMaxIndexBlockDetails.java   |  77 +++
 .../MinMaxDataMapExample.scala  |  77 +++
 .../testsuite/datamap/DataMapWriterSuite.scala  |   2 +-
 pom.xml |   2 +
 .../datamap/DataMapWriterListener.java  |   4 +-
 .../store/writer/AbstractFactDataWriter.java|   7 +-
 .../writer/v3/CarbonFactDataWriterImplV3.java   |   3 +
 19 files changed, 894 insertions(+), 29 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/6882f737/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
 
b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
index d30483a..90e5fff 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
@@ -26,6 +26,7 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datamap.dev.DataMapFactory;
 import org.apache.carbondata.core.indexstore.BlockletDetailsFetcher;
+import org.apache.carbondata.core.indexstore.SegmentPropertiesFetcher;
 import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMap;
 import 
org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapFactory;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
@@ -103,7 +104,7 @@ public final class DataMapStoreManager {
   tableDataMaps = new ArrayList<>();
 }
 TableDataMap dataMap = getTableDataMap(dataMapName, tableDataMaps);
-if (dataMap != null) {
+if (dataMap != null && 
dataMap.getDataMapName().equalsIgnoreCase(dataMapName)) {
   throw new RuntimeException("Already datamap exists in that path with 
type " + dataMapName);
 }
 
@@ -113,12 +114,15 @@ public final class DataMapStoreManager {
   DataMapFactory dataMapFactory = factoryClass.newInstance();
   dataMapFactory.init(identifier, dataMapName);
   BlockletDetailsFetcher blockletDetailsFetcher;
+  SegmentPropertiesFetcher segmentPropertiesFetcher = null;
   if (dataMapFactory instanceof BlockletDetailsFetcher) {
 blockletDetailsFetcher = (BlockletDetailsFetcher) dataMapFactory;
   } else {
 blockletDetailsFetcher = getBlockletDetailsFetcher(identifier);
   }
-  dataMap = new TableDataMap(identifier, dataMapName, dataMapFactory, 
blockletDetailsFetcher);
+  segmentPropertiesFetcher = (SegmentPropertiesFetcher) 
blockletDetailsFetcher;
+  dataMap = new TableDataMap(identifier, dataMapName, dataMapFactory, 
blockletDetailsFetcher,
+  segmentPropertiesFetcher);
 } catch (Exception e) {
   LOGGER.error(e);
   throw new RuntimeException(e);
@@ -128,11 +132,11 @@ public final class DataMapStoreManager {
 return dataMap;
   }
 
-  private TableDataMap getTableDataMap(String dataMapName,
-  List tableDataMaps) {
+  private TableDataMap getTableDataMap(String dataMapName, List 
tableDataMaps) {
 TableDataMap dataMap = null;
-for (TableDataMap tableDataMap: tableDataMaps) {
-  if (tableDataMap.getDataMapName().equals(dataMapName)) {
+for (TableDataMa

[48/50] [abbrv] carbondata git commit: [CARBONDATA-1114][Tests] Fix bugs in tests in windows env

2018-03-02 Thread jackylk
[CARBONDATA-1114][Tests] Fix bugs in tests in windows env

Fix bugs in tests that will cause failure under windows env

This closes #1994


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

Branch: refs/heads/carbonstore-rebase5
Commit: 8a9dd8b21907096ded81fb58e152eecc3f629bb8
Parents: b1c9628
Author: xuchuanyin 
Authored: Sat Feb 24 21:18:17 2018 +0800
Committer: Jacky Li 
Committed: Fri Mar 2 15:52:37 2018 +0800

--
 .../carbondata/core/locks/LocalFileLock.java| 30 +++-
 .../store/impl/DFSFileReaderImplUnitTest.java   | 11 ---
 .../store/impl/FileFactoryImplUnitTest.java | 28 +-
 .../filesystem/HDFSCarbonFileTest.java  |  3 +-
 .../filesystem/LocalCarbonFileTest.java | 20 +
 .../BooleanDataTypesInsertTest.scala|  5 ++--
 .../carbondata/lcm/locks/LocalFileLockTest.java |  2 +-
 .../loading/csvinput/CSVInputFormatTest.java|  1 +
 8 files changed, 64 insertions(+), 36 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/8a9dd8b2/core/src/main/java/org/apache/carbondata/core/locks/LocalFileLock.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/locks/LocalFileLock.java 
b/core/src/main/java/org/apache/carbondata/core/locks/LocalFileLock.java
index 75ea074..cb80877 100644
--- a/core/src/main/java/org/apache/carbondata/core/locks/LocalFileLock.java
+++ b/core/src/main/java/org/apache/carbondata/core/locks/LocalFileLock.java
@@ -17,17 +17,20 @@
 
 package org.apache.carbondata.core.locks;
 
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.nio.channels.FileChannel;
 import java.nio.channels.FileLock;
 import java.nio.channels.OverlappingFileLockException;
+import java.nio.file.Paths;
+import java.nio.file.StandardOpenOption;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.util.CarbonUtil;
 
 /**
  * This class handles the file locking in the local file system.
@@ -40,11 +43,6 @@ public class LocalFileLock extends AbstractCarbonLock {
   private String location;
 
   /**
-   * fileOutputStream of the local lock file
-   */
-  private FileOutputStream fileOutputStream;
-
-  /**
* channel is the FileChannel of the lock file.
*/
   private FileChannel channel;
@@ -104,8 +102,8 @@ public class LocalFileLock extends AbstractCarbonLock {
 FileFactory.createNewLockFile(lockFilePath, 
FileFactory.getFileType(location));
   }
 
-  fileOutputStream = new FileOutputStream(lockFilePath);
-  channel = fileOutputStream.getChannel();
+  channel = FileChannel.open(Paths.get(lockFilePath), 
StandardOpenOption.WRITE,
+  StandardOpenOption.APPEND);
   try {
 fileLock = channel.tryLock();
   } catch (OverlappingFileLockException e) {
@@ -137,11 +135,17 @@ public class LocalFileLock extends AbstractCarbonLock {
 } catch (IOException e) {
   status = false;
 } finally {
-  if (null != fileOutputStream) {
-try {
-  fileOutputStream.close();
-} catch (IOException e) {
-  LOGGER.error(e.getMessage());
+  CarbonUtil.closeStreams(channel);
+
+  // deleting the lock file after releasing the lock.
+  if (null != lockFilePath) {
+CarbonFile lockFile = FileFactory.getCarbonFile(lockFilePath,
+FileFactory.getFileType(lockFilePath));
+if (!lockFile.exists() || lockFile.delete()) {
+  LOGGER.info("Successfully deleted the lock file " + lockFilePath);
+} else {
+  LOGGER.error("Not able to delete the lock file " + lockFilePath);
+  status = false;
 }
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/8a9dd8b2/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileReaderImplUnitTest.java
--
diff --git 
a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileReaderImplUnitTest.java
 
b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileReaderImplUnitTest.java
index da61a94..30144c1 100644
--- 
a/core/src/test/java/org

[23/50] [abbrv] carbondata git commit: [CARBONDATA-2025] Unify all path construction through CarbonTablePath static method

2018-03-02 Thread jackylk
[CARBONDATA-2025] Unify all path construction through CarbonTablePath static 
method

Refactory CarbonTablePath:

1.Remove CarbonStorePath and use CarbonTablePath only.
2.Make CarbonTablePath an utility without object creation, it can avoid 
creating object before using it, thus code is cleaner and GC is less.

This closes #1768


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

Branch: refs/heads/carbonstore-rebase5
Commit: 9daad358a3878e8a2338bdd7c746507af0860cea
Parents: 636eb79
Author: Jacky Li 
Authored: Wed Jan 31 16:14:27 2018 +0800
Committer: Jacky Li 
Committed: Fri Mar 2 15:52:32 2018 +0800

--
 .../DictionaryColumnUniqueIdentifier.java   |  29 +-
 .../dictionary/ManageDictionaryAndBTree.java|  13 +-
 .../core/metadata/AbsoluteTableIdentifier.java  |   4 +-
 .../core/metadata/schema/table/CarbonTable.java |  11 +-
 .../core/mutate/CarbonUpdateUtil.java   |  58 ++--
 .../core/scan/executor/util/QueryUtil.java  |   7 +-
 .../core/service/CarbonCommonFactory.java   |  16 -
 .../carbondata/core/service/PathService.java|  35 ---
 .../core/service/impl/PathFactory.java  |  50 
 .../statusmanager/SegmentStatusManager.java |  30 +-
 .../SegmentUpdateStatusManager.java |  59 ++--
 .../apache/carbondata/core/util/CarbonUtil.java |  60 ++--
 .../util/path/CarbonSharedDictionaryPath.java   |  71 -
 .../core/util/path/CarbonStorePath.java |  71 -
 .../core/util/path/CarbonTablePath.java | 291 ++-
 .../dictionary/AbstractDictionaryCacheTest.java |  11 +-
 .../dictionary/ForwardDictionaryCacheTest.java  |   6 +-
 .../dictionary/ReverseDictionaryCacheTest.java  |   6 +-
 .../reader/CarbonDictionaryReaderImplTest.java  |   8 -
 .../CarbonFormatDirectoryStructureTest.java |  18 +-
 .../path/CarbonFormatSharedDictionaryTest.java  |  44 ---
 .../writer/CarbonDictionaryWriterImplTest.java  |  19 +-
 .../CarbonBatchSparkStreamingExample.scala  |   9 +-
 .../CarbonStructuredStreamingExample.scala  |  11 +-
 .../hadoop/api/CarbonTableInputFormat.java  |   8 +-
 .../streaming/CarbonStreamRecordWriter.java |   6 +-
 .../carbondata/hadoop/util/SchemaReader.java|  18 +-
 .../hadoop/test/util/StoreCreator.java  |  16 +-
 .../presto/CarbondataRecordSetProvider.java |   7 +-
 .../presto/impl/CarbonTableCacheModel.java  |  13 +-
 .../presto/impl/CarbonTableReader.java  |  49 ++--
 .../presto/util/CarbonDataStoreCreator.scala|  10 +-
 .../sdv/generated/MergeIndexTestCase.scala  |   8 +-
 .../dataload/TestLoadDataGeneral.scala  |  11 +-
 .../InsertIntoCarbonTableTestCase.scala |   8 +-
 .../createTable/TestCreateTableAsSelect.scala   |   2 +-
 .../datacompaction/DataCompactionLockTest.scala |   6 +-
 .../MajorCompactionIgnoreInMinorTest.scala  |  12 +-
 .../dataload/TestBatchSortDataLoad.scala|   5 +-
 .../dataload/TestDataLoadWithFileName.scala |   5 +-
 .../dataload/TestGlobalSortDataLoad.scala   |   8 +-
 .../dataretention/DataRetentionTestCase.scala   |   6 +-
 .../TestDataLoadingForPartitionTable.scala  |   4 +-
 .../StandardPartitionTableLoadingTestCase.scala |   6 +-
 .../org/apache/carbondata/api/CarbonStore.scala |   4 +-
 .../carbondata/spark/CarbonSparkFactory.scala   |   2 +-
 .../spark/DictionaryDetailHelper.scala  |   9 +-
 .../spark/rdd/AlterTableAddColumnRDD.scala  |  17 +-
 .../spark/rdd/CarbonGlobalDictionaryRDD.scala   |   6 -
 .../carbondata/spark/util/CommonUtil.scala  |  30 +-
 .../carbondata/spark/util/DataLoadingUtil.scala |   8 +-
 .../spark/util/GlobalDictionaryUtil.scala   |  16 +-
 .../command/carbonTableSchemaCommon.scala   |   9 +-
 .../org/apache/spark/util/PartitionUtils.scala  |   6 +-
 .../spark/rdd/AggregateDataMapCompactor.scala   |  11 +-
 .../spark/rdd/CarbonDataRDDFactory.scala|  14 +-
 .../spark/rdd/CarbonTableCompactor.scala|   8 +-
 .../CarbonAlterTableCompactionCommand.scala |  13 +-
 .../management/CarbonLoadDataCommand.scala  |  12 +-
 .../management/CarbonShowLoadsCommand.scala |   2 +-
 .../management/RefreshCarbonTableCommand.scala  |  17 +-
 .../command/mutation/DeleteExecution.scala  |   7 +-
 .../CarbonAlterTableDropPartitionCommand.scala  |  16 +-
 .../CarbonAlterTableSplitPartitionCommand.scala |  20 +-
 .../preaaggregate/PreAggregateUtil.scala|  35 ++-
 .../CarbonAlterTableAddColumnCommand.scala  |   8 +-
 .../CarbonAlterTableDataTypeChangeCommand.scala |   9 +-
 .../CarbonAlterTableDropColumnCommand.scala |   6 +-
 .../schema/CarbonAlterTableRenameCommand.scala  |  28 +-
 .../spark/sql/hive/CarbonFileMetastore.scala|  27

[03/50] [abbrv] carbondata git commit: [CARBONDATA-1827] S3 Carbon Implementation

2018-03-02 Thread jackylk
[CARBONDATA-1827] S3 Carbon Implementation

1.Provide support for s3 in carbondata.
2.Added S3Example to create carbon table on s3.
3.Added S3CSVExample to load carbon table using csv from s3.

This closes #1805


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

Branch: refs/heads/carbonstore-rebase5
Commit: 1921393f548b65d16cc44ff1d494cfd007365d8f
Parents: 2529e47
Author: SangeetaGulia 
Authored: Thu Sep 21 14:56:26 2017 +0530
Committer: Jacky Li 
Committed: Fri Mar 2 15:51:53 2018 +0800

--
 .../core/constants/CarbonCommonConstants.java   |  21 +++
 .../filesystem/AbstractDFSCarbonFile.java   |  20 ++-
 .../datastore/filesystem/HDFSCarbonFile.java|   5 +-
 .../core/datastore/impl/FileFactory.java|  11 +-
 .../core/locks/CarbonLockFactory.java   |  28 ++--
 .../carbondata/core/locks/S3FileLock.java   | 111 +
 .../carbondata/core/util/CarbonProperties.java  |   3 +-
 .../filesystem/HDFSCarbonFileTest.java  |   8 +-
 examples/spark2/pom.xml |   5 +
 examples/spark2/src/main/resources/data1.csv|  11 ++
 .../carbondata/examples/S3CsvExample.scala  |  99 +++
 .../apache/carbondata/examples/S3Example.scala  | 164 +++
 .../spark/rdd/NewCarbonDataLoadRDD.scala|  42 -
 integration/spark2/pom.xml  |  43 +
 .../spark/rdd/CarbonDataRDDFactory.scala|   3 +-
 .../org/apache/spark/sql/CarbonSession.scala|   3 +
 16 files changed, 554 insertions(+), 23 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/1921393f/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
 
b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index 0a06abc..18acc5b 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
@@ -167,6 +167,22 @@ public final class CarbonCommonConstants {
   public static final String S3N_PREFIX = "s3n://";
 
   public static final String S3A_PREFIX = "s3a://";
+  /**
+   * Access Key for s3n
+   */
+  public static final String S3N_ACCESS_KEY = "fs.s3n.awsAccessKeyId";
+  /**
+   * Secret Key for s3n
+   */
+  public static final String S3N_SECRET_KEY = "fs.s3n.awsSecretAccessKey";
+  /**
+   * Access Key for s3
+   */
+  public static final String S3_ACCESS_KEY = "fs.s3.awsAccessKeyId";
+  /**
+   * Secret Key for s3
+   */
+  public static final String S3_SECRET_KEY = "fs.s3.awsSecretAccessKey";
 
   /**
* FS_DEFAULT_FS
@@ -941,6 +957,11 @@ public final class CarbonCommonConstants {
   public static final String CARBON_LOCK_TYPE_HDFS = "HDFSLOCK";
 
   /**
+   * S3LOCK TYPE
+   */
+  public static final String CARBON_LOCK_TYPE_S3 = "S3LOCK";
+
+  /**
* Invalid filter member log string
*/
   public static final String FILTER_INVALID_MEMBER =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1921393f/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/AbstractDFSCarbonFile.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/AbstractDFSCarbonFile.java
 
b/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/AbstractDFSCarbonFile.java
index 68eaa21..fd5dc40 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/AbstractDFSCarbonFile.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/AbstractDFSCarbonFile.java
@@ -51,7 +51,7 @@ import org.apache.hadoop.io.compress.GzipCodec;
 import org.apache.hadoop.io.compress.Lz4Codec;
 import org.apache.hadoop.io.compress.SnappyCodec;
 
-public abstract  class AbstractDFSCarbonFile implements CarbonFile {
+public abstract class AbstractDFSCarbonFile implements CarbonFile {
   /**
* LOGGER
*/
@@ -262,18 +262,28 @@ public abstract  class AbstractDFSCarbonFile implements 
CarbonFile {
   @Override public DataOutputStream getDataOutputStream(String path, 
FileFactory.FileType fileType,
   int bufferSize, boolean append) throws IOException {
 Path pt = new Path(path);
-FileSystem fs = pt.getFileSystem(FileFactory.getConfiguration());
+FileSystem fileSystem = pt.getFileSystem(FileFactory.getConfiguration());
 FSDataOutputStream stream = null;
 if (append) {
   // appen

[30/50] [abbrv] carbondata git commit: Revert "[CARBONDATA-2023][DataLoad] Add size base block allocation in data loading"

2018-03-02 Thread jackylk
Revert "[CARBONDATA-2023][DataLoad] Add size base block allocation in data 
loading"

This reverts commit 6dd8b038fc898dbf48ad30adfc870c19eb38e3d0.


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

Branch: refs/heads/carbonstore-rebase5
Commit: 22bb333a53a6f256248be519874654a794e35d17
Parents: 9062931
Author: Jacky Li 
Authored: Sat Feb 10 10:34:59 2018 +0800
Committer: Jacky Li 
Committed: Fri Mar 2 15:52:35 2018 +0800

--
 .../constants/CarbonLoadOptionConstants.java|  10 -
 .../core/datastore/block/TableBlockInfo.java|  29 --
 .../carbondata/core/util/CarbonProperties.java  |  11 -
 docs/useful-tips-on-carbondata.md   |   1 -
 .../spark/rdd/NewCarbonDataLoadRDD.scala|   4 +-
 .../spark/sql/hive/DistributionUtil.scala   |   2 +-
 .../spark/rdd/CarbonDataRDDFactory.scala|  18 +-
 .../merger/NodeMultiBlockRelation.java  |  40 --
 .../processing/util/CarbonLoaderUtil.java   | 494 +++
 .../processing/util/CarbonLoaderUtilTest.java   | 125 -
 10 files changed, 183 insertions(+), 551 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/22bb333a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
 
b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
index a6bf60f..bcfeba0 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
@@ -114,14 +114,4 @@ public final class CarbonLoadOptionConstants {
*/
   public static final int MAX_EXTERNAL_DICTIONARY_SIZE = 1000;
 
-  /**
-   * enable block size based block allocation while loading data. By default, 
carbondata assigns
-   * blocks to node based on block number. If this option is set to `true`, 
carbondata will
-   * consider block size first and make sure that all the nodes will process 
almost equal size of
-   * data. This option is especially useful when you encounter skewed data.
-   */
-  @CarbonProperty
-  public static final String ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION
-  = "carbon.load.skewedDataOptimization.enabled";
-  public static final String 
ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION_DEFAULT = "false";
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/22bb333a/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
 
b/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
index c0cebe0..a7bfdba 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
@@ -19,8 +19,6 @@ package org.apache.carbondata.core.datastore.block;
 import java.io.IOException;
 import java.io.Serializable;
 import java.nio.charset.Charset;
-import java.util.Arrays;
-import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -100,20 +98,6 @@ public class TableBlockInfo implements Distributable, 
Serializable {
 
   private String dataMapWriterPath;
 
-  /**
-   * comparator to sort by block size in descending order.
-   * Since each line is not exactly the same, the size of a InputSplit may 
differs,
-   * so we allow some deviation for these splits.
-   */
-  public static final Comparator DATA_SIZE_DESC_COMPARATOR =
-  new Comparator() {
-@Override public int compare(Distributable o1, Distributable o2) {
-  long diff =
-  ((TableBlockInfo) o1).getBlockLength() - ((TableBlockInfo) 
o2).getBlockLength();
-  return diff < 0 ? 1 : (diff == 0 ? 0 : -1);
-}
-  };
-
   public TableBlockInfo(String filePath, long blockOffset, String segmentId,
   String[] locations, long blockLength, ColumnarFormatVersion version,
   String[] deletedDeltaFilePath) {
@@ -450,17 +434,4 @@ public class TableBlockInfo implements Distributable, 
Serializable {
   public void setDataMapWriterPath(String dataMapWriterPath) {
 this.dataMapWriterPath = dataMapWriterPath;
   }
-
-  @Override
-  public String toString() {
-final StringBuilder sb = new StringBuilder("TableBlockInfo{");
-sb.append("filePath='").append(filePa

[42/50] [abbrv] carbondata git commit: [CARBONDATA-2159] Remove carbon-spark dependency in store-sdk module

2018-03-02 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/c5740b19/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
--
diff --git 
a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
 
b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
new file mode 100644
index 000..fbb93b6
--- /dev/null
+++ 
b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
@@ -0,0 +1,322 @@
+/*
+ * 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.loading.model;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.carbondata.common.Maps;
+import org.apache.carbondata.common.Strings;
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.constants.LoggerAction;
+import org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
+import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.CarbonUtil;
+import 
org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants;
+import org.apache.carbondata.processing.loading.csvinput.CSVInputFormat;
+import org.apache.carbondata.processing.loading.sort.SortScopeOptions;
+import org.apache.carbondata.processing.util.TableOptionConstant;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Builder for {@link CarbonLoadModel}
+ */
+@InterfaceAudience.Developer
+public class CarbonLoadModelBuilder {
+
+  private CarbonTable table;
+
+  public CarbonLoadModelBuilder(CarbonTable table) {
+this.table = table;
+  }
+
+  /**
+   * build CarbonLoadModel for data loading
+   * @param options Load options from user input
+   * @return a new CarbonLoadModel instance
+   */
+  public CarbonLoadModel build(
+  Map options) throws InvalidLoadOptionException, 
IOException {
+Map optionsFinal = 
LoadOption.fillOptionWithDefaultValue(options);
+optionsFinal.put("sort_scope", "no_sort");
+if (!options.containsKey("fileheader")) {
+  List csvHeader = 
table.getCreateOrderColumn(table.getTableName());
+  String[] columns = new String[csvHeader.size()];
+  for (int i = 0; i < columns.length; i++) {
+columns[i] = csvHeader.get(i).getColName();
+  }
+  optionsFinal.put("fileheader", Strings.mkString(columns, ","));
+}
+CarbonLoadModel model = new CarbonLoadModel();
+
+// we have provided 'fileheader', so it hadoopConf can be null
+build(options, optionsFinal, model, null);
+
+// set default values
+
model.setTimestampformat(CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
+model.setDateFormat(CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT);
+model.setUseOnePass(Boolean.parseBoolean(Maps.getOrDefault(options, 
"onepass", "false")));
+model.setDictionaryServerHost(Maps.getOrDefault(options, "dicthost", 
null));
+try {
+  
model.setDictionaryServerPort(Integer.parseInt(Maps.getOrDefault(options, 
"dictport", "-1")));
+} catch (NumberFormatException e) {
+  throw new InvalidLoadOptionException(e.getMessage());
+}
+return model;
+  }
+
+  /**
+   * build CarbonLoadModel for data loading
+   * @param options Load options from user input
+   * @param optionsFinal Load options that populated with default values for 
optional options
+   * @param carbonLoadModel The output load model
+   * @param hadoopConf hadoopConf is needed to read CSV header if there 
'fileheader' is not set in
+   *   user provided load options
+   */
+  public void build(
+  Map options,
+  Map optionsFinal,
+  CarbonLoadModel carbonLoadModel,
+  Configuration hadoopConf) throws InvalidLoadOptionEx

[43/50] [abbrv] carbondata git commit: [CARBONDATA-2159] Remove carbon-spark dependency in store-sdk module

2018-03-02 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/c5740b19/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
--
diff --git 
a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
 
b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
index 8d394db..e69de29 100644
--- 
a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
+++ 
b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
@@ -1,610 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.util
-
-import java.text.SimpleDateFormat
-import java.util
-import java.util.{Date, List, Locale}
-
-import scala.collection.{immutable, mutable}
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ArrayBuffer
-
-import org.apache.commons.lang3.StringUtils
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.Path
-import org.apache.hadoop.mapred.JobConf
-import org.apache.hadoop.mapreduce.{TaskAttemptID, TaskType}
-import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat, FileSplit}
-import org.apache.hadoop.mapreduce.task.{JobContextImpl, 
TaskAttemptContextImpl}
-import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.GenericInternalRow
-import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, 
PartitionedFile}
-import org.apache.spark.sql.util.CarbonException
-import org.apache.spark.sql.util.SparkSQLUtil.sessionState
-
-import org.apache.carbondata.common.constants.LoggerAction
-import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
-import org.apache.carbondata.core.constants.{CarbonCommonConstants, 
CarbonLoadOptionConstants}
-import org.apache.carbondata.core.indexstore.PartitionSpec
-import org.apache.carbondata.core.locks.{CarbonLockFactory, CarbonLockUtil, 
LockUsage}
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, 
SegmentStatus, SegmentStatusManager}
-import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
-import 
org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants
-import org.apache.carbondata.processing.loading.csvinput.CSVInputFormat
-import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, 
CarbonLoadModel}
-import org.apache.carbondata.processing.util.{CarbonLoaderUtil, 
DeleteLoadFolders, TableOptionConstant}
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
-import org.apache.carbondata.spark.load.DataLoadProcessBuilderOnSpark.LOGGER
-import org.apache.carbondata.spark.load.ValidateUtil
-import org.apache.carbondata.spark.rdd.SerializableConfiguration
-
-/**
- * the util object of data loading
- */
-object DataLoadingUtil {
-
-  val LOGGER: LogService = 
LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-
-  /**
-   * get data loading options and initialise default value
-   */
-  def getDataLoadingOptions(
-  carbonProperty: CarbonProperties,
-  options: immutable.Map[String, String]): mutable.Map[String, String] = {
-val optionsFinal = scala.collection.mutable.Map[String, String]()
-optionsFinal.put("delimiter", options.getOrElse("delimiter", ","))
-optionsFinal.put("quotechar", options.getOrElse("quotechar", "\""))
-optionsFinal.put("fileheader", options.getOrElse("fileheader", ""))
-optionsFinal.put("commentchar", options.getOrElse("commentchar", "#"))
-optionsFinal.put("columndict", options.getOrElse("columndict", null))
-
-optionsFinal.put("escapechar",
-  CarbonLoaderUtil.getEscapeChar(options.getOrElse("escapechar", "\\")))
-
-optionsFinal.put(
-  "serialization_null_format",
-  options.getOrElse("ser

[25/50] [abbrv] carbondata git commit: [CARBONDATA-2080] [S3-Implementation] Propagated hadoopConf from driver to executor for s3 implementation in cluster mode.

2018-03-02 Thread jackylk
[CARBONDATA-2080] [S3-Implementation] Propagated hadoopConf from driver to 
executor for s3 implementation in cluster mode.

Problem : hadoopconf was not getting propagated from driver to the executor 
that's why load was failing to the distributed environment.
Solution: Setting the Hadoop conf in base class CarbonRDD
How to verify this PR :
Execute the load in the cluster mode It should be a success using location s3.

This closes #1860


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

Branch: refs/heads/carbonstore-rebase5
Commit: 8da481af6b2b14e16fe289f61efec4d5ae5e0378
Parents: 9daad35
Author: Jatin 
Authored: Thu Jan 25 16:53:00 2018 +0530
Committer: Jacky Li 
Committed: Fri Mar 2 15:52:34 2018 +0800

--
 .../spark/rdd/AlterTableAddColumnRDD.scala  |  2 +-
 .../spark/rdd/AlterTableDropColumnRDD.scala |  2 +-
 .../spark/rdd/CarbonCleanFilesRDD.scala |  2 +-
 .../spark/rdd/CarbonDeleteLoadByDateRDD.scala   |  2 +-
 .../spark/rdd/CarbonDeleteLoadRDD.scala |  2 +-
 .../spark/rdd/CarbonDropPartitionRDD.scala  |  2 +-
 .../spark/rdd/CarbonDropTableRDD.scala  |  2 +-
 .../spark/rdd/CarbonGlobalDictionaryRDD.scala   |  3 +-
 .../spark/rdd/CarbonMergeFilesRDD.scala |  0
 .../carbondata/spark/rdd/CarbonMergerRDD.scala  |  2 +-
 .../apache/carbondata/spark/rdd/CarbonRDD.scala | 32 ++--
 .../spark/rdd/NewCarbonDataLoadRDD.scala|  2 +-
 .../carbondata/spark/rdd/SparkDataMapJob.scala  |  2 +-
 .../apache/spark/rdd/DataLoadCoalescedRDD.scala |  3 +-
 .../apache/spark/rdd/UpdateCoalescedRDD.scala   |  2 +-
 .../carbondata/streaming/StreamHandoffRDD.scala |  2 +-
 16 files changed, 45 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/8da481af/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 56a66b9..7c1edea 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
@@ -50,7 +50,7 @@ class AddColumnPartition(rddId: Int, idx: Int, schema: 
ColumnSchema) extends Par
 class AlterTableAddColumnRDD[K, V](sc: SparkContext,
 @transient newColumns: Seq[ColumnSchema],
 identifier: AbsoluteTableIdentifier)
-  extends CarbonRDD[(Int, SegmentStatus)](sc, Nil) {
+  extends CarbonRDD[(Int, SegmentStatus)](sc, Nil, sc.hadoopConfiguration) {
 
   val lockType: String = 
CarbonProperties.getInstance.getProperty(CarbonCommonConstants.LOCK_TYPE,
 CarbonCommonConstants.CARBON_LOCK_TYPE_HDFS)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/8da481af/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 248f351..e14524e 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
@@ -48,7 +48,7 @@ class DropColumnPartition(rddId: Int, idx: Int, schema: 
ColumnSchema) extends Pa
 class AlterTableDropColumnRDD[K, V](sc: SparkContext,
 @transient newColumns: Seq[ColumnSchema],
 carbonTableIdentifier: AbsoluteTableIdentifier)
-  extends CarbonRDD[(Int, SegmentStatus)](sc, Nil) {
+  extends CarbonRDD[(Int, SegmentStatus)](sc, Nil, sc.hadoopConfiguration) {
 
   override def getPartitions: Array[Partition] = {
 newColumns.zipWithIndex.map { column =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/8da481af/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonCleanFilesRDD.scala
--
diff --git 
a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonCleanFilesRDD.scala
 
b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonCleanFilesRDD.scala
index 32523d8..9936a2a 100644
--- 
a/integration/spark-c

[29/50] [abbrv] carbondata git commit: [CARBONDATA-1544][Datamap] Datamap FineGrain implementation

2018-03-02 Thread jackylk
[CARBONDATA-1544][Datamap] Datamap FineGrain implementation

Implemented interfaces for FG datamap and integrated to filterscanner to use 
the pruned bitset from FG datamap.
FG Query flow as follows.
1.The user can add FG datamap to any table and implement there interfaces.
2. Any filter query which hits the table with datamap will call prune method of 
FGdatamap.
3. The prune method of FGDatamap return list FineGrainBlocklet , these 
blocklets contain the information of block, blocklet, page and rowids 
information as well.
4. The pruned blocklets are internally wriitten to file and returns only the 
block , blocklet and filepath information as part of Splits.
5. Based on the splits scanrdd schedule the tasks.
6. In filterscanner we check the datamapwriterpath from split and reNoteads the 
bitset if exists. And pass this bitset as input to it.

This closes #1471


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

Branch: refs/heads/carbonstore-rebase5
Commit: d099053285efa8267a050e299f9d8853735e4251
Parents: 6882f73
Author: ravipesala 
Authored: Wed Nov 15 19:48:40 2017 +0530
Committer: Jacky Li 
Committed: Fri Mar 2 15:52:35 2018 +0800

--
 .../carbondata/core/datamap/DataMapMeta.java|   8 +-
 .../core/datamap/DataMapStoreManager.java   |  30 +-
 .../carbondata/core/datamap/DataMapType.java|  21 +
 .../carbondata/core/datamap/TableDataMap.java   |  31 +-
 .../core/datamap/dev/AbstractDataMapWriter.java | 110 +
 .../core/datamap/dev/BlockletSerializer.java|  57 +++
 .../carbondata/core/datamap/dev/DataMap.java|   4 +-
 .../core/datamap/dev/DataMapFactory.java|  14 +-
 .../core/datamap/dev/DataMapWriter.java |  57 ---
 .../cgdatamap/AbstractCoarseGrainDataMap.java   |  24 +
 .../AbstractCoarseGrainDataMapFactory.java  |  34 ++
 .../dev/fgdatamap/AbstractFineGrainDataMap.java |  24 +
 .../AbstractFineGrainDataMapFactory.java|  38 ++
 .../carbondata/core/datastore/DataRefNode.java  |   6 +
 .../core/datastore/block/TableBlockInfo.java|  10 +
 .../impl/btree/AbstractBTreeLeafNode.java   |   5 +
 .../datastore/impl/btree/BTreeNonLeafNode.java  |   5 +
 .../carbondata/core/indexstore/Blocklet.java|  30 +-
 .../indexstore/BlockletDataMapIndexStore.java   |   6 -
 .../core/indexstore/BlockletDetailsFetcher.java |   8 +
 .../core/indexstore/ExtendedBlocklet.java   |  17 +
 .../core/indexstore/FineGrainBlocklet.java  | 120 +
 .../blockletindex/BlockletDataMap.java  |  15 +-
 .../blockletindex/BlockletDataMapFactory.java   |  55 ++-
 .../blockletindex/BlockletDataRefNode.java  |  27 +-
 .../indexstore/blockletindex/IndexWrapper.java  |  18 +
 .../core/indexstore/schema/FilterType.java  |  24 -
 .../executer/ExcludeFilterExecuterImpl.java |   3 +
 .../executer/IncludeFilterExecuterImpl.java |   3 +
 .../scanner/impl/BlockletFilterScanner.java |   2 +
 .../apache/carbondata/core/util/CarbonUtil.java |  98 +
 .../datamap/examples/MinMaxDataMap.java |  32 +-
 .../datamap/examples/MinMaxDataMapFactory.java  |  49 ++-
 .../datamap/examples/MinMaxDataWriter.java  |  36 +-
 .../examples/MinMaxIndexBlockDetails.java   |  13 -
 .../carbondata/hadoop/CarbonInputSplit.java |  21 +-
 .../hadoop/api/CarbonTableInputFormat.java  |  17 +-
 .../testsuite/datamap/CGDataMapTestCase.scala   | 361 +++
 .../testsuite/datamap/DataMapWriterSuite.scala  |  43 +-
 .../testsuite/datamap/FGDataMapTestCase.scala   | 440 +++
 .../TestInsertAndOtherCommandConcurrent.scala   |  21 +-
 .../carbondata/spark/rdd/CarbonScanRDD.scala|   7 +-
 .../TestStreamingTableOperation.scala   |   5 +-
 .../datamap/DataMapWriterListener.java  |  57 ++-
 .../store/CarbonFactDataHandlerModel.java   |  10 +-
 .../store/writer/AbstractFactDataWriter.java| 128 +-
 46 files changed, 1762 insertions(+), 382 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/d0990532/core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java 
b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java
index 7746acf..dd15ccb 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java
@@ -19,15 +19,15 @@ package org.apache.carbondata.core.datamap;
 
 import java.util.List;
 
-import org.apache.carbondata.core.indexstore.schema.FilterType;
+import

[02/50] [abbrv] carbondata git commit: [REBASE] Solve conflict after rebasing master

2018-03-02 Thread jackylk
[REBASE] Solve conflict after rebasing master


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

Branch: refs/heads/carbonstore-rebase5
Commit: 6944dd420a07f6fe694a7fed92ed256bea68d314
Parents: 1921393
Author: Jacky Li 
Authored: Thu Feb 1 00:25:31 2018 +0800
Committer: Jacky Li 
Committed: Fri Mar 2 15:51:53 2018 +0800

--
 .../hadoop/util/CarbonInputFormatUtil.java  | 20 +++
 .../spark/rdd/NewCarbonDataLoadRDD.scala| 21 ++--
 .../org/apache/spark/sql/CarbonSession.scala|  5 ++---
 3 files changed, 24 insertions(+), 22 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/6944dd42/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
--
diff --git 
a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
 
b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
index 514428b..056c27b 100644
--- 
a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
+++ 
b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
@@ -22,6 +22,8 @@ import java.text.SimpleDateFormat;
 import java.util.List;
 import java.util.Locale;
 
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
@@ -39,6 +41,7 @@ import org.apache.carbondata.core.scan.model.QueryMeasure;
 import org.apache.carbondata.core.scan.model.QueryModel;
 import org.apache.carbondata.hadoop.api.CarbonTableInputFormat;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobID;
@@ -159,4 +162,21 @@ public class CarbonInputFormatUtil {
 String jobtrackerID = createJobTrackerID(date);
 return new JobID(jobtrackerID, batch);
   }
+
+  public static void setS3Configurations(Configuration hadoopConf) {
+FileFactory.getConfiguration()
+.set("fs.s3a.access.key", hadoopConf.get("fs.s3a.access.key", ""));
+FileFactory.getConfiguration()
+.set("fs.s3a.secret.key", hadoopConf.get("fs.s3a.secret.key", ""));
+FileFactory.getConfiguration()
+.set("fs.s3a.endpoint", hadoopConf.get("fs.s3a.endpoint", ""));
+FileFactory.getConfiguration().set(CarbonCommonConstants.S3_ACCESS_KEY,
+hadoopConf.get(CarbonCommonConstants.S3_ACCESS_KEY, ""));
+FileFactory.getConfiguration().set(CarbonCommonConstants.S3_SECRET_KEY,
+hadoopConf.get(CarbonCommonConstants.S3_SECRET_KEY, ""));
+FileFactory.getConfiguration().set(CarbonCommonConstants.S3N_ACCESS_KEY,
+hadoopConf.get(CarbonCommonConstants.S3N_ACCESS_KEY, ""));
+FileFactory.getConfiguration().set(CarbonCommonConstants.S3N_SECRET_KEY,
+hadoopConf.get(CarbonCommonConstants.S3N_SECRET_KEY, ""));
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6944dd42/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 917fc88..e17824f 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
@@ -41,10 +41,10 @@ import 
org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.common.logging.impl.StandardLogService
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.compression.CompressorFactory
-import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, 
SegmentStatus}
 import org.apache.carbondata.core.util.{CarbonProperties, 
CarbonTimeStatisticsFactory, ThreadLocalTaskInfo}
 import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil
 import org.apache.carbondata.processing.loading.{DataLoadExecutor, 
FailureCauses, T

[40/50] [abbrv] carbondata git commit: [CARBONDATA-1997] Add CarbonWriter SDK API

2018-03-02 Thread jackylk
[CARBONDATA-1997] Add CarbonWriter SDK API

Added a new module called store-sdk, and added a CarbonWriter API, it can be 
used to write Carbondata files to a specified folder, without Spark and Hadoop 
dependency. User can use this API in any environment.

This closes #1967


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

Branch: refs/heads/carbonstore-rebase5
Commit: bf4973c62fee3e51749b3a1a2d939b05581bb069
Parents: 653c51b
Author: Jacky Li 
Authored: Sat Feb 10 19:44:23 2018 +0800
Committer: Jacky Li 
Committed: Fri Mar 2 15:52:36 2018 +0800

--
 .../org/apache/carbondata/common/Strings.java   |  40 
 .../apache/carbondata/common/StringsSuite.java  |  53 +
 .../core/metadata/schema/table/CarbonTable.java |   7 +
 .../schema/table/CarbonTableBuilder.java|  72 +++
 .../core/metadata/schema/table/TableSchema.java |   7 +
 .../schema/table/TableSchemaBuilder.java| 107 ++
 .../schema/table/CarbonTableBuilderSuite.java   |  86 
 .../metadata/schema/table/CarbonTableTest.java  |  12 +-
 .../schema/table/TableSchemaBuilderSuite.java   |  56 ++
 .../carbondata/spark/util/DataLoadingUtil.scala |  45 +
 pom.xml |   7 +
 store/sdk/pom.xml   | 130 +
 .../carbondata/sdk/file/CSVCarbonWriter.java|  89 +
 .../carbondata/sdk/file/CarbonWriter.java   |  51 +
 .../sdk/file/CarbonWriterBuilder.java   | 194 +++
 .../org/apache/carbondata/sdk/file/Field.java   |  74 +++
 .../org/apache/carbondata/sdk/file/Schema.java  |  74 +++
 .../sdk/file/CSVCarbonWriterSuite.java  | 127 
 18 files changed, 1225 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf4973c6/common/src/main/java/org/apache/carbondata/common/Strings.java
--
diff --git a/common/src/main/java/org/apache/carbondata/common/Strings.java 
b/common/src/main/java/org/apache/carbondata/common/Strings.java
new file mode 100644
index 000..23288dd
--- /dev/null
+++ b/common/src/main/java/org/apache/carbondata/common/Strings.java
@@ -0,0 +1,40 @@
+/*
+ * 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.common;
+
+import java.util.Objects;
+
+public class Strings {
+
+  /**
+   * Provide same function as mkString in Scala.
+   * This is added to avoid JDK 8 dependency.
+   */
+  public static String mkString(String[] strings, String delimeter) {
+Objects.requireNonNull(strings);
+Objects.requireNonNull(delimeter);
+StringBuilder builder = new StringBuilder();
+for (int i = 0; i < strings.length; i++) {
+  builder.append(strings[i]);
+  if (i != strings.length - 1) {
+builder.append(delimeter);
+  }
+}
+return builder.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf4973c6/common/src/test/java/org/apache/carbondata/common/StringsSuite.java
--
diff --git 
a/common/src/test/java/org/apache/carbondata/common/StringsSuite.java 
b/common/src/test/java/org/apache/carbondata/common/StringsSuite.java
new file mode 100644
index 000..65da32b
--- /dev/null
+++ b/common/src/test/java/org/apache/carbondata/common/StringsSuite.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org

[41/50] [abbrv] carbondata git commit: [CARBONDATA-2156] Add interface annotation

2018-03-02 Thread jackylk
[CARBONDATA-2156] Add interface annotation

InterfaceAudience and InterfaceStability annotation should be added for user 
and developer

1.InetfaceAudience can be User and Developer
2.InterfaceStability can be Stable, Evolving, Unstable

This closes #1968


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

Branch: refs/heads/carbonstore-rebase5
Commit: 653c51b1b64e6d55f9483bbca7e1e85869de665e
Parents: 78aa2cc
Author: Jacky Li 
Authored: Sun Feb 11 10:12:10 2018 +0800
Committer: Jacky Li 
Committed: Fri Mar 2 15:52:36 2018 +0800

--
 .../common/annotations/InterfaceAudience.java   | 58 
 .../common/annotations/InterfaceStability.java  | 69 
 2 files changed, 127 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/653c51b1/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceAudience.java
--
diff --git 
a/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceAudience.java
 
b/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceAudience.java
new file mode 100644
index 000..fa9729d
--- /dev/null
+++ 
b/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceAudience.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.common.annotations;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+
+/**
+ * This annotation is ported and modified from Apache Hadoop project.
+ *
+ * Annotation to inform users of a package, class or method's intended 
audience.
+ * Currently the audience can be {@link User}, {@link Developer}
+ *
+ * Public classes that are not marked with this annotation must be
+ * considered by default as {@link Developer}.
+ *
+ * External applications must only use classes that are marked {@link User}.
+ *
+ * Methods may have a different annotation that it is more restrictive
+ * compared to the audience classification of the class. Example: A class
+ * might be {@link User}, but a method may be {@link Developer}
+ */
+@InterfaceAudience.User
+@InterfaceStability.Evolving
+public class InterfaceAudience {
+  /**
+   * Intended for use by any project or application.
+   */
+  @Documented
+  @Retention(RetentionPolicy.RUNTIME)
+  public @interface User { }
+
+  /**
+   * Intended only for developers to extend interface for CarbonData project
+   * For example, new Datamap implementations.
+   */
+  @Documented
+  @Retention(RetentionPolicy.RUNTIME)
+  public @interface Developer { }
+
+  private InterfaceAudience() { } // Audience can't exist on its own
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/653c51b1/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceStability.java
--
diff --git 
a/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceStability.java
 
b/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceStability.java
new file mode 100644
index 000..b8e5e52
--- /dev/null
+++ 
b/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceStability.java
@@ -0,0 +1,69 @@
+/*
+ * 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, softw

[20/50] [abbrv] carbondata git commit: [CARBONDATA-2025] Unify all path construction through CarbonTablePath static method

2018-03-02 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/9daad358/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionUtil.java
--
diff --git 
a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionUtil.java
 
b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionUtil.java
index 2a69f0d..a4d3d2b 100644
--- 
a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionUtil.java
+++ 
b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionUtil.java
@@ -272,7 +272,7 @@ public class CarbonCompactionUtil {
   public static CarbonTable getNextTableToCompact(CarbonTable[] carbonTables,
   List skipList) {
 for (CarbonTable ctable : carbonTables) {
-  String metadataPath = ctable.getMetaDataFilepath();
+  String metadataPath = ctable.getMetadataPath();
   // check for the compaction required file and at the same time exclude 
the tables which are
   // present in the skip list.
   if (CarbonCompactionUtil.isCompactionRequiredForTable(metadataPath) && 
!skipList

http://git-wip-us.apache.org/repos/asf/carbondata/blob/9daad358/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
--
diff --git 
a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
 
b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
index c141636..89326a3 100644
--- 
a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
+++ 
b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
@@ -169,15 +169,13 @@ public final class CarbonDataMergerUtil {
 // End Timestamp.
 
 // Table Update Status Metadata Update.
-AbsoluteTableIdentifier absoluteTableIdentifier =
+AbsoluteTableIdentifier identifier =
 
carbonLoadModel.getCarbonDataLoadSchema().getCarbonTable().getAbsoluteTableIdentifier();
 
-CarbonTablePath carbonTablePath = 
CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
-
 SegmentUpdateStatusManager segmentUpdateStatusManager =
-new SegmentUpdateStatusManager(absoluteTableIdentifier);
+new SegmentUpdateStatusManager(identifier);
 
-SegmentStatusManager segmentStatusManager = new 
SegmentStatusManager(absoluteTableIdentifier);
+SegmentStatusManager segmentStatusManager = new 
SegmentStatusManager(identifier);
 
 ICarbonLock updateLock = 
segmentUpdateStatusManager.getTableUpdateStatusLock();
 ICarbonLock statusLock = segmentStatusManager.getTableStatusLock();
@@ -224,7 +222,7 @@ public final class CarbonDataMergerUtil {
   }
 
   LoadMetadataDetails[] loadDetails =
-  segmentStatusManager.readLoadMetadata(metaDataFilepath);
+  SegmentStatusManager.readLoadMetadata(metaDataFilepath);
 
   for (LoadMetadataDetails loadDetail : loadDetails) {
 if (loadsToMerge.contains(loadDetail)) {
@@ -237,18 +235,18 @@ public final class CarbonDataMergerUtil {
 }
   }
 
-  segmentUpdateStatusManager
-  .writeLoadDetailsIntoFile(Arrays.asList(updateLists), timestamp);
-  segmentStatusManager
-  
.writeLoadDetailsIntoFile(carbonTablePath.getTableStatusFilePath(), 
loadDetails);
+  segmentUpdateStatusManager.writeLoadDetailsIntoFile(
+  Arrays.asList(updateLists), timestamp);
+  SegmentStatusManager.writeLoadDetailsIntoFile(
+  
CarbonTablePath.getTableStatusFilePath(identifier.getTablePath()), loadDetails);
   status = true;
 } else {
   LOGGER.error("Not able to acquire the lock.");
   status = false;
 }
   } catch (IOException e) {
-LOGGER.error("Error while updating metadata. The metadata file path is 
" + carbonTablePath
-.getMetadataDirectoryPath());
+LOGGER.error("Error while updating metadata. The metadata file path is 
" +
+CarbonTablePath.getMetadataPath(identifier.getTablePath()));
 status = false;
 
   } finally {
@@ -284,9 +282,9 @@ public final class CarbonDataMergerUtil {
   String metaDataFilepath, String mergedLoadNumber, CarbonLoadModel 
carbonLoadModel,
   CompactionType compactionType, String segmentFile) throws IOException {
 boolean tableStatusUpdationStatus = false;
-AbsoluteTableIdentifier absoluteTableIdentifier =
+AbsoluteTableIdentifier identifier =
 
carbonLoadModel.getCarbonDataLoadSchema().getCarbonTable().getAbsoluteTableIdentifier();
-SegmentStatusManager segmentStatusManager = new 
SegmentStatusManager(absoluteTableIdentifier);
+SegmentStatusManager segmentStatusManager = new 
SegmentStatusManager(id

[32/50] [abbrv] carbondata git commit: Revert "[CARBONDATA-2018][DataLoad] Optimization in reading/writing for sort temp row"

2018-03-02 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/78aa2cc3/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
--
diff --git 
a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
 
b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
index 527452a..11b3d43 100644
--- 
a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
+++ 
b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
@@ -31,14 +31,15 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
-import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
+import org.apache.carbondata.core.util.DataTypeUtil;
+import 
org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
 import 
org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
-import 
org.apache.carbondata.processing.sort.sortdata.IntermediateSortTempRowComparator;
+import org.apache.carbondata.processing.sort.sortdata.NewRowComparator;
 import org.apache.carbondata.processing.sort.sortdata.SortParameters;
-import org.apache.carbondata.processing.sort.sortdata.TableFieldStat;
 
 public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
 
@@ -62,15 +63,21 @@ public class UnsafeSortTempFileChunkHolder implements 
SortTempChunkHolder {
* entry count
*/
   private int entryCount;
+
   /**
* return row
*/
-  private IntermediateSortTempRow returnRow;
+  private Object[] returnRow;
+  private int dimCnt;
+  private int complexCnt;
+  private int measureCnt;
+  private boolean[] isNoDictionaryDimensionColumn;
+  private DataType[] measureDataTypes;
   private int readBufferSize;
   private String compressorName;
-  private IntermediateSortTempRow[] currentBuffer;
+  private Object[][] currentBuffer;
 
-  private IntermediateSortTempRow[] backupBuffer;
+  private Object[][] backupBuffer;
 
   private boolean isBackupFilled;
 
@@ -93,21 +100,27 @@ public class UnsafeSortTempFileChunkHolder implements 
SortTempChunkHolder {
 
   private int numberOfObjectRead;
 
-  private TableFieldStat tableFieldStat;
-  private SortStepRowHandler sortStepRowHandler;
-  private Comparator comparator;
+  private int nullSetWordsLength;
+
+  private Comparator comparator;
+
   /**
* Constructor to initialize
*/
   public UnsafeSortTempFileChunkHolder(File tempFile, SortParameters 
parameters) {
 // set temp file
 this.tempFile = tempFile;
+this.dimCnt = parameters.getDimColCount();
+this.complexCnt = parameters.getComplexDimColCount();
+this.measureCnt = parameters.getMeasureColCount();
+this.isNoDictionaryDimensionColumn = 
parameters.getNoDictionaryDimnesionColumn();
+this.measureDataTypes = parameters.getMeasureDataType();
 this.readBufferSize = parameters.getBufferSize();
 this.compressorName = parameters.getSortTempCompressorName();
-this.tableFieldStat = new TableFieldStat(parameters);
-this.sortStepRowHandler = new SortStepRowHandler(tableFieldStat);
+
 this.executorService = Executors.newFixedThreadPool(1);
-comparator = new 
IntermediateSortTempRowComparator(parameters.getNoDictionarySortColumn());
+this.nullSetWordsLength = ((parameters.getMeasureColCount() - 1) >> 6) + 1;
+comparator = new NewRowComparator(parameters.getNoDictionarySortColumn());
 initialize();
   }
 
@@ -156,17 +169,11 @@ public class UnsafeSortTempFileChunkHolder implements 
SortTempChunkHolder {
*
* @throws CarbonSortKeyAndGroupByException problem while reading
*/
-  @Override
   public void readRow() throws CarbonSortKeyAndGroupByException {
 if (prefetch) {
   fillDataForPrefetch();
 } else {
-  try {
-this.returnRow = 
sortStepRowHandler.readIntermediateSortTempRowFromInputStream(stream);
-this.numberOfObjectRead++;
-  } catch (IOException e) {
-throw new CarbonSortKeyAndGroupByException("Problems while reading 
row", e);
-  }
+  this.returnRow = getRowFromStream();
 }
   }
 
@@ -200,22 +207,63 @@ public class UnsafeSortTempFileChunkHolder implements 
SortTempChunkHolder {
   }
 
   /**
-   * get a

[13/50] [abbrv] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

2018-03-02 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/636eb799/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 de97e82..540607d 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
@@ -34,8 +34,8 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.cache.dictionary.Dictionary;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
-import 
org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
+import 
org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionColumnPage;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import 
org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
@@ -58,7 +58,7 @@ import org.apache.carbondata.core.scan.filter.intf.RowImpl;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import 
org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import 
org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -68,20 +68,20 @@ public class RowLevelFilterExecuterImpl implements 
FilterExecuter {
 
   private static final LogService LOGGER =
   
LogServiceFactory.getLogService(RowLevelFilterExecuterImpl.class.getName());
-  protected List dimColEvaluatorInfoList;
-  protected List msrColEvalutorInfoList;
+  List dimColEvaluatorInfoList;
+  List msrColEvalutorInfoList;
   protected Expression exp;
   protected AbsoluteTableIdentifier tableIdentifier;
   protected SegmentProperties segmentProperties;
   /**
* it has index at which given dimension is stored in file
*/
-  protected int[] dimensionBlocksIndex;
+  int[] dimensionChunkIndex;
 
   /**
* it has index at which given measure is stored in file
*/
-  protected int[] measureBlocksIndex;
+  int[] measureChunkIndex;
 
   private Map complexDimensionInfoMap;
 
@@ -89,18 +89,18 @@ public class RowLevelFilterExecuterImpl implements 
FilterExecuter {
* flag to check whether the filter dimension is present in current block 
list of dimensions.
* Applicable for restructure scenarios
*/
-  protected boolean[] isDimensionPresentInCurrentBlock;
+  boolean[] isDimensionPresentInCurrentBlock;
 
   /**
* flag to check whether the filter measure is present in current block list 
of measures.
* Applicable for restructure scenarios
*/
-  protected boolean[] isMeasurePresentInCurrentBlock;
+  boolean[] isMeasurePresentInCurrentBlock;
 
   /**
* is dimension column data is natural sorted
*/
-  protected boolean isNaturalSorted;
+  boolean isNaturalSorted;
 
   /**
* date direct dictionary generator
@@ -124,10 +124,10 @@ public class RowLevelFilterExecuterImpl implements 
FilterExecuter {
 }
 if (this.dimColEvaluatorInfoList.size() > 0) {
   this.isDimensionPresentInCurrentBlock = new 
boolean[dimColEvaluatorInfoList.size()];
-  this.dimensionBlocksIndex = new int[dimColEvaluatorInfoList.size()];
+  this.dimensionChunkIndex = new int[dimColEvaluatorInfoList.size()];
 } else {
   this.isDimensionPresentInCurrentBlock = new boolean[]{false};
-  this.dimensionBlocksIndex = new int[]{0};
+  this.dimensionChunkIndex = new int[]{0};
 }
 if (null == msrColEvalutorInfoList) {
   this.msrColEvalutorInfoList = new 
ArrayList(20);
@@ -136,10 +136,10 @@ public class RowLevelFilterExecuterImpl implements 
FilterExecuter {
 }
 if (this.msrColEvalutorInfoList.size() > 0) {
   this.isMeasurePresentInCurrentBlock = new 
boolean[msrColEvalutorInfoList.size()];
-  this.measureBlocksIndex = new int[msrColEvalutorInfoList.size()];
+  this.measureChunkIndex = new int[msrColEvalutorInfoList.size()];
 } else {
   this.isMeasurePresentInCurrentBlock = new boolean[]{false};
-  this.measureBlocksIndex = new int[] {0};
+  this.measureChunkInde

[01/50] [abbrv] carbondata git commit: [CARBONDATA-2211] in case of DDL HandOff should not be execute in thread [Forced Update!]

2018-03-02 Thread jackylk
Repository: carbondata
Updated Branches:
  refs/heads/carbonstore-rebase5 8b94788a5 -> 7540cc9ca (forced update)


[CARBONDATA-2211] in case of DDL HandOff should not be execute in thread

1. DDL handoff will be executed in the blocking thread.
2. Auto handoff will be executed in a new non-blocking thread.

This closes #2008


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

Branch: refs/heads/carbonstore-rebase5
Commit: 7f7ea4d757d99b681512cb5f1369187f69f905c8
Parents: a0fc0be
Author: rahulforallp 
Authored: Tue Feb 27 21:50:20 2018 +0530
Committer: QiangCai 
Committed: Fri Mar 2 09:40:20 2018 +0800

--
 .../CarbonAlterTableCompactionCommand.scala|  2 +-
 .../carbondata/streaming/StreamHandoffRDD.scala| 17 +++--
 .../streaming/CarbonAppendableStreamSink.scala |  3 ++-
 3 files changed, 14 insertions(+), 8 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f7ea4d7/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
--
diff --git 
a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
 
b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
index f6019e4..9b9ca0e 100644
--- 
a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
+++ 
b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
@@ -181,7 +181,7 @@ case class CarbonAlterTableCompactionCommand(
 if (compactionType == CompactionType.STREAMING) {
   StreamHandoffRDD.startStreamingHandoffThread(
 carbonLoadModel,
-sqlContext.sparkSession)
+sqlContext.sparkSession, true)
   return
 }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f7ea4d7/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
--
diff --git 
a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
 
b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
index b03ee1e..a46ced5 100644
--- 
a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
+++ 
b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
@@ -279,15 +279,20 @@ object StreamHandoffRDD {
*/
   def startStreamingHandoffThread(
   carbonLoadModel: CarbonLoadModel,
-  sparkSession: SparkSession
+  sparkSession: SparkSession,
+  isDDL: Boolean
   ): Unit = {
-// start a new thread to execute streaming segment handoff
-val handoffThread = new Thread() {
-  override def run(): Unit = {
-iterateStreamingHandoff(carbonLoadModel, sparkSession)
+if (isDDL) {
+  iterateStreamingHandoff(carbonLoadModel, sparkSession)
+} else {
+  // start a new thread to execute streaming segment handoff
+  val handoffThread = new Thread() {
+override def run(): Unit = {
+  iterateStreamingHandoff(carbonLoadModel, sparkSession)
+}
   }
+  handoffThread.start()
 }
-handoffThread.start()
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f7ea4d7/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
--
diff --git 
a/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
 
b/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
index f2f9853..312d24e 100644
--- 
a/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
+++ 
b/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
@@ -178,7 +178,8 @@ class CarbonAppendableStreamSink(
   if (enableAutoHandoff) {
 StreamHandoffRDD.startStreamingHandoffThread(
   carbonLoadModel,
-  sparkSession)
+  sparkSession,
+  false)
   }
 }
   }



[26/50] [abbrv] carbondata git commit: [REBASE] Solve conflict after rebasing master

2018-03-02 Thread jackylk
[REBASE] Solve conflict after rebasing master


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

Branch: refs/heads/carbonstore-rebase5
Commit: 65daaca7de643f79e807a6d815a4cc6205ef2dad
Parents: d115c47
Author: Jacky Li 
Authored: Fri Feb 9 01:39:20 2018 +0800
Committer: Jacky Li 
Committed: Fri Mar 2 15:52:35 2018 +0800

--
 .../scan/filter/FilterExpressionProcessor.java  |  2 +-
 .../filter/executer/FalseFilterExecutor.java| 16 +++---
 .../FalseConditionalResolverImpl.java   |  4 ++--
 .../apache/carbondata/core/util/CarbonUtil.java |  1 -
 .../core/util/path/CarbonTablePath.java |  6 +++---
 .../spark/rdd/AggregateDataMapCompactor.scala   | 12 +--
 .../preaaggregate/PreAggregateListeners.scala   | 22 +---
 .../CarbonAlterTableDataTypeChangeCommand.scala | 17 ---
 .../schema/CarbonAlterTableRenameCommand.scala  |  3 ++-
 .../apache/spark/sql/hive/CarbonMetaStore.scala | 12 ++-
 .../processing/util/CarbonLoaderUtil.java   |  5 +++--
 11 files changed, 51 insertions(+), 49 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/65daaca7/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
 
b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
index b882b51..26b202f 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
@@ -398,7 +398,7 @@ public class FilterExpressionProcessor implements 
FilterProcessor {
 ConditionalExpression condExpression = null;
 switch (filterExpressionType) {
   case FALSE:
-return new FalseConditionalResolverImpl(expression, false, false, 
tableIdentifier);
+return new FalseConditionalResolverImpl(expression, false, false);
   case TRUE:
 return new TrueConditionalResolverImpl(expression, false, false);
   case EQUALS:

http://git-wip-us.apache.org/repos/asf/carbondata/blob/65daaca7/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FalseFilterExecutor.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FalseFilterExecutor.java
 
b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FalseFilterExecutor.java
index 2d2a15c..75a6ec3 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FalseFilterExecutor.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FalseFilterExecutor.java
@@ -21,7 +21,7 @@ import java.util.BitSet;
 
 import 
org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 
 /**
@@ -33,8 +33,8 @@ import org.apache.carbondata.core.util.BitSetGroup;
 public class FalseFilterExecutor implements FilterExecuter {
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blocksChunkHolder, boolean 
useBitsetPipeline)
-  throws FilterUnsupportedException, IOException {
+  public BitSetGroup applyFilter(RawBlockletColumnChunks blocksChunkHolder,
+  boolean useBitsetPipeline) throws FilterUnsupportedException, 
IOException {
 int numberOfPages = blocksChunkHolder.getDataBlock().numberOfPages();
 BitSetGroup group = new BitSetGroup(numberOfPages);
 for (int i = 0; i < numberOfPages; i++) {
@@ -44,17 +44,19 @@ public class FalseFilterExecutor implements FilterExecuter {
 return group;
   }
 
-  @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax)
+  @Override
+  public boolean applyFilter(RowIntf value, int dimOrdinalMax)
   throws FilterUnsupportedException, IOException {
 return false;
   }
 
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] 
blockMinValue) {
-
+  @Override
+  public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) 
{
 return new BitSet();
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws 
IOException {
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks block

[31/50] [abbrv] carbondata git commit: [REBASE] resolve conflict after rebasing to master

2018-03-02 Thread jackylk
[REBASE] resolve conflict after rebasing to master


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

Branch: refs/heads/carbonstore-rebase5
Commit: 90629314839bf639fda95c3d267983de1012cb05
Parents: b5908d2
Author: Jacky Li 
Authored: Tue Feb 27 08:51:25 2018 +0800
Committer: Jacky Li 
Committed: Fri Mar 2 15:52:35 2018 +0800

--
 .../carbondata/core/datamap/TableDataMap.java   |  2 +-
 .../core/datamap/dev/AbstractDataMapWriter.java |  5 ++--
 .../core/datamap/dev/DataMapFactory.java|  2 +-
 .../core/indexstore/BlockletDetailsFetcher.java |  2 +-
 .../indexstore/SegmentPropertiesFetcher.java|  3 +-
 .../blockletindex/BlockletDataMap.java  |  2 +-
 .../blockletindex/BlockletDataMapFactory.java   | 21 ++---
 .../core/metadata/SegmentFileStore.java |  2 +-
 .../RowLevelRangeGrtThanFiterExecuterImpl.java  |  1 +
 ...elRangeGrtrThanEquaToFilterExecuterImpl.java |  1 +
 ...velRangeLessThanEqualFilterExecuterImpl.java |  1 +
 .../RowLevelRangeLessThanFiterExecuterImpl.java |  1 +
 .../SegmentUpdateStatusManager.java | 26 
 .../apache/carbondata/core/util/CarbonUtil.java | 16 --
 .../testsuite/datamap/CGDataMapTestCase.scala   | 26 
 .../testsuite/datamap/DataMapWriterSuite.scala  | 19 ++--
 .../testsuite/datamap/FGDataMapTestCase.scala   | 31 +---
 .../iud/DeleteCarbonTableTestCase.scala |  2 +-
 .../TestInsertAndOtherCommandConcurrent.scala   | 14 +
 .../StandardPartitionTableCleanTestCase.scala   | 12 
 .../spark/rdd/NewCarbonDataLoadRDD.scala|  2 +-
 .../carbondata/spark/util/DataLoadingUtil.scala |  4 +--
 .../CreatePreAggregateTableCommand.scala|  2 +-
 .../apache/spark/sql/hive/CarbonRelation.scala  |  3 +-
 .../datamap/DataMapWriterListener.java  |  2 +-
 .../loading/model/CarbonLoadModel.java  |  2 +-
 .../processing/merger/CarbonDataMergerUtil.java | 15 +++---
 .../merger/CompactionResultSortProcessor.java   |  6 ++--
 .../merger/RowResultMergerProcessor.java|  6 ++--
 .../partition/spliter/RowResultProcessor.java   |  3 +-
 .../util/CarbonDataProcessorUtil.java   |  4 +--
 .../processing/util/CarbonLoaderUtil.java   |  2 +-
 32 files changed, 104 insertions(+), 136 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/90629314/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java 
b/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
index eed650e3..2a6ceaa 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
@@ -143,7 +143,7 @@ public final class TableDataMap extends 
OperationEventListener {
   blocklets.addAll(
   dataMap.prune(
   filterExp,
-  
segmentPropertiesFetcher.getSegmentProperties(distributable.getSegmentId()),
+  
segmentPropertiesFetcher.getSegmentProperties(distributable.getSegment()),
   partitions));
 }
 BlockletSerializer serializer = new BlockletSerializer();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/90629314/core/src/main/java/org/apache/carbondata/core/datamap/dev/AbstractDataMapWriter.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datamap/dev/AbstractDataMapWriter.java
 
b/core/src/main/java/org/apache/carbondata/core/datamap/dev/AbstractDataMapWriter.java
index bcc9bad..de6dcb1 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/datamap/dev/AbstractDataMapWriter.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/datamap/dev/AbstractDataMapWriter.java
@@ -18,6 +18,7 @@ package org.apache.carbondata.core.datamap.dev;
 
 import java.io.IOException;
 
+import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
@@ -35,10 +36,10 @@ public abstract class AbstractDataMapWriter {
 
   protected String writeDirectoryPath;
 
-  public AbstractDataMapWriter(AbsoluteTableIdentifier identifier, String 
segmentId,
+  public AbstractDataMapWriter(AbsoluteTableIdentifier identifier, Segment 
segment,
   String writeDirectoryPath) {
 this.identifier = identi

[05/50] [abbrv] carbondata git commit: [CARBONDATA-1992] Remove partitionId in CarbonTablePath

2018-03-02 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/2e28c156/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterProcessorStepImpl.java
--
diff --git 
a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterProcessorStepImpl.java
 
b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterProcessorStepImpl.java
index d321405..58009af 100644
--- 
a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterProcessorStepImpl.java
+++ 
b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterProcessorStepImpl.java
@@ -26,6 +26,7 @@ import org.apache.carbondata.core.datastore.row.CarbonRow;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.processing.loading.AbstractDataLoadProcessorStep;
 import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
 import org.apache.carbondata.processing.loading.DataField;
@@ -65,21 +66,21 @@ public class DataWriterProcessorStepImpl extends 
AbstractDataLoadProcessorStep {
 child.initialize();
   }
 
-  private String[] getStoreLocation(CarbonTableIdentifier tableIdentifier, 
String partitionId) {
+  private String[] getStoreLocation(CarbonTableIdentifier tableIdentifier) {
 String[] storeLocation = CarbonDataProcessorUtil
 .getLocalDataFolderLocation(tableIdentifier.getDatabaseName(),
-tableIdentifier.getTableName(), 
String.valueOf(configuration.getTaskNo()), partitionId,
-configuration.getSegmentId() + "", false, false);
+tableIdentifier.getTableName(), 
String.valueOf(configuration.getTaskNo()),
+configuration.getSegmentId(), false, false);
 CarbonDataProcessorUtil.createLocations(storeLocation);
 return storeLocation;
   }
 
-  public CarbonFactDataHandlerModel getDataHandlerModel(int partitionId) {
+  public CarbonFactDataHandlerModel getDataHandlerModel() {
 CarbonTableIdentifier tableIdentifier =
 configuration.getTableIdentifier().getCarbonTableIdentifier();
-String[] storeLocation = getStoreLocation(tableIdentifier, 
String.valueOf(partitionId));
+String[] storeLocation = getStoreLocation(tableIdentifier);
 return 
CarbonFactDataHandlerModel.createCarbonFactDataHandlerModel(configuration,
-storeLocation, partitionId, 0);
+storeLocation, 0, 0);
   }
 
   @Override public Iterator[] execute() throws 
CarbonDataLoadingException {
@@ -89,11 +90,11 @@ public class DataWriterProcessorStepImpl extends 
AbstractDataLoadProcessorStep {
 String tableName = tableIdentifier.getTableName();
 try {
   CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-  
.recordDictionaryValue2MdkAdd2FileTime(configuration.getPartitionId(),
+  
.recordDictionaryValue2MdkAdd2FileTime(CarbonTablePath.DEPRECATED_PATITION_ID,
   System.currentTimeMillis());
   int i = 0;
   for (Iterator iterator : iterators) {
-String[] storeLocation = getStoreLocation(tableIdentifier, 
String.valueOf(i));
+String[] storeLocation = getStoreLocation(tableIdentifier);
 
 CarbonFactDataHandlerModel model = CarbonFactDataHandlerModel
 .createCarbonFactDataHandlerModel(configuration, storeLocation, i, 
0);
@@ -147,10 +148,11 @@ public class DataWriterProcessorStepImpl extends 
AbstractDataLoadProcessorStep {
 
CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordTotalRecords(rowCounter.get());
 processingComplete(dataHandler);
 CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-.recordDictionaryValue2MdkAdd2FileTime(configuration.getPartitionId(),
+
.recordDictionaryValue2MdkAdd2FileTime(CarbonTablePath.DEPRECATED_PATITION_ID,
 System.currentTimeMillis());
 CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-.recordMdkGenerateTotalTime(configuration.getPartitionId(), 
System.currentTimeMillis());
+.recordMdkGenerateTotalTime(CarbonTablePath.DEPRECATED_PATITION_ID,
+System.currentTimeMillis());
   }
 
   private void processingComplete(CarbonFactHandler dataHandler) throws 
CarbonDataLoadingException {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2e28c156/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
--
diff --git 
a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
 
b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
index 3f1430d..c141636 100644
--- 
a/processing/src/main/

[22/50] [abbrv] carbondata git commit: [CARBONDATA-2025] Unify all path construction through CarbonTablePath static method

2018-03-02 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/9daad358/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatDirectoryStructureTest.java
--
diff --git 
a/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatDirectoryStructureTest.java
 
b/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatDirectoryStructureTest.java
index a1ccab3..4293536 100644
--- 
a/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatDirectoryStructureTest.java
+++ 
b/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatDirectoryStructureTest.java
@@ -39,21 +39,19 @@ public class CarbonFormatDirectoryStructureTest {
*/
   @Test public void testTablePathStructure() throws IOException {
 CarbonTableIdentifier tableIdentifier = new CarbonTableIdentifier("d1", 
"t1", UUID.randomUUID().toString());
-CarbonStorePath carbonStorePath = new CarbonStorePath(CARBON_STORE);
-AbsoluteTableIdentifier absoluteTableIdentifier =
+AbsoluteTableIdentifier identifier =
 AbsoluteTableIdentifier.from(CARBON_STORE + "/d1/t1", tableIdentifier);
-CarbonTablePath carbonTablePath = 
CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
-assertTrue(carbonTablePath.getPath().replace("\\", 
"/").equals(CARBON_STORE + "/d1/t1"));
-assertTrue(carbonTablePath.getSchemaFilePath().replace("\\", 
"/").equals(CARBON_STORE + "/d1/t1/Metadata/schema"));
-assertTrue(carbonTablePath.getTableStatusFilePath().replace("\\", "/")
+assertTrue(identifier.getTablePath().replace("\\", 
"/").equals(CARBON_STORE + "/d1/t1"));
+
assertTrue(CarbonTablePath.getSchemaFilePath(identifier.getTablePath()).replace("\\",
 "/").equals(CARBON_STORE + "/d1/t1/Metadata/schema"));
+
assertTrue(CarbonTablePath.getTableStatusFilePath(identifier.getTablePath()).replace("\\",
 "/")
 .equals(CARBON_STORE + "/d1/t1/Metadata/tablestatus"));
-assertTrue(carbonTablePath.getDictionaryFilePath("t1_c1").replace("\\", 
"/")
+
assertTrue(CarbonTablePath.getDictionaryFilePath(identifier.getTablePath(), 
"t1_c1").replace("\\", "/")
 .equals(CARBON_STORE + "/d1/t1/Metadata/t1_c1.dict"));
-
assertTrue(carbonTablePath.getDictionaryMetaFilePath("t1_c1").replace("\\", "/")
+
assertTrue(CarbonTablePath.getDictionaryMetaFilePath(identifier.getTablePath(), 
"t1_c1").replace("\\", "/")
 .equals(CARBON_STORE + "/d1/t1/Metadata/t1_c1.dictmeta"));
-assertTrue(carbonTablePath.getSortIndexFilePath("t1_c1").replace("\\", "/")
+
assertTrue(CarbonTablePath.getSortIndexFilePath(identifier.getTablePath(),"t1_c1").replace("\\",
 "/")
 .equals(CARBON_STORE + "/d1/t1/Metadata/t1_c1.sortindex"));
-assertTrue(carbonTablePath.getCarbonDataFilePath("2", 3, 4L,  0, 0, 
"999").replace("\\", "/")
+
assertTrue(CarbonTablePath.getCarbonDataFilePath(identifier.getTablePath(), 
"2", 3, 4L,  0, 0, "999").replace("\\", "/")
 .equals(CARBON_STORE + 
"/d1/t1/Fact/Part0/Segment_2/part-3-4_batchno0-0-999.carbondata"));
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/9daad358/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatSharedDictionaryTest.java
--
diff --git 
a/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatSharedDictionaryTest.java
 
b/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatSharedDictionaryTest.java
deleted file mode 100644
index 91384c1..000
--- 
a/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatSharedDictionaryTest.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.util.path;
-
-import java.io.IOException;
-
-import org.junit.Test;
-
-import static junit.framework.TestCase.assertTrue;
-
-/**
- * test shared dictionary paths
- */
-public class CarbonFormatSharedDictionaryTest {
-
-  private final String CARBON_STORE = "/opt/carbonstore";
-
-  /**
-   * test shared dictionary location
-   */
-  @Test public void testSharedDimentionLocation() throws IOException {
-as

[39/50] [abbrv] carbondata git commit: [CARBONDATA-2023][DataLoad] Add size base block allocation in data loading

2018-03-02 Thread jackylk
[CARBONDATA-2023][DataLoad] Add size base block allocation in data loading

Carbondata assign blocks to nodes at the beginning of data loading.
Previous block allocation strategy is block number based and it will
suffer skewed data problem if the size of input files differs a lot.

We introduced a size based block allocation strategy to optimize data
loading performance in skewed data scenario.

This closes #1808


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

Branch: refs/heads/carbonstore-rebase5
Commit: fbe97786a252a4502ac10e891292e35a08a2c11f
Parents: f9d46b1
Author: xuchuanyin 
Authored: Thu Feb 8 14:42:39 2018 +0800
Committer: Jacky Li 
Committed: Fri Mar 2 15:52:36 2018 +0800

--
 .../constants/CarbonLoadOptionConstants.java|  10 +
 .../core/datastore/block/TableBlockInfo.java|  29 ++
 .../carbondata/core/util/CarbonProperties.java  |  11 +
 docs/useful-tips-on-carbondata.md   |   1 +
 .../spark/rdd/NewCarbonDataLoadRDD.scala|   4 +-
 .../spark/sql/hive/DistributionUtil.scala   |   2 +-
 .../spark/rdd/CarbonDataRDDFactory.scala|  18 +-
 .../merger/NodeMultiBlockRelation.java  |  40 ++
 .../processing/util/CarbonLoaderUtil.java   | 480 ---
 .../processing/util/CarbonLoaderUtilTest.java   | 125 +
 10 files changed, 545 insertions(+), 175 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/fbe97786/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
 
b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
index bcfeba0..a6bf60f 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
@@ -114,4 +114,14 @@ public final class CarbonLoadOptionConstants {
*/
   public static final int MAX_EXTERNAL_DICTIONARY_SIZE = 1000;
 
+  /**
+   * enable block size based block allocation while loading data. By default, 
carbondata assigns
+   * blocks to node based on block number. If this option is set to `true`, 
carbondata will
+   * consider block size first and make sure that all the nodes will process 
almost equal size of
+   * data. This option is especially useful when you encounter skewed data.
+   */
+  @CarbonProperty
+  public static final String ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION
+  = "carbon.load.skewedDataOptimization.enabled";
+  public static final String 
ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION_DEFAULT = "false";
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fbe97786/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
 
b/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
index a7bfdba..c0cebe0 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
@@ -19,6 +19,8 @@ package org.apache.carbondata.core.datastore.block;
 import java.io.IOException;
 import java.io.Serializable;
 import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -98,6 +100,20 @@ public class TableBlockInfo implements Distributable, 
Serializable {
 
   private String dataMapWriterPath;
 
+  /**
+   * comparator to sort by block size in descending order.
+   * Since each line is not exactly the same, the size of a InputSplit may 
differs,
+   * so we allow some deviation for these splits.
+   */
+  public static final Comparator DATA_SIZE_DESC_COMPARATOR =
+  new Comparator() {
+@Override public int compare(Distributable o1, Distributable o2) {
+  long diff =
+  ((TableBlockInfo) o1).getBlockLength() - ((TableBlockInfo) 
o2).getBlockLength();
+  return diff < 0 ? 1 : (diff == 0 ? 0 : -1);
+}
+  };
+
   public TableBlockInfo(String filePath, long blockOffset, String segmentId,
   String[] locations, long blockLength, ColumnarFormatVersion version,
   String[] deletedDeltaFilePath) {
@@ -434,4 +450,17 @@ public class TableBlockInfo implements Distributable, 
Serializable {
   

[28/50] [abbrv] carbondata git commit: [CARBONDATA-1544][Datamap] Datamap FineGrain implementation

2018-03-02 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/d0990532/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataWriter.java
--
diff --git 
a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataWriter.java
 
b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataWriter.java
index 78544d3..fe0bbcf 100644
--- 
a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataWriter.java
+++ 
b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataWriter.java
@@ -19,7 +19,6 @@ package org.apache.carbondata.datamap.examples;
 
 import java.io.BufferedWriter;
 import java.io.DataOutputStream;
-import java.io.File;
 import java.io.IOException;
 import java.io.OutputStreamWriter;
 import java.util.ArrayList;
@@ -29,17 +28,18 @@ import java.util.Map;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datamap.dev.DataMapWriter;
+import org.apache.carbondata.core.datamap.dev.AbstractDataMapWriter;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.TableInfo;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
 
 import com.google.gson.Gson;
 
-public class MinMaxDataWriter implements DataMapWriter {
+public class MinMaxDataWriter extends AbstractDataMapWriter {
 
   private static final LogService LOGGER =
   LogServiceFactory.getLogService(TableInfo.class.getName());
@@ -50,17 +50,23 @@ public class MinMaxDataWriter implements DataMapWriter {
 
   private Map blockMinMaxMap;
 
-  private String blockPath;
+  private String dataWritePath;
 
+  public MinMaxDataWriter(AbsoluteTableIdentifier identifier, String segmentId,
+  String dataWritePath) {
+super(identifier, segmentId, dataWritePath);
+this.identifier = identifier;
+this.segmentId = segmentId;
+this.dataWritePath = dataWritePath;
+  }
 
-  @Override public void onBlockStart(String blockId, String blockPath) {
+  @Override public void onBlockStart(String blockId) {
 pageLevelMax = null;
 pageLevelMin = null;
 blockletLevelMax = null;
 blockletLevelMin = null;
 blockMinMaxMap = null;
 blockMinMaxMap = new HashMap();
-this.blockPath = blockPath;
   }
 
   @Override public void onBlockEnd(String blockId) {
@@ -161,7 +167,7 @@ public class MinMaxDataWriter implements DataMapWriter {
 List tempMinMaxIndexBlockDetails = null;
 tempMinMaxIndexBlockDetails = loadBlockDetails();
 try {
-  writeMinMaxIndexFile(tempMinMaxIndexBlockDetails, blockPath, blockId);
+  writeMinMaxIndexFile(tempMinMaxIndexBlockDetails, blockId);
 } catch (IOException ex) {
   LOGGER.info(" Unable to write the file");
 }
@@ -178,7 +184,6 @@ public class MinMaxDataWriter implements DataMapWriter {
   
tmpminMaxIndexBlockDetails.setMinValues(blockMinMaxMap.get(index).getMin());
   
tmpminMaxIndexBlockDetails.setMaxValues(blockMinMaxMap.get(index).getMax());
   tmpminMaxIndexBlockDetails.setBlockletId(index);
-  tmpminMaxIndexBlockDetails.setFilePath(this.blockPath);
   minMaxIndexBlockDetails.add(tmpminMaxIndexBlockDetails);
 }
 return minMaxIndexBlockDetails;
@@ -187,22 +192,19 @@ public class MinMaxDataWriter implements DataMapWriter {
   /**
* Write the data to a file. This is JSON format file.
* @param minMaxIndexBlockDetails
-   * @param blockPath
* @param blockId
* @throws IOException
*/
   public void writeMinMaxIndexFile(List 
minMaxIndexBlockDetails,
-  String blockPath, String blockId) throws IOException {
-String filePath = blockPath.substring(0, 
blockPath.lastIndexOf(File.separator) + 1) + blockId
-+ ".minmaxindex";
+  String blockId) throws IOException {
+String filePath = dataWritePath +"/" + blockId + ".minmaxindex";
 BufferedWriter brWriter = null;
 DataOutputStream dataOutStream = null;
 try {
   FileFactory.createNewFile(filePath, FileFactory.getFileType(filePath));
   dataOutStream = FileFactory.getDataOutputStream(filePath, 
FileFactory.getFileType(filePath));
   Gson gsonObjectToWrite = new Gson();
-  brWriter = new BufferedWriter(new OutputStreamWriter(dataOutStream,
-  CarbonCommonConstants.CARBON_DEFAULT_STREAM_ENCODEFORMAT));
+  brWriter = new BufferedWriter(new OutputStreamWriter(dataOutStream, 
"UTF-8"));
   String minmaxIndexData = 
g

[1/2] carbondata git commit: [REBASE] Solve conflict after merging master [Forced Update!]

2018-03-02 Thread jackylk
Repository: carbondata
Updated Branches:
  refs/heads/carbonstore-rebase5 7540cc9ca -> c305f309e (forced update)


http://git-wip-us.apache.org/repos/asf/carbondata/blob/c305f309/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
--
diff --git 
a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
 
b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
index d2faef5..142b2cb 100644
--- 
a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
+++ 
b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
@@ -21,7 +21,16 @@ import java.io.File;
 import java.io.IOException;
 import java.text.ParseException;
 import java.text.SimpleDateFormat;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Calendar;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
@@ -32,7 +41,6 @@ import 
org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.locks.ICarbonLock;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
 import org.apache.carbondata.core.mutate.DeleteDeltaBlockDetails;
@@ -592,10 +600,6 @@ public final class CarbonDataMergerUtil {
 List segmentsToBeMerged =
 new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
 
-CarbonTable carbonTable = 
carbonLoadModel.getCarbonDataLoadSchema().getCarbonTable();
-CarbonTableIdentifier tableIdentifier = 
carbonTable.getCarbonTableIdentifier();
-
-
 // total length
 long totalLength = 0;
 
@@ -1013,7 +1017,8 @@ public final class CarbonDataMergerUtil {
 CarbonFile[] updateDeltaFiles = null;
 Set uniqueBlocks = new HashSet();
 
-String segmentPath = 
CarbonTablePath.getSegmentPath(absoluteTableIdentifier.getTablePath(), 
seg.getSegmentNo());
+String segmentPath = CarbonTablePath.getSegmentPath(
+absoluteTableIdentifier.getTablePath(), seg.getSegmentNo());
 CarbonFile segDir =
 FileFactory.getCarbonFile(segmentPath, 
FileFactory.getFileType(segmentPath));
 CarbonFile[] allSegmentFiles = segDir.listFiles();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c305f309/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 ea11e22..ebcf944 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
@@ -405,8 +405,8 @@ public class CompactionResultSortProcessor extends 
AbstractResultProcessor {
   partitionSpec.getLocation().toString() + 
CarbonCommonConstants.FILE_SEPARATOR
   + carbonLoadModel.getFactTimeStamp() + ".tmp";
 } else {
-  carbonStoreLocation = CarbonDataProcessorUtil
-  .createCarbonStoreLocation(carbonLoadModel.getDatabaseName(), 
tableName, carbonLoadModel.getSegmentId());
+  carbonStoreLocation = CarbonDataProcessorUtil.createCarbonStoreLocation(
+  carbonLoadModel.getDatabaseName(), tableName, 
carbonLoadModel.getSegmentId());
 }
 CarbonFactDataHandlerModel carbonFactDataHandlerModel = 
CarbonFactDataHandlerModel
 .getCarbonFactDataHandlerModel(carbonLoadModel, carbonTable, 
segmentProperties, tableName,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c305f309/processing/src/main/java/org/apache/carbondata/processing/merger/RowResultMergerProcessor.java
--
diff --git 
a/processing/src/main/java/org/apache/carbondata/processing/merger/RowResultMergerProcessor.java
 
b/processing/src/main/java/org/apache/carbondata/processing/merger/RowResultMergerProcessor.java
index 278d5bb..2616def 100644
--- 
a/processing/src/main/java/org/apache/carbondata/processing/merger/RowResultMergerProcessor.java
+++ 
b/processing/src/main/java/org/apache/carbondata/processing/merger/RowResultMergerProcessor.java
@@

[2/2] carbondata git commit: [REBASE] Solve conflict after merging master

2018-03-02 Thread jackylk
[REBASE] Solve conflict after merging master


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

Branch: refs/heads/carbonstore-rebase5
Commit: c305f309e4556bfc98f01e0dff41b8411c992a12
Parents: 8a9dd8b
Author: Jacky Li 
Authored: Tue Feb 27 11:26:30 2018 +0800
Committer: Jacky Li 
Committed: Fri Mar 2 17:48:31 2018 +0800

--
 .../carbondata/core/datamap/dev/DataMap.java|   9 +-
 .../core/datamap/dev/DataMapFactory.java|   2 +-
 .../exception/ConcurrentOperationException.java |  16 +-
 .../core/indexstore/BlockletDetailsFetcher.java |   3 +-
 .../blockletindex/BlockletDataMap.java  |   3 +-
 .../blockletindex/SegmentIndexFileStore.java|   2 -
 .../core/metadata/PartitionMapFileStore.java|   0
 .../scan/executor/util/RestructureUtil.java |   6 +-
 .../statusmanager/SegmentStatusManager.java |  10 +-
 .../SegmentUpdateStatusManager.java |   7 +-
 datamap/examples/pom.xml| 145 +++--
 .../datamap/examples/MinMaxDataMap.java |   3 +-
 .../datamap/examples/MinMaxDataMapFactory.java  |  26 +--
 .../datamap/examples/MinMaxDataWriter.java  |   8 +-
 .../CarbonStreamSparkStreamingExample.scala |  14 +-
 ...CarbonStructuredStreamingWithRowParser.scala |   8 +-
 .../hadoop/api/CarbonTableInputFormat.java  |   5 +-
 .../preaggregate/TestPreAggCreateCommand.scala  |   2 +-
 .../TestInsertAndOtherCommandConcurrent.scala   |   2 +-
 .../StandardPartitionGlobalSortTestCase.scala   |   2 +-
 .../exception/ProcessMetaDataException.java |   2 +
 .../org/apache/carbondata/api/CarbonStore.scala |   6 +-
 .../carbondata/spark/load/CsvRDDHelper.scala| 157 +++
 .../load/DataLoadProcessBuilderOnSpark.scala|   3 +-
 .../carbondata/spark/util/CarbonScalaUtil.scala |   2 +-
 .../carbondata/spark/util/CommonUtil.scala  |   2 -
 .../command/carbonTableSchemaCommon.scala   |   6 +-
 .../CarbonAlterTableCompactionCommand.scala |   3 +-
 .../management/CarbonCleanFilesCommand.scala|   2 +-
 .../CarbonDeleteLoadByIdCommand.scala   |   2 +-
 .../CarbonDeleteLoadByLoadDateCommand.scala |   2 +-
 .../management/CarbonLoadDataCommand.scala  |  28 ++--
 .../CarbonProjectForDeleteCommand.scala |   2 +-
 .../CarbonProjectForUpdateCommand.scala |   2 +-
 .../schema/CarbonAlterTableRenameCommand.scala  |   2 +-
 .../command/table/CarbonDropTableCommand.scala  |   2 +-
 .../datasources/CarbonFileFormat.scala  |   3 -
 .../TestStreamingTableWithRowParser.scala   |   9 +-
 .../vectorreader/AddColumnTestCases.scala   |   1 +
 .../datamap/DataMapWriterListener.java  |   3 +-
 .../loading/model/CarbonLoadModelBuilder.java   |  34 +++-
 .../processing/loading/model/LoadOption.java|  15 +-
 .../processing/merger/CarbonDataMergerUtil.java |  19 ++-
 .../merger/CompactionResultSortProcessor.java   |   4 +-
 .../merger/RowResultMergerProcessor.java|   4 +-
 .../partition/spliter/RowResultProcessor.java   |   4 +-
 .../util/CarbonDataProcessorUtil.java   |   3 +-
 store/sdk/pom.xml   |   2 +-
 .../carbondata/sdk/file/CSVCarbonWriter.java|   8 +-
 49 files changed, 384 insertions(+), 221 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/c305f309/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java 
b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
index 4a68286..fdeacff 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
@@ -38,18 +38,13 @@ public interface DataMap {
   /**
* Prune the datamap with filter expression and partition information. It 
returns the list of
* blocklets where these filters can exist.
-   *
-   * @param filterExp
-   * @return
*/
-  List prune(FilterResolverIntf filterExp, SegmentProperties 
segmentProperties, List partitions);
+  List prune(FilterResolverIntf filterExp, SegmentProperties 
segmentProperties,
+  List partitions);
 
   // TODO Move this method to Abstract class
   /**
* Validate whether the current segment needs to be fetching the required 
data
-   *
-   * @param filterExp
-   * @return
*/
   boolean isScanRequired(FilterResolverIntf filterExp);
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c305f309/core/src/main/java/org/apache/carbondata/core/datamap/dev/

carbondata git commit: [CARBONDATA-2098] Optimize pre-aggregate documentation

2018-03-02 Thread jackylk
Repository: carbondata
Updated Branches:
  refs/heads/master 74f5d67c0 -> 3e36639ed


[CARBONDATA-2098] Optimize pre-aggregate documentation

optimize pre-aggregate documentation
move to separate file
add more examples

This closes #2022


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

Branch: refs/heads/master
Commit: 3e36639eda45933a036ea364dba1088c5f4e57ec
Parents: 74f5d67
Author: sraghunandan 
Authored: Fri Mar 2 17:02:39 2018 +0530
Committer: Jacky Li 
Committed: Sat Mar 3 09:44:02 2018 +0800

--
 docs/data-management-on-carbondata.md   | 242 --
 docs/preaggregate-guide.md  | 313 +++
 .../examples/PreAggregateTableExample.scala |  50 ++-
 3 files changed, 362 insertions(+), 243 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/3e36639e/docs/data-management-on-carbondata.md
--
diff --git a/docs/data-management-on-carbondata.md 
b/docs/data-management-on-carbondata.md
index ea80d41..2aa4a49 100644
--- a/docs/data-management-on-carbondata.md
+++ b/docs/data-management-on-carbondata.md
@@ -26,7 +26,6 @@ This tutorial is going to introduce all commands and data 
operations on CarbonDa
 * [UPDATE AND DELETE](#update-and-delete)
 * [COMPACTION](#compaction)
 * [PARTITION](#partition)
-* [PRE-AGGREGATE TABLES](#pre-aggregate-tables)
 * [BUCKETING](#bucketing)
 * [SEGMENT MANAGEMENT](#segment-management)
 
@@ -859,247 +858,6 @@ This tutorial is going to introduce all commands and data 
operations on CarbonDa
   * The partitioned column can be excluded from SORT_COLUMNS, this will let 
other columns to do the efficient sorting.
   * When writing SQL on a partition table, try to use filters on the partition 
column.
 
-
-## PRE-AGGREGATE TABLES
-  CarbonData supports pre aggregating of data so that OLAP kind of queries can 
fetch data 
-  much faster. Aggregate tables are created as datamaps so that the handling 
is as efficient as 
-  other indexing support. Users can create as many aggregate tables they 
require as datamaps to 
-  improve their query performance, provided the storage requirements and 
loading speeds are 
-  acceptable.
-  
-  For main table called **sales** which is defined as 
-  
-  ```
-  CREATE TABLE sales (
-order_time timestamp,
-user_id STRING,
-sex STRING,
-country STRING,
-quantity INT,
-price BIGINT)
-  STORED BY 'carbondata'
-  ```
-  
-  user can create pre-aggregate tables using the DDL
-  
-  ```
-  CREATE DATAMAP agg_sales
-  ON TABLE sales
-  USING "preaggregate"
-  AS
-  SELECT country, sex, sum(quantity), avg(price)
-  FROM sales
-  GROUP BY country, sex
-  ```
-  
-Functions supported in pre-aggregate tables
-
-| Function | Rollup supported |
-|---||
-| SUM | Yes |
-| AVG | Yes |
-| MAX | Yes |
-| MIN | Yes |
-| COUNT | Yes |
-
-
-# How pre-aggregate tables are selected
-For the main table **sales** and pre-aggregate table **agg_sales** created 
above, queries of the 
-kind
-```
-SELECT country, sex, sum(quantity), avg(price) from sales GROUP BY country, sex
-
-SELECT sex, sum(quantity) from sales GROUP BY sex
-
-SELECT sum(price), country from sales GROUP BY country
-``` 
-
-will be transformed by Query Planner to fetch data from pre-aggregate table 
**agg_sales**
-
-But queries of kind
-```
-SELECT user_id, country, sex, sum(quantity), avg(price) from sales GROUP BY 
user_id, country, sex
-
-SELECT sex, avg(quantity) from sales GROUP BY sex
-
-SELECT country, max(price) from sales GROUP BY country
-```
-
-will fetch the data from the main table **sales**
-
-# Loading data to pre-aggregate tables
-For existing table with loaded data, data load to pre-aggregate table will be 
triggered by the 
-CREATE DATAMAP statement when user creates the pre-aggregate table.
-For incremental loads after aggregates tables are created, loading data to 
main table triggers 
-the load to pre-aggregate tables once main table loading is complete. These 
loads are automic 
-meaning that data on main table and aggregate tables are only visible to the 
user after all tables 
-are loaded
-
-# Querying data from pre-aggregate tables
-Pre-aggregate tables cannot be queries directly. Queries are to be made on 
main table. Internally 
-carbondata will check associated pre-aggregate tables with the main table, and 
if the 
-pre-aggregate tables satisfy the query condition, the plan is transformed 
automatically to use 
-pre-aggregate table to fetch the data.
-
-# Compacting pre-aggreg

carbondata git commit: [CARBONDATA-2204] Optimized number of reads of tablestatus file while querying

2018-03-02 Thread jackylk
Repository: carbondata
Updated Branches:
  refs/heads/master bfd77f69f -> c125f0caa


[CARBONDATA-2204] Optimized number of reads of tablestatus file while querying

This PR avoid reading status file multiple times. For first time query, it 
reads 2 times(Needed for datamap refresher) and 1 time for second query onwards.

This closes #1999


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

Branch: refs/heads/master
Commit: c125f0caa58f1e7cfa7d10b52ab53364f1895c35
Parents: bfd77f6
Author: ravipesala 
Authored: Mon Feb 26 18:36:03 2018 +0530
Committer: Jacky Li 
Committed: Sat Mar 3 13:50:16 2018 +0800

--
 .../core/constants/CarbonCommonConstants.java   |   4 -
 .../statusmanager/SegmentStatusManager.java | 122 +--
 .../SegmentUpdateStatusManager.java |  28 +++--
 .../core/util/path/CarbonTablePath.java |   9 +-
 .../hadoop/api/CarbonTableInputFormat.java  |  24 ++--
 .../hadoop/test/util/StoreCreator.java  |   8 +-
 .../presto/util/CarbonDataStoreCreator.scala|   2 +-
 .../carbondata/processing/StoreCreator.java |   4 +-
 8 files changed, 108 insertions(+), 93 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/c125f0ca/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
 
b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index fa2b7d8..b2a3375 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
@@ -650,10 +650,6 @@ public final class CarbonCommonConstants {
*/
   public static final int DEFAULT_MAX_QUERY_EXECUTION_TIME = 60;
   /**
-   * LOADMETADATA_FILENAME
-   */
-  public static final String LOADMETADATA_FILENAME = "tablestatus";
-  /**
* TABLE UPDATE STATUS FILENAME
*/
   public static final String TABLEUPDATESTATUS_FILENAME = "tableupdatestatus";

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c125f0ca/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
 
b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
index 76c2dc7..ab849ce 100755
--- 
a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
@@ -98,6 +98,15 @@ public class SegmentStatusManager {
* @throws IOException
*/
   public ValidAndInvalidSegmentsInfo getValidAndInvalidSegments() throws 
IOException {
+return getValidAndInvalidSegments(null);
+  }
+
+  /**
+   * get valid segment for given load status details.
+   *
+   */
+  public ValidAndInvalidSegmentsInfo getValidAndInvalidSegments(
+  LoadMetadataDetails[] loadMetadataDetails) throws IOException {
 
 // @TODO: move reading LoadStatus file to separate class
 List listOfValidSegments = new ArrayList<>(10);
@@ -108,73 +117,56 @@ public class SegmentStatusManager {
 CarbonTablePath carbonTablePath = CarbonStorePath
 .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
 absoluteTableIdentifier.getCarbonTableIdentifier());
-String dataPath = carbonTablePath.getTableStatusFilePath();
-DataInputStream dataInputStream = null;
 
-// Use GSON to deserialize the load information
-Gson gson = new Gson();
-
-AtomicFileOperations fileOperation =
-new AtomicFileOperationsImpl(dataPath, 
FileFactory.getFileType(dataPath));
-LoadMetadataDetails[] loadFolderDetailsArray;
 try {
-  if (FileFactory.isFileExist(dataPath, 
FileFactory.getFileType(dataPath))) {
-dataInputStream = fileOperation.openForRead();
-BufferedReader buffReader =
-new BufferedReader(new InputStreamReader(dataInputStream, 
"UTF-8"));
-loadFolderDetailsArray = gson.fromJson(buffReader, 
LoadMetadataDetails[].class);
-// if loadFolderDetailsArray is null, assign a empty array
-if (null == loadFolderDetailsArray) {
-  loadFolderDetailsArray = new LoadMetadataDetails[0];
-}
-//just directly iterate Array
-for (LoadMetadataDetails segment : loadFolderDetailsArray) {
-  if (Segm

[47/50] [abbrv] carbondata git commit: Support generating assembling JAR for store-sdk module

2018-03-04 Thread jackylk
Support generating assembling JAR for store-sdk module

Support generating assembling JAR for store-sdk module and remove junit 
dependency

This closes #1976


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

Branch: refs/heads/carbonstore-rebase5
Commit: d32c0cf306a331f2f7e23c57438d2c4acff52620
Parents: e83d910
Author: Jacky Li 
Authored: Tue Feb 13 09:12:09 2018 +0800
Committer: Jacky Li 
Committed: Sun Mar 4 20:08:30 2018 +0800

--
 common/pom.xml|  2 +
 core/pom.xml  |  2 +
 hadoop/pom.xml|  1 +
 integration/presto/pom.xml|  3 +-
 integration/spark-common-cluster-test/pom.xml |  2 +-
 integration/spark-common-test/pom.xml |  3 +-
 integration/spark-common/pom.xml  |  2 +-
 integration/spark2/pom.xml|  2 +-
 pom.xml   |  5 +++
 processing/pom.xml|  1 +
 store/sdk/pom.xml | 50 +-
 streaming/pom.xml |  1 -
 12 files changed, 66 insertions(+), 8 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/d32c0cf3/common/pom.xml
--
diff --git a/common/pom.xml b/common/pom.xml
index 5550129..433d575 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -42,10 +42,12 @@
 
   junit
   junit
+  test
 
 
   org.jmockit
   jmockit
+  test
 
 
   org.apache.hadoop

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d32c0cf3/core/pom.xml
--
diff --git a/core/pom.xml b/core/pom.xml
index 92c9607..824de0d 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -70,10 +70,12 @@
 
   org.jmockit
   jmockit
+  test
 
 
   junit
   junit
+  test
 
 
   org.apache.spark

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d32c0cf3/hadoop/pom.xml
--
diff --git a/hadoop/pom.xml b/hadoop/pom.xml
index 2aaac99..c3964c5 100644
--- a/hadoop/pom.xml
+++ b/hadoop/pom.xml
@@ -42,6 +42,7 @@
 
   junit
   junit
+  test
 
   
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d32c0cf3/integration/presto/pom.xml
--
diff --git a/integration/presto/pom.xml b/integration/presto/pom.xml
index aaaf175..0abcf38 100644
--- a/integration/presto/pom.xml
+++ b/integration/presto/pom.xml
@@ -193,7 +193,7 @@
 
 
   org.scalatest
-  scalatest_2.11
+  scalatest_${scala.binary.version}
 
 
   org.apache.zookeeper
@@ -330,7 +330,6 @@
 
   org.scalatest
   scalatest_${scala.binary.version}
-  2.2.1
   test
 
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d32c0cf3/integration/spark-common-cluster-test/pom.xml
--
diff --git a/integration/spark-common-cluster-test/pom.xml 
b/integration/spark-common-cluster-test/pom.xml
index fd907a3..028da11 100644
--- a/integration/spark-common-cluster-test/pom.xml
+++ b/integration/spark-common-cluster-test/pom.xml
@@ -49,11 +49,11 @@
 
   junit
   junit
+  test
 
 
   org.scalatest
   scalatest_${scala.binary.version}
-  2.2.1
   test
 
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d32c0cf3/integration/spark-common-test/pom.xml
--
diff --git a/integration/spark-common-test/pom.xml 
b/integration/spark-common-test/pom.xml
index 67a2317..d1c04ae 100644
--- a/integration/spark-common-test/pom.xml
+++ b/integration/spark-common-test/pom.xml
@@ -106,16 +106,17 @@
 
   junit
   junit
+  test
 
 
   org.scalatest
   scalatest_${scala.binary.version}
-  2.2.1
   test
 
 
   org.jmockit
   jmockit
+  test
 
   
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d32c0cf3/integration/spark-common/pom.xml
--
diff --git a/integration/spark-common/pom.xml b/integration/spark-common/pom.xml
index 295d62b..16f327d 100644
--- a/integration/spark-common/pom.xml
+++ b/integration/spark-common/pom.xml
@@ -58,11 +58,11 @@
 
   junit
   junit
+  test

[20/50] [abbrv] carbondata git commit: [CARBONDATA-2025] Unify all path construction through CarbonTablePath static method

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
--
diff --git 
a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
 
b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
index 40b5cfc..753e637 100644
--- 
a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
+++ 
b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
@@ -34,7 +34,7 @@ import 
org.apache.carbondata.core.metadata.CarbonTableIdentifier
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
 import org.apache.carbondata.core.util.CarbonUtil
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.{AlterTableRenamePostEvent, 
AlterTableRenamePreEvent, OperationContext, OperationListenerBus}
 import org.apache.carbondata.format.SchemaEvolutionEntry
 import org.apache.carbondata.spark.exception.{ConcurrentOperationException, 
MalformedCarbonCommandException}
@@ -97,8 +97,7 @@ private[sql] case class CarbonAlterTableRenameCommand(
   val oldTableIdentifier = carbonTable.getAbsoluteTableIdentifier
   DataMapStoreManager.getInstance().clearDataMaps(oldTableIdentifier)
   // get the latest carbon table and check for column existence
-  val oldTablePath = CarbonStorePath.getCarbonTablePath(oldTableIdentifier)
-  val tableMetadataFile = oldTablePath.getPath
+  val tableMetadataFile = oldTableIdentifier.getTablePath
   val operationContext = new OperationContext
   // TODO: Pass new Table Path in pre-event.
   val alterTableRenamePreEvent: AlterTableRenamePreEvent = 
AlterTableRenamePreEvent(
@@ -108,7 +107,7 @@ private[sql] case class CarbonAlterTableRenameCommand(
 sparkSession)
   OperationListenerBus.getInstance().fireEvent(alterTableRenamePreEvent, 
operationContext)
   val tableInfo: org.apache.carbondata.format.TableInfo =
-metastore.getThriftTableInfo(oldTablePath)(sparkSession)
+metastore.getThriftTableInfo(carbonTable)(sparkSession)
   val schemaEvolutionEntry = new 
SchemaEvolutionEntry(System.currentTimeMillis)
   schemaEvolutionEntry.setTableName(newTableName)
   timeStamp = System.currentTimeMillis()
@@ -117,7 +116,8 @@ private[sql] case class CarbonAlterTableRenameCommand(
   val fileType = FileFactory.getFileType(tableMetadataFile)
   val newTableIdentifier = new CarbonTableIdentifier(oldDatabaseName,
 newTableName, carbonTable.getCarbonTableIdentifier.getTableId)
-  var newTablePath = CarbonUtil.getNewTablePath(oldTablePath, 
newTableIdentifier.getTableName)
+  var newTablePath = CarbonTablePath.getNewTablePath(
+oldTableIdentifier.getTablePath, newTableIdentifier.getTableName)
   metastore.removeTableFromMetadata(oldDatabaseName, oldTableName)
   val hiveClient = 
sparkSession.sessionState.catalog.asInstanceOf[CarbonSessionCatalog]
 .getClient()
@@ -139,9 +139,9 @@ private[sql] case class CarbonAlterTableRenameCommand(
   // changed the rename order to deal with situation when carbon table and 
hive table
   // will point to the same tablePath
   if (FileFactory.isFileExist(tableMetadataFile, fileType)) {
-val rename = FileFactory.getCarbonFile(oldTablePath.getPath, fileType)
-  .renameForce(oldTablePath.getParent.toString + 
CarbonCommonConstants.FILE_SEPARATOR +
-   newTableName)
+val rename = 
FileFactory.getCarbonFile(oldTableIdentifier.getTablePath, fileType)
+  .renameForce(
+CarbonTablePath.getNewTablePath(oldTableIdentifier.getTablePath, 
newTableName))
 if (!rename) {
   renameBadRecords(newTableName, oldTableName, oldDatabaseName)
   sys.error(s"Folder rename failed for table 
$oldDatabaseName.$oldTableName")
@@ -149,7 +149,7 @@ private[sql] case class CarbonAlterTableRenameCommand(
   }
   val updatedParts = updatePartitionLocations(
 partitions,
-oldTablePath.getPath,
+oldTableIdentifier.getTablePath,
 newTablePath,
 sparkSession)
 
@@ -191,13 +191,11 @@ private[sql] case class CarbonAlterTableRenameCommand(
   case e: Exception =>
 LOGGER.error(e, "Rename table failed: " + e.getMessage)
 if (carbonTable != null) {
-  AlterTableUtil
-.revertRenameTableChanges(oldTableIdentifier,
-  newTableName,
-  carbonTable.getTablePath,
-  carbonTable.getCarbo

[10/50] [abbrv] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/55c4e438/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
 
b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
index 447ab46..547ecaa 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
@@ -22,7 +22,7 @@ import java.util.List;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
@@ -35,12 +35,11 @@ import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.scan.expression.Expression;
-import 
org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import 
org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import 
org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -73,7 +72,7 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends 
RowLevelFilterExecut
   comparator = 
Comparator.getComparatorByDataTypeForMeasure(measure.getDataType());
 }
 ifDefaultValueMatchesFilter();
-if (isDimensionPresentInCurrentBlock[0] == true) {
+if (isDimensionPresentInCurrentBlock[0]) {
   isNaturalSorted = 
dimColEvaluatorInfoList.get(0).getDimension().isUseInvertedIndex()
   && dimColEvaluatorInfoList.get(0).getDimension().isSortColumn();
 }
@@ -120,11 +119,11 @@ public class RowLevelRangeLessThanFiterExecuterImpl 
extends RowLevelFilterExecut
 boolean isScanRequired = false;
 if (isMeasurePresentInCurrentBlock[0] || 
isDimensionPresentInCurrentBlock[0]) {
   if (isMeasurePresentInCurrentBlock[0]) {
-minValue = blockMinValue[measureBlocksIndex[0] + 
lastDimensionColOrdinal];
+minValue = blockMinValue[measureChunkIndex[0] + 
lastDimensionColOrdinal];
 isScanRequired =
 isScanRequired(minValue, msrFilterRangeValues, 
msrColEvalutorInfoList.get(0).getType());
   } else {
-minValue = blockMinValue[dimensionBlocksIndex[0]];
+minValue = blockMinValue[dimensionChunkIndex[0]];
 isScanRequired = isScanRequired(minValue, filterRangeValues);
   }
 } else {
@@ -170,67 +169,69 @@ public class RowLevelRangeLessThanFiterExecuterImpl 
extends RowLevelFilterExecut
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean 
useBitsetPipeLine)
-  throws FilterUnsupportedException, IOException {
+  public BitSetGroup applyFilter(RawBlockletColumnChunks 
rawBlockletColumnChunks,
+  boolean useBitsetPipeLine) throws IOException {
 // select all rows if dimension does not exists in the current block
 if (!isDimensionPresentInCurrentBlock[0] && 
!isMeasurePresentInCurrentBlock[0]) {
-  int numberOfRows = blockChunkHolder.getDataBlock().nodeSize();
+  int numberOfRows = rawBlockletColumnChunks.getDataBlock().numRows();
   return FilterUtil
-  
.createBitSetGroupWithDefaultValue(blockChunkHolder.getDataBlock().numberOfPages(),
+  
.createBitSetGroupWithDefaultValue(rawBlockletColumnChunks.getDataBlock().numberOfPages(),
   numberOfRows, true);
 }
 if (isDimensionPresentInCurrentBlock[0]) {
-  int blockIndex =
-  
segmentProperties.getDimensionOrdinalToBlockMapping().get(dimensionBlocksIndex[0]);
-  if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = 
blockChunkHolder.getDataBlock()
-.getDimensionC

[17/50] [abbrv] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

2018-03-04 Thread jackylk
[CARBONDATA-2099] Refactor query scan process to improve readability

Unified concepts in scan process flow:

1.QueryModel contains all parameter for scan, it is created by API in 
CarbonTable. (In future, CarbonTable will be the entry point for various table 
operations)
2.Use term ColumnChunk to represent one column in one blocklet, and use 
ChunkIndex in reader to read specified column chunk
3.Use term ColumnPage to represent one page in one ColumnChunk
4.QueryColumn => ProjectionColumn, indicating it is for projection

This closes #1874


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

Branch: refs/heads/carbonstore-rebase5
Commit: 55c4e43828d3918fb8657ca7279a89a3cad4d667
Parents: 03157f9
Author: Jacky Li 
Authored: Tue Jan 30 21:24:04 2018 +0800
Committer: Jacky Li 
Committed: Sun Mar 4 18:15:06 2018 +0800

--
 .../dictionary/AbstractDictionaryCache.java |   3 +-
 .../cache/dictionary/DictionaryCacheLoader.java |   7 +-
 .../dictionary/DictionaryCacheLoaderImpl.java   |  11 +-
 .../core/datastore/BTreeBuilderInfo.java|   6 -
 .../carbondata/core/datastore/DataRefNode.java  |  81 +--
 .../carbondata/core/datastore/FileHolder.java   | 118 
 .../carbondata/core/datastore/FileReader.java   | 114 +++
 .../core/datastore/block/SegmentProperties.java |  50 +-
 .../chunk/DimensionColumnDataChunk.java | 116 ---
 .../datastore/chunk/DimensionColumnPage.java| 111 +++
 .../chunk/impl/AbstractDimensionColumnPage.java |  89 +++
 .../chunk/impl/AbstractDimensionDataChunk.java  |  95 ---
 .../impl/ColumnGroupDimensionColumnPage.java| 194 ++
 .../impl/ColumnGroupDimensionDataChunk.java | 194 --
 .../chunk/impl/DimensionRawColumnChunk.java |  46 +-
 .../impl/FixedLengthDimensionColumnPage.java| 163 +
 .../impl/FixedLengthDimensionDataChunk.java | 163 -
 .../chunk/impl/MeasureRawColumnChunk.java   |  26 +-
 .../impl/VariableLengthDimensionColumnPage.java | 133 
 .../impl/VariableLengthDimensionDataChunk.java  | 140 
 .../reader/DimensionColumnChunkReader.java  |  14 +-
 .../chunk/reader/MeasureColumnChunkReader.java  |  12 +-
 .../AbstractChunkReaderV2V3Format.java  |  34 +-
 ...mpressedDimensionChunkFileBasedReaderV1.java |  38 +-
 ...mpressedDimensionChunkFileBasedReaderV2.java |  30 +-
 ...essedDimChunkFileBasedPageLevelReaderV3.java |  11 +-
 ...mpressedDimensionChunkFileBasedReaderV3.java |  49 +-
 .../AbstractMeasureChunkReaderV2V3Format.java   |  42 +-
 ...CompressedMeasureChunkFileBasedReaderV1.java |  16 +-
 ...CompressedMeasureChunkFileBasedReaderV2.java |  24 +-
 ...CompressedMeasureChunkFileBasedReaderV3.java |  45 +-
 ...essedMsrChunkFileBasedPageLevelReaderV3.java |   8 +-
 .../chunk/store/ColumnPageWrapper.java  |  30 +-
 .../chunk/store/DimensionDataChunkStore.java|   8 +-
 .../SafeFixedLengthDimensionDataChunkStore.java |   6 +-
 ...feVariableLengthDimensionDataChunkStore.java |   8 +-
 ...nsafeFixedLengthDimensionDataChunkStore.java |  10 +-
 ...afeVariableLengthDimesionDataChunkStore.java |  10 +-
 .../datastore/columnar/ColumnGroupModel.java|  26 -
 .../core/datastore/impl/DFSFileHolderImpl.java  | 166 -
 .../core/datastore/impl/DFSFileReaderImpl.java  | 155 
 .../datastore/impl/DefaultFileTypeProvider.java |  16 +-
 .../core/datastore/impl/FileFactory.java|   4 +-
 .../core/datastore/impl/FileHolderImpl.java | 224 --
 .../core/datastore/impl/FileReaderImpl.java | 215 ++
 .../core/datastore/impl/FileTypeInerface.java   |   4 +-
 .../impl/btree/AbstractBTreeLeafNode.java   |  60 +-
 .../impl/btree/BTreeDataRefNodeFinder.java  |   6 +-
 .../datastore/impl/btree/BTreeNonLeafNode.java  |  52 +-
 .../impl/btree/BlockBTreeLeafNode.java  |   6 +-
 .../impl/btree/BlockletBTreeLeafNode.java   |  46 +-
 .../page/encoding/EncodingFactory.java  |   8 +-
 .../server/NonSecureDictionaryServer.java   |   1 -
 .../core/indexstore/BlockletDetailInfo.java |   4 -
 .../blockletindex/BlockletDataRefNode.java  | 228 ++
 .../BlockletDataRefNodeWrapper.java | 241 ---
 .../indexstore/blockletindex/IndexWrapper.java  |   2 +-
 .../blockletindex/SegmentIndexFileStore.java|   7 +-
 .../core/memory/HeapMemoryAllocator.java|   2 +-
 .../core/metadata/blocklet/SegmentInfo.java |  19 -
 .../core/metadata/schema/table/CarbonTable.java | 130 +++-
 .../schema/table/RelationIdentifier.java|  16 -
 .../core/metadata/schema/table/TableInfo.java   |   6 +-
 .../schema/table/column/CarbonColumn.java   |   2 +-
 .../schema/table/column/CarbonDimension.java|  12 -
 .../core/mutate/CarbonUpdateUtil.java   |  20 +-
 .

[35/50] [abbrv] carbondata git commit: Revert "[CARBONDATA-2023][DataLoad] Add size base block allocation in data loading"

2018-03-04 Thread jackylk
Revert "[CARBONDATA-2023][DataLoad] Add size base block allocation in data 
loading"

This reverts commit 6dd8b038fc898dbf48ad30adfc870c19eb38e3d0.


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

Branch: refs/heads/carbonstore-rebase5
Commit: 3f1d101dc79de5680e6104d76625cd4d88d45011
Parents: 880bbce
Author: Jacky Li 
Authored: Sat Feb 10 10:34:59 2018 +0800
Committer: Jacky Li 
Committed: Sun Mar 4 20:04:49 2018 +0800

--
 .../constants/CarbonLoadOptionConstants.java|  10 -
 .../core/datastore/block/TableBlockInfo.java|  29 --
 .../carbondata/core/util/CarbonProperties.java  |  11 -
 docs/useful-tips-on-carbondata.md   |   1 -
 .../spark/rdd/NewCarbonDataLoadRDD.scala|   4 +-
 .../spark/sql/hive/DistributionUtil.scala   |   2 +-
 .../spark/rdd/CarbonDataRDDFactory.scala|  18 +-
 .../merger/NodeMultiBlockRelation.java  |  40 --
 .../processing/util/CarbonLoaderUtil.java   | 494 +++
 .../processing/util/CarbonLoaderUtilTest.java   | 125 -
 10 files changed, 183 insertions(+), 551 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/3f1d101d/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
 
b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
index a6bf60f..bcfeba0 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
@@ -114,14 +114,4 @@ public final class CarbonLoadOptionConstants {
*/
   public static final int MAX_EXTERNAL_DICTIONARY_SIZE = 1000;
 
-  /**
-   * enable block size based block allocation while loading data. By default, 
carbondata assigns
-   * blocks to node based on block number. If this option is set to `true`, 
carbondata will
-   * consider block size first and make sure that all the nodes will process 
almost equal size of
-   * data. This option is especially useful when you encounter skewed data.
-   */
-  @CarbonProperty
-  public static final String ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION
-  = "carbon.load.skewedDataOptimization.enabled";
-  public static final String 
ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION_DEFAULT = "false";
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/3f1d101d/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
 
b/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
index c0cebe0..a7bfdba 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
@@ -19,8 +19,6 @@ package org.apache.carbondata.core.datastore.block;
 import java.io.IOException;
 import java.io.Serializable;
 import java.nio.charset.Charset;
-import java.util.Arrays;
-import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -100,20 +98,6 @@ public class TableBlockInfo implements Distributable, 
Serializable {
 
   private String dataMapWriterPath;
 
-  /**
-   * comparator to sort by block size in descending order.
-   * Since each line is not exactly the same, the size of a InputSplit may 
differs,
-   * so we allow some deviation for these splits.
-   */
-  public static final Comparator DATA_SIZE_DESC_COMPARATOR =
-  new Comparator() {
-@Override public int compare(Distributable o1, Distributable o2) {
-  long diff =
-  ((TableBlockInfo) o1).getBlockLength() - ((TableBlockInfo) 
o2).getBlockLength();
-  return diff < 0 ? 1 : (diff == 0 ? 0 : -1);
-}
-  };
-
   public TableBlockInfo(String filePath, long blockOffset, String segmentId,
   String[] locations, long blockLength, ColumnarFormatVersion version,
   String[] deletedDeltaFilePath) {
@@ -450,17 +434,4 @@ public class TableBlockInfo implements Distributable, 
Serializable {
   public void setDataMapWriterPath(String dataMapWriterPath) {
 this.dataMapWriterPath = dataMapWriterPath;
   }
-
-  @Override
-  public String toString() {
-final StringBuilder sb = new StringBuilder("TableBlockInfo{");
-sb.append("filePath='").append(filePa

[02/50] [abbrv] carbondata git commit: [CARBONDATA-1992] Remove partitionId in CarbonTablePath

2018-03-04 Thread jackylk
[CARBONDATA-1992] Remove partitionId in CarbonTablePath

In CarbonTablePath, there is a deprecated partition id which is always 0, it 
should be removed to avoid confusion.

This closes #1765


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

Branch: refs/heads/carbonstore-rebase5
Commit: c770ff686dd969b7bd7019efaacc908a36c1b9cc
Parents: f5df308
Author: Jacky Li 
Authored: Sat Jan 6 20:28:44 2018 +0800
Committer: Jacky Li 
Committed: Sun Mar 4 18:15:03 2018 +0800

--
 .../core/metadata/PartitionMapFileStore.java|   0
 .../core/mutate/CarbonUpdateUtil.java   |   8 +-
 .../core/statusmanager/LoadMetadataDetails.java |   1 +
 .../SegmentUpdateStatusManager.java |   8 +-
 .../apache/carbondata/core/util/CarbonUtil.java |   6 +-
 .../core/util/path/CarbonTablePath.java |  55 ---
 .../CarbonFormatDirectoryStructureTest.java |   4 +-
 .../hadoop/api/CarbonTableInputFormat.java  |   2 +-
 .../streaming/CarbonStreamRecordWriter.java |   2 +-
 .../hadoop/test/util/StoreCreator.java  |   1 -
 .../presto/util/CarbonDataStoreCreator.scala|   1 -
 .../dataload/TestLoadDataGeneral.scala  |   2 +-
 .../InsertIntoCarbonTableTestCase.scala |   4 +-
 .../dataload/TestBatchSortDataLoad.scala|   3 +-
 .../dataload/TestDataLoadWithFileName.scala |   2 +-
 .../dataload/TestGlobalSortDataLoad.scala   |   4 +-
 .../testsuite/datamap/TestDataMapCommand.scala  |  34 ++--
 .../TestDataLoadingForPartitionTable.scala  |   3 +-
 .../load/DataLoadProcessBuilderOnSpark.scala|   1 -
 .../load/DataLoadProcessorStepOnSpark.scala |   2 +-
 .../spark/rdd/AlterTableLoadPartitionRDD.scala  | 154 +++
 .../carbondata/spark/rdd/CarbonMergerRDD.scala  |  11 +-
 .../spark/rdd/NewCarbonDataLoadRDD.scala|  25 ++-
 .../org/apache/spark/util/PartitionUtils.scala  |   5 +-
 .../spark/rdd/CarbonDataRDDFactory.scala|   5 +-
 .../management/CarbonLoadDataCommand.scala  |   1 -
 .../datasources/CarbonFileFormat.scala  |   1 -
 .../partition/TestAlterPartitionTable.scala |   2 +-
 .../bucketing/TableBucketingTestCase.scala  |   2 +
 .../loading/CarbonDataLoadConfiguration.java|  10 --
 .../loading/DataLoadProcessBuilder.java |   1 -
 .../loading/TableProcessingOperations.java  |   3 +-
 .../loading/model/CarbonLoadModel.java  |  73 +
 .../sort/impl/ParallelReadMergeSorterImpl.java  |   4 +-
 ...arallelReadMergeSorterWithBucketingImpl.java |  15 +-
 .../UnsafeBatchParallelReadMergeSorterImpl.java |   7 +-
 ...arallelReadMergeSorterWithBucketingImpl.java |  21 ++-
 .../CarbonRowDataWriterProcessorStepImpl.java   |  33 ++--
 .../steps/DataWriterBatchProcessorStepImpl.java |  25 +--
 .../steps/DataWriterProcessorStepImpl.java  |  22 +--
 .../processing/merger/CarbonDataMergerUtil.java |   6 +-
 .../merger/CompactionResultSortProcessor.java   |   8 +-
 .../merger/RowResultMergerProcessor.java|   2 +-
 .../partition/spliter/RowResultProcessor.java   |   2 +-
 .../sort/sortdata/SortParameters.java   |  16 +-
 .../store/CarbonFactDataHandlerModel.java   |   3 +-
 .../util/CarbonDataProcessorUtil.java   |  20 +--
 .../processing/util/CarbonLoaderUtil.java   |  12 +-
 .../processing/util/DeleteLoadFolders.java  |   7 +-
 .../carbondata/processing/StoreCreator.java |   1 -
 .../carbondata/streaming/StreamHandoffRDD.scala |   1 -
 .../streaming/StreamSinkFactory.scala   |   2 +-
 .../streaming/CarbonAppendableStreamSink.scala  |   8 +-
 53 files changed, 285 insertions(+), 366 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/c770ff68/core/src/main/java/org/apache/carbondata/core/metadata/PartitionMapFileStore.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/metadata/PartitionMapFileStore.java
 
b/core/src/main/java/org/apache/carbondata/core/metadata/PartitionMapFileStore.java
new file mode 100644
index 000..e69de29

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c770ff68/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java 
b/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
index de98fa8..18eae11 100644
--- a/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdat

[05/50] [abbrv] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/55c4e438/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionExecutor.java
--
diff --git 
a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionExecutor.java
 
b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionExecutor.java
index f51ced3..6a401d8 100644
--- 
a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionExecutor.java
+++ 
b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionExecutor.java
@@ -34,20 +34,16 @@ import 
org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.datastore.block.TaskBlockInfo;
 import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.scan.executor.QueryExecutor;
 import org.apache.carbondata.core.scan.executor.QueryExecutorFactory;
 import 
org.apache.carbondata.core.scan.executor.exception.QueryExecutionException;
-import org.apache.carbondata.core.scan.model.QueryDimension;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
 import org.apache.carbondata.core.scan.model.QueryModel;
-import org.apache.carbondata.core.scan.result.BatchResult;
+import org.apache.carbondata.core.scan.result.RowBatch;
 import org.apache.carbondata.core.scan.result.iterator.RawResultIterator;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.DataTypeUtil;
+import org.apache.carbondata.core.util.DataTypeConverter;
 
 /**
  * Executor class for executing the query on the selected segments to be 
merged.
@@ -70,6 +66,9 @@ public class CarbonCompactionExecutor {
*/
   private boolean restructuredBlockExists;
 
+  // converter for UTF8String and decimal conversion
+  private DataTypeConverter dataTypeConverter;
+
   /**
* Constructor
*
@@ -82,13 +81,14 @@ public class CarbonCompactionExecutor {
   public CarbonCompactionExecutor(Map segmentMapping,
   SegmentProperties segmentProperties, CarbonTable carbonTable,
   Map> dataFileMetadataSegMapping,
-  boolean restructuredBlockExists) {
+  boolean restructuredBlockExists, DataTypeConverter dataTypeConverter) {
 this.segmentMapping = segmentMapping;
 this.destinationSegProperties = segmentProperties;
 this.carbonTable = carbonTable;
 this.dataFileMetadataSegMapping = dataFileMetadataSegMapping;
 this.restructuredBlockExists = restructuredBlockExists;
-queryExecutorList = new 
ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+this.queryExecutorList = new 
ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+this.dataTypeConverter = dataTypeConverter;
   }
 
   /**
@@ -100,7 +100,9 @@ public class CarbonCompactionExecutor {
 List resultList =
 new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
 List list = null;
-queryModel = prepareQueryModel(list);
+queryModel = 
carbonTable.createQueryModelWithProjectAllColumns(dataTypeConverter);
+queryModel.setReadPageByPage(enablePageLevelReaderForCompaction());
+queryModel.setForcedDetailRawQuery(true);
 // iterate each seg ID
 for (Map.Entry taskMap : segmentMapping.entrySet()) 
{
   String segmentId = taskMap.getKey();
@@ -156,7 +158,7 @@ public class CarbonCompactionExecutor {
* @param blockList
* @return
*/
-  private CarbonIterator executeBlockList(List 
blockList)
+  private CarbonIterator executeBlockList(List 
blockList)
   throws QueryExecutionException, IOException {
 queryModel.setTableBlockInfos(blockList);
 QueryExecutor queryExecutor = 
QueryExecutorFactory.getQueryExecutor(queryModel);
@@ -195,48 +197,6 @@ public class CarbonCompactionExecutor {
   }
 
   /**
-   * Preparing of the query model.
-   *
-   * @param blockList
-   * @return
-   */
-  private QueryModel prepareQueryModel(List blockList) {
-QueryModel model = new QueryModel();
-model.setTableBlockInfos(blockList);
-model.setForcedDetailRawQuery(true);
-model.setFilterExpressionResolverTree(null);
-model.setConverter(DataTypeUtil.getDataTypeConverter());
-model.setReadPageByPage(enablePageLevelReaderForCompaction());
-
-List dims = new 
ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-List dimensions =
-carbonTable.getDimensionByTableName(carbonTable.getTableName());
-for (CarbonDimension dim : dimensions) {
-  // check if dimension is deleted
-  QueryDimension

[29/50] [abbrv] carbondata git commit: [CARBONDATA-2018][DataLoad] Optimization in reading/writing for sort temp row

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/8fcde02e/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
--
diff --git 
a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
 
b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
index 11b3d43..527452a 100644
--- 
a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
+++ 
b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
@@ -31,15 +31,14 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.DataTypeUtil;
-import 
org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
+import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
+import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
 import 
org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
-import org.apache.carbondata.processing.sort.sortdata.NewRowComparator;
+import 
org.apache.carbondata.processing.sort.sortdata.IntermediateSortTempRowComparator;
 import org.apache.carbondata.processing.sort.sortdata.SortParameters;
+import org.apache.carbondata.processing.sort.sortdata.TableFieldStat;
 
 public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
 
@@ -63,21 +62,15 @@ public class UnsafeSortTempFileChunkHolder implements 
SortTempChunkHolder {
* entry count
*/
   private int entryCount;
-
   /**
* return row
*/
-  private Object[] returnRow;
-  private int dimCnt;
-  private int complexCnt;
-  private int measureCnt;
-  private boolean[] isNoDictionaryDimensionColumn;
-  private DataType[] measureDataTypes;
+  private IntermediateSortTempRow returnRow;
   private int readBufferSize;
   private String compressorName;
-  private Object[][] currentBuffer;
+  private IntermediateSortTempRow[] currentBuffer;
 
-  private Object[][] backupBuffer;
+  private IntermediateSortTempRow[] backupBuffer;
 
   private boolean isBackupFilled;
 
@@ -100,27 +93,21 @@ public class UnsafeSortTempFileChunkHolder implements 
SortTempChunkHolder {
 
   private int numberOfObjectRead;
 
-  private int nullSetWordsLength;
-
-  private Comparator comparator;
-
+  private TableFieldStat tableFieldStat;
+  private SortStepRowHandler sortStepRowHandler;
+  private Comparator comparator;
   /**
* Constructor to initialize
*/
   public UnsafeSortTempFileChunkHolder(File tempFile, SortParameters 
parameters) {
 // set temp file
 this.tempFile = tempFile;
-this.dimCnt = parameters.getDimColCount();
-this.complexCnt = parameters.getComplexDimColCount();
-this.measureCnt = parameters.getMeasureColCount();
-this.isNoDictionaryDimensionColumn = 
parameters.getNoDictionaryDimnesionColumn();
-this.measureDataTypes = parameters.getMeasureDataType();
 this.readBufferSize = parameters.getBufferSize();
 this.compressorName = parameters.getSortTempCompressorName();
-
+this.tableFieldStat = new TableFieldStat(parameters);
+this.sortStepRowHandler = new SortStepRowHandler(tableFieldStat);
 this.executorService = Executors.newFixedThreadPool(1);
-this.nullSetWordsLength = ((parameters.getMeasureColCount() - 1) >> 6) + 1;
-comparator = new NewRowComparator(parameters.getNoDictionarySortColumn());
+comparator = new 
IntermediateSortTempRowComparator(parameters.getNoDictionarySortColumn());
 initialize();
   }
 
@@ -169,11 +156,17 @@ public class UnsafeSortTempFileChunkHolder implements 
SortTempChunkHolder {
*
* @throws CarbonSortKeyAndGroupByException problem while reading
*/
+  @Override
   public void readRow() throws CarbonSortKeyAndGroupByException {
 if (prefetch) {
   fillDataForPrefetch();
 } else {
-  this.returnRow = getRowFromStream();
+  try {
+this.returnRow = 
sortStepRowHandler.readIntermediateSortTempRowFromInputStream(stream);
+this.numberOfObjectRead++;
+  } catch (IOException e) {
+throw new CarbonSortKeyAndGroupByException("Problems while reading 
row", e);
+  }
 }
   }
 
@@ -207,63 +200,22 @@ public class UnsafeSortTempFileChunkHolder implements 
SortTempChunkHolder {
   }
 
   /**
-   * @retu

[36/50] [abbrv] carbondata git commit: Revert "[CARBONDATA-2018][DataLoad] Optimization in reading/writing for sort temp row"

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/bfdf3e35/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
--
diff --git 
a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
 
b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
index 527452a..11b3d43 100644
--- 
a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
+++ 
b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
@@ -31,14 +31,15 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
-import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
+import org.apache.carbondata.core.util.DataTypeUtil;
+import 
org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
 import 
org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
-import 
org.apache.carbondata.processing.sort.sortdata.IntermediateSortTempRowComparator;
+import org.apache.carbondata.processing.sort.sortdata.NewRowComparator;
 import org.apache.carbondata.processing.sort.sortdata.SortParameters;
-import org.apache.carbondata.processing.sort.sortdata.TableFieldStat;
 
 public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
 
@@ -62,15 +63,21 @@ public class UnsafeSortTempFileChunkHolder implements 
SortTempChunkHolder {
* entry count
*/
   private int entryCount;
+
   /**
* return row
*/
-  private IntermediateSortTempRow returnRow;
+  private Object[] returnRow;
+  private int dimCnt;
+  private int complexCnt;
+  private int measureCnt;
+  private boolean[] isNoDictionaryDimensionColumn;
+  private DataType[] measureDataTypes;
   private int readBufferSize;
   private String compressorName;
-  private IntermediateSortTempRow[] currentBuffer;
+  private Object[][] currentBuffer;
 
-  private IntermediateSortTempRow[] backupBuffer;
+  private Object[][] backupBuffer;
 
   private boolean isBackupFilled;
 
@@ -93,21 +100,27 @@ public class UnsafeSortTempFileChunkHolder implements 
SortTempChunkHolder {
 
   private int numberOfObjectRead;
 
-  private TableFieldStat tableFieldStat;
-  private SortStepRowHandler sortStepRowHandler;
-  private Comparator comparator;
+  private int nullSetWordsLength;
+
+  private Comparator comparator;
+
   /**
* Constructor to initialize
*/
   public UnsafeSortTempFileChunkHolder(File tempFile, SortParameters 
parameters) {
 // set temp file
 this.tempFile = tempFile;
+this.dimCnt = parameters.getDimColCount();
+this.complexCnt = parameters.getComplexDimColCount();
+this.measureCnt = parameters.getMeasureColCount();
+this.isNoDictionaryDimensionColumn = 
parameters.getNoDictionaryDimnesionColumn();
+this.measureDataTypes = parameters.getMeasureDataType();
 this.readBufferSize = parameters.getBufferSize();
 this.compressorName = parameters.getSortTempCompressorName();
-this.tableFieldStat = new TableFieldStat(parameters);
-this.sortStepRowHandler = new SortStepRowHandler(tableFieldStat);
+
 this.executorService = Executors.newFixedThreadPool(1);
-comparator = new 
IntermediateSortTempRowComparator(parameters.getNoDictionarySortColumn());
+this.nullSetWordsLength = ((parameters.getMeasureColCount() - 1) >> 6) + 1;
+comparator = new NewRowComparator(parameters.getNoDictionarySortColumn());
 initialize();
   }
 
@@ -156,17 +169,11 @@ public class UnsafeSortTempFileChunkHolder implements 
SortTempChunkHolder {
*
* @throws CarbonSortKeyAndGroupByException problem while reading
*/
-  @Override
   public void readRow() throws CarbonSortKeyAndGroupByException {
 if (prefetch) {
   fillDataForPrefetch();
 } else {
-  try {
-this.returnRow = 
sortStepRowHandler.readIntermediateSortTempRowFromInputStream(stream);
-this.numberOfObjectRead++;
-  } catch (IOException e) {
-throw new CarbonSortKeyAndGroupByException("Problems while reading 
row", e);
-  }
+  this.returnRow = getRowFromStream();
 }
   }
 
@@ -200,22 +207,63 @@ public class UnsafeSortTempFileChunkHolder implements 
SortTempChunkHolder {
   }
 
   /**
-   * get a

[23/50] [abbrv] carbondata git commit: [CARBONDATA-2025] Unify all path construction through CarbonTablePath static method

2018-03-04 Thread jackylk
[CARBONDATA-2025] Unify all path construction through CarbonTablePath static 
method

Refactory CarbonTablePath:

1.Remove CarbonStorePath and use CarbonTablePath only.
2.Make CarbonTablePath an utility without object creation, it can avoid 
creating object before using it, thus code is cleaner and GC is less.

This closes #1768


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

Branch: refs/heads/carbonstore-rebase5
Commit: 9b9125b6dfa4373364faca9b605569fb2d775c17
Parents: 55c4e43
Author: Jacky Li 
Authored: Wed Jan 31 16:14:27 2018 +0800
Committer: Jacky Li 
Committed: Sun Mar 4 19:31:36 2018 +0800

--
 .../DictionaryColumnUniqueIdentifier.java   |  29 +-
 .../dictionary/ManageDictionaryAndBTree.java|  13 +-
 .../core/metadata/AbsoluteTableIdentifier.java  |   4 +-
 .../core/metadata/SegmentFileStore.java |   3 +-
 .../core/metadata/schema/table/CarbonTable.java |  11 +-
 .../core/mutate/CarbonUpdateUtil.java   |  58 ++--
 .../core/scan/executor/util/QueryUtil.java  |   7 +-
 .../scan/executor/util/RestructureUtil.java |   6 +-
 .../scan/filter/FilterExpressionProcessor.java  |   2 +-
 .../filter/executer/FalseFilterExecutor.java|   8 +-
 .../RowLevelRangeGrtThanFiterExecuterImpl.java  |   1 +
 ...elRangeGrtrThanEquaToFilterExecuterImpl.java |   1 +
 ...velRangeLessThanEqualFilterExecuterImpl.java |   1 +
 .../RowLevelRangeLessThanFiterExecuterImpl.java |   1 +
 .../FalseConditionalResolverImpl.java   |   4 +-
 .../core/service/CarbonCommonFactory.java   |  16 -
 .../carbondata/core/service/PathService.java|  35 ---
 .../core/service/impl/PathFactory.java  |  50 
 .../statusmanager/SegmentStatusManager.java |  31 +-
 .../SegmentUpdateStatusManager.java |  70 ++---
 .../apache/carbondata/core/util/CarbonUtil.java |  98 ++
 .../util/path/CarbonSharedDictionaryPath.java   |  71 -
 .../core/util/path/CarbonStorePath.java |  71 -
 .../core/util/path/CarbonTablePath.java | 298 ++-
 .../dictionary/AbstractDictionaryCacheTest.java |  11 +-
 .../dictionary/ForwardDictionaryCacheTest.java  |   6 +-
 .../dictionary/ReverseDictionaryCacheTest.java  |   6 +-
 .../reader/CarbonDictionaryReaderImplTest.java  |   8 -
 .../CarbonFormatDirectoryStructureTest.java |  18 +-
 .../path/CarbonFormatSharedDictionaryTest.java  |  44 ---
 .../writer/CarbonDictionaryWriterImplTest.java  |  19 +-
 .../CarbonBatchSparkStreamingExample.scala  |   9 +-
 .../CarbonStreamSparkStreamingExample.scala |  10 +-
 .../CarbonStructuredStreamingExample.scala  |  11 +-
 ...CarbonStructuredStreamingWithRowParser.scala |   9 +-
 .../hadoop/api/CarbonTableInputFormat.java  |   8 +-
 .../streaming/CarbonStreamRecordWriter.java |   6 +-
 .../carbondata/hadoop/util/SchemaReader.java|  18 +-
 .../hadoop/test/util/StoreCreator.java  |  16 +-
 .../presto/CarbondataRecordSetProvider.java |   7 +-
 .../presto/impl/CarbonTableCacheModel.java  |  13 +-
 .../presto/impl/CarbonTableReader.java  |  49 ++-
 .../presto/util/CarbonDataStoreCreator.scala|  10 +-
 .../sdv/generated/MergeIndexTestCase.scala  |  19 +-
 .../dataload/TestLoadDataGeneral.scala  |  11 +-
 .../InsertIntoCarbonTableTestCase.scala |   8 +-
 .../createTable/TestCreateTableAsSelect.scala   |   2 +-
 .../datacompaction/DataCompactionLockTest.scala |   6 +-
 .../MajorCompactionIgnoreInMinorTest.scala  |  12 +-
 .../dataload/TestBatchSortDataLoad.scala|   5 +-
 .../dataload/TestDataLoadWithFileName.scala |   5 +-
 .../dataload/TestGlobalSortDataLoad.scala   |   8 +-
 .../dataretention/DataRetentionTestCase.scala   |   6 +-
 .../TestDataLoadingForPartitionTable.scala  |   4 +-
 .../StandardPartitionTableLoadingTestCase.scala |   6 +-
 .../org/apache/carbondata/api/CarbonStore.scala |   4 +-
 .../carbondata/spark/CarbonSparkFactory.scala   |   2 +-
 .../spark/DictionaryDetailHelper.scala  |   9 +-
 .../spark/rdd/AlterTableAddColumnRDD.scala  |  17 +-
 .../spark/rdd/CarbonGlobalDictionaryRDD.scala   |   6 -
 .../carbondata/spark/util/CommonUtil.scala  |  30 +-
 .../carbondata/spark/util/DataLoadingUtil.scala |  10 +-
 .../spark/util/GlobalDictionaryUtil.scala   |  16 +-
 .../command/carbonTableSchemaCommon.scala   |   9 +-
 .../org/apache/spark/util/PartitionUtils.scala  |   6 +-
 .../spark/rdd/AggregateDataMapCompactor.scala   |  19 +-
 .../spark/rdd/CarbonDataRDDFactory.scala|  14 +-
 .../spark/rdd/CarbonTableCompactor.scala|   8 +-
 .../CarbonAlterTableCompactionCommand.scala |  13 +-
 .../management/CarbonLoadDataCommand.scala  |  12 +-

[25/50] [abbrv] carbondata git commit: [CARBONDATA-2080] [S3-Implementation] Propagated hadoopConf from driver to executor for s3 implementation in cluster mode.

2018-03-04 Thread jackylk
[CARBONDATA-2080] [S3-Implementation] Propagated hadoopConf from driver to 
executor for s3 implementation in cluster mode.

Problem : hadoopconf was not getting propagated from driver to the executor 
that's why load was failing to the distributed environment.
Solution: Setting the Hadoop conf in base class CarbonRDD
How to verify this PR :
Execute the load in the cluster mode It should be a success using location s3.

This closes #1860


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

Branch: refs/heads/carbonstore-rebase5
Commit: f8841d873d281046657a2afcf6efd343c0f154e4
Parents: 9b9125b
Author: Jatin 
Authored: Thu Jan 25 16:53:00 2018 +0530
Committer: Jacky Li 
Committed: Sun Mar 4 19:31:39 2018 +0800

--
 .../spark/rdd/AlterTableAddColumnRDD.scala  |  2 +-
 .../spark/rdd/AlterTableDropColumnRDD.scala |  2 +-
 .../spark/rdd/CarbonCleanFilesRDD.scala |  2 +-
 .../spark/rdd/CarbonDeleteLoadByDateRDD.scala   |  2 +-
 .../spark/rdd/CarbonDeleteLoadRDD.scala |  2 +-
 .../spark/rdd/CarbonDropPartitionRDD.scala  |  2 +-
 .../spark/rdd/CarbonDropTableRDD.scala  |  2 +-
 .../spark/rdd/CarbonGlobalDictionaryRDD.scala   |  3 +-
 .../spark/rdd/CarbonMergeFilesRDD.scala |  0
 .../carbondata/spark/rdd/CarbonMergerRDD.scala  |  2 +-
 .../apache/carbondata/spark/rdd/CarbonRDD.scala | 32 ++--
 .../spark/rdd/NewCarbonDataLoadRDD.scala|  2 +-
 .../carbondata/spark/rdd/SparkDataMapJob.scala  |  2 +-
 .../apache/spark/rdd/DataLoadCoalescedRDD.scala |  3 +-
 .../apache/spark/rdd/UpdateCoalescedRDD.scala   |  2 +-
 .../carbondata/streaming/StreamHandoffRDD.scala |  2 +-
 16 files changed, 45 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/f8841d87/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 56a66b9..7c1edea 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
@@ -50,7 +50,7 @@ class AddColumnPartition(rddId: Int, idx: Int, schema: 
ColumnSchema) extends Par
 class AlterTableAddColumnRDD[K, V](sc: SparkContext,
 @transient newColumns: Seq[ColumnSchema],
 identifier: AbsoluteTableIdentifier)
-  extends CarbonRDD[(Int, SegmentStatus)](sc, Nil) {
+  extends CarbonRDD[(Int, SegmentStatus)](sc, Nil, sc.hadoopConfiguration) {
 
   val lockType: String = 
CarbonProperties.getInstance.getProperty(CarbonCommonConstants.LOCK_TYPE,
 CarbonCommonConstants.CARBON_LOCK_TYPE_HDFS)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f8841d87/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 248f351..e14524e 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
@@ -48,7 +48,7 @@ class DropColumnPartition(rddId: Int, idx: Int, schema: 
ColumnSchema) extends Pa
 class AlterTableDropColumnRDD[K, V](sc: SparkContext,
 @transient newColumns: Seq[ColumnSchema],
 carbonTableIdentifier: AbsoluteTableIdentifier)
-  extends CarbonRDD[(Int, SegmentStatus)](sc, Nil) {
+  extends CarbonRDD[(Int, SegmentStatus)](sc, Nil, sc.hadoopConfiguration) {
 
   override def getPartitions: Array[Partition] = {
 newColumns.zipWithIndex.map { column =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/f8841d87/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonCleanFilesRDD.scala
--
diff --git 
a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonCleanFilesRDD.scala
 
b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonCleanFilesRDD.scala
index 32523d8..9936a2a 100644
--- 
a/integration/spark-c

[28/50] [abbrv] carbondata git commit: [HotFix][CheckStyle] Fix import related checkstyle

2018-03-04 Thread jackylk
[HotFix][CheckStyle] Fix import related checkstyle

This closes #1952


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

Branch: refs/heads/carbonstore-rebase5
Commit: 0feaeb545bb9e45a1cca35b75d825d8b86aa938d
Parents: 828ae5e
Author: xuchuanyin 
Authored: Thu Feb 8 15:39:45 2018 +0800
Committer: Jacky Li 
Committed: Sun Mar 4 19:44:12 2018 +0800

--
 .../core/indexstore/blockletindex/BlockletDataRefNode.java | 2 +-
 .../org/apache/carbondata/core/memory/HeapMemoryAllocator.java | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/0feaeb54/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
 
b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
index b8fd6ff..50862a7 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
@@ -33,8 +33,8 @@ import 
org.apache.carbondata.core.datastore.chunk.reader.MeasureColumnChunkReade
 import org.apache.carbondata.core.indexstore.BlockletDetailInfo;
 import org.apache.carbondata.core.indexstore.FineGrainBlocklet;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
-import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.metadata.blocklet.index.BlockletIndex;
+import org.apache.carbondata.core.util.BitSetGroup;
 
 /**
  * wrapper for blocklet data map data

http://git-wip-us.apache.org/repos/asf/carbondata/blob/0feaeb54/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java 
b/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
index 53cbb1d..242995b 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
@@ -17,11 +17,11 @@
 
 package org.apache.carbondata.core.memory;
 
-import javax.annotation.concurrent.GuardedBy;
 import java.lang.ref.WeakReference;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.Map;
+import javax.annotation.concurrent.GuardedBy;
 
 import org.apache.carbondata.core.util.CarbonProperties;
 



[42/50] [abbrv] carbondata git commit: [CARBONDATA-2018][DataLoad] Optimization in reading/writing for sort temp row

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/daecc774/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
--
diff --git 
a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
 
b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
index 11b3d43..527452a 100644
--- 
a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
+++ 
b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
@@ -31,15 +31,14 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.DataTypeUtil;
-import 
org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
+import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
+import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
 import 
org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
-import org.apache.carbondata.processing.sort.sortdata.NewRowComparator;
+import 
org.apache.carbondata.processing.sort.sortdata.IntermediateSortTempRowComparator;
 import org.apache.carbondata.processing.sort.sortdata.SortParameters;
+import org.apache.carbondata.processing.sort.sortdata.TableFieldStat;
 
 public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
 
@@ -63,21 +62,15 @@ public class UnsafeSortTempFileChunkHolder implements 
SortTempChunkHolder {
* entry count
*/
   private int entryCount;
-
   /**
* return row
*/
-  private Object[] returnRow;
-  private int dimCnt;
-  private int complexCnt;
-  private int measureCnt;
-  private boolean[] isNoDictionaryDimensionColumn;
-  private DataType[] measureDataTypes;
+  private IntermediateSortTempRow returnRow;
   private int readBufferSize;
   private String compressorName;
-  private Object[][] currentBuffer;
+  private IntermediateSortTempRow[] currentBuffer;
 
-  private Object[][] backupBuffer;
+  private IntermediateSortTempRow[] backupBuffer;
 
   private boolean isBackupFilled;
 
@@ -100,27 +93,21 @@ public class UnsafeSortTempFileChunkHolder implements 
SortTempChunkHolder {
 
   private int numberOfObjectRead;
 
-  private int nullSetWordsLength;
-
-  private Comparator comparator;
-
+  private TableFieldStat tableFieldStat;
+  private SortStepRowHandler sortStepRowHandler;
+  private Comparator comparator;
   /**
* Constructor to initialize
*/
   public UnsafeSortTempFileChunkHolder(File tempFile, SortParameters 
parameters) {
 // set temp file
 this.tempFile = tempFile;
-this.dimCnt = parameters.getDimColCount();
-this.complexCnt = parameters.getComplexDimColCount();
-this.measureCnt = parameters.getMeasureColCount();
-this.isNoDictionaryDimensionColumn = 
parameters.getNoDictionaryDimnesionColumn();
-this.measureDataTypes = parameters.getMeasureDataType();
 this.readBufferSize = parameters.getBufferSize();
 this.compressorName = parameters.getSortTempCompressorName();
-
+this.tableFieldStat = new TableFieldStat(parameters);
+this.sortStepRowHandler = new SortStepRowHandler(tableFieldStat);
 this.executorService = Executors.newFixedThreadPool(1);
-this.nullSetWordsLength = ((parameters.getMeasureColCount() - 1) >> 6) + 1;
-comparator = new NewRowComparator(parameters.getNoDictionarySortColumn());
+comparator = new 
IntermediateSortTempRowComparator(parameters.getNoDictionarySortColumn());
 initialize();
   }
 
@@ -169,11 +156,17 @@ public class UnsafeSortTempFileChunkHolder implements 
SortTempChunkHolder {
*
* @throws CarbonSortKeyAndGroupByException problem while reading
*/
+  @Override
   public void readRow() throws CarbonSortKeyAndGroupByException {
 if (prefetch) {
   fillDataForPrefetch();
 } else {
-  this.returnRow = getRowFromStream();
+  try {
+this.returnRow = 
sortStepRowHandler.readIntermediateSortTempRowFromInputStream(stream);
+this.numberOfObjectRead++;
+  } catch (IOException e) {
+throw new CarbonSortKeyAndGroupByException("Problems while reading 
row", e);
+  }
 }
   }
 
@@ -207,63 +200,22 @@ public class UnsafeSortTempFileChunkHolder implements 
SortTempChunkHolder {
   }
 
   /**
-   * @retu

[18/50] [abbrv] carbondata git commit: [REBASE] Solve conflict after rebasing master

2018-03-04 Thread jackylk
[REBASE] Solve conflict after rebasing master


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

Branch: refs/heads/carbonstore-rebase5
Commit: 03157f91120e3de1e12cca1cf7230aafc6856e0d
Parents: 4628d2e
Author: Jacky Li 
Authored: Thu Feb 1 00:25:31 2018 +0800
Committer: Jacky Li 
Committed: Sun Mar 4 18:15:06 2018 +0800

--
 .../hadoop/util/CarbonInputFormatUtil.java  | 20 +++
 .../spark/rdd/NewCarbonDataLoadRDD.scala| 21 ++--
 .../org/apache/spark/sql/CarbonSession.scala|  5 ++---
 3 files changed, 24 insertions(+), 22 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/03157f91/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
--
diff --git 
a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
 
b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
index 514428b..056c27b 100644
--- 
a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
+++ 
b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
@@ -22,6 +22,8 @@ import java.text.SimpleDateFormat;
 import java.util.List;
 import java.util.Locale;
 
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
@@ -39,6 +41,7 @@ import org.apache.carbondata.core.scan.model.QueryMeasure;
 import org.apache.carbondata.core.scan.model.QueryModel;
 import org.apache.carbondata.hadoop.api.CarbonTableInputFormat;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobID;
@@ -159,4 +162,21 @@ public class CarbonInputFormatUtil {
 String jobtrackerID = createJobTrackerID(date);
 return new JobID(jobtrackerID, batch);
   }
+
+  public static void setS3Configurations(Configuration hadoopConf) {
+FileFactory.getConfiguration()
+.set("fs.s3a.access.key", hadoopConf.get("fs.s3a.access.key", ""));
+FileFactory.getConfiguration()
+.set("fs.s3a.secret.key", hadoopConf.get("fs.s3a.secret.key", ""));
+FileFactory.getConfiguration()
+.set("fs.s3a.endpoint", hadoopConf.get("fs.s3a.endpoint", ""));
+FileFactory.getConfiguration().set(CarbonCommonConstants.S3_ACCESS_KEY,
+hadoopConf.get(CarbonCommonConstants.S3_ACCESS_KEY, ""));
+FileFactory.getConfiguration().set(CarbonCommonConstants.S3_SECRET_KEY,
+hadoopConf.get(CarbonCommonConstants.S3_SECRET_KEY, ""));
+FileFactory.getConfiguration().set(CarbonCommonConstants.S3N_ACCESS_KEY,
+hadoopConf.get(CarbonCommonConstants.S3N_ACCESS_KEY, ""));
+FileFactory.getConfiguration().set(CarbonCommonConstants.S3N_SECRET_KEY,
+hadoopConf.get(CarbonCommonConstants.S3N_SECRET_KEY, ""));
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/03157f91/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 917fc88..e17824f 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
@@ -41,10 +41,10 @@ import 
org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.common.logging.impl.StandardLogService
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.compression.CompressorFactory
-import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, 
SegmentStatus}
 import org.apache.carbondata.core.util.{CarbonProperties, 
CarbonTimeStatisticsFactory, ThreadLocalTaskInfo}
 import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil
 import org.apache.carbondata.processing.loading.{DataLoadExecutor, 
FailureCauses, T

[09/50] [abbrv] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/55c4e438/core/src/main/java/org/apache/carbondata/core/scan/processor/DataBlockIterator.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/scan/processor/DataBlockIterator.java
 
b/core/src/main/java/org/apache/carbondata/core/scan/processor/DataBlockIterator.java
new file mode 100644
index 000..fde4e55
--- /dev/null
+++ 
b/core/src/main/java/org/apache/carbondata/core/scan/processor/DataBlockIterator.java
@@ -0,0 +1,269 @@
+/*
+ * 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.scan.processor;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.carbondata.common.CarbonIterator;
+import org.apache.carbondata.core.datastore.DataRefNode;
+import org.apache.carbondata.core.datastore.FileReader;
+import org.apache.carbondata.core.scan.collector.ResultCollectorFactory;
+import org.apache.carbondata.core.scan.collector.ScannedResultCollector;
+import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
+import org.apache.carbondata.core.scan.scanner.BlockletScanner;
+import org.apache.carbondata.core.scan.scanner.impl.BlockletFilterScanner;
+import org.apache.carbondata.core.scan.scanner.impl.BlockletFullScanner;
+import org.apache.carbondata.core.stats.QueryStatisticsModel;
+import org.apache.carbondata.core.util.TaskMetricsMap;
+
+/**
+ * This abstract class provides a skeletal implementation of the
+ * Block iterator.
+ */
+public class DataBlockIterator extends CarbonIterator> {
+
+  /**
+   * iterator which will be used to iterate over blocklets
+   */
+  private BlockletIterator blockletIterator;
+
+  /**
+   * result collector which will be used to aggregate the scanned result
+   */
+  private ScannedResultCollector scannerResultAggregator;
+
+  /**
+   * processor which will be used to process the block processing can be
+   * filter processing or non filter processing
+   */
+  private BlockletScanner blockletScanner;
+
+  /**
+   * batch size of result
+   */
+  private int batchSize;
+
+  private ExecutorService executorService;
+
+  private Future future;
+
+  private Future futureIo;
+
+  private BlockletScannedResult scannedResult;
+
+  private BlockExecutionInfo blockExecutionInfo;
+
+  private FileReader fileReader;
+
+  private AtomicBoolean nextBlock;
+
+  private AtomicBoolean nextRead;
+
+  public DataBlockIterator(BlockExecutionInfo blockExecutionInfo, FileReader 
fileReader,
+  int batchSize, QueryStatisticsModel queryStatisticsModel, 
ExecutorService executorService) {
+this.blockExecutionInfo = blockExecutionInfo;
+this.fileReader = fileReader;
+blockletIterator = new 
BlockletIterator(blockExecutionInfo.getFirstDataBlock(),
+blockExecutionInfo.getNumberOfBlockToScan());
+if (blockExecutionInfo.getFilterExecuterTree() != null) {
+  blockletScanner = new BlockletFilterScanner(blockExecutionInfo, 
queryStatisticsModel);
+} else {
+  blockletScanner = new BlockletFullScanner(blockExecutionInfo, 
queryStatisticsModel);
+}
+this.scannerResultAggregator =
+ResultCollectorFactory.getScannedResultCollector(blockExecutionInfo);
+this.batchSize = batchSize;
+this.executorService = executorService;
+this.nextBlock = new AtomicBoolean(false);
+this.nextRead = new AtomicBoolean(false);
+  }
+
+  @Override
+  public List next() {
+List collectedResult = null;
+if (updateScanner()) {
+  collectedResult = 
this.scannerResultAggregator.collectResultInRow(scannedResult, batchSize);
+  while (collectedResult.size() < batchSize && updateScanner()) {
+List data = this.scannerResultAggregator
+.collectResultInRow(scannedResult, batchSize - 
collectedResul

[07/50] [abbrv] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/55c4e438/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
 
b/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
index 94a041a..b74c279 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
@@ -378,7 +378,6 @@ public abstract class AbstractDataFileFooterConverter {
   cardinality[i] = segmentInfo.getColumn_cardinalities().get(i);
 }
 info.setColumnCardinality(cardinality);
-info.setNumberOfColumns(segmentInfo.getNum_cols());
 return info;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/55c4e438/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 52305bd..0cc783e 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
@@ -48,10 +48,10 @@ import 
org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentif
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.constants.CarbonLoadOptionConstants;
 import org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.block.AbstractIndex;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.columnar.ColumnGroupModel;
@@ -82,7 +82,7 @@ import 
org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.mutate.UpdateVO;
 import org.apache.carbondata.core.reader.ThriftReader;
 import org.apache.carbondata.core.reader.ThriftReader.TBaseCreator;
-import org.apache.carbondata.core.scan.model.QueryDimension;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatus;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
@@ -248,16 +248,13 @@ public final class CarbonUtil {
   public static ColumnGroupModel getColGroupModel(int[][] columnGroups) {
 int[] columnSplit = new int[columnGroups.length];
 int noOfColumnStore = columnSplit.length;
-boolean[] columnarStore = new boolean[noOfColumnStore];
 
 for (int i = 0; i < columnGroups.length; i++) {
   columnSplit[i] = columnGroups[i].length;
-  columnarStore[i] = columnGroups[i].length <= 1;
 }
 ColumnGroupModel colGroupModel = new ColumnGroupModel();
 colGroupModel.setNoOfColumnStore(noOfColumnStore);
 colGroupModel.setColumnSplit(columnSplit);
-colGroupModel.setColumnarStore(columnarStore);
 colGroupModel.setColumnGroup(columnGroups);
 return colGroupModel;
   }
@@ -418,7 +415,7 @@ public final class CarbonUtil {
 }
   }
 
-  public static int getFirstIndexUsingBinarySearch(DimensionColumnDataChunk 
dimColumnDataChunk,
+  public static int getFirstIndexUsingBinarySearch(DimensionColumnPage 
dimColumnDataChunk,
   int low, int high, byte[] compareValue, boolean matchUpLimit) {
 int cmpResult = 0;
 while (high >= low) {
@@ -457,7 +454,7 @@ public final class CarbonUtil {
* @return the compareValue's range index in the dimColumnDataChunk
*/
   public static int[] getRangeIndexUsingBinarySearch(
-  DimensionColumnDataChunk dimColumnDataChunk, int low, int high, byte[] 
compareValue) {
+  DimensionColumnPage dimColumnDataChunk, int low, int high, byte[] 
compareValue) {
 
 int[] rangeIndex = new int[2];
 int cmpResult = 0;
@@ -551,7 +548,7 @@ public final class CarbonUtil {
* @return index value
*/
   public static int nextLesserValueToTarget(int currentIndex,
-  DimensionColumnDataChunk dimColumnDataChunk, byte[] compareValue) {
+  DimensionColumnPage dimColumnDataChunk, byte[] compareValue) {
 while (currentIndex - 1 >= 0
 && dimColumnDataChunk.compareTo(currentIndex - 1, compareValue) >= 0) {
   --currentIndex;
@@ -571,7 +568,7 @@ public

[41/50] [abbrv] carbondata git commit: [CARBONDATA-2159] Remove carbon-spark dependency in store-sdk module

2018-03-04 Thread jackylk
[CARBONDATA-2159] Remove carbon-spark dependency in store-sdk module

To make assembling JAR of store-sdk module, it should not depend on 
carbon-spark module

This closes #1970


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

Branch: refs/heads/carbonstore-rebase5
Commit: 83df87ddd74eda674a7343fd3f39e6c81650c0cf
Parents: a950863
Author: Jacky Li 
Authored: Sun Feb 11 21:37:04 2018 +0800
Committer: Jacky Li 
Committed: Sun Mar 4 20:08:28 2018 +0800

--
 .../java/org/apache/carbondata/common/Maps.java |  39 ++
 .../org/apache/carbondata/common/Strings.java   |   3 +
 .../exceptions/TableStatusLockException.java|  34 ++
 .../sql/InvalidLoadOptionException.java |  33 +
 .../sql/MalformedCarbonCommandException.java|  75 +++
 .../sql/MalformedDataMapCommandException.java   |  37 ++
 .../exceptions/sql/NoSuchDataMapException.java  |  39 ++
 .../carbondata/core/datamap/TableDataMap.java   |   5 +-
 .../exception/ConcurrentOperationException.java |  50 ++
 .../statusmanager/SegmentStatusManager.java | 124 
 .../carbondata/core/util/DeleteLoadFolders.java | 210 +++
 .../preaggregate/TestPreAggCreateCommand.scala  |   2 +-
 .../preaggregate/TestPreAggregateDrop.scala |   2 +-
 .../timeseries/TestTimeSeriesCreateTable.scala  |   2 +-
 .../timeseries/TestTimeSeriesDropSuite.scala|   2 +-
 .../TestTimeseriesTableSelection.scala  |   2 +-
 .../TestDataLoadWithColumnsMoreThanSchema.scala |   3 +-
 .../dataload/TestGlobalSortDataLoad.scala   |   2 +-
 .../TestLoadDataWithDiffTimestampFormat.scala   |   2 +-
 .../TestLoadDataWithFileHeaderException.scala   |  11 +-
 ...ataWithMalformedCarbonCommandException.scala |   3 +-
 .../testsuite/dataload/TestLoadOptions.scala|   2 +-
 .../dataload/TestTableLevelBlockSize.scala  |   4 +-
 .../testsuite/datamap/TestDataMapCommand.scala  |   2 +-
 .../dataretention/DataRetentionTestCase.scala   |   2 +-
 .../spark/testsuite/datetype/DateTypeTest.scala |   2 +-
 .../testsuite/sortcolumns/TestSortColumns.scala |   3 +-
 integration/spark-common/pom.xml|   5 -
 .../exception/ConcurrentOperationException.java |  38 --
 .../MalformedCarbonCommandException.java|  69 ---
 .../MalformedDataMapCommandException.java   |  32 -
 .../spark/exception/NoSuchDataMapException.java |  33 -
 .../org/apache/carbondata/api/CarbonStore.scala |   3 +-
 .../spark/CarbonColumnValidator.scala   |   8 +-
 .../carbondata/spark/load/ValidateUtil.scala|  72 ---
 .../carbondata/spark/rdd/CarbonMergerRDD.scala  |   6 +-
 .../carbondata/spark/util/CommonUtil.scala  |  70 +--
 .../carbondata/spark/util/DataLoadingUtil.scala | 610 ---
 .../spark/util/GlobalDictionaryUtil.scala   |   2 +-
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala |   2 +-
 .../spark/rdd/CarbonDataRDDFactory.scala|   4 +-
 .../spark/rdd/CarbonTableCompactor.scala|   2 +-
 .../org/apache/spark/sql/CarbonSource.scala |   2 +-
 .../datamap/CarbonCreateDataMapCommand.scala|   2 +-
 .../datamap/CarbonDropDataMapCommand.scala  |   2 +-
 .../CarbonAlterTableCompactionCommand.scala |  13 +-
 .../management/CarbonLoadDataCommand.scala  |  17 +-
 .../CarbonProjectForDeleteCommand.scala |   2 +-
 .../CarbonProjectForUpdateCommand.scala |   2 +-
 .../command/mutation/IUDCommonUtil.scala|   2 +-
 .../CreatePreAggregateTableCommand.scala|   7 +-
 .../preaaggregate/PreAggregateUtil.scala|   2 +-
 .../schema/CarbonAlterTableRenameCommand.scala  |   3 +-
 .../command/timeseries/TimeSeriesUtil.scala |   2 +-
 .../datasources/CarbonFileFormat.scala  |  14 +-
 .../sql/execution/strategy/DDLStrategy.scala|   2 +-
 .../strategy/StreamingTableStrategy.scala   |   2 +-
 .../execution/command/CarbonHiveCommands.scala  |   2 +-
 .../sql/parser/CarbonSpark2SqlParser.scala  |   2 +-
 .../spark/sql/parser/CarbonSparkSqlParser.scala |   2 +-
 .../org/apache/spark/util/AlterTableUtil.scala  |   2 +-
 .../org/apache/spark/util/TableAPIUtil.scala|   2 +-
 .../spark/sql/hive/CarbonSessionState.scala |   7 +-
 .../segmentreading/TestSegmentReading.scala |   2 +-
 .../spark/util/AllDictionaryTestCase.scala  |   4 +-
 .../util/ExternalColumnDictionaryTestCase.scala |   6 +-
 .../TestStreamingTableOperation.scala   |   4 +-
 .../bucketing/TableBucketingTestCase.scala  |   2 +-
 .../vectorreader/AddColumnTestCases.scala   |   2 +-
 .../loading/model/CarbonLoadModel.java  |  14 +-
 .../loading/model/CarbonLoadModelBuilder.java   | 322 ++
 .../processing/loading/model/LoadOption.java| 251 
 .../processing/util/CarbonLo

[01/50] [abbrv] carbondata git commit: [CARBONDATA-1992] Remove partitionId in CarbonTablePath [Forced Update!]

2018-03-04 Thread jackylk
Repository: carbondata
Updated Branches:
  refs/heads/carbonstore-rebase5 c305f309e -> 7f92fde49 (forced update)


http://git-wip-us.apache.org/repos/asf/carbondata/blob/c770ff68/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterBatchProcessorStepImpl.java
--
diff --git 
a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterBatchProcessorStepImpl.java
 
b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterBatchProcessorStepImpl.java
index f030d52..369c1f2 100644
--- 
a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterBatchProcessorStepImpl.java
+++ 
b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterBatchProcessorStepImpl.java
@@ -24,6 +24,7 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datastore.row.CarbonRow;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.processing.loading.AbstractDataLoadProcessorStep;
 import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
 import org.apache.carbondata.processing.loading.DataField;
@@ -59,13 +60,11 @@ public class DataWriterBatchProcessorStepImpl extends 
AbstractDataLoadProcessorS
 child.initialize();
   }
 
-  private String[] getStoreLocation(CarbonTableIdentifier tableIdentifier, 
String partitionId) {
-String[] storeLocation = CarbonDataProcessorUtil
-.getLocalDataFolderLocation(tableIdentifier.getDatabaseName(),
-tableIdentifier.getTableName(), 
String.valueOf(configuration.getTaskNo()), partitionId,
-configuration.getSegmentId() + "", false, false);
-CarbonDataProcessorUtil.createLocations(storeLocation);
-return storeLocation;
+  private String[] getStoreLocation(CarbonTableIdentifier tableIdentifier) {
+return CarbonDataProcessorUtil.getLocalDataFolderLocation(
+tableIdentifier.getDatabaseName(), tableIdentifier.getTableName(),
+String.valueOf(configuration.getTaskNo()),
+configuration.getSegmentId(), false, false);
   }
 
   @Override public Iterator[] execute() throws 
CarbonDataLoadingException {
@@ -75,18 +74,19 @@ public class DataWriterBatchProcessorStepImpl extends 
AbstractDataLoadProcessorS
 String tableName = tableIdentifier.getTableName();
 try {
   CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-  
.recordDictionaryValue2MdkAdd2FileTime(configuration.getPartitionId(),
+  
.recordDictionaryValue2MdkAdd2FileTime(CarbonTablePath.DEPRECATED_PATITION_ID,
   System.currentTimeMillis());
   int i = 0;
+  String[] storeLocation = getStoreLocation(tableIdentifier);
+  CarbonDataProcessorUtil.createLocations(storeLocation);
   for (Iterator iterator : iterators) {
-String[] storeLocation = getStoreLocation(tableIdentifier, 
String.valueOf(i));
 int k = 0;
 while (iterator.hasNext()) {
   CarbonRowBatch next = iterator.next();
   // If no rows from merge sorter, then don't create a file in fact 
column handler
   if (next.hasNext()) {
 CarbonFactDataHandlerModel model = CarbonFactDataHandlerModel
-.createCarbonFactDataHandlerModel(configuration, 
storeLocation, i, k++);
+.createCarbonFactDataHandlerModel(configuration, 
storeLocation, 0, k++);
 CarbonFactHandler dataHandler = CarbonFactHandlerFactory
 .createCarbonFactHandler(model, 
CarbonFactHandlerFactory.FactHandlerType.COLUMNAR);
 dataHandler.initialise();
@@ -119,10 +119,11 @@ public class DataWriterBatchProcessorStepImpl extends 
AbstractDataLoadProcessorS
 
CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordTotalRecords(rowCounter.get());
 processingComplete(dataHandler);
 CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-.recordDictionaryValue2MdkAdd2FileTime(configuration.getPartitionId(),
+
.recordDictionaryValue2MdkAdd2FileTime(CarbonTablePath.DEPRECATED_PATITION_ID,
 System.currentTimeMillis());
 CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-.recordMdkGenerateTotalTime(configuration.getPartitionId(), 
System.currentTimeMillis());
+.recordMdkGenerateTotalTime(CarbonTablePath.DEPRECATED_PATITION_ID,
+System.currentTimeMillis());
   }
 
   private void processingComplete(CarbonFactHandler dataHandler) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c770ff68/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterProcessorStepImpl.java
--
diff --git 
a/pr

[46/50] [abbrv] carbondata git commit: [CARBONDATA-2091][DataLoad] Support specifying sort column bounds in data loading

2018-03-04 Thread jackylk
[CARBONDATA-2091][DataLoad] Support specifying sort column bounds in data 
loading

Enhance data loading performance by specifying sort column bounds
1. Add row range number during convert-process-step
2. Dispatch rows to each sorter by range number
3. Sort/Write process step can be done concurrently in each range
4. Since all sorttemp files will be written in one folders, we add range
number to the file name to distingush them

Tests added and docs updated

This closes #1953


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

Branch: refs/heads/carbonstore-rebase5
Commit: 49d06c205a06817139fe6b0bb99240013d33a92d
Parents: 8996cd4
Author: xuchuanyin 
Authored: Tue Feb 13 10:58:06 2018 +0800
Committer: Jacky Li 
Committed: Sun Mar 4 20:08:30 2018 +0800

--
 .../constants/CarbonLoadOptionConstants.java|  10 +
 .../core/datastore/row/CarbonRow.java   |  10 +-
 .../ThriftWrapperSchemaConverterImpl.java   |   2 +-
 .../core/metadata/schema/BucketingInfo.java |  24 +-
 .../core/metadata/schema/ColumnRangeInfo.java   |  29 ++
 .../metadata/schema/SortColumnRangeInfo.java|  83 +
 docs/data-management-on-carbondata.md   |  11 +
 .../TestLoadDataWithSortColumnBounds.scala  | 348 +++
 .../carbondata/spark/rdd/CarbonScanRDD.scala|   2 +-
 .../carbondata/spark/rdd/PartitionDropper.scala |   2 +-
 .../spark/rdd/PartitionSplitter.scala   |   2 +-
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala |   3 +-
 .../strategy/CarbonLateDecodeStrategy.scala |   2 +-
 .../loading/CarbonDataLoadConfiguration.java|  11 +
 .../loading/DataLoadProcessBuilder.java |  77 +++-
 .../loading/converter/RowConverter.java |   2 +-
 .../converter/impl/RowConverterImpl.java|   5 +
 .../loading/model/CarbonLoadModel.java  |  14 +
 .../loading/model/CarbonLoadModelBuilder.java   |   1 +
 .../processing/loading/model/LoadOption.java|   1 +
 .../partition/impl/HashPartitionerImpl.java |  10 +-
 .../partition/impl/RangePartitionerImpl.java|  71 
 .../partition/impl/RawRowComparator.java|  63 
 .../processing/loading/sort/SorterFactory.java  |  16 +-
 ...arallelReadMergeSorterWithBucketingImpl.java | 272 ---
 ...allelReadMergeSorterWithColumnRangeImpl.java | 289 +++
 ...arallelReadMergeSorterWithBucketingImpl.java | 263 --
 ...allelReadMergeSorterWithColumnRangeImpl.java | 293 
 .../loading/sort/unsafe/UnsafeSortDataRows.java |   6 +-
 .../unsafe/merger/UnsafeIntermediateMerger.java |   6 +-
 .../UnsafeSingleThreadFinalSortFilesMerger.java |  11 +-
 .../steps/DataConverterProcessorStepImpl.java   | 102 +-
 ...ConverterProcessorWithBucketingStepImpl.java | 161 -
 .../steps/DataWriterProcessorStepImpl.java  |  70 +++-
 .../SingleThreadFinalSortFilesMerger.java   |   3 +-
 .../processing/sort/sortdata/SortDataRows.java  |  11 +-
 .../sortdata/SortIntermediateFileMerger.java|   6 +-
 .../sort/sortdata/SortParameters.java   |  10 +
 .../store/CarbonFactDataHandlerColumnar.java|   6 +-
 39 files changed, 1558 insertions(+), 750 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/49d06c20/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
 
b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
index a6bf60f..8ff8dc4 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
@@ -124,4 +124,14 @@ public final class CarbonLoadOptionConstants {
   public static final String ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION
   = "carbon.load.skewedDataOptimization.enabled";
   public static final String 
ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION_DEFAULT = "false";
+
+  /**
+   * field delimiter for each field in one bound
+   */
+  public static final String SORT_COLUMN_BOUNDS_FIELD_DELIMITER = ",";
+
+  /**
+   * row delimiter for each sort column bounds
+   */
+  public static final String SORT_COLUMN_BOUNDS_ROW_DELIMITER = ";";
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/49d06c20/core/src/main/java/org/apache/carbondata/core/datastore/row/CarbonRow.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata

[27/50] [abbrv] carbondata git commit: [CARBONDATA-1544][Datamap] Datamap FineGrain implementation

2018-03-04 Thread jackylk
[CARBONDATA-1544][Datamap] Datamap FineGrain implementation

Implemented interfaces for FG datamap and integrated to filterscanner to use 
the pruned bitset from FG datamap.
FG Query flow as follows.
1.The user can add FG datamap to any table and implement there interfaces.
2. Any filter query which hits the table with datamap will call prune method of 
FGdatamap.
3. The prune method of FGDatamap return list FineGrainBlocklet , these 
blocklets contain the information of block, blocklet, page and rowids 
information as well.
4. The pruned blocklets are internally wriitten to file and returns only the 
block , blocklet and filepath information as part of Splits.
5. Based on the splits scanrdd schedule the tasks.
6. In filterscanner we check the datamapwriterpath from split and reNoteads the 
bitset if exists. And pass this bitset as input to it.

This closes #1471


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

Branch: refs/heads/carbonstore-rebase5
Commit: 828ae5ecef68e97824693c60743a781c22f08d34
Parents: 3a0a9e6
Author: ravipesala 
Authored: Wed Nov 15 19:48:40 2017 +0530
Committer: Jacky Li 
Committed: Sun Mar 4 19:44:10 2018 +0800

--
 .../carbondata/core/datamap/DataMapMeta.java|   8 +-
 .../core/datamap/DataMapStoreManager.java   |  30 +-
 .../carbondata/core/datamap/DataMapType.java|  21 +
 .../carbondata/core/datamap/TableDataMap.java   |  33 +-
 .../core/datamap/dev/AbstractDataMapWriter.java | 110 +
 .../core/datamap/dev/BlockletSerializer.java|  57 +++
 .../carbondata/core/datamap/dev/DataMap.java|   5 +-
 .../core/datamap/dev/DataMapFactory.java|  14 +-
 .../core/datamap/dev/DataMapWriter.java |  57 ---
 .../cgdatamap/AbstractCoarseGrainDataMap.java   |  24 +
 .../AbstractCoarseGrainDataMapFactory.java  |  34 ++
 .../dev/fgdatamap/AbstractFineGrainDataMap.java |  24 +
 .../AbstractFineGrainDataMapFactory.java|  38 ++
 .../carbondata/core/datastore/DataRefNode.java  |   6 +
 .../core/datastore/block/TableBlockInfo.java|  10 +
 .../impl/btree/AbstractBTreeLeafNode.java   |   5 +
 .../datastore/impl/btree/BTreeNonLeafNode.java  |   5 +
 .../carbondata/core/indexstore/Blocklet.java|  30 +-
 .../indexstore/BlockletDataMapIndexStore.java   |   6 -
 .../core/indexstore/BlockletDetailsFetcher.java |   9 +
 .../core/indexstore/ExtendedBlocklet.java   |  17 +
 .../core/indexstore/FineGrainBlocklet.java  | 120 +
 .../indexstore/SegmentPropertiesFetcher.java|   3 +-
 .../blockletindex/BlockletDataMap.java  |  18 +-
 .../blockletindex/BlockletDataMapFactory.java   |  66 ++-
 .../blockletindex/BlockletDataRefNode.java  |  27 +-
 .../indexstore/blockletindex/IndexWrapper.java  |  18 +
 .../core/indexstore/schema/FilterType.java  |  24 -
 .../executer/ExcludeFilterExecuterImpl.java |   3 +
 .../executer/IncludeFilterExecuterImpl.java |   3 +
 .../scanner/impl/BlockletFilterScanner.java |   2 +
 .../apache/carbondata/core/util/CarbonUtil.java |  97 
 datamap/examples/pom.xml|   2 +-
 .../datamap/examples/MinMaxDataMap.java |  33 +-
 .../datamap/examples/MinMaxDataMapFactory.java  |  67 +--
 .../datamap/examples/MinMaxDataWriter.java  |  36 +-
 .../examples/MinMaxIndexBlockDetails.java   |  13 -
 .../carbondata/hadoop/CarbonInputSplit.java |  21 +-
 .../hadoop/api/CarbonTableInputFormat.java  |  17 +-
 .../testsuite/datamap/CGDataMapTestCase.scala   | 361 +++
 .../testsuite/datamap/DataMapWriterSuite.scala  |  43 +-
 .../testsuite/datamap/FGDataMapTestCase.scala   | 440 +++
 .../TestInsertAndOtherCommandConcurrent.scala   |  21 +-
 .../carbondata/spark/rdd/CarbonScanRDD.scala|   7 +-
 .../TestStreamingTableOperation.scala   |   5 +-
 .../datamap/DataMapWriterListener.java  |  57 ++-
 .../store/CarbonFactDataHandlerModel.java   |  10 +-
 .../store/writer/AbstractFactDataWriter.java| 128 +-
 48 files changed, 1784 insertions(+), 401 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/828ae5ec/core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java 
b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java
index 7746acf..dd15ccb 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java
@@ -19,15 +19,15 @@ package org.apache.carbondata

[11/50] [abbrv] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/55c4e438/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 de97e82..540607d 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
@@ -34,8 +34,8 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.cache.dictionary.Dictionary;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
-import 
org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
+import 
org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionColumnPage;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import 
org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
@@ -58,7 +58,7 @@ import org.apache.carbondata.core.scan.filter.intf.RowImpl;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import 
org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import 
org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -68,20 +68,20 @@ public class RowLevelFilterExecuterImpl implements 
FilterExecuter {
 
   private static final LogService LOGGER =
   
LogServiceFactory.getLogService(RowLevelFilterExecuterImpl.class.getName());
-  protected List dimColEvaluatorInfoList;
-  protected List msrColEvalutorInfoList;
+  List dimColEvaluatorInfoList;
+  List msrColEvalutorInfoList;
   protected Expression exp;
   protected AbsoluteTableIdentifier tableIdentifier;
   protected SegmentProperties segmentProperties;
   /**
* it has index at which given dimension is stored in file
*/
-  protected int[] dimensionBlocksIndex;
+  int[] dimensionChunkIndex;
 
   /**
* it has index at which given measure is stored in file
*/
-  protected int[] measureBlocksIndex;
+  int[] measureChunkIndex;
 
   private Map complexDimensionInfoMap;
 
@@ -89,18 +89,18 @@ public class RowLevelFilterExecuterImpl implements 
FilterExecuter {
* flag to check whether the filter dimension is present in current block 
list of dimensions.
* Applicable for restructure scenarios
*/
-  protected boolean[] isDimensionPresentInCurrentBlock;
+  boolean[] isDimensionPresentInCurrentBlock;
 
   /**
* flag to check whether the filter measure is present in current block list 
of measures.
* Applicable for restructure scenarios
*/
-  protected boolean[] isMeasurePresentInCurrentBlock;
+  boolean[] isMeasurePresentInCurrentBlock;
 
   /**
* is dimension column data is natural sorted
*/
-  protected boolean isNaturalSorted;
+  boolean isNaturalSorted;
 
   /**
* date direct dictionary generator
@@ -124,10 +124,10 @@ public class RowLevelFilterExecuterImpl implements 
FilterExecuter {
 }
 if (this.dimColEvaluatorInfoList.size() > 0) {
   this.isDimensionPresentInCurrentBlock = new 
boolean[dimColEvaluatorInfoList.size()];
-  this.dimensionBlocksIndex = new int[dimColEvaluatorInfoList.size()];
+  this.dimensionChunkIndex = new int[dimColEvaluatorInfoList.size()];
 } else {
   this.isDimensionPresentInCurrentBlock = new boolean[]{false};
-  this.dimensionBlocksIndex = new int[]{0};
+  this.dimensionChunkIndex = new int[]{0};
 }
 if (null == msrColEvalutorInfoList) {
   this.msrColEvalutorInfoList = new 
ArrayList(20);
@@ -136,10 +136,10 @@ public class RowLevelFilterExecuterImpl implements 
FilterExecuter {
 }
 if (this.msrColEvalutorInfoList.size() > 0) {
   this.isMeasurePresentInCurrentBlock = new 
boolean[msrColEvalutorInfoList.size()];
-  this.measureBlocksIndex = new int[msrColEvalutorInfoList.size()];
+  this.measureChunkIndex = new int[msrColEvalutorInfoList.size()];
 } else {
   this.isMeasurePresentInCurrentBlock = new boolean[]{false};
-  this.measureBlocksIndex = new int[] {0};
+  this.measureChunkInde

[08/50] [abbrv] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/55c4e438/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java
 
b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java
index 553f85e..773fbd7 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java
@@ -17,20 +17,15 @@
 package org.apache.carbondata.core.scan.result.iterator;
 
 import org.apache.carbondata.common.CarbonIterator;
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.scan.result.BatchResult;
+import org.apache.carbondata.core.scan.result.RowBatch;
 
 public class PartitionSpliterRawResultIterator extends 
CarbonIterator {
 
-  private CarbonIterator iterator;
-  private BatchResult batch;
+  private CarbonIterator iterator;
+  private RowBatch batch;
   private int counter;
 
-  private static final LogService LOGGER =
-  
LogServiceFactory.getLogService(PartitionSpliterRawResultIterator.class.getName());
-
-  public PartitionSpliterRawResultIterator(CarbonIterator 
iterator) {
+  public PartitionSpliterRawResultIterator(CarbonIterator iterator) {
 this.iterator = iterator;
   }
 
@@ -65,7 +60,7 @@ public class PartitionSpliterRawResultIterator extends 
CarbonIterator
* @param batch
* @return
*/
-  private boolean checkBatchEnd(BatchResult batch) {
+  private boolean checkBatchEnd(RowBatch batch) {
 return !(counter < batch.getSize());
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/55c4e438/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java
 
b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java
index 70d0958..1dd1595 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java
@@ -21,7 +21,7 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
-import org.apache.carbondata.core.scan.result.BatchResult;
+import org.apache.carbondata.core.scan.result.RowBatch;
 import org.apache.carbondata.core.scan.wrappers.ByteArrayWrapper;
 
 /**
@@ -37,7 +37,7 @@ public class RawResultIterator extends 
CarbonIterator {
   /**
* Iterator of the Batch raw result.
*/
-  private CarbonIterator detailRawQueryResultIterator;
+  private CarbonIterator detailRawQueryResultIterator;
 
   /**
* Counter to maintain the row counter.
@@ -55,9 +55,9 @@ public class RawResultIterator extends 
CarbonIterator {
   /**
* batch of the result.
*/
-  private BatchResult batch;
+  private RowBatch batch;
 
-  public RawResultIterator(CarbonIterator 
detailRawQueryResultIterator,
+  public RawResultIterator(CarbonIterator 
detailRawQueryResultIterator,
   SegmentProperties sourceSegProperties, SegmentProperties 
destinationSegProperties) {
 this.detailRawQueryResultIterator = detailRawQueryResultIterator;
 this.sourceSegProperties = sourceSegProperties;
@@ -155,7 +155,7 @@ public class RawResultIterator extends 
CarbonIterator {
* @param batch
* @return
*/
-  private boolean checkIfBatchIsProcessedCompletely(BatchResult batch) {
+  private boolean checkIfBatchIsProcessedCompletely(RowBatch batch) {
 if (counter < batch.getSize()) {
   return false;
 } else {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/55c4e438/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/VectorDetailQueryResultIterator.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/VectorDetailQueryResultIterator.java
 
b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/VectorDetailQueryResultIterator.java
index cc9710e..c7cb00d 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/VectorDetailQueryResultIterator.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/VectorDetailQueryResultIterator.java
@@ -35,10 +35,12 @@ public class VectorDetailQueryResultIterator ext

[22/50] [abbrv] carbondata git commit: [CARBONDATA-2025] Unify all path construction through CarbonTablePath static method

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java 
b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
index 9799ac2..b7b5e43 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
@@ -22,16 +22,14 @@ import 
org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
 
 import org.apache.hadoop.fs.Path;
 
-
 /**
  * Helps to get Table content paths.
  */
-public class CarbonTablePath extends Path {
+public class CarbonTablePath {
 
   private static final String METADATA_DIR = "Metadata";
   private static final String DICTIONARY_EXT = ".dict";
@@ -54,19 +52,10 @@ public class CarbonTablePath extends Path {
   private static final String STREAMING_LOG_DIR = "log";
   private static final String STREAMING_CHECKPOINT_DIR = "checkpoint";
 
-  private String tablePath;
-  private CarbonTableIdentifier carbonTableIdentifier;
-
   /**
-   * structure CarbonTablePath object to manage table paths
-   *
-   * @param carbonTableIdentifier identifier of carbon table that the segment 
belong to
-   * @param tablePathString the store path of the segment
+   * This class provides static utility only.
*/
-  public CarbonTablePath(CarbonTableIdentifier carbonTableIdentifier, String 
tablePathString) {
-super(tablePathString);
-this.carbonTableIdentifier = carbonTableIdentifier;
-this.tablePath = tablePathString;
+  private CarbonTablePath() {
   }
 
   /**
@@ -130,40 +119,21 @@ public class CarbonTablePath extends Path {
   }
 
   /**
-   * gets table path
+   * Return absolute path of dictionary file
*/
-  public String getPath() {
-return tablePath;
+  public static String getDictionaryFilePath(String tablePath, String 
columnId) {
+return getMetadataPath(tablePath) + File.separator + 
getDictionaryFileName(columnId);
   }
 
   /**
-   * @param columnId unique column identifier
-   * @return absolute path of dictionary file
+   * Return absolute path of dictionary file
*/
-  public String getDictionaryFilePath(String columnId) {
-return getMetaDataDir() + File.separator + getDictionaryFileName(columnId);
-  }
-
-  /**
-   * @param dictionaryPath
-   * @param columnId unique column identifier
-   * @return absolute path of dictionary file
-   */
-  public String getDictionaryFilePath(String dictionaryPath, String columnId) {
+  public static String getExternalDictionaryFilePath(String dictionaryPath, 
String columnId) {
 return dictionaryPath + File.separator + getDictionaryFileName(columnId);
   }
 
   /**
-   * This method will return the metadata directory location for a table
-   *
-   * @return
-   */
-  public String getMetadataDirectoryPath() {
-return getMetaDataDir();
-  }
-
-  /**
-   * Return metadata path based on `tablePath`
+   * Return metadata path
*/
   public static String getMetadataPath(String tablePath) {
 return tablePath + File.separator + METADATA_DIR;
@@ -184,67 +154,42 @@ public class CarbonTablePath extends Path {
   }
 
   /**
-   * @param columnId unique column identifier
-   * @return absolute path of dictionary meta file
+   * Return absolute path of dictionary meta file
*/
-  public String getDictionaryMetaFilePath(String columnId) {
-return getMetaDataDir() + File.separator + columnId + DICTIONARY_META_EXT;
-  }
-
-  /**
-   * @param dictionaryPath
-   * @param columnId unique column identifier
-   * @return absolute path of dictionary file
-   */
-  public String getDictionaryMetaFilePath(String dictionaryPath, String 
columnId) {
+  public static String getExternalDictionaryMetaFilePath(String 
dictionaryPath, String columnId) {
 return dictionaryPath + File.separator + columnId + DICTIONARY_META_EXT;
   }
 
   /**
-   * @param columnId unique column identifier
-   * @return absolute path of sort index file
+   * Return absolute path of dictionary meta file
*/
-  public String getSortIndexFilePath(String columnId) {
-return getMetaDataDir() + File.separator + columnId + SORT_INDEX_EXT;
+  public static String getDictionaryMetaFilePath(String tablePath, String 
columnId) {
+return getMetadataPath(tablePath) + File.separator + columnId + 
DICTIONARY_META_EXT;
   }
 
   /**
-   * @param dictionaryPath
-   * @param columnId unique column identifier
-   * @return absolute path of dictionary 

[14/50] [abbrv] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/55c4e438/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/SegmentInfo.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/SegmentInfo.java
 
b/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/SegmentInfo.java
index 0cb2918..099fffd 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/SegmentInfo.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/SegmentInfo.java
@@ -29,31 +29,12 @@ public class SegmentInfo implements Serializable {
   private static final long serialVersionUID = -174987462709431L;
 
   /**
-   * number of column in the segment
-   */
-  private int numberOfColumns;
-
-  /**
* cardinality of each columns
* column which is not participating in the multidimensional key cardinality 
will be -1;
*/
   private int[] columnCardinality;
 
   /**
-   * @return the numberOfColumns
-   */
-  public int getNumberOfColumns() {
-return numberOfColumns;
-  }
-
-  /**
-   * @param numberOfColumns the numberOfColumns to set
-   */
-  public void setNumberOfColumns(int numberOfColumns) {
-this.numberOfColumns = numberOfColumns;
-  }
-
-  /**
* @return the columnCardinality
*/
   public int[] getColumnCardinality() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/55c4e438/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
 
b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
index 6036569..d17d865 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
@@ -19,7 +19,13 @@ package org.apache.carbondata.core.metadata.schema.table;
 
 import java.io.IOException;
 import java.io.Serializable;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
@@ -33,7 +39,10 @@ import 
org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import 
org.apache.carbondata.core.metadata.schema.table.column.CarbonImplicitDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.scan.model.QueryModel;
+import org.apache.carbondata.core.scan.model.QueryProjection;
 import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.DataTypeConverter;
 import org.apache.carbondata.core.util.DataTypeUtil;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 
@@ -136,10 +145,7 @@ public class CarbonTable implements Serializable {
   /**
* During creation of TableInfo from hivemetastore the DataMapSchemas and 
the columns
* DataTypes are not converted to the appropriate child classes.
-   *
* This method will cast the same to the appropriate classes
-   *
-   * @param tableInfo
*/
   public static void updateTableInfo(TableInfo tableInfo) {
 List dataMapSchemas = new ArrayList<>();
@@ -153,8 +159,9 @@ public class CarbonTable implements Serializable {
 }
 tableInfo.setDataMapSchemaList(dataMapSchemas);
 for (ColumnSchema columnSchema : 
tableInfo.getFactTable().getListOfColumns()) {
-  columnSchema.setDataType(DataTypeUtil.valueOf(columnSchema.getDataType(),
-  columnSchema.getPrecision(), columnSchema.getScale()));
+  columnSchema.setDataType(
+  DataTypeUtil.valueOf(
+  columnSchema.getDataType(), columnSchema.getPrecision(), 
columnSchema.getScale()));
 }
 List childSchema = tableInfo.getDataMapSchemaList();
 for (DataMapSchema dataMapSchema : childSchema) {
@@ -168,10 +175,11 @@ public class CarbonTable implements Serializable {
   }
 }
 if (tableInfo.getFactTable().getBucketingInfo() != null) {
-  for (ColumnSchema columnSchema : tableInfo.getFactTable()
-  .getBucketingInfo().getListOfColumns()) {
-
columnSchema.setDataType(DataTypeUtil.valueOf(columnSchema.getDataType(),
-columnSchema.getPrecision(), columnSchema.getScale()));
+  for (ColumnSchema columnSchema :
+  tableInfo.getFactTable().getBucketingInfo().getListOfColumns()) {
+columnSchema.setDataType(
+DataTypeUtil.valueOf(
+columnSchema.getDataType(), columnSchema.getPrecision(), 

[37/50] [abbrv] carbondata git commit: Revert "[CARBONDATA-2018][DataLoad] Optimization in reading/writing for sort temp row"

2018-03-04 Thread jackylk
Revert "[CARBONDATA-2018][DataLoad] Optimization in reading/writing for sort 
temp row"

This reverts commit de92ea9a123b17d903f2d1d4662299315c792954.


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

Branch: refs/heads/carbonstore-rebase5
Commit: bfdf3e354b83b173a3325204074bdee3f72cd71a
Parents: 3f1d101
Author: Jacky Li 
Authored: Sat Feb 10 20:11:25 2018 +0800
Committer: Jacky Li 
Committed: Sun Mar 4 20:04:49 2018 +0800

--
 .../carbondata/core/util/NonDictionaryUtil.java |  67 ++-
 .../presto/util/CarbonDataStoreCreator.scala|   1 +
 .../load/DataLoadProcessorStepOnSpark.scala |   6 +-
 .../loading/row/IntermediateSortTempRow.java| 117 -
 .../loading/sort/SortStepRowHandler.java| 466 ---
 .../loading/sort/SortStepRowUtil.java   | 103 
 .../sort/unsafe/UnsafeCarbonRowPage.java| 331 +++--
 .../loading/sort/unsafe/UnsafeSortDataRows.java |  57 ++-
 .../unsafe/comparator/UnsafeRowComparator.java  |  95 ++--
 .../UnsafeRowComparatorForNormalDIms.java   |  59 +++
 .../UnsafeRowComparatorForNormalDims.java   |  59 ---
 .../sort/unsafe/holder/SortTempChunkHolder.java |   3 +-
 .../holder/UnsafeFinalMergePageHolder.java  |  19 +-
 .../unsafe/holder/UnsafeInmemoryHolder.java |  21 +-
 .../holder/UnsafeSortTempFileChunkHolder.java   | 138 --
 .../merger/UnsafeIntermediateFileMerger.java| 118 -
 .../UnsafeSingleThreadFinalSortFilesMerger.java |  27 +-
 .../merger/CompactionResultSortProcessor.java   |   1 +
 .../sort/sortdata/IntermediateFileMerger.java   |  95 +++-
 .../IntermediateSortTempRowComparator.java  |  73 ---
 .../sort/sortdata/NewRowComparator.java |   5 +-
 .../sortdata/NewRowComparatorForNormalDims.java |   3 +-
 .../processing/sort/sortdata/RowComparator.java |  94 
 .../sortdata/RowComparatorForNormalDims.java|  62 +++
 .../SingleThreadFinalSortFilesMerger.java   |  25 +-
 .../processing/sort/sortdata/SortDataRows.java  |  85 +++-
 .../sort/sortdata/SortTempFileChunkHolder.java  | 174 +--
 .../sort/sortdata/TableFieldStat.java   | 176 ---
 28 files changed, 1294 insertions(+), 1186 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/bfdf3e35/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java 
b/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
index fca1244..d6ecfbc 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
@@ -82,26 +82,18 @@ public class NonDictionaryUtil {
   }
 
   /**
-   * Method to get the required dictionary Dimension from obj []
+   * Method to get the required Dimension from obj []
*
* @param index
* @param row
* @return
*/
-  public static int getDictDimension(int index, Object[] row) {
-int[] dimensions = (int[]) row[WriteStepRowUtil.DICTIONARY_DIMENSION];
+  public static Integer getDimension(int index, Object[] row) {
+
+Integer[] dimensions = (Integer[]) 
row[WriteStepRowUtil.DICTIONARY_DIMENSION];
+
 return dimensions[index];
-  }
 
-  /**
-   * Method to get the required non-dictionary & complex from 3-parted row
-   * @param index
-   * @param row
-   * @return
-   */
-  public static byte[] getNoDictOrComplex(int index, Object[] row) {
-byte[][] nonDictArray = (byte[][]) 
row[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX];
-return nonDictArray[index];
   }
 
   /**
@@ -116,11 +108,60 @@ public class NonDictionaryUtil {
 return measures[index];
   }
 
+  public static byte[] getByteArrayForNoDictionaryCols(Object[] row) {
+
+return (byte[]) row[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX];
+  }
+
   public static void prepareOutObj(Object[] out, int[] dimArray, byte[][] 
byteBufferArr,
   Object[] measureArray) {
+
 out[WriteStepRowUtil.DICTIONARY_DIMENSION] = dimArray;
 out[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX] = byteBufferArr;
 out[WriteStepRowUtil.MEASURE] = measureArray;
 
   }
+
+  /**
+   * This method will extract the single dimension from the complete high card 
dims byte[].+ *
+   * The format of the byte [] will be,  Totallength,CompleteStartOffsets,Dat
+   *
+   * @param highCardArr
+   * @param index
+   * @param highCardinalityCount
+   * @param outBuffer
+   */
+  public static void extractSingleHighCardDims(byte[] highCardArr, int index,
+  int highCardin

[15/50] [abbrv] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/55c4e438/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeFixedLengthDimensionDataChunkStore.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeFixedLengthDimensionDataChunkStore.java
 
b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeFixedLengthDimensionDataChunkStore.java
index 8c8d08f..a689d8e 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeFixedLengthDimensionDataChunkStore.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeFixedLengthDimensionDataChunkStore.java
@@ -124,22 +124,22 @@ public class UnsafeFixedLengthDimensionDataChunkStore
   /**
* to compare the two byte array
*
-   * @param indexindex of first byte array
+   * @param rowIdindex of first byte array
* @param compareValue value of to be compared
* @return compare result
*/
-  @Override public int compareTo(int index, byte[] compareValue) {
+  @Override public int compareTo(int rowId, byte[] compareValue) {
 // based on index we need to calculate the actual position in memory block
-index = index * columnValueSize;
+rowId = rowId * columnValueSize;
 int compareResult = 0;
 for (int i = 0; i < compareValue.length; i++) {
   compareResult = (CarbonUnsafe.getUnsafe()
-  .getByte(dataPageMemoryBlock.getBaseObject(), 
dataPageMemoryBlock.getBaseOffset() + index)
+  .getByte(dataPageMemoryBlock.getBaseObject(), 
dataPageMemoryBlock.getBaseOffset() + rowId)
   & 0xff) - (compareValue[i] & 0xff);
   if (compareResult != 0) {
 break;
   }
-  index++;
+  rowId++;
 }
 return compareResult;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/55c4e438/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 36b2bd8..e1eb378 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
@@ -189,11 +189,11 @@ public class UnsafeVariableLengthDimesionDataChunkStore
   /**
* to compare the two byte array
*
-   * @param index index of first byte array
+   * @param rowId index of first byte array
* @param compareValue value of to be compared
* @return compare result
*/
-  @Override public int compareTo(int index, byte[] compareValue) {
+  @Override public int compareTo(int rowId, byte[] compareValue) {
 // 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
@@ -201,13 +201,13 @@ public class UnsafeVariableLengthDimesionDataChunkStore
 // else subtract the current row offset
 // with complete data length get the offset of set of data
 int currentDataOffset = 
CarbonUnsafe.getUnsafe().getInt(dataPageMemoryBlock.getBaseObject(),
-dataPageMemoryBlock.getBaseOffset() + this.dataPointersOffsets + 
((long)index
+dataPageMemoryBlock.getBaseOffset() + this.dataPointersOffsets + 
((long) rowId
 * CarbonCommonConstants.INT_SIZE_IN_BYTE * 1L));
 short length = 0;
 // calculating the length of data
-if (index < numberOfRows - 1) {
+if (rowId < numberOfRows - 1) {
   int OffsetOfNextdata = 
CarbonUnsafe.getUnsafe().getInt(dataPageMemoryBlock.getBaseObject(),
-  dataPageMemoryBlock.getBaseOffset() + this.dataPointersOffsets + 
((index + 1)
+  dataPageMemoryBlock.getBaseOffset() + this.dataPointersOffsets + 
((rowId + 1)
   * CarbonCommonConstants.INT_SIZE_IN_BYTE));
   length = (short) (OffsetOfNextdata - (currentDataOffset
   + CarbonCommonConstants.SHORT_SIZE_IN_BYTE));

http://git-wip-us.apache.org/repos/asf/carbondata/blob/55c4e438/core/src/main/java/org/apache/carbondata/core/datastore/columnar/ColumnGroupModel.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datastore/columnar/ColumnGroupModel.java
 
b/core/src/main/java/org/apache/carbondata/core/datastore/columnar/ColumnGroupModel.java
index 74d268a..e2a4161 100644
--- 
a/core/src/main/java/org/apache/

[44/50] [abbrv] carbondata git commit: [CARBONDATA-2023][DataLoad] Add size base block allocation in data loading

2018-03-04 Thread jackylk
[CARBONDATA-2023][DataLoad] Add size base block allocation in data loading

Carbondata assign blocks to nodes at the beginning of data loading.
Previous block allocation strategy is block number based and it will
suffer skewed data problem if the size of input files differs a lot.

We introduced a size based block allocation strategy to optimize data
loading performance in skewed data scenario.

This closes #1808


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

Branch: refs/heads/carbonstore-rebase5
Commit: e83d910b4c9085c0dc6c062a3c05dcd9059e1978
Parents: daecc77
Author: xuchuanyin 
Authored: Thu Feb 8 14:42:39 2018 +0800
Committer: Jacky Li 
Committed: Sun Mar 4 20:08:30 2018 +0800

--
 .../constants/CarbonLoadOptionConstants.java|  10 +
 .../core/datastore/block/TableBlockInfo.java|  29 ++
 .../carbondata/core/util/CarbonProperties.java  |  11 +
 docs/useful-tips-on-carbondata.md   |   1 +
 .../spark/rdd/NewCarbonDataLoadRDD.scala|   4 +-
 .../spark/sql/hive/DistributionUtil.scala   |   2 +-
 .../spark/rdd/CarbonDataRDDFactory.scala|  18 +-
 .../merger/NodeMultiBlockRelation.java  |  40 ++
 .../processing/util/CarbonLoaderUtil.java   | 480 ---
 .../processing/util/CarbonLoaderUtilTest.java   | 125 +
 10 files changed, 545 insertions(+), 175 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/e83d910b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
 
b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
index bcfeba0..a6bf60f 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
@@ -114,4 +114,14 @@ public final class CarbonLoadOptionConstants {
*/
   public static final int MAX_EXTERNAL_DICTIONARY_SIZE = 1000;
 
+  /**
+   * enable block size based block allocation while loading data. By default, 
carbondata assigns
+   * blocks to node based on block number. If this option is set to `true`, 
carbondata will
+   * consider block size first and make sure that all the nodes will process 
almost equal size of
+   * data. This option is especially useful when you encounter skewed data.
+   */
+  @CarbonProperty
+  public static final String ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION
+  = "carbon.load.skewedDataOptimization.enabled";
+  public static final String 
ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION_DEFAULT = "false";
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e83d910b/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
 
b/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
index a7bfdba..c0cebe0 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
@@ -19,6 +19,8 @@ package org.apache.carbondata.core.datastore.block;
 import java.io.IOException;
 import java.io.Serializable;
 import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -98,6 +100,20 @@ public class TableBlockInfo implements Distributable, 
Serializable {
 
   private String dataMapWriterPath;
 
+  /**
+   * comparator to sort by block size in descending order.
+   * Since each line is not exactly the same, the size of a InputSplit may 
differs,
+   * so we allow some deviation for these splits.
+   */
+  public static final Comparator DATA_SIZE_DESC_COMPARATOR =
+  new Comparator() {
+@Override public int compare(Distributable o1, Distributable o2) {
+  long diff =
+  ((TableBlockInfo) o1).getBlockLength() - ((TableBlockInfo) 
o2).getBlockLength();
+  return diff < 0 ? 1 : (diff == 0 ? 0 : -1);
+}
+  };
+
   public TableBlockInfo(String filePath, long blockOffset, String segmentId,
   String[] locations, long blockLength, ColumnarFormatVersion version,
   String[] deletedDeltaFilePath) {
@@ -434,4 +450,17 @@ public class TableBlockInfo implements Distributable, 
Serializable {
   

[13/50] [abbrv] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/55c4e438/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 69f5ceb..22d1df1 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
@@ -43,10 +43,9 @@ import 
org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.datastore.block.TableBlockUniqueIdentifier;
 import org.apache.carbondata.core.indexstore.BlockletDetailInfo;
-import 
org.apache.carbondata.core.indexstore.blockletindex.BlockletDataRefNodeWrapper;
+import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataRefNode;
 import org.apache.carbondata.core.indexstore.blockletindex.IndexWrapper;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
-import org.apache.carbondata.core.keygenerator.KeyGenerator;
 import org.apache.carbondata.core.memory.UnsafeMemoryManager;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.blocklet.BlockletInfo;
@@ -64,8 +63,8 @@ import 
org.apache.carbondata.core.scan.executor.util.RestructureUtil;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.SingleTableProvider;
 import org.apache.carbondata.core.scan.filter.TableProvider;
-import org.apache.carbondata.core.scan.model.QueryDimension;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
+import org.apache.carbondata.core.scan.model.ProjectionMeasure;
 import org.apache.carbondata.core.scan.model.QueryModel;
 import org.apache.carbondata.core.stats.QueryStatistic;
 import org.apache.carbondata.core.stats.QueryStatisticsConstants;
@@ -121,7 +120,6 @@ public abstract class AbstractQueryExecutor implements 
QueryExecutor {
 queryProperties.queryStatisticsRecorder =
 
CarbonTimeStatisticsFactory.createExecutorRecorder(queryModel.getQueryId());
 queryModel.setStatisticsRecorder(queryProperties.queryStatisticsRecorder);
-QueryUtil.resolveQueryModel(queryModel);
 QueryStatistic queryStatistic = new QueryStatistic();
 // sort the block info
 // so block will be loaded in sorted order this will be required for
@@ -168,12 +166,12 @@ public abstract class AbstractQueryExecutor implements 
QueryExecutor {
 .addStatistics(QueryStatisticsConstants.LOAD_BLOCKS_EXECUTOR, 
System.currentTimeMillis());
 queryProperties.queryStatisticsRecorder.recordStatistics(queryStatistic);
 // calculating the total number of aggeragted columns
-int measureCount = queryModel.getQueryMeasures().size();
+int measureCount = queryModel.getProjectionMeasures().size();
 
 int currentIndex = 0;
 DataType[] dataTypes = new DataType[measureCount];
 
-for (QueryMeasure carbonMeasure : queryModel.getQueryMeasures()) {
+for (ProjectionMeasure carbonMeasure : queryModel.getProjectionMeasures()) 
{
   // adding the data type and aggregation type of all the measure this
   // can be used
   // to select the aggregator
@@ -198,9 +196,11 @@ public abstract class AbstractQueryExecutor implements 
QueryExecutor {
 queryStatistic = new QueryStatistic();
 // dictionary column unique column id to dictionary mapping
 // which will be used to get column actual data
-queryProperties.columnToDictionayMapping = QueryUtil
-.getDimensionDictionaryDetail(queryModel.getQueryDimension(),
-queryProperties.complexFilterDimension, 
queryModel.getAbsoluteTableIdentifier(),
+queryProperties.columnToDictionayMapping =
+QueryUtil.getDimensionDictionaryDetail(
+queryModel.getProjectionDimensions(),
+queryProperties.complexFilterDimension,
+queryModel.getAbsoluteTableIdentifier(),
 tableProvider);
 queryStatistic
 .addStatistics(QueryStatisticsConstants.LOAD_DICTIONARY, 
System.currentTimeMillis());
@@ -263,8 +263,8 @@ public abstract class AbstractQueryExecutor implements 
QueryExecutor {
 // and query will be executed based on that infos
 for (int i = 0; i < queryProperties.dataBlocks.size(); i++) {
   AbstractIndex abstractIndex = queryProperties.dataBlocks.get(i);
-  BlockletDataRefNodeWrapper dataRefNode =
-  (BlockletDataRefNodeWrapper) abstractIndex.getDataRefNode();
+  BlockletDataRefNode dataRefNode =
+  (BlockletDataRefNode) abstractIndex.getDataRefNode();
   blockExec

[31/50] [abbrv] carbondata git commit: [REBASE] Solve conflict after rebasing master

2018-03-04 Thread jackylk
[REBASE] Solve conflict after rebasing master


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

Branch: refs/heads/carbonstore-rebase5
Commit: ef812484f159945cd7e8921965698f67a6d53a52
Parents: 8fcde02
Author: Jacky Li 
Authored: Fri Feb 9 01:39:20 2018 +0800
Committer: Jacky Li 
Committed: Sun Mar 4 19:52:34 2018 +0800

--
 .../filter/executer/FalseFilterExecutor.java| 10 +
 .../core/util/path/CarbonTablePath.java | 23 +++-
 .../CarbonAlterTableDataTypeChangeCommand.scala |  7 +++---
 .../schema/CarbonAlterTableRenameCommand.scala  |  3 ++-
 .../apache/spark/sql/hive/CarbonMetaStore.scala |  9 +---
 .../processing/util/CarbonLoaderUtil.java   |  4 ++--
 6 files changed, 28 insertions(+), 28 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/ef812484/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FalseFilterExecutor.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FalseFilterExecutor.java
 
b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FalseFilterExecutor.java
index 1c918bc..ee3b0fc 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FalseFilterExecutor.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FalseFilterExecutor.java
@@ -44,17 +44,19 @@ public class FalseFilterExecutor implements FilterExecuter {
 return group;
   }
 
-  @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax)
+  @Override
+  public boolean applyFilter(RowIntf value, int dimOrdinalMax)
   throws FilterUnsupportedException, IOException {
 return false;
   }
 
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] 
blockMinValue) {
-
+  @Override
+  public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) 
{
 return new BitSet();
   }
 
-  @Override public void readColumnChunks(RawBlockletColumnChunks 
blockChunkHolder) {
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks blockChunkHolder) {
 // Do Nothing
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ef812484/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java 
b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
index b7b5e43..50c5a31 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
@@ -36,13 +36,13 @@ public class CarbonTablePath {
   private static final String DICTIONARY_META_EXT = ".dictmeta";
   private static final String SORT_INDEX_EXT = ".sortindex";
   private static final String SCHEMA_FILE = "schema";
-  public static final String TABLE_STATUS_FILE = "tablestatus";
   private static final String FACT_DIR = "Fact";
   private static final String SEGMENT_PREFIX = "Segment_";
   private static final String PARTITION_PREFIX = "Part";
   private static final String DATA_PART_PREFIX = "part-";
   private static final String BATCH_PREFIX = "_batchno";
 
+  public static final String TABLE_STATUS_FILE = "tablestatus";
   public static final String CARBON_DATA_EXT = ".carbondata";
   public static final String INDEX_FILE_EXT = ".carbonindex";
   public static final String MERGE_INDEX_FILE_EXT = ".carbonindexmerge";
@@ -140,20 +140,6 @@ public class CarbonTablePath {
   }
 
   /**
-   * Return metadata path based on `tablePath`
-   */
-  public static String getTableStatusPath(String tablePath) {
-return getMetadataPath(tablePath) + File.separator + TABLE_STATUS_FILE;
-  }
-
-  /**
-   * Return table status file path based on `tablePath`
-   */
-  public static String getTableStatusFilePath(String tablePath) {
-return getMetadataPath(tablePath) + CarbonCommonConstants.FILE_SEPARATOR + 
TABLE_STATUS_FILE;
-  }
-
-  /**
* Return absolute path of dictionary meta file
*/
   public static String getExternalDictionaryMetaFilePath(String 
dictionaryPath, String columnId) {
@@ -213,6 +199,13 @@ public class CarbonTablePath {
 }
   }
 
+  /**
+   * Return absolute path of table status file
+   */
+  public static String getTableStatusFilePath(String tablePath) {
+return getMetadataPath(tablePath) + File.separator + TABLE_STATUS_FILE;
+  }
+
   public static String getTableStatusFilePathWithUUID(

[40/50] [abbrv] carbondata git commit: [CARBONDATA-2159] Remove carbon-spark dependency in store-sdk module

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/83df87dd/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
--
diff --git 
a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
 
b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
index 8d394db..e69de29 100644
--- 
a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
+++ 
b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
@@ -1,610 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.util
-
-import java.text.SimpleDateFormat
-import java.util
-import java.util.{Date, List, Locale}
-
-import scala.collection.{immutable, mutable}
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ArrayBuffer
-
-import org.apache.commons.lang3.StringUtils
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.Path
-import org.apache.hadoop.mapred.JobConf
-import org.apache.hadoop.mapreduce.{TaskAttemptID, TaskType}
-import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat, FileSplit}
-import org.apache.hadoop.mapreduce.task.{JobContextImpl, 
TaskAttemptContextImpl}
-import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.GenericInternalRow
-import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, 
PartitionedFile}
-import org.apache.spark.sql.util.CarbonException
-import org.apache.spark.sql.util.SparkSQLUtil.sessionState
-
-import org.apache.carbondata.common.constants.LoggerAction
-import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
-import org.apache.carbondata.core.constants.{CarbonCommonConstants, 
CarbonLoadOptionConstants}
-import org.apache.carbondata.core.indexstore.PartitionSpec
-import org.apache.carbondata.core.locks.{CarbonLockFactory, CarbonLockUtil, 
LockUsage}
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, 
SegmentStatus, SegmentStatusManager}
-import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
-import 
org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants
-import org.apache.carbondata.processing.loading.csvinput.CSVInputFormat
-import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, 
CarbonLoadModel}
-import org.apache.carbondata.processing.util.{CarbonLoaderUtil, 
DeleteLoadFolders, TableOptionConstant}
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
-import org.apache.carbondata.spark.load.DataLoadProcessBuilderOnSpark.LOGGER
-import org.apache.carbondata.spark.load.ValidateUtil
-import org.apache.carbondata.spark.rdd.SerializableConfiguration
-
-/**
- * the util object of data loading
- */
-object DataLoadingUtil {
-
-  val LOGGER: LogService = 
LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-
-  /**
-   * get data loading options and initialise default value
-   */
-  def getDataLoadingOptions(
-  carbonProperty: CarbonProperties,
-  options: immutable.Map[String, String]): mutable.Map[String, String] = {
-val optionsFinal = scala.collection.mutable.Map[String, String]()
-optionsFinal.put("delimiter", options.getOrElse("delimiter", ","))
-optionsFinal.put("quotechar", options.getOrElse("quotechar", "\""))
-optionsFinal.put("fileheader", options.getOrElse("fileheader", ""))
-optionsFinal.put("commentchar", options.getOrElse("commentchar", "#"))
-optionsFinal.put("columndict", options.getOrElse("columndict", null))
-
-optionsFinal.put("escapechar",
-  CarbonLoaderUtil.getEscapeChar(options.getOrElse("escapechar", "\\")))
-
-optionsFinal.put(
-  "serialization_null_format",
-  options.getOrElse("ser

[30/50] [abbrv] carbondata git commit: [CARBONDATA-2018][DataLoad] Optimization in reading/writing for sort temp row

2018-03-04 Thread jackylk
[CARBONDATA-2018][DataLoad] Optimization in reading/writing for sort temp row

Pick up the no-sort fields in the row and pack them as bytes array and skip 
parsing them during merge sort to reduce CPU consumption

This closes #1792


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

Branch: refs/heads/carbonstore-rebase5
Commit: 8fcde02eefbd9ff176ceed2c2a33fad4d48fb332
Parents: 0feaeb5
Author: xuchuanyin 
Authored: Thu Feb 8 14:35:14 2018 +0800
Committer: Jacky Li 
Committed: Sun Mar 4 19:44:12 2018 +0800

--
 .../carbondata/core/util/NonDictionaryUtil.java |  67 +--
 .../presto/util/CarbonDataStoreCreator.scala|   1 -
 .../load/DataLoadProcessorStepOnSpark.scala |   6 +-
 .../loading/row/IntermediateSortTempRow.java| 117 +
 .../loading/sort/SortStepRowHandler.java| 466 +++
 .../loading/sort/SortStepRowUtil.java   | 103 
 .../sort/unsafe/UnsafeCarbonRowPage.java| 331 ++---
 .../loading/sort/unsafe/UnsafeSortDataRows.java |  57 +--
 .../unsafe/comparator/UnsafeRowComparator.java  |  95 ++--
 .../UnsafeRowComparatorForNormalDIms.java   |  59 ---
 .../UnsafeRowComparatorForNormalDims.java   |  59 +++
 .../sort/unsafe/holder/SortTempChunkHolder.java |   3 +-
 .../holder/UnsafeFinalMergePageHolder.java  |  19 +-
 .../unsafe/holder/UnsafeInmemoryHolder.java |  21 +-
 .../holder/UnsafeSortTempFileChunkHolder.java   | 138 ++
 .../merger/UnsafeIntermediateFileMerger.java| 118 +
 .../UnsafeSingleThreadFinalSortFilesMerger.java |  27 +-
 .../merger/CompactionResultSortProcessor.java   |   1 -
 .../sort/sortdata/IntermediateFileMerger.java   |  95 +---
 .../IntermediateSortTempRowComparator.java  |  73 +++
 .../sort/sortdata/NewRowComparator.java |   5 +-
 .../sortdata/NewRowComparatorForNormalDims.java |   3 +-
 .../processing/sort/sortdata/RowComparator.java |  94 
 .../sortdata/RowComparatorForNormalDims.java|  62 ---
 .../SingleThreadFinalSortFilesMerger.java   |  25 +-
 .../processing/sort/sortdata/SortDataRows.java  |  85 +---
 .../sort/sortdata/SortTempFileChunkHolder.java  | 174 ++-
 .../sort/sortdata/TableFieldStat.java   | 176 +++
 28 files changed, 1186 insertions(+), 1294 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/8fcde02e/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java 
b/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
index d6ecfbc..fca1244 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
@@ -82,18 +82,26 @@ public class NonDictionaryUtil {
   }
 
   /**
-   * Method to get the required Dimension from obj []
+   * Method to get the required dictionary Dimension from obj []
*
* @param index
* @param row
* @return
*/
-  public static Integer getDimension(int index, Object[] row) {
-
-Integer[] dimensions = (Integer[]) 
row[WriteStepRowUtil.DICTIONARY_DIMENSION];
-
+  public static int getDictDimension(int index, Object[] row) {
+int[] dimensions = (int[]) row[WriteStepRowUtil.DICTIONARY_DIMENSION];
 return dimensions[index];
+  }
 
+  /**
+   * Method to get the required non-dictionary & complex from 3-parted row
+   * @param index
+   * @param row
+   * @return
+   */
+  public static byte[] getNoDictOrComplex(int index, Object[] row) {
+byte[][] nonDictArray = (byte[][]) 
row[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX];
+return nonDictArray[index];
   }
 
   /**
@@ -108,60 +116,11 @@ public class NonDictionaryUtil {
 return measures[index];
   }
 
-  public static byte[] getByteArrayForNoDictionaryCols(Object[] row) {
-
-return (byte[]) row[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX];
-  }
-
   public static void prepareOutObj(Object[] out, int[] dimArray, byte[][] 
byteBufferArr,
   Object[] measureArray) {
-
 out[WriteStepRowUtil.DICTIONARY_DIMENSION] = dimArray;
 out[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX] = byteBufferArr;
 out[WriteStepRowUtil.MEASURE] = measureArray;
 
   }
-
-  /**
-   * This method will extract the single dimension from the complete high card 
dims byte[].+ *
-   * The format of the byte [] will be,  Totallength,CompleteStartOffsets,Dat
-   *
-   * @param highCardArr
-   * @param index
-   * @param highCardinalityCount
-   * @param outBuffer
-   */
-  public static v

[03/50] [abbrv] carbondata git commit: [CARBONDATA-1827] S3 Carbon Implementation

2018-03-04 Thread jackylk
[CARBONDATA-1827] S3 Carbon Implementation

1.Provide support for s3 in carbondata.
2.Added S3Example to create carbon table on s3.
3.Added S3CSVExample to load carbon table using csv from s3.

This closes #1805


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

Branch: refs/heads/carbonstore-rebase5
Commit: 4628d2ec7608eabea5970c2106e6b4e785b5096b
Parents: 39d6748
Author: SangeetaGulia 
Authored: Thu Sep 21 14:56:26 2017 +0530
Committer: Jacky Li 
Committed: Sun Mar 4 18:15:05 2018 +0800

--
 .../core/constants/CarbonCommonConstants.java   |  21 +++
 .../filesystem/AbstractDFSCarbonFile.java   |  20 ++-
 .../datastore/filesystem/HDFSCarbonFile.java|   5 +-
 .../core/datastore/impl/FileFactory.java|  11 +-
 .../core/locks/CarbonLockFactory.java   |  28 ++--
 .../carbondata/core/locks/S3FileLock.java   | 111 +
 .../carbondata/core/util/CarbonProperties.java  |   3 +-
 .../filesystem/HDFSCarbonFileTest.java  |   8 +-
 examples/spark2/pom.xml |   5 +
 examples/spark2/src/main/resources/data1.csv|  11 ++
 .../carbondata/examples/S3CsvExample.scala  |  99 +++
 .../apache/carbondata/examples/S3Example.scala  | 164 +++
 .../spark/rdd/NewCarbonDataLoadRDD.scala|  42 -
 integration/spark2/pom.xml  |  43 +
 .../spark/rdd/CarbonDataRDDFactory.scala|   3 +-
 .../org/apache/spark/sql/CarbonSession.scala|   3 +
 16 files changed, 554 insertions(+), 23 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/4628d2ec/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
 
b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index b2a3375..af3ed99 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
@@ -167,6 +167,22 @@ public final class CarbonCommonConstants {
   public static final String S3N_PREFIX = "s3n://";
 
   public static final String S3A_PREFIX = "s3a://";
+  /**
+   * Access Key for s3n
+   */
+  public static final String S3N_ACCESS_KEY = "fs.s3n.awsAccessKeyId";
+  /**
+   * Secret Key for s3n
+   */
+  public static final String S3N_SECRET_KEY = "fs.s3n.awsSecretAccessKey";
+  /**
+   * Access Key for s3
+   */
+  public static final String S3_ACCESS_KEY = "fs.s3.awsAccessKeyId";
+  /**
+   * Secret Key for s3
+   */
+  public static final String S3_SECRET_KEY = "fs.s3.awsSecretAccessKey";
 
   /**
* FS_DEFAULT_FS
@@ -937,6 +953,11 @@ public final class CarbonCommonConstants {
   public static final String CARBON_LOCK_TYPE_HDFS = "HDFSLOCK";
 
   /**
+   * S3LOCK TYPE
+   */
+  public static final String CARBON_LOCK_TYPE_S3 = "S3LOCK";
+
+  /**
* Invalid filter member log string
*/
   public static final String FILTER_INVALID_MEMBER =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4628d2ec/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/AbstractDFSCarbonFile.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/AbstractDFSCarbonFile.java
 
b/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/AbstractDFSCarbonFile.java
index 68eaa21..fd5dc40 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/AbstractDFSCarbonFile.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/AbstractDFSCarbonFile.java
@@ -51,7 +51,7 @@ import org.apache.hadoop.io.compress.GzipCodec;
 import org.apache.hadoop.io.compress.Lz4Codec;
 import org.apache.hadoop.io.compress.SnappyCodec;
 
-public abstract  class AbstractDFSCarbonFile implements CarbonFile {
+public abstract class AbstractDFSCarbonFile implements CarbonFile {
   /**
* LOGGER
*/
@@ -262,18 +262,28 @@ public abstract  class AbstractDFSCarbonFile implements 
CarbonFile {
   @Override public DataOutputStream getDataOutputStream(String path, 
FileFactory.FileType fileType,
   int bufferSize, boolean append) throws IOException {
 Path pt = new Path(path);
-FileSystem fs = pt.getFileSystem(FileFactory.getConfiguration());
+FileSystem fileSystem = pt.getFileSystem(FileFactory.getConfiguration());
 FSDataOutputStream stream = null;
 if (append) {
   // appen

[49/50] [abbrv] carbondata git commit: [CARBONDATA-1114][Tests] Fix bugs in tests in windows env

2018-03-04 Thread jackylk
[CARBONDATA-1114][Tests] Fix bugs in tests in windows env

Fix bugs in tests that will cause failure under windows env

This closes #1994


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

Branch: refs/heads/carbonstore-rebase5
Commit: 87bda9601ed23871454640432025858799bff1f6
Parents: 49d06c2
Author: xuchuanyin 
Authored: Sat Feb 24 21:18:17 2018 +0800
Committer: Jacky Li 
Committed: Sun Mar 4 20:08:31 2018 +0800

--
 .../carbondata/core/locks/LocalFileLock.java| 30 +++-
 .../store/impl/DFSFileReaderImplUnitTest.java   | 11 ---
 .../store/impl/FileFactoryImplUnitTest.java | 28 +-
 .../filesystem/HDFSCarbonFileTest.java  |  3 +-
 .../filesystem/LocalCarbonFileTest.java | 20 +
 .../BooleanDataTypesInsertTest.scala|  5 ++--
 .../carbondata/lcm/locks/LocalFileLockTest.java |  2 +-
 .../loading/csvinput/CSVInputFormatTest.java|  1 +
 8 files changed, 64 insertions(+), 36 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/87bda960/core/src/main/java/org/apache/carbondata/core/locks/LocalFileLock.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/locks/LocalFileLock.java 
b/core/src/main/java/org/apache/carbondata/core/locks/LocalFileLock.java
index 75ea074..cb80877 100644
--- a/core/src/main/java/org/apache/carbondata/core/locks/LocalFileLock.java
+++ b/core/src/main/java/org/apache/carbondata/core/locks/LocalFileLock.java
@@ -17,17 +17,20 @@
 
 package org.apache.carbondata.core.locks;
 
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.nio.channels.FileChannel;
 import java.nio.channels.FileLock;
 import java.nio.channels.OverlappingFileLockException;
+import java.nio.file.Paths;
+import java.nio.file.StandardOpenOption;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.util.CarbonUtil;
 
 /**
  * This class handles the file locking in the local file system.
@@ -40,11 +43,6 @@ public class LocalFileLock extends AbstractCarbonLock {
   private String location;
 
   /**
-   * fileOutputStream of the local lock file
-   */
-  private FileOutputStream fileOutputStream;
-
-  /**
* channel is the FileChannel of the lock file.
*/
   private FileChannel channel;
@@ -104,8 +102,8 @@ public class LocalFileLock extends AbstractCarbonLock {
 FileFactory.createNewLockFile(lockFilePath, 
FileFactory.getFileType(location));
   }
 
-  fileOutputStream = new FileOutputStream(lockFilePath);
-  channel = fileOutputStream.getChannel();
+  channel = FileChannel.open(Paths.get(lockFilePath), 
StandardOpenOption.WRITE,
+  StandardOpenOption.APPEND);
   try {
 fileLock = channel.tryLock();
   } catch (OverlappingFileLockException e) {
@@ -137,11 +135,17 @@ public class LocalFileLock extends AbstractCarbonLock {
 } catch (IOException e) {
   status = false;
 } finally {
-  if (null != fileOutputStream) {
-try {
-  fileOutputStream.close();
-} catch (IOException e) {
-  LOGGER.error(e.getMessage());
+  CarbonUtil.closeStreams(channel);
+
+  // deleting the lock file after releasing the lock.
+  if (null != lockFilePath) {
+CarbonFile lockFile = FileFactory.getCarbonFile(lockFilePath,
+FileFactory.getFileType(lockFilePath));
+if (!lockFile.exists() || lockFile.delete()) {
+  LOGGER.info("Successfully deleted the lock file " + lockFilePath);
+} else {
+  LOGGER.error("Not able to delete the lock file " + lockFilePath);
+  status = false;
 }
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87bda960/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileReaderImplUnitTest.java
--
diff --git 
a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileReaderImplUnitTest.java
 
b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileReaderImplUnitTest.java
index da61a94..30144c1 100644
--- 
a/core/src/test/java/org

[06/50] [abbrv] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/55c4e438/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
--
diff --git 
a/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java 
b/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
index f4450e3..5f8d199 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
@@ -31,7 +31,7 @@ import java.util.Map;
 
 import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
-import 
org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionDataChunk;
+import 
org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionColumnPage;
 import org.apache.carbondata.core.datastore.columnar.ColumnGroupModel;
 import org.apache.carbondata.core.datastore.filesystem.LocalCarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
@@ -45,7 +45,7 @@ import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.scan.model.QueryDimension;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
 
 import mockit.Mock;
 import mockit.MockUp;
@@ -267,8 +267,8 @@ public class CarbonUtilTest {
   @Test public void testToGetNextLesserValue() {
 byte[] dataChunks = { 5, 6, 7, 8, 9 };
 byte[] compareValues = { 7 };
-FixedLengthDimensionDataChunk fixedLengthDataChunk =
-new FixedLengthDimensionDataChunk(dataChunks, null, null, 5, 1);
+FixedLengthDimensionColumnPage fixedLengthDataChunk =
+new FixedLengthDimensionColumnPage(dataChunks, null, null, 5, 1);
 int result = CarbonUtil.nextLesserValueToTarget(2, fixedLengthDataChunk, 
compareValues);
 assertEquals(result, 1);
   }
@@ -276,8 +276,8 @@ public class CarbonUtilTest {
   @Test public void testToGetNextLesserValueToTarget() {
 byte[] dataChunks = { 7, 7, 7, 8, 9 };
 byte[] compareValues = { 7 };
-FixedLengthDimensionDataChunk fixedLengthDataChunk =
-new FixedLengthDimensionDataChunk(dataChunks, null, null, 5, 1);
+FixedLengthDimensionColumnPage fixedLengthDataChunk =
+new FixedLengthDimensionColumnPage(dataChunks, null, null, 5, 1);
 int result = CarbonUtil.nextLesserValueToTarget(2, fixedLengthDataChunk, 
compareValues);
 assertEquals(result, -1);
   }
@@ -285,8 +285,8 @@ public class CarbonUtilTest {
   @Test public void testToGetnextGreaterValue() {
 byte[] dataChunks = { 5, 6, 7, 8, 9 };
 byte[] compareValues = { 7 };
-FixedLengthDimensionDataChunk fixedLengthDataChunk =
-new FixedLengthDimensionDataChunk(dataChunks, null, null, 5, 1);
+FixedLengthDimensionColumnPage fixedLengthDataChunk =
+new FixedLengthDimensionColumnPage(dataChunks, null, null, 5, 1);
 int result = CarbonUtil.nextGreaterValueToTarget(2, fixedLengthDataChunk, 
compareValues, 5);
 assertEquals(result, 3);
   }
@@ -302,8 +302,8 @@ public class CarbonUtilTest {
   @Test public void testToGetnextGreaterValueToTarget() {
 byte[] dataChunks = { 5, 6, 7, 7, 7 };
 byte[] compareValues = { 7 };
-FixedLengthDimensionDataChunk fixedLengthDataChunk =
-new FixedLengthDimensionDataChunk(dataChunks, null, null, 5, 1);
+FixedLengthDimensionColumnPage fixedLengthDataChunk =
+new FixedLengthDimensionColumnPage(dataChunks, null, null, 5, 1);
 int result = CarbonUtil.nextGreaterValueToTarget(2, fixedLengthDataChunk, 
compareValues, 5);
 assertEquals(result, 5);
   }
@@ -525,23 +525,23 @@ public class CarbonUtilTest {
   }
 
   @Test public void testToGetDictionaryEncodingArray() {
-QueryDimension column1 = new QueryDimension("Column1");
-QueryDimension column2 = new QueryDimension("Column2");
 ColumnSchema column1Schema = new ColumnSchema();
 ColumnSchema column2Schema = new ColumnSchema();
 column1Schema.setColumnName("Column1");
 List encoding = new ArrayList<>();
 encoding.add(Encoding.DICTIONARY);
 column1Schema.setEncodingList(encoding);
-column1.setDimension(new CarbonDimension(column1Schema, 1, 1, 1, 1));
+ProjectionDimension
+column1 = new ProjectionDimension(new CarbonDimension(column1Schema, 
1, 1, 1, 1));
 
 column2Schema.setColumnName("Column2");
 List encoding2 = new ArrayList<>();
 encoding2.add(Encoding.DELTA);
 column2Schema.setEncodingList(encoding2);
-column2.setDimension(new CarbonDimension(column2Schema, 1, 1, 1, 1));
+ProjectionDimension
+column2 = new ProjectionDimension(new CarbonDimension(column2Schema, 
1, 1, 1, 1));
 
-Que

[50/50] [abbrv] carbondata git commit: [REBASE] Solve conflict after merging master

2018-03-04 Thread jackylk
[REBASE] Solve conflict after merging master


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

Branch: refs/heads/carbonstore-rebase5
Commit: 7f92fde4953a67470661e8ece7cb65bb0c9624bc
Parents: 87bda96
Author: Jacky Li 
Authored: Tue Feb 27 11:26:30 2018 +0800
Committer: Jacky Li 
Committed: Sun Mar 4 20:23:37 2018 +0800

--
 .../carbondata/core/datamap/dev/DataMap.java|   6 -
 .../core/datamap/dev/DataMapFactory.java|   2 +-
 .../exception/ConcurrentOperationException.java |  16 +-
 .../core/metadata/PartitionMapFileStore.java|   0
 .../statusmanager/SegmentStatusManager.java |  10 +-
 .../SegmentUpdateStatusManager.java |   1 -
 datamap/examples/pom.xml| 145 +++--
 .../datamap/examples/MinMaxDataWriter.java  |   1 -
 examples/flink/pom.xml  |   4 +-
 .../carbondata/examples/FlinkExample.scala  |  10 +-
 .../CarbonStreamSparkStreamingExample.scala |   1 -
 .../hadoop/api/CarbonTableInputFormat.java  |   5 +-
 .../TestInsertAndOtherCommandConcurrent.scala   |   2 +-
 .../StandardPartitionGlobalSortTestCase.scala   |   2 +-
 .../exception/ProcessMetaDataException.java |   2 +
 .../org/apache/carbondata/api/CarbonStore.scala |   6 +-
 .../carbondata/spark/load/CsvRDDHelper.scala| 157 +++
 .../load/DataLoadProcessBuilderOnSpark.scala|   3 +-
 .../carbondata/spark/util/CarbonScalaUtil.scala |   2 +-
 .../carbondata/spark/util/CommonUtil.scala  |   2 -
 .../command/carbonTableSchemaCommon.scala   |   6 +-
 .../CarbonAlterTableCompactionCommand.scala |   3 +-
 .../management/CarbonCleanFilesCommand.scala|   2 +-
 .../CarbonDeleteLoadByIdCommand.scala   |   2 +-
 .../CarbonDeleteLoadByLoadDateCommand.scala |   2 +-
 .../management/CarbonLoadDataCommand.scala  |  28 ++--
 .../CarbonProjectForDeleteCommand.scala |   2 +-
 .../CarbonProjectForUpdateCommand.scala |   2 +-
 .../schema/CarbonAlterTableRenameCommand.scala  |   2 +-
 .../command/table/CarbonDropTableCommand.scala  |   2 +-
 .../datasources/CarbonFileFormat.scala  |   3 -
 .../vectorreader/AddColumnTestCases.scala   |   1 +
 .../datamap/DataMapWriterListener.java  |   3 +-
 .../loading/model/CarbonLoadModelBuilder.java   |  34 +++-
 .../processing/loading/model/LoadOption.java|  15 +-
 .../processing/merger/CarbonDataMergerUtil.java |   3 +-
 .../util/CarbonDataProcessorUtil.java   |   3 +-
 .../processing/util/CarbonLoaderUtil.java   |   8 +
 store/sdk/pom.xml   |   2 +-
 .../carbondata/sdk/file/CSVCarbonWriter.java|   8 +-
 40 files changed, 336 insertions(+), 172 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f92fde4/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java 
b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
index 02db8af..dd5507c 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
@@ -38,9 +38,6 @@ public interface DataMap {
   /**
* Prune the datamap with filter expression and partition information. It 
returns the list of
* blocklets where these filters can exist.
-   *
-   * @param filterExp
-   * @return
*/
   List prune(FilterResolverIntf filterExp, SegmentProperties 
segmentProperties,
   List partitions);
@@ -48,9 +45,6 @@ public interface DataMap {
   // TODO Move this method to Abstract class
   /**
* Validate whether the current segment needs to be fetching the required 
data
-   *
-   * @param filterExp
-   * @return
*/
   boolean isScanRequired(FilterResolverIntf filterExp);
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f92fde4/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java 
b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
index 50ac279..d8a467f 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
@@ -21,8 +21,8 @@ import java.util.List;
 
 import org.apache.carbondata.core.datamap.DataMapDistributable;
 import org.apa

[21/50] [abbrv] carbondata git commit: [CARBONDATA-2025] Unify all path construction through CarbonTablePath static method

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/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 31a08fc..9afb890 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
@@ -26,7 +26,7 @@ import 
org.apache.carbondata.core.datastore.TableSegmentUniqueIdentifier
 import org.apache.carbondata.core.metadata.CarbonMetadata
 import org.apache.carbondata.core.statusmanager.{SegmentStatus, 
SegmentStatusManager}
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.hadoop.CacheClient
 import org.apache.spark.sql.test.util.QueryTest
 
@@ -112,11 +112,9 @@ class MajorCompactionIgnoreInMinorTest extends QueryTest 
with BeforeAndAfterAll
   CarbonCommonConstants.DATABASE_DEFAULT_NAME,
   "ignoremajor"
 )
-val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
 
-val carbontablePath = 
CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier)
-  .getMetadataDirectoryPath
-val segs = SegmentStatusManager.readLoadMetadata(carbontablePath)
+val carbonTablePath = carbonTable.getMetadataPath
+val segs = SegmentStatusManager.readLoadMetadata(carbonTablePath)
 
 // status should remain as compacted.
 assertResult(SegmentStatus.COMPACTED)(segs(3).getSegmentStatus)
@@ -134,9 +132,7 @@ class MajorCompactionIgnoreInMinorTest extends QueryTest 
with BeforeAndAfterAll
   CarbonCommonConstants.DATABASE_DEFAULT_NAME,
   "ignoremajor"
 )
-val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
-val carbontablePath = CarbonStorePath
-  .getCarbonTablePath(absoluteTableIdentifier).getMetadataDirectoryPath
+val carbontablePath = carbonTable.getMetadataPath
 val segs = SegmentStatusManager.readLoadMetadata(carbontablePath)
 
 // status should remain as compacted for segment 2.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestBatchSortDataLoad.scala
--
diff --git 
a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestBatchSortDataLoad.scala
 
b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestBatchSortDataLoad.scala
index 42ac4df..68a3058 100644
--- 
a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestBatchSortDataLoad.scala
+++ 
b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestBatchSortDataLoad.scala
@@ -29,7 +29,7 @@ import org.apache.spark.sql.test.util.QueryTest
 import 
org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore
 import org.apache.carbondata.core.metadata.CarbonMetadata
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 
 class TestBatchSortDataLoad extends QueryTest with BeforeAndAfterAll {
   var filePath: String = _
@@ -193,8 +193,7 @@ class TestBatchSortDataLoad extends QueryTest with 
BeforeAndAfterAll {
   CarbonCommonConstants.DATABASE_DEFAULT_NAME,
   tableName
 )
-val carbonTablePath = 
CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-val segmentDir = carbonTablePath.getCarbonDataDirectoryPath(segmentNo)
+val segmentDir = carbonTable.getSemgentPath(segmentNo)
 new SegmentIndexFileStore().getIndexFilesFromSegment(segmentDir).size()
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithFileName.scala
--
diff --git 
a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithFileName.scala
 
b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithFileName.scala
index db0a62c..b9d8e12 100644
--- 
a

[04/50] [abbrv] carbondata git commit: [CARBONDATA-1968] Add external table support

2018-03-04 Thread jackylk
[CARBONDATA-1968] Add external table support

This PR adds support for creating external table with existing carbondata 
files, using Hive syntax.
CREATE EXTERNAL TABLE tableName STORED BY 'carbondata' LOCATION 'path'

This closes #1749


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

Branch: refs/heads/carbonstore-rebase5
Commit: 39d6748dbc91200cb2e37afd7bb82af2bdea794f
Parents: c770ff6
Author: Jacky Li 
Authored: Tue Jan 2 23:46:14 2018 +0800
Committer: Jacky Li 
Committed: Sun Mar 4 18:15:05 2018 +0800

--
 .../core/metadata/schema/table/CarbonTable.java |  9 ++
 .../createTable/TestCreateExternalTable.scala   | 91 
 .../TestDataWithDicExcludeAndInclude.scala  | 10 ---
 .../command/table/CarbonDropTableCommand.scala  |  5 +-
 .../spark/sql/parser/CarbonSparkSqlParser.scala | 64 +-
 5 files changed, 147 insertions(+), 32 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/39d6748d/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
 
b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
index 09ff440..6036569 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
@@ -785,6 +785,15 @@ public class CarbonTable implements Serializable {
 && !tableInfo.getParentRelationIdentifiers().isEmpty();
   }
 
+  /**
+   * Return true if this is an external table (table with property 
"_external"="true", this is
+   * an internal table property set during table creation)
+   */
+  public boolean isExternalTable() {
+String external = 
tableInfo.getFactTable().getTableProperties().get("_external");
+return external != null && external.equalsIgnoreCase("true");
+  }
+
   public long size() throws IOException {
 Map dataIndexSize = CarbonUtil.calculateDataIndexSize(this);
 Long dataSize = 
dataIndexSize.get(CarbonCommonConstants.CARBON_TOTAL_DATA_SIZE);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/39d6748d/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateExternalTable.scala
--
diff --git 
a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateExternalTable.scala
 
b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateExternalTable.scala
new file mode 100644
index 000..67370eb
--- /dev/null
+++ 
b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateExternalTable.scala
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.spark.testsuite.createTable
+
+import java.io.File
+
+import org.apache.spark.sql.{AnalysisException, CarbonEnv}
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+class TestCreateExternalTable extends QueryTest with BeforeAndAfterAll {
+
+  var originDataPath: String = _
+
+  override def beforeAll(): Unit = {
+sql("DROP TABLE IF EXISTS origin")
+// create carbon table and insert data
+sql("CREATE TABLE origin(key INT, value STRING) STORED BY 'carbondata'")
+sql("INSERT INTO origin select 100,'spark'")
+sql("INSERT INTO origin select 200,'hive'")
+originDataPath = s"$storeLocation/origin"
+  }
+
+  override def afterAll(): Unit = {
+sql("DROP TABLE IF EXISTS origin")
+  }
+
+  test("create external table with existing files") {
+assert(new

[12/50] [abbrv] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/55c4e438/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeColGroupFilterExecuterImpl.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeColGroupFilterExecuterImpl.java
 
b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeColGroupFilterExecuterImpl.java
index 9391ebd..44f7c07 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeColGroupFilterExecuterImpl.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeColGroupFilterExecuterImpl.java
@@ -16,20 +16,10 @@
  */
 package org.apache.carbondata.core.scan.filter.executer;
 
-import java.util.ArrayList;
 import java.util.BitSet;
-import java.util.List;
 
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
-import org.apache.carbondata.core.keygenerator.KeyGenException;
-import org.apache.carbondata.core.keygenerator.KeyGenerator;
-import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
-import org.apache.carbondata.core.scan.executor.util.QueryUtil;
 import 
org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.apache.carbondata.core.util.ByteUtil;
 
 /**
  * It checks if filter is required on given block and if required, it does
@@ -38,12 +28,6 @@ import org.apache.carbondata.core.util.ByteUtil;
 public class ExcludeColGroupFilterExecuterImpl extends 
ExcludeFilterExecuterImpl {
 
   /**
-   * LOGGER
-   */
-  private static final LogService LOGGER =
-  
LogServiceFactory.getLogService(ExcludeColGroupFilterExecuterImpl.class.getName());
-
-  /**
* @param dimColResolvedFilterInfo
* @param segmentProperties
*/
@@ -53,54 +37,6 @@ public class ExcludeColGroupFilterExecuterImpl extends 
ExcludeFilterExecuterImpl
   }
 
   /**
-   * It fills BitSet with row index which matches filter key
-   */
-  protected BitSet getFilteredIndexes(DimensionColumnDataChunk 
dimensionColumnDataChunk,
-  int numerOfRows) {
-BitSet bitSet = new BitSet(numerOfRows);
-bitSet.flip(0, numerOfRows);
-try {
-  KeyStructureInfo keyStructureInfo = getKeyStructureInfo();
-  byte[][] filterValues = dimColumnExecuterInfo.getExcludeFilterKeys();
-  for (int i = 0; i < filterValues.length; i++) {
-byte[] filterVal = filterValues[i];
-for (int rowId = 0; rowId < numerOfRows; rowId++) {
-  byte[] colData = new 
byte[keyStructureInfo.getMaskByteRanges().length];
-  dimensionColumnDataChunk.fillChunkData(colData, 0, rowId, 
keyStructureInfo);
-  if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterVal, colData) 
== 0) {
-bitSet.flip(rowId);
-  }
-}
-  }
-
-} catch (Exception e) {
-  LOGGER.error(e);
-}
-
-return bitSet;
-  }
-
-  /**
-   * It is required for extracting column data from columngroup chunk
-   *
-   * @return
-   * @throws KeyGenException
-   */
-  private KeyStructureInfo getKeyStructureInfo() throws KeyGenException {
-int colGrpId = getColumnGroupId(dimColEvaluatorInfo.getColumnIndex());
-KeyGenerator keyGenerator = 
segmentProperties.getColumnGroupAndItsKeygenartor().get(colGrpId);
-List mdKeyOrdinal = new ArrayList();
-mdKeyOrdinal.add(getMdkeyOrdinal(dimColEvaluatorInfo.getColumnIndex(), 
colGrpId));
-int[] maskByteRanges = 
QueryUtil.getMaskedByteRangeBasedOrdinal(mdKeyOrdinal, keyGenerator);
-byte[] maxKey = QueryUtil.getMaxKeyBasedOnOrinal(mdKeyOrdinal, 
keyGenerator);
-KeyStructureInfo restructureInfos = new KeyStructureInfo();
-restructureInfos.setKeyGenerator(keyGenerator);
-restructureInfos.setMaskByteRanges(maskByteRanges);
-restructureInfos.setMaxKey(maxKey);
-return restructureInfos;
-  }
-
-  /**
* Check if scan is required on given block based on min and max value
*/
   public BitSet isScanRequired(byte[][] blkMaxVal, byte[][] blkMinVal) {
@@ -109,25 +45,4 @@ public class ExcludeColGroupFilterExecuterImpl extends 
ExcludeFilterExecuterImpl
 return bitSet;
   }
 
-  private int getMdkeyOrdinal(int ordinal, int colGrpId) {
-return segmentProperties.getColumnGroupMdKeyOrdinal(colGrpId, ordinal);
-  }
-
-  private int getColumnGroupId(int ordinal) {
-int[][] columnGroups = segmentProperties.getColumnGroups();
-int colGrpId = -1;
-for (int i = 0; i < columnGroups.length; i++) {
-  if (columnGroups[i].length > 1) {
-colGrpId++;
-if (QueryUtil.searchInArray(columnGroups[i], ordinal)) {
-  break;
-}
-  }
-}
-return colGrpId;
-  }
-
-  public KeyGenerator getKeyGenerator(int co

[24/50] [abbrv] carbondata git commit: [CARBONDATA-1480]Min Max Index Example for DataMap

2018-03-04 Thread jackylk
[CARBONDATA-1480]Min Max Index Example for DataMap

Datamap Example. Implementation of Min Max Index through Datamap. And Using the 
Index while prunning.

This closes #1359


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

Branch: refs/heads/carbonstore-rebase5
Commit: 3a0a9e64a75bb4f1f4644775708fc55c256a0d8d
Parents: f8841d8
Author: sounakr 
Authored: Thu Sep 28 16:21:05 2017 +0530
Committer: Jacky Li 
Committed: Sun Mar 4 19:31:39 2018 +0800

--
 .../core/datamap/DataMapStoreManager.java   |  16 +-
 .../carbondata/core/datamap/TableDataMap.java   |  18 +-
 .../carbondata/core/datamap/dev/DataMap.java|  11 +-
 .../core/datamap/dev/DataMapWriter.java |   3 +-
 .../indexstore/SegmentPropertiesFetcher.java|  36 +++
 .../blockletindex/BlockletDataMap.java  |   5 +-
 .../blockletindex/BlockletDataMapFactory.java   |  32 ++-
 datamap/examples/pom.xml| 111 ++
 .../datamap/examples/BlockletMinMax.java|  41 
 .../datamap/examples/MinMaxDataMap.java | 143 
 .../datamap/examples/MinMaxDataMapFactory.java  | 114 ++
 .../datamap/examples/MinMaxDataWriter.java  | 221 +++
 .../examples/MinMaxIndexBlockDetails.java   |  77 +++
 .../MinMaxDataMapExample.scala  |  77 +++
 .../testsuite/datamap/DataMapWriterSuite.scala  |   2 +-
 pom.xml |   2 +
 .../datamap/DataMapWriterListener.java  |   4 +-
 .../store/writer/AbstractFactDataWriter.java|   7 +-
 .../writer/v3/CarbonFactDataWriterImplV3.java   |   3 +
 19 files changed, 894 insertions(+), 29 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/3a0a9e64/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
 
b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
index d30483a..90e5fff 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
@@ -26,6 +26,7 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datamap.dev.DataMapFactory;
 import org.apache.carbondata.core.indexstore.BlockletDetailsFetcher;
+import org.apache.carbondata.core.indexstore.SegmentPropertiesFetcher;
 import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMap;
 import 
org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapFactory;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
@@ -103,7 +104,7 @@ public final class DataMapStoreManager {
   tableDataMaps = new ArrayList<>();
 }
 TableDataMap dataMap = getTableDataMap(dataMapName, tableDataMaps);
-if (dataMap != null) {
+if (dataMap != null && 
dataMap.getDataMapName().equalsIgnoreCase(dataMapName)) {
   throw new RuntimeException("Already datamap exists in that path with 
type " + dataMapName);
 }
 
@@ -113,12 +114,15 @@ public final class DataMapStoreManager {
   DataMapFactory dataMapFactory = factoryClass.newInstance();
   dataMapFactory.init(identifier, dataMapName);
   BlockletDetailsFetcher blockletDetailsFetcher;
+  SegmentPropertiesFetcher segmentPropertiesFetcher = null;
   if (dataMapFactory instanceof BlockletDetailsFetcher) {
 blockletDetailsFetcher = (BlockletDetailsFetcher) dataMapFactory;
   } else {
 blockletDetailsFetcher = getBlockletDetailsFetcher(identifier);
   }
-  dataMap = new TableDataMap(identifier, dataMapName, dataMapFactory, 
blockletDetailsFetcher);
+  segmentPropertiesFetcher = (SegmentPropertiesFetcher) 
blockletDetailsFetcher;
+  dataMap = new TableDataMap(identifier, dataMapName, dataMapFactory, 
blockletDetailsFetcher,
+  segmentPropertiesFetcher);
 } catch (Exception e) {
   LOGGER.error(e);
   throw new RuntimeException(e);
@@ -128,11 +132,11 @@ public final class DataMapStoreManager {
 return dataMap;
   }
 
-  private TableDataMap getTableDataMap(String dataMapName,
-  List tableDataMaps) {
+  private TableDataMap getTableDataMap(String dataMapName, List 
tableDataMaps) {
 TableDataMap dataMap = null;
-for (TableDataMap tableDataMap: tableDataMaps) {
-  if (tableDataMap.getDataMapName().equals(dataMapName)) {
+for (TableDataMa

[45/50] [abbrv] carbondata git commit: [CARBONDATA-2091][DataLoad] Support specifying sort column bounds in data loading

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/49d06c20/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeParallelReadMergeSorterWithBucketingImpl.java
--
diff --git 
a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeParallelReadMergeSorterWithBucketingImpl.java
 
b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeParallelReadMergeSorterWithBucketingImpl.java
deleted file mode 100644
index f605b22..000
--- 
a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeParallelReadMergeSorterWithBucketingImpl.java
+++ /dev/null
@@ -1,263 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.loading.sort.impl;
-
-import java.io.File;
-import java.util.Iterator;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.carbondata.common.CarbonIterator;
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.row.CarbonRow;
-import org.apache.carbondata.core.memory.MemoryException;
-import org.apache.carbondata.core.metadata.schema.BucketingInfo;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
-import org.apache.carbondata.processing.loading.DataField;
-import 
org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
-import org.apache.carbondata.processing.loading.row.CarbonRowBatch;
-import org.apache.carbondata.processing.loading.sort.AbstractMergeSorter;
-import 
org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
-import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeSortDataRows;
-import 
org.apache.carbondata.processing.loading.sort.unsafe.merger.UnsafeIntermediateMerger;
-import 
org.apache.carbondata.processing.loading.sort.unsafe.merger.UnsafeSingleThreadFinalSortFilesMerger;
-import org.apache.carbondata.processing.sort.sortdata.SortParameters;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-/**
- * It parallely reads data from array of iterates and do merge sort.
- * First it sorts the data and write to temp files. These temp files will be 
merge sorted to get
- * final merge sort result.
- * This step is specifically for bucketing, it sorts each bucket data 
separately and write to
- * temp files.
- */
-public class UnsafeParallelReadMergeSorterWithBucketingImpl extends 
AbstractMergeSorter {
-
-  private static final LogService LOGGER =
-  LogServiceFactory.getLogService(
-
UnsafeParallelReadMergeSorterWithBucketingImpl.class.getName());
-
-  private SortParameters sortParameters;
-
-  private BucketingInfo bucketingInfo;
-
-  public UnsafeParallelReadMergeSorterWithBucketingImpl(DataField[] 
inputDataFields,
-  BucketingInfo bucketingInfo) {
-this.bucketingInfo = bucketingInfo;
-  }
-
-  @Override public void initialize(SortParameters sortParameters) {
-this.sortParameters = sortParameters;
-  }
-
-  @Override public Iterator[] sort(Iterator[] 
iterators)
-  throws CarbonDataLoadingException {
-UnsafeSortDataRows[] sortDataRows = new 
UnsafeSortDataRows[bucketingInfo.getNumberOfBuckets()];
-UnsafeIntermediateMerger[] intermediateFileMergers =
-new UnsafeIntermediateMerger[sortDataRows.length];
-int inMemoryChunkSizeInMB = 
CarbonProperties.getInstance().getSortMemoryChunkSizeInMB();
-inMemoryChunkSizeInMB = inMemoryChunkSizeInMB / 
bucketingInfo.getNumberOfBuckets();
-if (inMemoryChunkSizeInMB < 5) {
-  inMemoryChunkSizeInMB = 5;
-}
-try {
-  for (int i = 0; i < bucketingInfo.getNumberOfBuckets(); i++) {
-SortParameters parameters = sortParameters.getCopy();
-parameters.setPartitionID(i + "");
-setTempLocation(parameters);
-intermediateFileMergers[

[26/50] [abbrv] carbondata git commit: [CARBONDATA-1544][Datamap] Datamap FineGrain implementation

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/828ae5ec/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMapFactory.java
--
diff --git 
a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMapFactory.java
 
b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMapFactory.java
index b196d0d..266c107 100644
--- 
a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMapFactory.java
+++ 
b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMapFactory.java
@@ -17,7 +17,6 @@
 
 package org.apache.carbondata.datamap.examples;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -25,90 +24,94 @@ import java.util.List;
 
 import org.apache.carbondata.core.datamap.DataMapDistributable;
 import org.apache.carbondata.core.datamap.DataMapMeta;
-import org.apache.carbondata.core.datamap.dev.DataMap;
-import org.apache.carbondata.core.datamap.dev.DataMapFactory;
-import org.apache.carbondata.core.datamap.dev.DataMapWriter;
-import org.apache.carbondata.core.events.ChangeEvent;
-import org.apache.carbondata.core.indexstore.schema.FilterType;
+import org.apache.carbondata.core.datamap.Segment;
+import org.apache.carbondata.core.datamap.dev.AbstractDataMapWriter;
+import org.apache.carbondata.core.datamap.dev.DataMapModel;
+import 
org.apache.carbondata.core.datamap.dev.cgdatamap.AbstractCoarseGrainDataMap;
+import 
org.apache.carbondata.core.datamap.dev.cgdatamap.AbstractCoarseGrainDataMapFactory;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-
+import org.apache.carbondata.core.scan.filter.intf.ExpressionType;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+import org.apache.carbondata.events.Event;
 
 /**
  * Min Max DataMap Factory
  */
-public class MinMaxDataMapFactory implements DataMapFactory {
+public class MinMaxDataMapFactory extends AbstractCoarseGrainDataMapFactory {
 
   private AbsoluteTableIdentifier identifier;
 
-  @Override
-  public void init(AbsoluteTableIdentifier identifier, String dataMapName) {
+  @Override public void init(AbsoluteTableIdentifier identifier, String 
dataMapName) {
 this.identifier = identifier;
   }
 
   /**
* createWriter will return the MinMaxDataWriter.
-   * @param segmentId
+   *
+   * @param segment
* @return
*/
-  @Override
-  public DataMapWriter createWriter(String segmentId) {
-return new MinMaxDataWriter();
+  @Override public AbstractDataMapWriter createWriter(Segment segment) {
+return new MinMaxDataWriter(identifier, segment.getSegmentNo(),
+CarbonTablePath.getSegmentPath(identifier.getTablePath(), 
segment.getSegmentNo()));
   }
 
   /**
* getDataMaps Factory method Initializes the Min Max Data Map and returns.
-   * @param segmentId
+   *
+   * @param segment
* @return
* @throws IOException
*/
-  @Override public List getDataMaps(String segmentId) throws 
IOException {
-List dataMapList = new ArrayList<>();
+  @Override public List getDataMaps(Segment 
segment)
+  throws IOException {
+List dataMapList = new ArrayList<>();
 // Form a dataMap of Type MinMaxDataMap.
 MinMaxDataMap dataMap = new MinMaxDataMap();
 try {
-  dataMap.init(identifier.getTablePath() + "/Fact/Part0/Segment_" + 
segmentId + File.separator);
+  dataMap.init(new DataMapModel(
+  CarbonTablePath.getSegmentPath(identifier.getTablePath(), 
segment.getSegmentNo(;
 } catch (MemoryException ex) {
-
+  throw new IOException(ex);
 }
 dataMapList.add(dataMap);
 return dataMapList;
   }
 
   /**
-   *
-   * @param segmentId
+   * @param segment
* @return
*/
-  @Override public List toDistributable(String 
segmentId) {
+  @Override public List toDistributable(Segment segment) 
{
 return null;
   }
 
   /**
* Clear the DataMap.
-   * @param segmentId
+   *
+   * @param segment
*/
-  @Override public void clear(String segmentId) {
+  @Override public void clear(Segment segment) {
   }
 
   /**
* Clearing the data map.
*/
-  @Override
-  public void clear() {
+  @Override public void clear() {
   }
 
-  @Override public DataMap getDataMap(DataMapDistributable distributable) {
+  @Override public List 
getDataMaps(DataMapDistributable distributable)
+  throws IOException {
 return null;
   }
 
-  @Override
-  public void fireEvent(ChangeEvent event) {
+  @Override public void fireEvent(Event event) {
 
   }
 
-  @Override
-  public DataMapMeta getMeta() {
-return new DataMapMeta(new ArrayList(Arrays.asList("c2")), 
FilterType.EQUALTO);
+  @Override public DataMapMeta getMeta() {
+return new DataMapMeta(new ArrayList(Arr

[32/50] [abbrv] carbondata git commit: [CARBONDATA-2023][DataLoad] Add size base block allocation in data loading

2018-03-04 Thread jackylk
[CARBONDATA-2023][DataLoad] Add size base block allocation in data loading

Carbondata assign blocks to nodes at the beginning of data loading.
Previous block allocation strategy is block number based and it will
suffer skewed data problem if the size of input files differs a lot.

We introduced a size based block allocation strategy to optimize data
loading performance in skewed data scenario.

This closes #1808


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

Branch: refs/heads/carbonstore-rebase5
Commit: 111bb5c41946d72604964658ec8562f7722dec14
Parents: ef81248
Author: xuchuanyin 
Authored: Thu Feb 8 14:42:39 2018 +0800
Committer: Jacky Li 
Committed: Sun Mar 4 19:54:03 2018 +0800

--
 .../constants/CarbonLoadOptionConstants.java|  10 +
 .../core/datastore/block/TableBlockInfo.java|  29 ++
 .../carbondata/core/util/CarbonProperties.java  |  11 +
 docs/useful-tips-on-carbondata.md   |   1 +
 .../CarbonIndexFileMergeTestCase.scala  |   4 -
 .../StandardPartitionTableLoadingTestCase.scala |   2 +-
 .../spark/rdd/NewCarbonDataLoadRDD.scala|   4 +-
 .../spark/sql/hive/DistributionUtil.scala   |   2 +-
 .../spark/rdd/CarbonDataRDDFactory.scala|  18 +-
 .../merger/NodeMultiBlockRelation.java  |  40 ++
 .../processing/util/CarbonLoaderUtil.java   | 494 ---
 .../processing/util/CarbonLoaderUtilTest.java   | 125 +
 12 files changed, 552 insertions(+), 188 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/111bb5c4/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
 
b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
index bcfeba0..a6bf60f 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
@@ -114,4 +114,14 @@ public final class CarbonLoadOptionConstants {
*/
   public static final int MAX_EXTERNAL_DICTIONARY_SIZE = 1000;
 
+  /**
+   * enable block size based block allocation while loading data. By default, 
carbondata assigns
+   * blocks to node based on block number. If this option is set to `true`, 
carbondata will
+   * consider block size first and make sure that all the nodes will process 
almost equal size of
+   * data. This option is especially useful when you encounter skewed data.
+   */
+  @CarbonProperty
+  public static final String ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION
+  = "carbon.load.skewedDataOptimization.enabled";
+  public static final String 
ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION_DEFAULT = "false";
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/111bb5c4/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
 
b/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
index a7bfdba..c0cebe0 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
@@ -19,6 +19,8 @@ package org.apache.carbondata.core.datastore.block;
 import java.io.IOException;
 import java.io.Serializable;
 import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -98,6 +100,20 @@ public class TableBlockInfo implements Distributable, 
Serializable {
 
   private String dataMapWriterPath;
 
+  /**
+   * comparator to sort by block size in descending order.
+   * Since each line is not exactly the same, the size of a InputSplit may 
differs,
+   * so we allow some deviation for these splits.
+   */
+  public static final Comparator DATA_SIZE_DESC_COMPARATOR =
+  new Comparator() {
+@Override public int compare(Distributable o1, Distributable o2) {
+  long diff =
+  ((TableBlockInfo) o1).getBlockLength() - ((TableBlockInfo) 
o2).getBlockLength();
+  return diff < 0 ? 1 : (diff == 0 ? 0 : -1);
+}
+  };
+
   public TableBlockInfo(String filePath, long blockOffset, String segmentId,
   String[] locations, long blockLength, ColumnarFormatVersion version,
   String[] del

[01/50] [abbrv] carbondata git commit: [REBASE] Solve conflict after rebasing master [Forced Update!]

2018-03-04 Thread jackylk
Repository: carbondata
Updated Branches:
  refs/heads/carbonstore-rebase5 7f92fde49 -> 8104735fd (forced update)


[REBASE] Solve conflict after rebasing master


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

Branch: refs/heads/carbonstore-rebase5
Commit: bd40a0d73d2a7086caaa6773a2c6a1a45e24334c
Parents: 9086a1b
Author: Jacky Li 
Authored: Thu Feb 1 00:25:31 2018 +0800
Committer: Jacky Li 
Committed: Sun Mar 4 20:30:31 2018 +0800

--
 .../hadoop/util/CarbonInputFormatUtil.java  | 20 +++
 .../spark/rdd/NewCarbonDataLoadRDD.scala| 21 ++--
 .../org/apache/spark/sql/CarbonSession.scala|  5 ++---
 3 files changed, 24 insertions(+), 22 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/bd40a0d7/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
--
diff --git 
a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
 
b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
index 514428b..056c27b 100644
--- 
a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
+++ 
b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
@@ -22,6 +22,8 @@ import java.text.SimpleDateFormat;
 import java.util.List;
 import java.util.Locale;
 
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
@@ -39,6 +41,7 @@ import org.apache.carbondata.core.scan.model.QueryMeasure;
 import org.apache.carbondata.core.scan.model.QueryModel;
 import org.apache.carbondata.hadoop.api.CarbonTableInputFormat;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobID;
@@ -159,4 +162,21 @@ public class CarbonInputFormatUtil {
 String jobtrackerID = createJobTrackerID(date);
 return new JobID(jobtrackerID, batch);
   }
+
+  public static void setS3Configurations(Configuration hadoopConf) {
+FileFactory.getConfiguration()
+.set("fs.s3a.access.key", hadoopConf.get("fs.s3a.access.key", ""));
+FileFactory.getConfiguration()
+.set("fs.s3a.secret.key", hadoopConf.get("fs.s3a.secret.key", ""));
+FileFactory.getConfiguration()
+.set("fs.s3a.endpoint", hadoopConf.get("fs.s3a.endpoint", ""));
+FileFactory.getConfiguration().set(CarbonCommonConstants.S3_ACCESS_KEY,
+hadoopConf.get(CarbonCommonConstants.S3_ACCESS_KEY, ""));
+FileFactory.getConfiguration().set(CarbonCommonConstants.S3_SECRET_KEY,
+hadoopConf.get(CarbonCommonConstants.S3_SECRET_KEY, ""));
+FileFactory.getConfiguration().set(CarbonCommonConstants.S3N_ACCESS_KEY,
+hadoopConf.get(CarbonCommonConstants.S3N_ACCESS_KEY, ""));
+FileFactory.getConfiguration().set(CarbonCommonConstants.S3N_SECRET_KEY,
+hadoopConf.get(CarbonCommonConstants.S3N_SECRET_KEY, ""));
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bd40a0d7/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 917fc88..e17824f 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
@@ -41,10 +41,10 @@ import 
org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.common.logging.impl.StandardLogService
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.compression.CompressorFactory
-import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, 
SegmentStatus}
 import org.apache.carbondata.core.util.{CarbonProperties, 
CarbonTimeStatisticsFactory, ThreadLocalTaskInfo}
 import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.hado

[07/50] [abbrv] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/92c9f224/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
--
diff --git 
a/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java 
b/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
index f4450e3..5f8d199 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
@@ -31,7 +31,7 @@ import java.util.Map;
 
 import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
-import 
org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionDataChunk;
+import 
org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionColumnPage;
 import org.apache.carbondata.core.datastore.columnar.ColumnGroupModel;
 import org.apache.carbondata.core.datastore.filesystem.LocalCarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
@@ -45,7 +45,7 @@ import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.scan.model.QueryDimension;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
 
 import mockit.Mock;
 import mockit.MockUp;
@@ -267,8 +267,8 @@ public class CarbonUtilTest {
   @Test public void testToGetNextLesserValue() {
 byte[] dataChunks = { 5, 6, 7, 8, 9 };
 byte[] compareValues = { 7 };
-FixedLengthDimensionDataChunk fixedLengthDataChunk =
-new FixedLengthDimensionDataChunk(dataChunks, null, null, 5, 1);
+FixedLengthDimensionColumnPage fixedLengthDataChunk =
+new FixedLengthDimensionColumnPage(dataChunks, null, null, 5, 1);
 int result = CarbonUtil.nextLesserValueToTarget(2, fixedLengthDataChunk, 
compareValues);
 assertEquals(result, 1);
   }
@@ -276,8 +276,8 @@ public class CarbonUtilTest {
   @Test public void testToGetNextLesserValueToTarget() {
 byte[] dataChunks = { 7, 7, 7, 8, 9 };
 byte[] compareValues = { 7 };
-FixedLengthDimensionDataChunk fixedLengthDataChunk =
-new FixedLengthDimensionDataChunk(dataChunks, null, null, 5, 1);
+FixedLengthDimensionColumnPage fixedLengthDataChunk =
+new FixedLengthDimensionColumnPage(dataChunks, null, null, 5, 1);
 int result = CarbonUtil.nextLesserValueToTarget(2, fixedLengthDataChunk, 
compareValues);
 assertEquals(result, -1);
   }
@@ -285,8 +285,8 @@ public class CarbonUtilTest {
   @Test public void testToGetnextGreaterValue() {
 byte[] dataChunks = { 5, 6, 7, 8, 9 };
 byte[] compareValues = { 7 };
-FixedLengthDimensionDataChunk fixedLengthDataChunk =
-new FixedLengthDimensionDataChunk(dataChunks, null, null, 5, 1);
+FixedLengthDimensionColumnPage fixedLengthDataChunk =
+new FixedLengthDimensionColumnPage(dataChunks, null, null, 5, 1);
 int result = CarbonUtil.nextGreaterValueToTarget(2, fixedLengthDataChunk, 
compareValues, 5);
 assertEquals(result, 3);
   }
@@ -302,8 +302,8 @@ public class CarbonUtilTest {
   @Test public void testToGetnextGreaterValueToTarget() {
 byte[] dataChunks = { 5, 6, 7, 7, 7 };
 byte[] compareValues = { 7 };
-FixedLengthDimensionDataChunk fixedLengthDataChunk =
-new FixedLengthDimensionDataChunk(dataChunks, null, null, 5, 1);
+FixedLengthDimensionColumnPage fixedLengthDataChunk =
+new FixedLengthDimensionColumnPage(dataChunks, null, null, 5, 1);
 int result = CarbonUtil.nextGreaterValueToTarget(2, fixedLengthDataChunk, 
compareValues, 5);
 assertEquals(result, 5);
   }
@@ -525,23 +525,23 @@ public class CarbonUtilTest {
   }
 
   @Test public void testToGetDictionaryEncodingArray() {
-QueryDimension column1 = new QueryDimension("Column1");
-QueryDimension column2 = new QueryDimension("Column2");
 ColumnSchema column1Schema = new ColumnSchema();
 ColumnSchema column2Schema = new ColumnSchema();
 column1Schema.setColumnName("Column1");
 List encoding = new ArrayList<>();
 encoding.add(Encoding.DICTIONARY);
 column1Schema.setEncodingList(encoding);
-column1.setDimension(new CarbonDimension(column1Schema, 1, 1, 1, 1));
+ProjectionDimension
+column1 = new ProjectionDimension(new CarbonDimension(column1Schema, 
1, 1, 1, 1));
 
 column2Schema.setColumnName("Column2");
 List encoding2 = new ArrayList<>();
 encoding2.add(Encoding.DELTA);
 column2Schema.setEncodingList(encoding2);
-column2.setDimension(new CarbonDimension(column2Schema, 1, 1, 1, 1));
+ProjectionDimension
+column2 = new ProjectionDimension(new CarbonDimension(column2Schema, 
1, 1, 1, 1));
 
-Que

[16/50] [abbrv] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/92c9f224/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeFixedLengthDimensionDataChunkStore.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeFixedLengthDimensionDataChunkStore.java
 
b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeFixedLengthDimensionDataChunkStore.java
index 8c8d08f..a689d8e 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeFixedLengthDimensionDataChunkStore.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeFixedLengthDimensionDataChunkStore.java
@@ -124,22 +124,22 @@ public class UnsafeFixedLengthDimensionDataChunkStore
   /**
* to compare the two byte array
*
-   * @param indexindex of first byte array
+   * @param rowIdindex of first byte array
* @param compareValue value of to be compared
* @return compare result
*/
-  @Override public int compareTo(int index, byte[] compareValue) {
+  @Override public int compareTo(int rowId, byte[] compareValue) {
 // based on index we need to calculate the actual position in memory block
-index = index * columnValueSize;
+rowId = rowId * columnValueSize;
 int compareResult = 0;
 for (int i = 0; i < compareValue.length; i++) {
   compareResult = (CarbonUnsafe.getUnsafe()
-  .getByte(dataPageMemoryBlock.getBaseObject(), 
dataPageMemoryBlock.getBaseOffset() + index)
+  .getByte(dataPageMemoryBlock.getBaseObject(), 
dataPageMemoryBlock.getBaseOffset() + rowId)
   & 0xff) - (compareValue[i] & 0xff);
   if (compareResult != 0) {
 break;
   }
-  index++;
+  rowId++;
 }
 return compareResult;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/92c9f224/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 36b2bd8..e1eb378 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
@@ -189,11 +189,11 @@ public class UnsafeVariableLengthDimesionDataChunkStore
   /**
* to compare the two byte array
*
-   * @param index index of first byte array
+   * @param rowId index of first byte array
* @param compareValue value of to be compared
* @return compare result
*/
-  @Override public int compareTo(int index, byte[] compareValue) {
+  @Override public int compareTo(int rowId, byte[] compareValue) {
 // 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
@@ -201,13 +201,13 @@ public class UnsafeVariableLengthDimesionDataChunkStore
 // else subtract the current row offset
 // with complete data length get the offset of set of data
 int currentDataOffset = 
CarbonUnsafe.getUnsafe().getInt(dataPageMemoryBlock.getBaseObject(),
-dataPageMemoryBlock.getBaseOffset() + this.dataPointersOffsets + 
((long)index
+dataPageMemoryBlock.getBaseOffset() + this.dataPointersOffsets + 
((long) rowId
 * CarbonCommonConstants.INT_SIZE_IN_BYTE * 1L));
 short length = 0;
 // calculating the length of data
-if (index < numberOfRows - 1) {
+if (rowId < numberOfRows - 1) {
   int OffsetOfNextdata = 
CarbonUnsafe.getUnsafe().getInt(dataPageMemoryBlock.getBaseObject(),
-  dataPageMemoryBlock.getBaseOffset() + this.dataPointersOffsets + 
((index + 1)
+  dataPageMemoryBlock.getBaseOffset() + this.dataPointersOffsets + 
((rowId + 1)
   * CarbonCommonConstants.INT_SIZE_IN_BYTE));
   length = (short) (OffsetOfNextdata - (currentDataOffset
   + CarbonCommonConstants.SHORT_SIZE_IN_BYTE));

http://git-wip-us.apache.org/repos/asf/carbondata/blob/92c9f224/core/src/main/java/org/apache/carbondata/core/datastore/columnar/ColumnGroupModel.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datastore/columnar/ColumnGroupModel.java
 
b/core/src/main/java/org/apache/carbondata/core/datastore/columnar/ColumnGroupModel.java
index 74d268a..e2a4161 100644
--- 
a/core/src/main/java/org/apache/

[08/50] [abbrv] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/92c9f224/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
 
b/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
index 94a041a..b74c279 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
@@ -378,7 +378,6 @@ public abstract class AbstractDataFileFooterConverter {
   cardinality[i] = segmentInfo.getColumn_cardinalities().get(i);
 }
 info.setColumnCardinality(cardinality);
-info.setNumberOfColumns(segmentInfo.getNum_cols());
 return info;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/92c9f224/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 52305bd..0cc783e 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
@@ -48,10 +48,10 @@ import 
org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentif
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.constants.CarbonLoadOptionConstants;
 import org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.block.AbstractIndex;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.columnar.ColumnGroupModel;
@@ -82,7 +82,7 @@ import 
org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.mutate.UpdateVO;
 import org.apache.carbondata.core.reader.ThriftReader;
 import org.apache.carbondata.core.reader.ThriftReader.TBaseCreator;
-import org.apache.carbondata.core.scan.model.QueryDimension;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatus;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
@@ -248,16 +248,13 @@ public final class CarbonUtil {
   public static ColumnGroupModel getColGroupModel(int[][] columnGroups) {
 int[] columnSplit = new int[columnGroups.length];
 int noOfColumnStore = columnSplit.length;
-boolean[] columnarStore = new boolean[noOfColumnStore];
 
 for (int i = 0; i < columnGroups.length; i++) {
   columnSplit[i] = columnGroups[i].length;
-  columnarStore[i] = columnGroups[i].length <= 1;
 }
 ColumnGroupModel colGroupModel = new ColumnGroupModel();
 colGroupModel.setNoOfColumnStore(noOfColumnStore);
 colGroupModel.setColumnSplit(columnSplit);
-colGroupModel.setColumnarStore(columnarStore);
 colGroupModel.setColumnGroup(columnGroups);
 return colGroupModel;
   }
@@ -418,7 +415,7 @@ public final class CarbonUtil {
 }
   }
 
-  public static int getFirstIndexUsingBinarySearch(DimensionColumnDataChunk 
dimColumnDataChunk,
+  public static int getFirstIndexUsingBinarySearch(DimensionColumnPage 
dimColumnDataChunk,
   int low, int high, byte[] compareValue, boolean matchUpLimit) {
 int cmpResult = 0;
 while (high >= low) {
@@ -457,7 +454,7 @@ public final class CarbonUtil {
* @return the compareValue's range index in the dimColumnDataChunk
*/
   public static int[] getRangeIndexUsingBinarySearch(
-  DimensionColumnDataChunk dimColumnDataChunk, int low, int high, byte[] 
compareValue) {
+  DimensionColumnPage dimColumnDataChunk, int low, int high, byte[] 
compareValue) {
 
 int[] rangeIndex = new int[2];
 int cmpResult = 0;
@@ -551,7 +548,7 @@ public final class CarbonUtil {
* @return index value
*/
   public static int nextLesserValueToTarget(int currentIndex,
-  DimensionColumnDataChunk dimColumnDataChunk, byte[] compareValue) {
+  DimensionColumnPage dimColumnDataChunk, byte[] compareValue) {
 while (currentIndex - 1 >= 0
 && dimColumnDataChunk.compareTo(currentIndex - 1, compareValue) >= 0) {
   --currentIndex;
@@ -571,7 +568,7 @@ public

[22/50] [abbrv] carbondata git commit: [CARBONDATA-2025] Unify all path construction through CarbonTablePath static method

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f06824e9/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java 
b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
index 9799ac2..b7b5e43 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
@@ -22,16 +22,14 @@ import 
org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
 
 import org.apache.hadoop.fs.Path;
 
-
 /**
  * Helps to get Table content paths.
  */
-public class CarbonTablePath extends Path {
+public class CarbonTablePath {
 
   private static final String METADATA_DIR = "Metadata";
   private static final String DICTIONARY_EXT = ".dict";
@@ -54,19 +52,10 @@ public class CarbonTablePath extends Path {
   private static final String STREAMING_LOG_DIR = "log";
   private static final String STREAMING_CHECKPOINT_DIR = "checkpoint";
 
-  private String tablePath;
-  private CarbonTableIdentifier carbonTableIdentifier;
-
   /**
-   * structure CarbonTablePath object to manage table paths
-   *
-   * @param carbonTableIdentifier identifier of carbon table that the segment 
belong to
-   * @param tablePathString the store path of the segment
+   * This class provides static utility only.
*/
-  public CarbonTablePath(CarbonTableIdentifier carbonTableIdentifier, String 
tablePathString) {
-super(tablePathString);
-this.carbonTableIdentifier = carbonTableIdentifier;
-this.tablePath = tablePathString;
+  private CarbonTablePath() {
   }
 
   /**
@@ -130,40 +119,21 @@ public class CarbonTablePath extends Path {
   }
 
   /**
-   * gets table path
+   * Return absolute path of dictionary file
*/
-  public String getPath() {
-return tablePath;
+  public static String getDictionaryFilePath(String tablePath, String 
columnId) {
+return getMetadataPath(tablePath) + File.separator + 
getDictionaryFileName(columnId);
   }
 
   /**
-   * @param columnId unique column identifier
-   * @return absolute path of dictionary file
+   * Return absolute path of dictionary file
*/
-  public String getDictionaryFilePath(String columnId) {
-return getMetaDataDir() + File.separator + getDictionaryFileName(columnId);
-  }
-
-  /**
-   * @param dictionaryPath
-   * @param columnId unique column identifier
-   * @return absolute path of dictionary file
-   */
-  public String getDictionaryFilePath(String dictionaryPath, String columnId) {
+  public static String getExternalDictionaryFilePath(String dictionaryPath, 
String columnId) {
 return dictionaryPath + File.separator + getDictionaryFileName(columnId);
   }
 
   /**
-   * This method will return the metadata directory location for a table
-   *
-   * @return
-   */
-  public String getMetadataDirectoryPath() {
-return getMetaDataDir();
-  }
-
-  /**
-   * Return metadata path based on `tablePath`
+   * Return metadata path
*/
   public static String getMetadataPath(String tablePath) {
 return tablePath + File.separator + METADATA_DIR;
@@ -184,67 +154,42 @@ public class CarbonTablePath extends Path {
   }
 
   /**
-   * @param columnId unique column identifier
-   * @return absolute path of dictionary meta file
+   * Return absolute path of dictionary meta file
*/
-  public String getDictionaryMetaFilePath(String columnId) {
-return getMetaDataDir() + File.separator + columnId + DICTIONARY_META_EXT;
-  }
-
-  /**
-   * @param dictionaryPath
-   * @param columnId unique column identifier
-   * @return absolute path of dictionary file
-   */
-  public String getDictionaryMetaFilePath(String dictionaryPath, String 
columnId) {
+  public static String getExternalDictionaryMetaFilePath(String 
dictionaryPath, String columnId) {
 return dictionaryPath + File.separator + columnId + DICTIONARY_META_EXT;
   }
 
   /**
-   * @param columnId unique column identifier
-   * @return absolute path of sort index file
+   * Return absolute path of dictionary meta file
*/
-  public String getSortIndexFilePath(String columnId) {
-return getMetaDataDir() + File.separator + columnId + SORT_INDEX_EXT;
+  public static String getDictionaryMetaFilePath(String tablePath, String 
columnId) {
+return getMetadataPath(tablePath) + File.separator + columnId + 
DICTIONARY_META_EXT;
   }
 
   /**
-   * @param dictionaryPath
-   * @param columnId unique column identifier
-   * @return absolute path of dictionary 

[34/50] [abbrv] carbondata git commit: [CARBONDATA-2023][DataLoad] Add size base block allocation in data loading

2018-03-04 Thread jackylk
[CARBONDATA-2023][DataLoad] Add size base block allocation in data loading

Carbondata assign blocks to nodes at the beginning of data loading.
Previous block allocation strategy is block number based and it will
suffer skewed data problem if the size of input files differs a lot.

We introduced a size based block allocation strategy to optimize data
loading performance in skewed data scenario.

This closes #1808


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

Branch: refs/heads/carbonstore-rebase5
Commit: 8fe8ab4c078de0ccd218f8ba41352896aebd5202
Parents: 28b5720
Author: xuchuanyin 
Authored: Thu Feb 8 14:42:39 2018 +0800
Committer: Jacky Li 
Committed: Sun Mar 4 20:32:13 2018 +0800

--
 .../constants/CarbonLoadOptionConstants.java|  10 +
 .../core/datastore/block/TableBlockInfo.java|  29 ++
 .../carbondata/core/util/CarbonProperties.java  |  11 +
 docs/useful-tips-on-carbondata.md   |   1 +
 .../spark/rdd/NewCarbonDataLoadRDD.scala|   4 +-
 .../spark/sql/hive/DistributionUtil.scala   |   2 +-
 .../spark/rdd/CarbonDataRDDFactory.scala|  18 +-
 .../merger/NodeMultiBlockRelation.java  |  40 ++
 .../processing/util/CarbonLoaderUtil.java   | 480 ---
 .../processing/util/CarbonLoaderUtilTest.java   | 125 +
 10 files changed, 545 insertions(+), 175 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/8fe8ab4c/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
 
b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
index bcfeba0..a6bf60f 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
@@ -114,4 +114,14 @@ public final class CarbonLoadOptionConstants {
*/
   public static final int MAX_EXTERNAL_DICTIONARY_SIZE = 1000;
 
+  /**
+   * enable block size based block allocation while loading data. By default, 
carbondata assigns
+   * blocks to node based on block number. If this option is set to `true`, 
carbondata will
+   * consider block size first and make sure that all the nodes will process 
almost equal size of
+   * data. This option is especially useful when you encounter skewed data.
+   */
+  @CarbonProperty
+  public static final String ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION
+  = "carbon.load.skewedDataOptimization.enabled";
+  public static final String 
ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION_DEFAULT = "false";
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/8fe8ab4c/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
 
b/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
index a7bfdba..c0cebe0 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
@@ -19,6 +19,8 @@ package org.apache.carbondata.core.datastore.block;
 import java.io.IOException;
 import java.io.Serializable;
 import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -98,6 +100,20 @@ public class TableBlockInfo implements Distributable, 
Serializable {
 
   private String dataMapWriterPath;
 
+  /**
+   * comparator to sort by block size in descending order.
+   * Since each line is not exactly the same, the size of a InputSplit may 
differs,
+   * so we allow some deviation for these splits.
+   */
+  public static final Comparator DATA_SIZE_DESC_COMPARATOR =
+  new Comparator() {
+@Override public int compare(Distributable o1, Distributable o2) {
+  long diff =
+  ((TableBlockInfo) o1).getBlockLength() - ((TableBlockInfo) 
o2).getBlockLength();
+  return diff < 0 ? 1 : (diff == 0 ? 0 : -1);
+}
+  };
+
   public TableBlockInfo(String filePath, long blockOffset, String segmentId,
   String[] locations, long blockLength, ColumnarFormatVersion version,
   String[] deletedDeltaFilePath) {
@@ -434,4 +450,17 @@ public class TableBlockInfo implements Distributable, 
Serializable {
   

[13/50] [abbrv] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/92c9f224/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeColGroupFilterExecuterImpl.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeColGroupFilterExecuterImpl.java
 
b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeColGroupFilterExecuterImpl.java
index 9391ebd..44f7c07 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeColGroupFilterExecuterImpl.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeColGroupFilterExecuterImpl.java
@@ -16,20 +16,10 @@
  */
 package org.apache.carbondata.core.scan.filter.executer;
 
-import java.util.ArrayList;
 import java.util.BitSet;
-import java.util.List;
 
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
-import org.apache.carbondata.core.keygenerator.KeyGenException;
-import org.apache.carbondata.core.keygenerator.KeyGenerator;
-import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
-import org.apache.carbondata.core.scan.executor.util.QueryUtil;
 import 
org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.apache.carbondata.core.util.ByteUtil;
 
 /**
  * It checks if filter is required on given block and if required, it does
@@ -38,12 +28,6 @@ import org.apache.carbondata.core.util.ByteUtil;
 public class ExcludeColGroupFilterExecuterImpl extends 
ExcludeFilterExecuterImpl {
 
   /**
-   * LOGGER
-   */
-  private static final LogService LOGGER =
-  
LogServiceFactory.getLogService(ExcludeColGroupFilterExecuterImpl.class.getName());
-
-  /**
* @param dimColResolvedFilterInfo
* @param segmentProperties
*/
@@ -53,54 +37,6 @@ public class ExcludeColGroupFilterExecuterImpl extends 
ExcludeFilterExecuterImpl
   }
 
   /**
-   * It fills BitSet with row index which matches filter key
-   */
-  protected BitSet getFilteredIndexes(DimensionColumnDataChunk 
dimensionColumnDataChunk,
-  int numerOfRows) {
-BitSet bitSet = new BitSet(numerOfRows);
-bitSet.flip(0, numerOfRows);
-try {
-  KeyStructureInfo keyStructureInfo = getKeyStructureInfo();
-  byte[][] filterValues = dimColumnExecuterInfo.getExcludeFilterKeys();
-  for (int i = 0; i < filterValues.length; i++) {
-byte[] filterVal = filterValues[i];
-for (int rowId = 0; rowId < numerOfRows; rowId++) {
-  byte[] colData = new 
byte[keyStructureInfo.getMaskByteRanges().length];
-  dimensionColumnDataChunk.fillChunkData(colData, 0, rowId, 
keyStructureInfo);
-  if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterVal, colData) 
== 0) {
-bitSet.flip(rowId);
-  }
-}
-  }
-
-} catch (Exception e) {
-  LOGGER.error(e);
-}
-
-return bitSet;
-  }
-
-  /**
-   * It is required for extracting column data from columngroup chunk
-   *
-   * @return
-   * @throws KeyGenException
-   */
-  private KeyStructureInfo getKeyStructureInfo() throws KeyGenException {
-int colGrpId = getColumnGroupId(dimColEvaluatorInfo.getColumnIndex());
-KeyGenerator keyGenerator = 
segmentProperties.getColumnGroupAndItsKeygenartor().get(colGrpId);
-List mdKeyOrdinal = new ArrayList();
-mdKeyOrdinal.add(getMdkeyOrdinal(dimColEvaluatorInfo.getColumnIndex(), 
colGrpId));
-int[] maskByteRanges = 
QueryUtil.getMaskedByteRangeBasedOrdinal(mdKeyOrdinal, keyGenerator);
-byte[] maxKey = QueryUtil.getMaxKeyBasedOnOrinal(mdKeyOrdinal, 
keyGenerator);
-KeyStructureInfo restructureInfos = new KeyStructureInfo();
-restructureInfos.setKeyGenerator(keyGenerator);
-restructureInfos.setMaskByteRanges(maskByteRanges);
-restructureInfos.setMaxKey(maxKey);
-return restructureInfos;
-  }
-
-  /**
* Check if scan is required on given block based on min and max value
*/
   public BitSet isScanRequired(byte[][] blkMaxVal, byte[][] blkMinVal) {
@@ -109,25 +45,4 @@ public class ExcludeColGroupFilterExecuterImpl extends 
ExcludeFilterExecuterImpl
 return bitSet;
   }
 
-  private int getMdkeyOrdinal(int ordinal, int colGrpId) {
-return segmentProperties.getColumnGroupMdKeyOrdinal(colGrpId, ordinal);
-  }
-
-  private int getColumnGroupId(int ordinal) {
-int[][] columnGroups = segmentProperties.getColumnGroups();
-int colGrpId = -1;
-for (int i = 0; i < columnGroups.length; i++) {
-  if (columnGroups[i].length > 1) {
-colGrpId++;
-if (QueryUtil.searchInArray(columnGroups[i], ordinal)) {
-  break;
-}
-  }
-}
-return colGrpId;
-  }
-
-  public KeyGenerator getKeyGenerator(int co

[33/50] [abbrv] carbondata git commit: [CARBONDATA-1544][Datamap] Datamap FineGrain implementation

2018-03-04 Thread jackylk
[CARBONDATA-1544][Datamap] Datamap FineGrain implementation

Implemented interfaces for FG datamap and integrated to filterscanner to use 
the pruned bitset from FG datamap.
FG Query flow as follows.
1.The user can add FG datamap to any table and implement there interfaces.
2. Any filter query which hits the table with datamap will call prune method of 
FGdatamap.
3. The prune method of FGDatamap return list FineGrainBlocklet , these 
blocklets contain the information of block, blocklet, page and rowids 
information as well.
4. The pruned blocklets are internally wriitten to file and returns only the 
block , blocklet and filepath information as part of Splits.
5. Based on the splits scanrdd schedule the tasks.
6. In filterscanner we check the datamapwriterpath from split and reNoteads the 
bitset if exists. And pass this bitset as input to it.

This closes #1471


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

Branch: refs/heads/carbonstore-rebase5
Commit: bb5bb00af982831ea83c73a2c437aa4aea8a5422
Parents: f9d15a2
Author: ravipesala 
Authored: Wed Nov 15 19:48:40 2017 +0530
Committer: Jacky Li 
Committed: Sun Mar 4 20:32:12 2018 +0800

--
 .../carbondata/core/datamap/DataMapMeta.java|   8 +-
 .../core/datamap/DataMapStoreManager.java   |  30 +-
 .../carbondata/core/datamap/DataMapType.java|  21 +
 .../carbondata/core/datamap/TableDataMap.java   |  33 +-
 .../core/datamap/dev/AbstractDataMapWriter.java | 110 +
 .../core/datamap/dev/BlockletSerializer.java|  57 +++
 .../carbondata/core/datamap/dev/DataMap.java|   5 +-
 .../core/datamap/dev/DataMapFactory.java|  14 +-
 .../core/datamap/dev/DataMapWriter.java |  57 ---
 .../cgdatamap/AbstractCoarseGrainDataMap.java   |  24 +
 .../AbstractCoarseGrainDataMapFactory.java  |  34 ++
 .../dev/fgdatamap/AbstractFineGrainDataMap.java |  24 +
 .../AbstractFineGrainDataMapFactory.java|  38 ++
 .../carbondata/core/datastore/DataRefNode.java  |   6 +
 .../core/datastore/block/TableBlockInfo.java|  10 +
 .../impl/btree/AbstractBTreeLeafNode.java   |   5 +
 .../datastore/impl/btree/BTreeNonLeafNode.java  |   5 +
 .../carbondata/core/indexstore/Blocklet.java|  30 +-
 .../indexstore/BlockletDataMapIndexStore.java   |   6 -
 .../core/indexstore/BlockletDetailsFetcher.java |   9 +
 .../core/indexstore/ExtendedBlocklet.java   |  17 +
 .../core/indexstore/FineGrainBlocklet.java  | 120 +
 .../indexstore/SegmentPropertiesFetcher.java|   3 +-
 .../blockletindex/BlockletDataMap.java  |  18 +-
 .../blockletindex/BlockletDataMapFactory.java   |  66 ++-
 .../blockletindex/BlockletDataRefNode.java  |  27 +-
 .../indexstore/blockletindex/IndexWrapper.java  |  18 +
 .../core/indexstore/schema/FilterType.java  |  24 -
 .../executer/ExcludeFilterExecuterImpl.java |   3 +
 .../executer/IncludeFilterExecuterImpl.java |   3 +
 .../scanner/impl/BlockletFilterScanner.java |   2 +
 .../apache/carbondata/core/util/CarbonUtil.java |  97 
 datamap/examples/pom.xml|   2 +-
 .../datamap/examples/MinMaxDataMap.java |  33 +-
 .../datamap/examples/MinMaxDataMapFactory.java  |  67 +--
 .../datamap/examples/MinMaxDataWriter.java  |  36 +-
 .../examples/MinMaxIndexBlockDetails.java   |  13 -
 .../carbondata/hadoop/CarbonInputSplit.java |  21 +-
 .../hadoop/api/CarbonTableInputFormat.java  |  17 +-
 .../testsuite/datamap/CGDataMapTestCase.scala   | 361 +++
 .../testsuite/datamap/DataMapWriterSuite.scala  |  43 +-
 .../testsuite/datamap/FGDataMapTestCase.scala   | 440 +++
 .../TestInsertAndOtherCommandConcurrent.scala   |  21 +-
 .../carbondata/spark/rdd/CarbonScanRDD.scala|   7 +-
 .../TestStreamingTableOperation.scala   |   5 +-
 .../datamap/DataMapWriterListener.java  |  57 ++-
 .../store/CarbonFactDataHandlerModel.java   |  10 +-
 .../store/writer/AbstractFactDataWriter.java| 128 +-
 48 files changed, 1784 insertions(+), 401 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/bb5bb00a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java 
b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java
index 7746acf..dd15ccb 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java
@@ -19,15 +19,15 @@ package org.apache.carbondata

[32/50] [abbrv] carbondata git commit: [CARBONDATA-1544][Datamap] Datamap FineGrain implementation

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/bb5bb00a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMapFactory.java
--
diff --git 
a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMapFactory.java
 
b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMapFactory.java
index b196d0d..266c107 100644
--- 
a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMapFactory.java
+++ 
b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMapFactory.java
@@ -17,7 +17,6 @@
 
 package org.apache.carbondata.datamap.examples;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -25,90 +24,94 @@ import java.util.List;
 
 import org.apache.carbondata.core.datamap.DataMapDistributable;
 import org.apache.carbondata.core.datamap.DataMapMeta;
-import org.apache.carbondata.core.datamap.dev.DataMap;
-import org.apache.carbondata.core.datamap.dev.DataMapFactory;
-import org.apache.carbondata.core.datamap.dev.DataMapWriter;
-import org.apache.carbondata.core.events.ChangeEvent;
-import org.apache.carbondata.core.indexstore.schema.FilterType;
+import org.apache.carbondata.core.datamap.Segment;
+import org.apache.carbondata.core.datamap.dev.AbstractDataMapWriter;
+import org.apache.carbondata.core.datamap.dev.DataMapModel;
+import 
org.apache.carbondata.core.datamap.dev.cgdatamap.AbstractCoarseGrainDataMap;
+import 
org.apache.carbondata.core.datamap.dev.cgdatamap.AbstractCoarseGrainDataMapFactory;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-
+import org.apache.carbondata.core.scan.filter.intf.ExpressionType;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+import org.apache.carbondata.events.Event;
 
 /**
  * Min Max DataMap Factory
  */
-public class MinMaxDataMapFactory implements DataMapFactory {
+public class MinMaxDataMapFactory extends AbstractCoarseGrainDataMapFactory {
 
   private AbsoluteTableIdentifier identifier;
 
-  @Override
-  public void init(AbsoluteTableIdentifier identifier, String dataMapName) {
+  @Override public void init(AbsoluteTableIdentifier identifier, String 
dataMapName) {
 this.identifier = identifier;
   }
 
   /**
* createWriter will return the MinMaxDataWriter.
-   * @param segmentId
+   *
+   * @param segment
* @return
*/
-  @Override
-  public DataMapWriter createWriter(String segmentId) {
-return new MinMaxDataWriter();
+  @Override public AbstractDataMapWriter createWriter(Segment segment) {
+return new MinMaxDataWriter(identifier, segment.getSegmentNo(),
+CarbonTablePath.getSegmentPath(identifier.getTablePath(), 
segment.getSegmentNo()));
   }
 
   /**
* getDataMaps Factory method Initializes the Min Max Data Map and returns.
-   * @param segmentId
+   *
+   * @param segment
* @return
* @throws IOException
*/
-  @Override public List getDataMaps(String segmentId) throws 
IOException {
-List dataMapList = new ArrayList<>();
+  @Override public List getDataMaps(Segment 
segment)
+  throws IOException {
+List dataMapList = new ArrayList<>();
 // Form a dataMap of Type MinMaxDataMap.
 MinMaxDataMap dataMap = new MinMaxDataMap();
 try {
-  dataMap.init(identifier.getTablePath() + "/Fact/Part0/Segment_" + 
segmentId + File.separator);
+  dataMap.init(new DataMapModel(
+  CarbonTablePath.getSegmentPath(identifier.getTablePath(), 
segment.getSegmentNo(;
 } catch (MemoryException ex) {
-
+  throw new IOException(ex);
 }
 dataMapList.add(dataMap);
 return dataMapList;
   }
 
   /**
-   *
-   * @param segmentId
+   * @param segment
* @return
*/
-  @Override public List toDistributable(String 
segmentId) {
+  @Override public List toDistributable(Segment segment) 
{
 return null;
   }
 
   /**
* Clear the DataMap.
-   * @param segmentId
+   *
+   * @param segment
*/
-  @Override public void clear(String segmentId) {
+  @Override public void clear(Segment segment) {
   }
 
   /**
* Clearing the data map.
*/
-  @Override
-  public void clear() {
+  @Override public void clear() {
   }
 
-  @Override public DataMap getDataMap(DataMapDistributable distributable) {
+  @Override public List 
getDataMaps(DataMapDistributable distributable)
+  throws IOException {
 return null;
   }
 
-  @Override
-  public void fireEvent(ChangeEvent event) {
+  @Override public void fireEvent(Event event) {
 
   }
 
-  @Override
-  public DataMapMeta getMeta() {
-return new DataMapMeta(new ArrayList(Arrays.asList("c2")), 
FilterType.EQUALTO);
+  @Override public DataMapMeta getMeta() {
+return new DataMapMeta(new ArrayList(Arr

[26/50] [abbrv] carbondata git commit: [REBASE] resolve conflict after rebasing to master

2018-03-04 Thread jackylk
[REBASE] resolve conflict after rebasing to master


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

Branch: refs/heads/carbonstore-rebase5
Commit: 6216294c1e28c1db05e572f0aac3a991d345e085
Parents: 3fdd5d0
Author: Jacky Li 
Authored: Tue Feb 27 08:51:25 2018 +0800
Committer: Jacky Li 
Committed: Sun Mar 4 20:32:12 2018 +0800

--
 .../core/datamap/dev/AbstractDataMapWriter.java |  5 ++--
 .../core/datamap/dev/DataMapFactory.java|  2 +-
 .../blockletindex/BlockletDataMapFactory.java   |  2 +-
 .../SegmentUpdateStatusManager.java |  9 +-
 .../datamap/examples/MinMaxDataMapFactory.java  |  5 ++--
 .../datamap/examples/MinMaxDataWriter.java  |  7 +++--
 .../testsuite/datamap/CGDataMapTestCase.scala   | 26 
 .../testsuite/datamap/DataMapWriterSuite.scala  | 19 ++--
 .../testsuite/datamap/FGDataMapTestCase.scala   | 31 +---
 .../iud/DeleteCarbonTableTestCase.scala |  2 +-
 .../TestInsertAndOtherCommandConcurrent.scala   | 14 +
 .../StandardPartitionTableCleanTestCase.scala   | 12 
 .../carbondata/spark/util/DataLoadingUtil.scala |  2 +-
 .../datamap/DataMapWriterListener.java  |  2 +-
 .../processing/merger/CarbonDataMergerUtil.java |  8 +
 .../merger/CompactionResultSortProcessor.java   |  4 +--
 .../merger/RowResultMergerProcessor.java|  5 ++--
 .../partition/spliter/RowResultProcessor.java   |  5 ++--
 .../util/CarbonDataProcessorUtil.java   |  4 +--
 .../processing/util/CarbonLoaderUtil.java   |  9 --
 20 files changed, 73 insertions(+), 100 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/6216294c/core/src/main/java/org/apache/carbondata/core/datamap/dev/AbstractDataMapWriter.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datamap/dev/AbstractDataMapWriter.java
 
b/core/src/main/java/org/apache/carbondata/core/datamap/dev/AbstractDataMapWriter.java
index bcc9bad..de6dcb1 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/datamap/dev/AbstractDataMapWriter.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/datamap/dev/AbstractDataMapWriter.java
@@ -18,6 +18,7 @@ package org.apache.carbondata.core.datamap.dev;
 
 import java.io.IOException;
 
+import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
@@ -35,10 +36,10 @@ public abstract class AbstractDataMapWriter {
 
   protected String writeDirectoryPath;
 
-  public AbstractDataMapWriter(AbsoluteTableIdentifier identifier, String 
segmentId,
+  public AbstractDataMapWriter(AbsoluteTableIdentifier identifier, Segment 
segment,
   String writeDirectoryPath) {
 this.identifier = identifier;
-this.segmentId = segmentId;
+this.segmentId = segment.getSegmentNo();
 this.writeDirectoryPath = writeDirectoryPath;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6216294c/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java 
b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
index df5670d..50ac279 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
@@ -39,7 +39,7 @@ public interface DataMapFactory {
   /**
* Return a new write for this datamap
*/
-  AbstractDataMapWriter createWriter(Segment segment);
+  AbstractDataMapWriter createWriter(Segment segment, String 
writeDirectoryPath);
 
   /**
* Get the datamap for segmentid

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6216294c/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
 
b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
index efe2b71..ee849bd 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFacto

[12/50] [abbrv] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/92c9f224/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 de97e82..540607d 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
@@ -34,8 +34,8 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.cache.dictionary.Dictionary;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
-import 
org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
+import 
org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionColumnPage;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import 
org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
@@ -58,7 +58,7 @@ import org.apache.carbondata.core.scan.filter.intf.RowImpl;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import 
org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import 
org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -68,20 +68,20 @@ public class RowLevelFilterExecuterImpl implements 
FilterExecuter {
 
   private static final LogService LOGGER =
   
LogServiceFactory.getLogService(RowLevelFilterExecuterImpl.class.getName());
-  protected List dimColEvaluatorInfoList;
-  protected List msrColEvalutorInfoList;
+  List dimColEvaluatorInfoList;
+  List msrColEvalutorInfoList;
   protected Expression exp;
   protected AbsoluteTableIdentifier tableIdentifier;
   protected SegmentProperties segmentProperties;
   /**
* it has index at which given dimension is stored in file
*/
-  protected int[] dimensionBlocksIndex;
+  int[] dimensionChunkIndex;
 
   /**
* it has index at which given measure is stored in file
*/
-  protected int[] measureBlocksIndex;
+  int[] measureChunkIndex;
 
   private Map complexDimensionInfoMap;
 
@@ -89,18 +89,18 @@ public class RowLevelFilterExecuterImpl implements 
FilterExecuter {
* flag to check whether the filter dimension is present in current block 
list of dimensions.
* Applicable for restructure scenarios
*/
-  protected boolean[] isDimensionPresentInCurrentBlock;
+  boolean[] isDimensionPresentInCurrentBlock;
 
   /**
* flag to check whether the filter measure is present in current block list 
of measures.
* Applicable for restructure scenarios
*/
-  protected boolean[] isMeasurePresentInCurrentBlock;
+  boolean[] isMeasurePresentInCurrentBlock;
 
   /**
* is dimension column data is natural sorted
*/
-  protected boolean isNaturalSorted;
+  boolean isNaturalSorted;
 
   /**
* date direct dictionary generator
@@ -124,10 +124,10 @@ public class RowLevelFilterExecuterImpl implements 
FilterExecuter {
 }
 if (this.dimColEvaluatorInfoList.size() > 0) {
   this.isDimensionPresentInCurrentBlock = new 
boolean[dimColEvaluatorInfoList.size()];
-  this.dimensionBlocksIndex = new int[dimColEvaluatorInfoList.size()];
+  this.dimensionChunkIndex = new int[dimColEvaluatorInfoList.size()];
 } else {
   this.isDimensionPresentInCurrentBlock = new boolean[]{false};
-  this.dimensionBlocksIndex = new int[]{0};
+  this.dimensionChunkIndex = new int[]{0};
 }
 if (null == msrColEvalutorInfoList) {
   this.msrColEvalutorInfoList = new 
ArrayList(20);
@@ -136,10 +136,10 @@ public class RowLevelFilterExecuterImpl implements 
FilterExecuter {
 }
 if (this.msrColEvalutorInfoList.size() > 0) {
   this.isMeasurePresentInCurrentBlock = new 
boolean[msrColEvalutorInfoList.size()];
-  this.measureBlocksIndex = new int[msrColEvalutorInfoList.size()];
+  this.measureChunkIndex = new int[msrColEvalutorInfoList.size()];
 } else {
   this.isMeasurePresentInCurrentBlock = new boolean[]{false};
-  this.measureBlocksIndex = new int[] {0};
+  this.measureChunkInde

[17/50] [abbrv] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/92c9f224/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunk.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunk.java
 
b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunk.java
deleted file mode 100644
index 6629d31..000
--- 
a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunk.java
+++ /dev/null
@@ -1,163 +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.datastore.chunk.impl;
-
-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;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
-import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
-import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
-
-/**
- * This class is gives access to fixed length dimension data chunk store
- */
-public class FixedLengthDimensionDataChunk extends AbstractDimensionDataChunk {
-
-  /**
-   * Constructor
-   *
-   * @param dataChunkdata chunk
-   * @param invertedIndexinverted index
-   * @param invertedIndexReverse reverse inverted index
-   * @param numberOfRows number of rows
-   * @param columnValueSize  size of each column value
-   */
-  public FixedLengthDimensionDataChunk(byte[] dataChunk, int[] invertedIndex,
-  int[] invertedIndexReverse, int numberOfRows, int columnValueSize) {
-long totalSize = null != invertedIndex ?
-dataChunk.length + (2 * numberOfRows * 
CarbonCommonConstants.INT_SIZE_IN_BYTE) :
-dataChunk.length;
-dataChunkStore = DimensionChunkStoreFactory.INSTANCE
-.getDimensionChunkStore(columnValueSize, null != invertedIndex, 
numberOfRows, totalSize,
-DimensionStoreType.FIXEDLENGTH);
-dataChunkStore.putArray(invertedIndex, invertedIndexReverse, dataChunk);
-  }
-
-  /**
-   * Below method will be used to fill the data based on offset and row id
-   *
-   * @param data data to filed
-   * @param offset   offset from which data need to be filed
-   * @param indexrow id of the chunk
-   * @param keyStructureInfo define the structure of the key
-   * @return how many bytes was copied
-   */
-  @Override public int fillChunkData(byte[] data, int offset, int index,
-  KeyStructureInfo keyStructureInfo) {
-dataChunkStore.fillRow(index, data, offset);
-return dataChunkStore.getColumnValueSize();
-  }
-
-  /**
-   * Converts to column dictionary integer value
-   *
-   * @param rowId
-   * @param columnIndex
-   * @param row
-   * @param restructuringInfo
-   * @return
-   */
-  @Override public int fillConvertedChunkData(int rowId, int columnIndex, 
int[] row,
-  KeyStructureInfo restructuringInfo) {
-row[columnIndex] = dataChunkStore.getSurrogate(rowId);
-return columnIndex + 1;
-  }
-
-  /**
-   * Fill the data to vector
-   *
-   * @param vectorInfo
-   * @param column
-   * @param restructuringInfo
-   * @return next column index
-   */
-  @Override public int fillConvertedChunkData(ColumnVectorInfo[] vectorInfo, 
int column,
-  KeyStructureInfo restructuringInfo) {
-ColumnVectorInfo columnVectorInfo = vectorInfo[column];
-int offset = columnVectorInfo.offset;
-int vectorOffset = columnVectorInfo.vectorOffset;
-int len = columnVectorInfo.size + offset;
-CarbonColumnVector vector = columnVectorInfo.vector;
-for (int j = offset; j < len; j++) {
-  int dict = dataChunkStore.getSurrogate(j);
-  if (columnVectorInfo.directDictionaryGenerator == null) {
-vector.putInt(vectorOffset++, dict);
-  } else {
-Object valueFromSurr

[09/50] [abbrv] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

2018-03-04 Thread jackylk
http://git-wip-us.apache.org/repos/asf/carbondata/blob/92c9f224/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java
 
b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java
index 553f85e..773fbd7 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java
@@ -17,20 +17,15 @@
 package org.apache.carbondata.core.scan.result.iterator;
 
 import org.apache.carbondata.common.CarbonIterator;
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.scan.result.BatchResult;
+import org.apache.carbondata.core.scan.result.RowBatch;
 
 public class PartitionSpliterRawResultIterator extends 
CarbonIterator {
 
-  private CarbonIterator iterator;
-  private BatchResult batch;
+  private CarbonIterator iterator;
+  private RowBatch batch;
   private int counter;
 
-  private static final LogService LOGGER =
-  
LogServiceFactory.getLogService(PartitionSpliterRawResultIterator.class.getName());
-
-  public PartitionSpliterRawResultIterator(CarbonIterator 
iterator) {
+  public PartitionSpliterRawResultIterator(CarbonIterator iterator) {
 this.iterator = iterator;
   }
 
@@ -65,7 +60,7 @@ public class PartitionSpliterRawResultIterator extends 
CarbonIterator
* @param batch
* @return
*/
-  private boolean checkBatchEnd(BatchResult batch) {
+  private boolean checkBatchEnd(RowBatch batch) {
 return !(counter < batch.getSize());
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/92c9f224/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java
 
b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java
index 70d0958..1dd1595 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java
@@ -21,7 +21,7 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
-import org.apache.carbondata.core.scan.result.BatchResult;
+import org.apache.carbondata.core.scan.result.RowBatch;
 import org.apache.carbondata.core.scan.wrappers.ByteArrayWrapper;
 
 /**
@@ -37,7 +37,7 @@ public class RawResultIterator extends 
CarbonIterator {
   /**
* Iterator of the Batch raw result.
*/
-  private CarbonIterator detailRawQueryResultIterator;
+  private CarbonIterator detailRawQueryResultIterator;
 
   /**
* Counter to maintain the row counter.
@@ -55,9 +55,9 @@ public class RawResultIterator extends 
CarbonIterator {
   /**
* batch of the result.
*/
-  private BatchResult batch;
+  private RowBatch batch;
 
-  public RawResultIterator(CarbonIterator 
detailRawQueryResultIterator,
+  public RawResultIterator(CarbonIterator 
detailRawQueryResultIterator,
   SegmentProperties sourceSegProperties, SegmentProperties 
destinationSegProperties) {
 this.detailRawQueryResultIterator = detailRawQueryResultIterator;
 this.sourceSegProperties = sourceSegProperties;
@@ -155,7 +155,7 @@ public class RawResultIterator extends 
CarbonIterator {
* @param batch
* @return
*/
-  private boolean checkIfBatchIsProcessedCompletely(BatchResult batch) {
+  private boolean checkIfBatchIsProcessedCompletely(RowBatch batch) {
 if (counter < batch.getSize()) {
   return false;
 } else {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/92c9f224/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/VectorDetailQueryResultIterator.java
--
diff --git 
a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/VectorDetailQueryResultIterator.java
 
b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/VectorDetailQueryResultIterator.java
index cc9710e..c7cb00d 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/VectorDetailQueryResultIterator.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/VectorDetailQueryResultIterator.java
@@ -35,10 +35,12 @@ public class VectorDetailQueryResultIterator ext

[29/50] [abbrv] carbondata git commit: [CARBONDATA-2080] [S3-Implementation] Propagated hadoopConf from driver to executor for s3 implementation in cluster mode.

2018-03-04 Thread jackylk
[CARBONDATA-2080] [S3-Implementation] Propagated hadoopConf from driver to 
executor for s3 implementation in cluster mode.

Problem : hadoopconf was not getting propagated from driver to the executor 
that's why load was failing to the distributed environment.
Solution: Setting the Hadoop conf in base class CarbonRDD
How to verify this PR :
Execute the load in the cluster mode It should be a success using location s3.

This closes #1860


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

Branch: refs/heads/carbonstore-rebase5
Commit: dfbdf3db00cbb488e49d3125b4ec93ff9e0dc9b2
Parents: f06824e
Author: Jatin 
Authored: Thu Jan 25 16:53:00 2018 +0530
Committer: Jacky Li 
Committed: Sun Mar 4 20:32:12 2018 +0800

--
 .../spark/rdd/AlterTableAddColumnRDD.scala  |  2 +-
 .../spark/rdd/AlterTableDropColumnRDD.scala |  2 +-
 .../spark/rdd/CarbonCleanFilesRDD.scala |  2 +-
 .../spark/rdd/CarbonDeleteLoadByDateRDD.scala   |  2 +-
 .../spark/rdd/CarbonDeleteLoadRDD.scala |  2 +-
 .../spark/rdd/CarbonDropPartitionRDD.scala  |  2 +-
 .../spark/rdd/CarbonDropTableRDD.scala  |  2 +-
 .../spark/rdd/CarbonGlobalDictionaryRDD.scala   |  3 +-
 .../spark/rdd/CarbonMergeFilesRDD.scala |  0
 .../carbondata/spark/rdd/CarbonMergerRDD.scala  |  2 +-
 .../apache/carbondata/spark/rdd/CarbonRDD.scala | 32 ++--
 .../spark/rdd/NewCarbonDataLoadRDD.scala|  2 +-
 .../carbondata/spark/rdd/SparkDataMapJob.scala  |  2 +-
 .../apache/spark/rdd/DataLoadCoalescedRDD.scala |  3 +-
 .../apache/spark/rdd/UpdateCoalescedRDD.scala   |  2 +-
 .../carbondata/streaming/StreamHandoffRDD.scala |  2 +-
 16 files changed, 45 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/carbondata/blob/dfbdf3db/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 56a66b9..7c1edea 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
@@ -50,7 +50,7 @@ class AddColumnPartition(rddId: Int, idx: Int, schema: 
ColumnSchema) extends Par
 class AlterTableAddColumnRDD[K, V](sc: SparkContext,
 @transient newColumns: Seq[ColumnSchema],
 identifier: AbsoluteTableIdentifier)
-  extends CarbonRDD[(Int, SegmentStatus)](sc, Nil) {
+  extends CarbonRDD[(Int, SegmentStatus)](sc, Nil, sc.hadoopConfiguration) {
 
   val lockType: String = 
CarbonProperties.getInstance.getProperty(CarbonCommonConstants.LOCK_TYPE,
 CarbonCommonConstants.CARBON_LOCK_TYPE_HDFS)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/dfbdf3db/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 248f351..e14524e 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
@@ -48,7 +48,7 @@ class DropColumnPartition(rddId: Int, idx: Int, schema: 
ColumnSchema) extends Pa
 class AlterTableDropColumnRDD[K, V](sc: SparkContext,
 @transient newColumns: Seq[ColumnSchema],
 carbonTableIdentifier: AbsoluteTableIdentifier)
-  extends CarbonRDD[(Int, SegmentStatus)](sc, Nil) {
+  extends CarbonRDD[(Int, SegmentStatus)](sc, Nil, sc.hadoopConfiguration) {
 
   override def getPartitions: Array[Partition] = {
 newColumns.zipWithIndex.map { column =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/dfbdf3db/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonCleanFilesRDD.scala
--
diff --git 
a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonCleanFilesRDD.scala
 
b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonCleanFilesRDD.scala
index 32523d8..9936a2a 100644
--- 
a/integration/spark-c

  1   2   3   4   5   6   7   8   9   10   >