[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-09-18 Thread dhatchayani
Github user dhatchayani commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r218669857
  
--- Diff: 
datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java
 ---
@@ -331,8 +332,18 @@ private BloomQueryModel 
buildQueryModelInternal(CarbonColumn carbonColumn,
   // for dictionary/date columns, convert the surrogate key to bytes
   internalFilterValue = CarbonUtil.getValueAsBytes(DataTypes.INT, 
convertedValue);
 } else {
-  // for non dictionary dimensions, is already bytes,
-  internalFilterValue = (byte[]) convertedValue;
+  // for non dictionary dimensions, numeric columns will be of 
original data,
+  // so convert the data to bytes
+  if (DataTypeUtil.isPrimitiveColumn(carbonColumn.getDataType())) {
+if (convertedValue == null) {
+  convertedValue = 
DataConvertUtil.getNullValueForMeasure(carbonColumn.getDataType(),
+  carbonColumn.getColumnSchema().getScale());
+}
+internalFilterValue =
+CarbonUtil.getValueAsBytes(carbonColumn.getDataType(), 
convertedValue);
--- End diff --

> I ran a test on table with bloom datamap created before applying this PR, 
and query it after this PR merged, but the answer is not correct. Can you check 
it?
> 
> Procedure to reproduce:
> 
> * switch master code before this PR merged
> * create table with no-dict measure column (set the measure column as 
sort column)
> * create bloom datamap on the measure column
> * load some data into table
> * query on the measure column, get a result
> * switch to code after this PR merged
> * do the same query and compare the result

I will check this issue and update asap


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-09-18 Thread dhatchayani
Github user dhatchayani commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r218669311
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/carbondata/datamap/IndexDataMapRebuildRDD.scala
 ---
@@ -264,8 +264,17 @@ class RawBytesReadSupport(segmentProperties: 
SegmentProperties, indexColumns: Ar
   rtn(i) = if (indexCol2IdxInDictArray.contains(col.getColName)) {
 
surrogatKeys(indexCol2IdxInDictArray(col.getColName)).toInt.asInstanceOf[Integer]
   } else if (indexCol2IdxInNoDictArray.contains(col.getColName)) {
-data(0).asInstanceOf[ByteArrayWrapper].getNoDictionaryKeyByIndex(
+val bytes = 
data(0).asInstanceOf[ByteArrayWrapper].getNoDictionaryKeyByIndex(
   indexCol2IdxInNoDictArray(col.getColName))
+// no dictionary primitive columns are expected to be in original 
data while loading,
+// so convert it to original data
+if (DataTypeUtil.isPrimitiveColumn(col.getDataType)) {
+  val dataFromBytes = DataTypeUtil
+.getDataBasedOnDataTypeForNoDictionaryColumn(bytes, 
col.getDataType)
+  dataFromBytes
--- End diff --

i think measure null and no dictionary null values are different, can u 
please give me any scenario which fall into no dictionary null case?


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-09-18 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/carbondata/pull/2654


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-09-06 Thread kevinjmh
Github user kevinjmh commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r215829032
  
--- Diff: 
datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java
 ---
@@ -331,8 +332,18 @@ private BloomQueryModel 
buildQueryModelInternal(CarbonColumn carbonColumn,
   // for dictionary/date columns, convert the surrogate key to bytes
   internalFilterValue = CarbonUtil.getValueAsBytes(DataTypes.INT, 
convertedValue);
 } else {
-  // for non dictionary dimensions, is already bytes,
-  internalFilterValue = (byte[]) convertedValue;
+  // for non dictionary dimensions, numeric columns will be of 
original data,
+  // so convert the data to bytes
+  if (DataTypeUtil.isPrimitiveColumn(carbonColumn.getDataType())) {
+if (convertedValue == null) {
+  convertedValue = 
DataConvertUtil.getNullValueForMeasure(carbonColumn.getDataType(),
+  carbonColumn.getColumnSchema().getScale());
+}
+internalFilterValue =
+CarbonUtil.getValueAsBytes(carbonColumn.getDataType(), 
convertedValue);
--- End diff --

The above problem is similar to your second commit for minmax


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-09-06 Thread kevinjmh
Github user kevinjmh commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r215827023
  
--- Diff: 
datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java
 ---
@@ -331,8 +332,18 @@ private BloomQueryModel 
buildQueryModelInternal(CarbonColumn carbonColumn,
   // for dictionary/date columns, convert the surrogate key to bytes
   internalFilterValue = CarbonUtil.getValueAsBytes(DataTypes.INT, 
convertedValue);
 } else {
-  // for non dictionary dimensions, is already bytes,
-  internalFilterValue = (byte[]) convertedValue;
+  // for non dictionary dimensions, numeric columns will be of 
original data,
+  // so convert the data to bytes
+  if (DataTypeUtil.isPrimitiveColumn(carbonColumn.getDataType())) {
+if (convertedValue == null) {
+  convertedValue = 
DataConvertUtil.getNullValueForMeasure(carbonColumn.getDataType(),
+  carbonColumn.getColumnSchema().getScale());
+}
+internalFilterValue =
+CarbonUtil.getValueAsBytes(carbonColumn.getDataType(), 
convertedValue);
--- End diff --

Result of `getValueAsBytes`  conflicts with existing bloom index data which 
will affect query result.
For measure in  'NoDict',  original implementation used 
`NonDictionaryFieldConverterImpl` to convert value, but now it uses 
`MeasureFieldConverterImpl` to convert value


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-09-05 Thread kevinjmh
Github user kevinjmh commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r215153728
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/carbondata/datamap/IndexDataMapRebuildRDD.scala
 ---
@@ -264,8 +264,17 @@ class RawBytesReadSupport(segmentProperties: 
SegmentProperties, indexColumns: Ar
   rtn(i) = if (indexCol2IdxInDictArray.contains(col.getColName)) {
 
surrogatKeys(indexCol2IdxInDictArray(col.getColName)).toInt.asInstanceOf[Integer]
   } else if (indexCol2IdxInNoDictArray.contains(col.getColName)) {
-data(0).asInstanceOf[ByteArrayWrapper].getNoDictionaryKeyByIndex(
+val bytes = 
data(0).asInstanceOf[ByteArrayWrapper].getNoDictionaryKeyByIndex(
   indexCol2IdxInNoDictArray(col.getColName))
+// no dictionary primitive columns are expected to be in original 
data while loading,
+// so convert it to original data
+if (DataTypeUtil.isPrimitiveColumn(col.getDataType)) {
+  val dataFromBytes = DataTypeUtil
+.getDataBasedOnDataTypeForNoDictionaryColumn(bytes, 
col.getDataType)
+  dataFromBytes
--- End diff --

if isPrimitiveColumn, need null check and get null value for measure


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-09-04 Thread kumarvishal09
Github user kumarvishal09 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214809720
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java
 ---
@@ -58,26 +70,64 @@ public int fillSurrogateKey(int rowId, int chunkIndex, 
int[] outputSurrogateKey)
 
   @Override
   public int fillVector(ColumnVectorInfo[] vectorInfo, int chunkIndex) {
-throw new UnsupportedOperationException("internal error");
+ColumnVectorInfo columnVectorInfo = vectorInfo[chunkIndex];
+CarbonColumnVector vector = columnVectorInfo.vector;
+int offset = columnVectorInfo.offset;
+int vectorOffset = columnVectorInfo.vectorOffset;
+int len = offset + columnVectorInfo.size;
+for (int i = offset; i < len; i++) {
+  fillRow(i, vector, vectorOffset++);
+}
+return chunkIndex + 1;
+  }
+
+  /**
+   * Fill the data to the vector
+   *
+   * @param rowId
+   * @param vector
+   * @param vectorRow
+   */
+  private void fillRow(int rowId, CarbonColumnVector vector, int 
vectorRow) {
+byte[] value = getChunkData(rowId);
+int length = value.length;
+DimensionDataVectorProcessor.putDataToVector(vector, value, vectorRow, 
length);
   }
 
   @Override
   public int fillVector(int[] filteredRowId, ColumnVectorInfo[] 
vectorInfo, int chunkIndex) {
-throw new UnsupportedOperationException("internal error");
+ColumnVectorInfo columnVectorInfo = vectorInfo[chunkIndex];
+CarbonColumnVector vector = columnVectorInfo.vector;
+int offset = columnVectorInfo.offset;
+int vectorOffset = columnVectorInfo.vectorOffset;
+int len = offset + columnVectorInfo.size;
+for (int i = offset; i < len; i++) {
+  fillRow(filteredRowId[i], vector, vectorOffset++);
+}
+return chunkIndex + 1;
   }
 
   @Override public byte[] getChunkData(int rowId) {
+int rowIdCopy = rowId;
--- End diff --

Can u add comment why we need to store actual rowid, and please change the 
variable name 


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-31 Thread kumarvishal09
Github user kumarvishal09 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214504899
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/datastore/columnar/BlockIndexerStorage.java
 ---
@@ -0,0 +1,90 @@
+/*
+ * 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.columnar;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+
+public class BlockIndexerStorage {
+
+  /**
+   * It compresses depends up on the sequence numbers.
+   * [1,2,3,4,6,8,10,11,12,13] is translated to [1,4,6,8,10,13] and [0,6]. 
In
+   * first array the start and end of sequential numbers and second array
+   * keeps the indexes of where sequential numbers starts. If there is no
+   * sequential numbers then the same array it returns with empty second
+   * array.
+   *
+   * @param rowIds
+   */
+  public static Map rleEncodeOnRowId(short[] rowIds, 
short[] rowIdPage,
--- End diff --

move this code to carbonutil


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-31 Thread kumarvishal09
Github user kumarvishal09 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214504900
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/datastore/columnar/BlockIndexerStorage.java
 ---
@@ -0,0 +1,90 @@
+/*
+ * 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.columnar;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+
+public class BlockIndexerStorage {
+
+  /**
+   * It compresses depends up on the sequence numbers.
+   * [1,2,3,4,6,8,10,11,12,13] is translated to [1,4,6,8,10,13] and [0,6]. 
In
+   * first array the start and end of sequential numbers and second array
+   * keeps the indexes of where sequential numbers starts. If there is no
+   * sequential numbers then the same array it returns with empty second
+   * array.
+   *
+   * @param rowIds
+   */
+  public static Map rleEncodeOnRowId(short[] rowIds, 
short[] rowIdPage,
+  short[] rowIdRlePage) {
+List list = new 
ArrayList(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+List map = new 
ArrayList(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+int k = 0;
+int i = 1;
+for (; i < rowIds.length; i++) {
+  if (rowIds[i] - rowIds[i - 1] == 1) {
+k++;
+  } else {
+if (k > 0) {
+  map.add(((short) list.size()));
+  list.add(rowIds[i - k - 1]);
+  list.add(rowIds[i - 1]);
+} else {
+  list.add(rowIds[i - 1]);
+}
+k = 0;
+  }
+}
+if (k > 0) {
+  map.add(((short) list.size()));
+  list.add(rowIds[i - k - 1]);
+  list.add(rowIds[i - 1]);
+} else {
+  list.add(rowIds[i - 1]);
+}
+int compressionPercentage = (((list.size() + map.size()) * 100) / 
rowIds.length);
+if (compressionPercentage > 70) {
+  rowIdPage = rowIds;
+} else {
+  rowIdPage = convertToArray(list);
+}
+if (rowIds.length == rowIdPage.length) {
+  rowIdRlePage = new short[0];
+} else {
+  rowIdRlePage = convertToArray(map);
+}
+Map rowIdAndRowRleIdPages = new HashMap<>(2);
+rowIdAndRowRleIdPages.put("rowIdPage", rowIdPage);
+rowIdAndRowRleIdPages.put("rowRlePage", rowIdRlePage);
+return rowIdAndRowRleIdPages;
+  }
+
+  public static short[] convertToArray(List list) {
--- End diff --

move this code to carbonutil


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-31 Thread manishgupta88
Github user manishgupta88 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214371546
  
--- Diff: 
datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java
 ---
@@ -331,8 +332,18 @@ private BloomQueryModel 
buildQueryModelInternal(CarbonColumn carbonColumn,
   // for dictionary/date columns, convert the surrogate key to bytes
   internalFilterValue = CarbonUtil.getValueAsBytes(DataTypes.INT, 
convertedValue);
 } else {
-  // for non dictionary dimensions, is already bytes,
-  internalFilterValue = (byte[]) convertedValue;
+  // for non dictionary dimensions, numeric columns will be of 
original data,
+  // so convert the data to bytes
+  if (DataTypeUtil.isPrimitiveColumn(carbonColumn.getDataType())) {
+if (convertedValue == null) {
--- End diff --

if possible initialize and store the flag  in constructor and remove the 
check `DataTypeUtil.isPrimitiveColumn` wherever applicable in the below code


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-31 Thread manishgupta88
Github user manishgupta88 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214361007
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java
 ---
@@ -110,8 +112,19 @@ public BitSetGroup applyFilter(RawBlockletColumnChunks 
rawBlockletColumnChunks,
   boolean isDecoded = false;
   for (int i = 0; i < dimensionRawColumnChunk.getPagesCount(); i++) {
 if (dimensionRawColumnChunk.getMaxValues() != null) {
-  if (isScanRequired(dimensionRawColumnChunk.getMaxValues()[i],
-  dimensionRawColumnChunk.getMinValues()[i], 
dimColumnExecuterInfo.getFilterKeys())) {
+  boolean scanRequired;
+  // for no dictionary measure column comparison can be done
+  // on the original data as like measure column
+  if 
(DataTypeUtil.isPrimitiveColumn(dimColumnEvaluatorInfo.getDimension().getDataType())
+  && 
!dimColumnEvaluatorInfo.getDimension().hasEncoding(Encoding.DICTIONARY)) {
+scanRequired = 
isScanRequired(dimensionRawColumnChunk.getMaxValues()[i],
--- End diff --

You can create a `isPrimitiveNoDictionaryColumn` flag and check 
`DataTypeUtil.isPrimitiveColum` in the constructor. This will avoid the check 
for every page. Do this for all the filters


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-31 Thread manishgupta88
Github user manishgupta88 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214356896
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java
 ---
@@ -239,12 +239,25 @@ private boolean isEncodedWithMeta(DataChunk2 
pageMetadata) {
   protected DimensionColumnPage decodeDimension(DimensionRawColumnChunk 
rawColumnPage,
   ByteBuffer pageData, DataChunk2 pageMetadata, int offset)
   throws IOException, MemoryException {
+List encodings = pageMetadata.getEncoders();
 if (isEncodedWithMeta(pageMetadata)) {
   ColumnPage decodedPage = decodeDimensionByMeta(pageMetadata, 
pageData, offset,
   null != rawColumnPage.getLocalDictionary());
   
decodedPage.setNullBits(QueryUtil.getNullBitSet(pageMetadata.presence));
-  return new ColumnPageWrapper(decodedPage, 
rawColumnPage.getLocalDictionary(),
-  isEncodedWithAdaptiveMeta(pageMetadata));
+  int[] invertedIndexes = new int[0];
--- End diff --

add a comment to explain that this scenario is to handle no dictionary 
primitive type columns where inverted index can be created on row id's during 
data load


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-31 Thread manishgupta88
Github user manishgupta88 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214354541
  
--- Diff: 
processing/src/main/java/org/apache/carbondata/processing/store/TablePage.java 
---
@@ -363,7 +398,16 @@ public EncodedTablePage getEncodedTablePage() {
   columnPageEncoder = encodingFactory.createEncoder(
   spec,
   noDictDimensionPages[noDictIndex]);
-  encodedPage = 
columnPageEncoder.encode(noDictDimensionPages[noDictIndex++]);
+  encodedPage = 
columnPageEncoder.encode(noDictDimensionPages[noDictIndex]);
+  DataType targetDataType =
+  
columnPageEncoder.getTargetDataType(noDictDimensionPages[noDictIndex]);
+  if (null != targetDataType) {
+LOGGER.info("Encoder result ---> Source data type: " + 
noDictDimensionPages[noDictIndex]
--- End diff --

make this logger debug and check for debugenabled


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-31 Thread manishgupta88
Github user manishgupta88 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214352965
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/DefaultEncodingFactory.java
 ---
@@ -346,12 +371,21 @@ static ColumnPageCodec 
selectCodecByAlgorithmForDecimal(SimpleStatsResult stats,
   // no effect to use adaptive or delta, use compression only
   return new DirectCompressCodec(stats.getDataType());
 }
+boolean isSort = false;
+boolean isInvertedIndex = false;
+if (columnSpec instanceof TableSpec.DimensionSpec
+&& columnSpec.getColumnType() != ColumnType.COMPLEX_PRIMITIVE) {
+  isSort = ((TableSpec.DimensionSpec) columnSpec).isInSortColumns();
+  isInvertedIndex = isSort && ((TableSpec.DimensionSpec) 
columnSpec).isDoInvertedIndex();
+}
--- End diff --

Put the above changes in one method as the same code is used in above 
places also and then call this method while creating the encoding type


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-31 Thread manishgupta88
Github user manishgupta88 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214351650
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java ---
@@ -91,6 +92,30 @@ private void addMeasures(List measures) {
 }
   }
 
+  /**
+   * No dictionary and complex dimensions of the table
+   *
+   * @return
+   */
+  public DimensionSpec[] getNoDictAndComplexDimensions() {
+List noDicOrCompIndexes = new 
ArrayList<>(dimensionSpec.length);
+int noDicCount = 0;
+for (int i = 0; i < dimensionSpec.length; i++) {
+  if (dimensionSpec[i].getColumnType() == ColumnType.PLAIN_VALUE
+  || dimensionSpec[i].getColumnType() == 
ColumnType.COMPLEX_PRIMITIVE
+  || dimensionSpec[i].getColumnType() == ColumnType.COMPLEX) {
+noDicOrCompIndexes.add(i);
+noDicCount++;
+  }
+}
+
+DimensionSpec[] dims = new DimensionSpec[noDicCount];
+for (int i = 0; i < dims.length; i++) {
+  dims[i] = dimensionSpec[noDicOrCompIndexes.get(i)];
+}
+return dims;
--- End diff --

Avoid the below for loop in this method


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-31 Thread manishgupta88
Github user manishgupta88 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214338168
  
--- Diff: 
processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowHandler.java
 ---
@@ -375,6 +454,47 @@ public void 
writeRawRowAsIntermediateSortTempRowToOutputStream(Object[] row,
 outputStream.write(rowBuffer.array(), 0, packSize);
   }
 
+  /**
+   * Write the data to stream
+   *
+   * @param data
+   * @param outputStream
+   * @param idx
+   * @throws IOException
+   */
+  private void writeDataToStream(Object data, DataOutputStream 
outputStream, int idx)
+  throws IOException {
+DataType dataType = noDicSortDataTypes[idx];
+if (null == data) {
+  outputStream.writeBoolean(false);
+  return;
--- End diff --

do not use return statement instead use the if else block wisely


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-31 Thread manishgupta88
Github user manishgupta88 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214336180
  
--- Diff: 
processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowHandler.java
 ---
@@ -224,10 +237,15 @@ public IntermediateSortTempRow 
readWithNoSortFieldConvert(
 
 // read no-dict & sort data
 for (int idx = 0; idx < this.noDictSortDimCnt; idx++) {
-  short len = inputStream.readShort();
-  byte[] bytes = new byte[len];
-  inputStream.readFully(bytes);
-  noDictSortDims[idx] = bytes;
+  // for no dict measure column get the original data
+  if (DataTypeUtil.isPrimitiveColumn(noDicSortDataTypes[idx])) {
+noDictSortDims[idx] = readDataFromStream(inputStream, idx);
+  } else {
+short len = inputStream.readShort();
+byte[] bytes = new byte[len];
+inputStream.readFully(bytes);
+noDictSortDims[idx] = bytes;
+  }
--- End diff --

Above also there is a similar..refactor the code to one method and call 
from both these places


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-31 Thread manishgupta88
Github user manishgupta88 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214341633
  
--- Diff: 
processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/RawRowComparator.java
 ---
@@ -30,24 +33,39 @@
 public class RawRowComparator implements Comparator {
   private int[] sortColumnIndices;
   private boolean[] isSortColumnNoDict;
+  private DataType[] noDicDataTypes;
 
-  public RawRowComparator(int[] sortColumnIndices, boolean[] 
isSortColumnNoDict) {
+  public RawRowComparator(int[] sortColumnIndices, boolean[] 
isSortColumnNoDict,
+  DataType[] noDicDataTypes) {
 this.sortColumnIndices = sortColumnIndices;
 this.isSortColumnNoDict = isSortColumnNoDict;
+this.noDicDataTypes = noDicDataTypes;
   }
 
   @Override
   public int compare(CarbonRow o1, CarbonRow o2) {
 int diff = 0;
 int i = 0;
+int noDicIdx = 0;
 for (int colIdx : sortColumnIndices) {
   if (isSortColumnNoDict[i]) {
-byte[] colA = (byte[]) o1.getObject(colIdx);
-byte[] colB = (byte[]) o2.getObject(colIdx);
-diff = UnsafeComparer.INSTANCE.compareTo(colA, colB);
-if (diff != 0) {
-  return diff;
+if (DataTypeUtil.isPrimitiveColumn(noDicDataTypes[noDicIdx])) {
+  // for no dictionary numeric column get comparator based on the 
data type
+  SerializableComparator comparator = 
org.apache.carbondata.core.util.comparator.Comparator
--- End diff --

increment `noDicIdx` in if block and remove from method end


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-31 Thread manishgupta88
Github user manishgupta88 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214341135
  
--- Diff: 
processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateSortTempRowComparator.java
 ---
@@ -45,18 +52,31 @@ public int compare(IntermediateSortTempRow rowA, 
IntermediateSortTempRow rowB) {
 int diff = 0;
 int dictIndex = 0;
 int nonDictIndex = 0;
+int noDicTypeIdx = 0;
 
 for (boolean isNoDictionary : isSortColumnNoDictionary) {
 
   if (isNoDictionary) {
-byte[] byteArr1 = rowA.getNoDictSortDims()[nonDictIndex];
-byte[] byteArr2 = rowB.getNoDictSortDims()[nonDictIndex];
-nonDictIndex++;
+if 
(DataTypeUtil.isPrimitiveColumn(noDicSortDataTypes[noDicTypeIdx])) {
+  // use data types based comparator for the no dictionary measure 
columns
+  SerializableComparator comparator = 
org.apache.carbondata.core.util.comparator.Comparator
--- End diff --

Increment the no dictionary type index here `noDicTypeIdx` in if block and 
not at the end


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-31 Thread manishgupta88
Github user manishgupta88 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214341442
  
--- Diff: 
processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparator.java
 ---
@@ -43,15 +53,31 @@ public NewRowComparator(boolean[] 
noDictionarySortColumnMaping) {
   public int compare(Object[] rowA, Object[] rowB) {
 int diff = 0;
 int index = 0;
+int dataTypeIdx = 0;
+int noDicSortIdx = 0;
 
-for (boolean isNoDictionary : noDictionarySortColumnMaping) {
-  if (isNoDictionary) {
-byte[] byteArr1 = (byte[]) rowA[index];
-byte[] byteArr2 = (byte[]) rowB[index];
+for (int i = 0; i < noDicDimColMapping.length; i++) {
+  if (noDicDimColMapping[i]) {
+if (noDicSortColumnMapping[noDicSortIdx++]) {
+  if (DataTypeUtil.isPrimitiveColumn(noDicDataTypes[dataTypeIdx])) 
{
+// use data types based comparator for the no dictionary 
measure columns
+SerializableComparator comparator =
--- End diff --

increment `dataTypeIdx` in if block and remove from method end


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-31 Thread manishgupta88
Github user manishgupta88 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214337384
  
--- Diff: 
processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowHandler.java
 ---
@@ -359,9 +433,14 @@ public void 
writeRawRowAsIntermediateSortTempRowToOutputStream(Object[] row,
 
 // write no-dict & sort
 for (int idx = 0; idx < this.noDictSortDimCnt; idx++) {
-  byte[] bytes = (byte[]) row[this.noDictSortDimIdx[idx]];
-  outputStream.writeShort(bytes.length);
-  outputStream.write(bytes);
+  if (DataTypeUtil.isPrimitiveColumn(noDicSortDataTypes[idx])) {
--- End diff --

I can see that at multiple places for every row 
DataTypeUtil.isPrimitiveColumn is getting used. Please check the load 
performance impact of this


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-31 Thread manishgupta88
Github user manishgupta88 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214341815
  
--- Diff: 
processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparator.java
 ---
@@ -60,26 +64,50 @@ public int compare(UnsafeCarbonRow rowL, Object 
baseObjectL, UnsafeCarbonRow row
   if (isNoDictionary) {
 short lengthA = CarbonUnsafe.getUnsafe().getShort(baseObjectL,
 rowA + dictSizeInMemory + sizeInNonDictPartA);
-byte[] byteArr1 = new byte[lengthA];
 sizeInNonDictPartA += 2;
-CarbonUnsafe.getUnsafe()
-.copyMemory(baseObjectL, rowA + dictSizeInMemory + 
sizeInNonDictPartA,
-byteArr1, CarbonUnsafe.BYTE_ARRAY_OFFSET, lengthA);
-sizeInNonDictPartA += lengthA;
-
 short lengthB = CarbonUnsafe.getUnsafe().getShort(baseObjectR,
 rowB + dictSizeInMemory + sizeInNonDictPartB);
-byte[] byteArr2 = new byte[lengthB];
 sizeInNonDictPartB += 2;
-CarbonUnsafe.getUnsafe()
-.copyMemory(baseObjectR, rowB + dictSizeInMemory + 
sizeInNonDictPartB,
-byteArr2, CarbonUnsafe.BYTE_ARRAY_OFFSET, lengthB);
-sizeInNonDictPartB += lengthB;
+DataType dataType = 
tableFieldStat.getNoDicSortDataType()[noDicSortIdx];
+if (DataTypeUtil.isPrimitiveColumn(dataType)) {
+  Object data1 = null;
--- End diff --

increment `noDicSortIdx` in if block and remove from method end


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-31 Thread dhatchayani
Github user dhatchayani commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214313592
  
--- Diff: 
processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowHandler.java
 ---
@@ -240,8 +258,44 @@ public IntermediateSortTempRow 
readWithNoSortFieldConvert(
 return new IntermediateSortTempRow(dictSortDims, 
noDictSortDims,measure);
   }
 
+  /**
+   * Read the data from the stream
+   *
+   * @param inputStream
+   * @param idx
+   * @return
+   * @throws IOException
+   */
+  private Object readDataFromStream(DataInputStream inputStream, int idx) 
throws IOException {
--- End diff --

For measures, it will always be packed/unpacked to/from bytebuffer


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-30 Thread kumarvishal09
Github user kumarvishal09 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214092679
  
--- Diff: 
processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowHandler.java
 ---
@@ -240,8 +258,44 @@ public IntermediateSortTempRow 
readWithNoSortFieldConvert(
 return new IntermediateSortTempRow(dictSortDims, 
noDictSortDims,measure);
   }
 
+  /**
+   * Read the data from the stream
+   *
+   * @param inputStream
+   * @param idx
+   * @return
+   * @throws IOException
+   */
+  private Object readDataFromStream(DataInputStream inputStream, int idx) 
throws IOException {
--- End diff --

is it refactor code ?? I think same code is also present for measures, can 
u please check and combine


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-30 Thread kumarvishal09
Github user kumarvishal09 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214090481
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaIntegralCodec.java
 ---
@@ -103,9 +137,33 @@ protected ColumnPageEncoderMeta 
getEncoderMeta(ColumnPage inputPage) {
   protected List getEncodingList() {
 List encodings = new ArrayList<>();
 encodings.add(Encoding.ADAPTIVE_DELTA_INTEGRAL);
+if (isInvertedIndex) {
+  encodings.add(Encoding.INVERTED_INDEX);
+}
 return encodings;
   }
 
+  @Override
+  protected void fillLegacyFields(DataChunk2 dataChunk) throws 
IOException {
--- End diff --

can u please move this code to some utility and as below classes also has 
same logic


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-30 Thread kumarvishal09
Github user kumarvishal09 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214090412
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaIntegralCodec.java
 ---
@@ -88,7 +97,32 @@ public ColumnPageEncoder createEncoder(Map parameter) {
 encodedPage = ColumnPage.newPage(input.getColumnSpec(), 
targetDataType,
 input.getPageSize());
 input.convertValue(converter);
-byte[] result = encodedPage.compress(compressor);
+result = encodedPage.compress(compressor);
+if (isInvertedIndex) {
--- End diff --

can u please move this code to some utility and as below classes also has 
same logic


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-30 Thread kumarvishal09
Github user kumarvishal09 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214089736
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveCodec.java
 ---
@@ -38,17 +40,56 @@
   // the data type specified in schema
   protected final DataType srcDataType;
 
+  protected boolean isSort;
+  protected boolean isInvertedIndex;
+
   protected AdaptiveCodec(DataType srcDataType, DataType targetDataType,
-  SimpleStatsResult stats) {
+  SimpleStatsResult stats, boolean isSort, boolean isInvertedIndex) {
 this.stats = stats;
 this.srcDataType = srcDataType;
 this.targetDataType = targetDataType;
+this.isSort = isSort;
+this.isInvertedIndex = isInvertedIndex;
   }
 
   public DataType getTargetDataType() {
 return targetDataType;
   }
 
+  public Object[] getPageBasedOnDataType(ColumnPage input) {
--- End diff --

what is the purpose of this method??


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-30 Thread kumarvishal09
Github user kumarvishal09 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214088957
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java
 ---
@@ -81,25 +81,29 @@ public ColumnPageDecoder createDecoder(List 
encodings, List

[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-30 Thread kumarvishal09
Github user kumarvishal09 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214087899
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java
 ---
@@ -73,11 +113,16 @@ public int fillVector(int[] filteredRowId, 
ColumnVectorInfo[] vectorInfo, int ch
 if (null != localDictionary) {
   return localDictionary
   
.getDictionaryValue(CarbonUtil.getSurrogateInternal(columnPage.getBytes(rowId), 
0, 3));
-} else if (columnType == ColumnType.COMPLEX_PRIMITIVE && 
this.isAdaptiveComplexPrimitive()) {
-  if (columnPage.getNullBits().get(rowId)) {
+} else if (columnType == ColumnType.COMPLEX_PRIMITIVE
--- End diff --

In case of explict sorted data will be return based on invertedIndexReverse 
can u please check the same, Please refer 
UnsafeFixedLengthDimensionDataChunkStore.java getRowMethod


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-30 Thread kumarvishal09
Github user kumarvishal09 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214086463
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/datastore/columnar/BlockIndexerStorageForNoDictionary.java
 ---
@@ -0,0 +1,157 @@
+/*
+ * 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.columnar;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+
+public class BlockIndexerStorageForNoDictionary {
+
+  private short[] rowIdPage;
+
+  private short[] rowIdRlePage;
+
+  private DataType dataType;
+
+  public BlockIndexerStorageForNoDictionary(Object[] dataPage, DataType 
dataType,
+  boolean isNoDictionary, boolean isSortRequired) {
+this.dataType = dataType;
+ColumnWithRowIdForNoDictionary[] dataWithRowId =
+createColumnWithRowId(dataPage, isNoDictionary);
+if (isSortRequired) {
+  Arrays.sort(dataWithRowId);
+}
+short[] rowIds = extractDataAndReturnRowId(dataWithRowId, dataPage);
+rleEncodeOnRowId(rowIds);
+  }
+
+  /**
+   * Create an object with each column array and respective rowId
+   *
+   * @return
+   */
+  private ColumnWithRowIdForNoDictionary[] 
createColumnWithRowId(Object[] dataPage,
+  boolean isNoDictionary) {
+ColumnWithRowIdForNoDictionary[] columnWithIndexs =
+new ColumnWithRowIdForNoDictionary[dataPage.length];
+if (isNoDictionary) {
+  for (short i = 0; i < columnWithIndexs.length; i++) {
+columnWithIndexs[i] = new 
ColumnWithRowIdForNoDictionary<>(dataPage[i], i, dataType);
+  }
+}
+return columnWithIndexs;
+  }
+
+  private short[] 
extractDataAndReturnRowId(ColumnWithRowIdForNoDictionary[] dataWithRowId,
+  Object[] dataPage) {
+short[] indexes = new short[dataWithRowId.length];
+for (int i = 0; i < indexes.length; i++) {
+  indexes[i] = dataWithRowId[i].getIndex();
+  dataPage[i] = dataWithRowId[i].getColumn();
+}
+return indexes;
+  }
+
+  /**
+   * It compresses depends up on the sequence numbers.
+   * [1,2,3,4,6,8,10,11,12,13] is translated to [1,4,6,8,10,13] and [0,6]. 
In
+   * first array the start and end of sequential numbers and second array
+   * keeps the indexes of where sequential numbers starts. If there is no
+   * sequential numbers then the same array it returns with empty second
+   * array.
+   *
+   * @param rowIds
+   */
+  private void rleEncodeOnRowId(short[] rowIds) {
--- End diff --

This method is common for BlockIndexStorageForShot please move it to some 
util class


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-30 Thread kumarvishal09
Github user kumarvishal09 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214085963
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/datastore/columnar/BlockIndexerStorageForNoDictionary.java
 ---
@@ -0,0 +1,157 @@
+/*
+ * 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.columnar;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+
+public class BlockIndexerStorageForNoDictionary {
+
+  private short[] rowIdPage;
+
+  private short[] rowIdRlePage;
+
+  private DataType dataType;
+
+  public BlockIndexerStorageForNoDictionary(Object[] dataPage, DataType 
dataType,
+  boolean isNoDictionary, boolean isSortRequired) {
--- End diff --

isNoDictionary is not required as it this class will be called for No 
dictionary primitive columns


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-30 Thread manishgupta88
Github user manishgupta88 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214047186
  
--- Diff: 
processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortParameters.java
 ---
@@ -88,6 +88,12 @@
 
   private DataType[] measureDataType;
 
+  private DataType[] noDicDataType;
+
+  private DataType[] noDicSortDataType;
+
+  private DataType[] noDicNoSortDataType;
--- End diff --

write the usage of each type to get a better understanding


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-30 Thread manishgupta88
Github user manishgupta88 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214044420
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java ---
@@ -976,4 +978,122 @@ public static long 
getDataBasedOnRestructuredDataType(byte[] data, DataType rest
 return value;
   }
 
+  /**
+   * Check if the column is a no dictionary primitive column
+   *
+   * @param dataType
+   * @return
+   */
+  public static boolean isPrimitiveColumn(DataType dataType) {
+if (dataType == DataTypes.BOOLEAN || dataType == DataTypes.BYTE || 
dataType == DataTypes.SHORT
+|| dataType == DataTypes.INT || dataType == DataTypes.LONG || 
DataTypes.isDecimal(dataType)
+|| dataType == DataTypes.FLOAT || dataType == DataTypes.DOUBLE
+|| dataType == DataTypes.BYTE_ARRAY) {
+  return true;
+}
+return false;
+  }
+
+  /**
+   * Put the data to unsafe memory
+   *
+   * @param dataType
+   * @param data
+   * @param baseObject
+   * @param address
+   * @param size
+   * @param sizeInBytes
+   */
+  public static void putDataToUnsafe(DataType dataType, Object data, 
Object baseObject,
+  long address, int size, int sizeInBytes) {
+dataType = DataTypeUtil.valueOf(dataType.getName());
+if (dataType == DataTypes.BOOLEAN) {
+  CarbonUnsafe.getUnsafe().putBoolean(baseObject, address + size, 
(boolean) data);
+} else if (dataType == DataTypes.BYTE) {
+  CarbonUnsafe.getUnsafe().putByte(baseObject, address + size, (byte) 
data);
+} else if (dataType == DataTypes.SHORT) {
+  CarbonUnsafe.getUnsafe().putShort(baseObject, address + size, 
(short) data);
+} else if (dataType == DataTypes.INT) {
+  CarbonUnsafe.getUnsafe().putInt(baseObject, address + size, (int) 
data);
+} else if (dataType == DataTypes.LONG) {
+  CarbonUnsafe.getUnsafe().putLong(baseObject, address + size, (long) 
data);
+} else if (DataTypes.isDecimal(dataType) || dataType == 
DataTypes.DOUBLE) {
+  CarbonUnsafe.getUnsafe().putDouble(baseObject, address + size, 
(double) data);
+} else if (dataType == DataTypes.FLOAT) {
+  CarbonUnsafe.getUnsafe().putFloat(baseObject, address + size, 
(float) data);
+} else if (dataType == DataTypes.BYTE_ARRAY) {
+  CarbonUnsafe.getUnsafe()
+  .copyMemory(data, CarbonUnsafe.BYTE_ARRAY_OFFSET, baseObject, 
address + size,
+  sizeInBytes);
+}
+  }
+
+  /**
+   * Retrieve/Get the data from unsafe memory
+   *
+   * @param dataType
+   * @param baseObject
+   * @param address
+   * @param size
+   * @param sizeInBytes
+   * @return
+   */
+  public static Object getDataFromUnsafe(DataType dataType, Object 
baseObject, long address,
+  int size, int sizeInBytes) {
+dataType = DataTypeUtil.valueOf(dataType.getName());
+Object data = new Object();
+if (dataType == DataTypes.BOOLEAN) {
+  data = CarbonUnsafe.getUnsafe().getBoolean(baseObject, address + 
size);
+} else if (dataType == DataTypes.BYTE) {
+  data = CarbonUnsafe.getUnsafe().getByte(baseObject, address + size);
+} else if (dataType == DataTypes.SHORT) {
+  data = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
+} else if (dataType == DataTypes.INT) {
+  data = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
+} else if (dataType == DataTypes.LONG) {
+  data = CarbonUnsafe.getUnsafe().getLong(baseObject, address + size);
+} else if (DataTypes.isDecimal(dataType) || dataType == 
DataTypes.DOUBLE) {
+  data = CarbonUnsafe.getUnsafe().getDouble(baseObject, address + 
size);
+} else if (dataType == DataTypes.FLOAT) {
+  data = CarbonUnsafe.getUnsafe().getDouble(baseObject, address + 
size);
+} else if (dataType == DataTypes.BYTE_ARRAY) {
+  CarbonUnsafe.getUnsafe()
+  .copyMemory(baseObject, address + size, data, 
CarbonUnsafe.BYTE_ARRAY_OFFSET,
+  sizeInBytes);
+}
+return data;
+  }
+
+  /**
+   * Put the data to vector
+   *
+   * @param vector
+   * @param value
+   * @param vectorRow
+   * @param length
+   */
+  public static void putDataToVector(CarbonColumnVector vector, byte[] 
value, int vectorRow,
--- End diff --

Move this method to DimensionVectorDataProcessor.java


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-30 Thread manishgupta88
Github user manishgupta88 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214051061
  
--- Diff: 
processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
 ---
@@ -450,6 +450,114 @@ public static boolean isHeaderValid(String tableName, 
String[] csvHeader,
 return type;
   }
 
+  /**
+   * Get the no dictionary data types on the table
+   *
+   * @param databaseName
+   * @param tableName
+   * @return
+   */
+  public static DataType[] getNoDicDataTypes(String databaseName, String 
tableName) {
+CarbonTable carbonTable = 
CarbonMetadata.getInstance().getCarbonTable(databaseName, tableName);
+List dimensions = 
carbonTable.getDimensionByTableName(tableName);
+int noDicCount = 0;
+for (int i = 0; i < dimensions.size(); i++) {
+  if (!dimensions.get(i).hasEncoding(Encoding.DICTIONARY)) {
+noDicCount++;
+  }
+}
+DataType[] type = new DataType[noDicCount];
+noDicCount = 0;
+for (int i = 0; i < dimensions.size(); i++) {
+  if (!dimensions.get(i).hasEncoding(Encoding.DICTIONARY)) {
+type[noDicCount++] = dimensions.get(i).getDataType();
+  }
+}
+return type;
+  }
+
+  /**
+   * Get the no dictionary sort column mapping of the table
+   *
+   * @param databaseName
+   * @param tableName
+   * @return
+   */
+  public static boolean[] getNoDicSortColMapping(String databaseName, 
String tableName) {
+CarbonTable carbonTable = 
CarbonMetadata.getInstance().getCarbonTable(databaseName, tableName);
+List dimensions = 
carbonTable.getDimensionByTableName(tableName);
+List noDicIndexes = new ArrayList<>(dimensions.size());
+int noDicCount = 0;
+for (int i = 0; i < dimensions.size(); i++) {
+  if (!dimensions.get(i).hasEncoding(Encoding.DICTIONARY)) {
+noDicIndexes.add(i);
+noDicCount++;
+  }
+}
+
+boolean[] noDicSortColMapping = new boolean[noDicCount];
+for (int i = 0; i < noDicSortColMapping.length; i++) {
+  if (dimensions.get(noDicIndexes.get(i)).isSortColumn()) {
+noDicSortColMapping[i] = true;
+  }
+}
+return noDicSortColMapping;
+  }
+
+  /**
+   * Get the data types of the no dictionary sort columns
+   *
+   * @param databaseName
+   * @param tableName
+   * @return
+   */
+  public static DataType[] getNoDicSortDataTypes(String databaseName, 
String tableName) {
+CarbonTable carbonTable = 
CarbonMetadata.getInstance().getCarbonTable(databaseName, tableName);
+List dimensions = 
carbonTable.getDimensionByTableName(tableName);
+int noDicSortCount = 0;
+for (int i = 0; i < dimensions.size(); i++) {
+  if (!dimensions.get(i).hasEncoding(Encoding.DICTIONARY) && 
dimensions.get(i).isSortColumn()) {
+noDicSortCount++;
+  }
+}
+DataType[] type = new DataType[noDicSortCount];
+noDicSortCount = 0;
+for (int i = 0; i < dimensions.size(); i++) {
+  if (!dimensions.get(i).hasEncoding(Encoding.DICTIONARY) && 
dimensions.get(i).isSortColumn()) {
+type[noDicSortCount++] = dimensions.get(i).getDataType();
+  }
+}
+return type;
+  }
+
+  /**
+   * Get the data types of the no dictionary no sort columns
+   *
+   * @param databaseName
+   * @param tableName
+   * @return
+   */
+  public static DataType[] getNoDicNoSortDataTypes(String databaseName, 
String tableName) {
+CarbonTable carbonTable = 
CarbonMetadata.getInstance().getCarbonTable(databaseName, tableName);
+List dimensions = 
carbonTable.getDimensionByTableName(tableName);
+int noDicNoSortCount = 0;
+for (int i = 0; i < dimensions.size(); i++) {
+  if (!dimensions.get(i).hasEncoding(Encoding.DICTIONARY) && 
!dimensions.get(i)
+  .isSortColumn()) {
+noDicNoSortCount++;
+  }
+}
+DataType[] type = new DataType[noDicNoSortCount];
+noDicNoSortCount = 0;
+for (int i = 0; i < dimensions.size(); i++) {
+  if (!dimensions.get(i).hasEncoding(Encoding.DICTIONARY) && 
!dimensions.get(i)
+  .isSortColumn()) {
+type[noDicNoSortCount++] = dimensions.get(i).getDataType();
+  }
+}
+return type;
--- End diff --

same comment as above


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-30 Thread manishgupta88
Github user manishgupta88 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214049718
  
--- Diff: 
processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
 ---
@@ -450,6 +450,114 @@ public static boolean isHeaderValid(String tableName, 
String[] csvHeader,
 return type;
   }
 
+  /**
+   * Get the no dictionary data types on the table
+   *
+   * @param databaseName
+   * @param tableName
+   * @return
+   */
+  public static DataType[] getNoDicDataTypes(String databaseName, String 
tableName) {
+CarbonTable carbonTable = 
CarbonMetadata.getInstance().getCarbonTable(databaseName, tableName);
+List dimensions = 
carbonTable.getDimensionByTableName(tableName);
+int noDicCount = 0;
+for (int i = 0; i < dimensions.size(); i++) {
+  if (!dimensions.get(i).hasEncoding(Encoding.DICTIONARY)) {
+noDicCount++;
+  }
+}
+DataType[] type = new DataType[noDicCount];
+noDicCount = 0;
+for (int i = 0; i < dimensions.size(); i++) {
+  if (!dimensions.get(i).hasEncoding(Encoding.DICTIONARY)) {
+type[noDicCount++] = dimensions.get(i).getDataType();
+  }
+}
+return type;
+  }
+
+  /**
+   * Get the no dictionary sort column mapping of the table
+   *
+   * @param databaseName
+   * @param tableName
+   * @return
+   */
+  public static boolean[] getNoDicSortColMapping(String databaseName, 
String tableName) {
+CarbonTable carbonTable = 
CarbonMetadata.getInstance().getCarbonTable(databaseName, tableName);
+List dimensions = 
carbonTable.getDimensionByTableName(tableName);
+List noDicIndexes = new ArrayList<>(dimensions.size());
+int noDicCount = 0;
+for (int i = 0; i < dimensions.size(); i++) {
+  if (!dimensions.get(i).hasEncoding(Encoding.DICTIONARY)) {
+noDicIndexes.add(i);
+noDicCount++;
+  }
+}
+
+boolean[] noDicSortColMapping = new boolean[noDicCount];
+for (int i = 0; i < noDicSortColMapping.length; i++) {
+  if (dimensions.get(noDicIndexes.get(i)).isSortColumn()) {
+noDicSortColMapping[i] = true;
+  }
+}
+return noDicSortColMapping;
+  }
+
+  /**
+   * Get the data types of the no dictionary sort columns
+   *
+   * @param databaseName
+   * @param tableName
+   * @return
+   */
+  public static DataType[] getNoDicSortDataTypes(String databaseName, 
String tableName) {
+CarbonTable carbonTable = 
CarbonMetadata.getInstance().getCarbonTable(databaseName, tableName);
+List dimensions = 
carbonTable.getDimensionByTableName(tableName);
+int noDicSortCount = 0;
+for (int i = 0; i < dimensions.size(); i++) {
+  if (!dimensions.get(i).hasEncoding(Encoding.DICTIONARY) && 
dimensions.get(i).isSortColumn()) {
+noDicSortCount++;
+  }
+}
+DataType[] type = new DataType[noDicSortCount];
+noDicSortCount = 0;
+for (int i = 0; i < dimensions.size(); i++) {
+  if (!dimensions.get(i).hasEncoding(Encoding.DICTIONARY) && 
dimensions.get(i).isSortColumn()) {
+type[noDicSortCount++] = dimensions.get(i).getDataType();
+  }
--- End diff --

same comment as above. Check on the callers and merge all 3 methods into 
one if possible


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-30 Thread manishgupta88
Github user manishgupta88 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214055038
  
--- Diff: 
processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
 ---
@@ -320,12 +325,22 @@ public static CarbonFactDataHandlerModel 
getCarbonFactDataHandlerModel(CarbonLoa
 List allDimensions = carbonTable.getDimensions();
 int dictDimCount = allDimensions.size() - 
segmentProperties.getNumberOfNoDictionaryDimension()
 - segmentProperties.getComplexDimensions().size();
+CarbonColumn[] noDicAndComplexColumns =
+new 
CarbonColumn[segmentProperties.getNumberOfNoDictionaryDimension() + 
segmentProperties
+.getComplexDimensions().size()];
+int noDic = 0;
--- End diff --

Remove this unused variable


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-30 Thread manishgupta88
Github user manishgupta88 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214038515
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java ---
@@ -976,4 +978,122 @@ public static long 
getDataBasedOnRestructuredDataType(byte[] data, DataType rest
 return value;
   }
 
+  /**
+   * Check if the column is a no dictionary primitive column
+   *
+   * @param dataType
+   * @return
+   */
+  public static boolean isPrimitiveColumn(DataType dataType) {
+if (dataType == DataTypes.BOOLEAN || dataType == DataTypes.BYTE || 
dataType == DataTypes.SHORT
+|| dataType == DataTypes.INT || dataType == DataTypes.LONG || 
DataTypes.isDecimal(dataType)
+|| dataType == DataTypes.FLOAT || dataType == DataTypes.DOUBLE
+|| dataType == DataTypes.BYTE_ARRAY) {
+  return true;
+}
+return false;
+  }
+
+  /**
+   * Put the data to unsafe memory
+   *
+   * @param dataType
+   * @param data
+   * @param baseObject
+   * @param address
+   * @param size
+   * @param sizeInBytes
+   */
+  public static void putDataToUnsafe(DataType dataType, Object data, 
Object baseObject,
--- End diff --

Create a new class CarbonUnsafeUtil and move this method there


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-30 Thread manishgupta88
Github user manishgupta88 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214038624
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java ---
@@ -976,4 +978,122 @@ public static long 
getDataBasedOnRestructuredDataType(byte[] data, DataType rest
 return value;
   }
 
+  /**
+   * Check if the column is a no dictionary primitive column
+   *
+   * @param dataType
+   * @return
+   */
+  public static boolean isPrimitiveColumn(DataType dataType) {
+if (dataType == DataTypes.BOOLEAN || dataType == DataTypes.BYTE || 
dataType == DataTypes.SHORT
+|| dataType == DataTypes.INT || dataType == DataTypes.LONG || 
DataTypes.isDecimal(dataType)
+|| dataType == DataTypes.FLOAT || dataType == DataTypes.DOUBLE
+|| dataType == DataTypes.BYTE_ARRAY) {
+  return true;
+}
+return false;
+  }
+
+  /**
+   * Put the data to unsafe memory
+   *
+   * @param dataType
+   * @param data
+   * @param baseObject
+   * @param address
+   * @param size
+   * @param sizeInBytes
+   */
+  public static void putDataToUnsafe(DataType dataType, Object data, 
Object baseObject,
+  long address, int size, int sizeInBytes) {
+dataType = DataTypeUtil.valueOf(dataType.getName());
+if (dataType == DataTypes.BOOLEAN) {
+  CarbonUnsafe.getUnsafe().putBoolean(baseObject, address + size, 
(boolean) data);
+} else if (dataType == DataTypes.BYTE) {
+  CarbonUnsafe.getUnsafe().putByte(baseObject, address + size, (byte) 
data);
+} else if (dataType == DataTypes.SHORT) {
+  CarbonUnsafe.getUnsafe().putShort(baseObject, address + size, 
(short) data);
+} else if (dataType == DataTypes.INT) {
+  CarbonUnsafe.getUnsafe().putInt(baseObject, address + size, (int) 
data);
+} else if (dataType == DataTypes.LONG) {
+  CarbonUnsafe.getUnsafe().putLong(baseObject, address + size, (long) 
data);
+} else if (DataTypes.isDecimal(dataType) || dataType == 
DataTypes.DOUBLE) {
+  CarbonUnsafe.getUnsafe().putDouble(baseObject, address + size, 
(double) data);
+} else if (dataType == DataTypes.FLOAT) {
+  CarbonUnsafe.getUnsafe().putFloat(baseObject, address + size, 
(float) data);
+} else if (dataType == DataTypes.BYTE_ARRAY) {
+  CarbonUnsafe.getUnsafe()
+  .copyMemory(data, CarbonUnsafe.BYTE_ARRAY_OFFSET, baseObject, 
address + size,
+  sizeInBytes);
+}
+  }
+
+  /**
+   * Retrieve/Get the data from unsafe memory
+   *
+   * @param dataType
+   * @param baseObject
+   * @param address
+   * @param size
+   * @param sizeInBytes
+   * @return
+   */
+  public static Object getDataFromUnsafe(DataType dataType, Object 
baseObject, long address,
--- End diff --

same comment as above


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-30 Thread manishgupta88
Github user manishgupta88 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214049002
  
--- Diff: 
processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
 ---
@@ -450,6 +450,114 @@ public static boolean isHeaderValid(String tableName, 
String[] csvHeader,
 return type;
   }
 
+  /**
+   * Get the no dictionary data types on the table
+   *
+   * @param databaseName
+   * @param tableName
+   * @return
+   */
+  public static DataType[] getNoDicDataTypes(String databaseName, String 
tableName) {
+CarbonTable carbonTable = 
CarbonMetadata.getInstance().getCarbonTable(databaseName, tableName);
+List dimensions = 
carbonTable.getDimensionByTableName(tableName);
+int noDicCount = 0;
+for (int i = 0; i < dimensions.size(); i++) {
+  if (!dimensions.get(i).hasEncoding(Encoding.DICTIONARY)) {
+noDicCount++;
+  }
+}
+DataType[] type = new DataType[noDicCount];
+noDicCount = 0;
+for (int i = 0; i < dimensions.size(); i++) {
+  if (!dimensions.get(i).hasEncoding(Encoding.DICTIONARY)) {
+type[noDicCount++] = dimensions.get(i).getDataType();
+  }
+}
--- End diff --

The below for loop can be removed and the filling of data type can be done 
in single iteration by taking an arraylist and while sending back you can 
convert back to array


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-30 Thread manishgupta88
Github user manishgupta88 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214049631
  
--- Diff: 
processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
 ---
@@ -450,6 +450,114 @@ public static boolean isHeaderValid(String tableName, 
String[] csvHeader,
 return type;
   }
 
+  /**
+   * Get the no dictionary data types on the table
+   *
+   * @param databaseName
+   * @param tableName
+   * @return
+   */
+  public static DataType[] getNoDicDataTypes(String databaseName, String 
tableName) {
+CarbonTable carbonTable = 
CarbonMetadata.getInstance().getCarbonTable(databaseName, tableName);
+List dimensions = 
carbonTable.getDimensionByTableName(tableName);
+int noDicCount = 0;
+for (int i = 0; i < dimensions.size(); i++) {
+  if (!dimensions.get(i).hasEncoding(Encoding.DICTIONARY)) {
+noDicCount++;
+  }
+}
+DataType[] type = new DataType[noDicCount];
+noDicCount = 0;
+for (int i = 0; i < dimensions.size(); i++) {
+  if (!dimensions.get(i).hasEncoding(Encoding.DICTIONARY)) {
+type[noDicCount++] = dimensions.get(i).getDataType();
+  }
+}
+return type;
+  }
+
+  /**
+   * Get the no dictionary sort column mapping of the table
+   *
+   * @param databaseName
+   * @param tableName
+   * @return
+   */
+  public static boolean[] getNoDicSortColMapping(String databaseName, 
String tableName) {
+CarbonTable carbonTable = 
CarbonMetadata.getInstance().getCarbonTable(databaseName, tableName);
+List dimensions = 
carbonTable.getDimensionByTableName(tableName);
+List noDicIndexes = new ArrayList<>(dimensions.size());
+int noDicCount = 0;
+for (int i = 0; i < dimensions.size(); i++) {
+  if (!dimensions.get(i).hasEncoding(Encoding.DICTIONARY)) {
+noDicIndexes.add(i);
+noDicCount++;
+  }
+}
+
+boolean[] noDicSortColMapping = new boolean[noDicCount];
+for (int i = 0; i < noDicSortColMapping.length; i++) {
+  if (dimensions.get(noDicIndexes.get(i)).isSortColumn()) {
+noDicSortColMapping[i] = true;
+  }
--- End diff --

same comment as above


---


[GitHub] carbondata pull request #2654: [CARBONDATA-2896] Adaptive Encoding for Primi...

2018-08-30 Thread manishgupta88
Github user manishgupta88 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/2654#discussion_r214055654
  
--- Diff: 
processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
 ---
@@ -320,12 +325,22 @@ public static CarbonFactDataHandlerModel 
getCarbonFactDataHandlerModel(CarbonLoa
 List allDimensions = carbonTable.getDimensions();
 int dictDimCount = allDimensions.size() - 
segmentProperties.getNumberOfNoDictionaryDimension()
 - segmentProperties.getComplexDimensions().size();
+CarbonColumn[] noDicAndComplexColumns =
+new 
CarbonColumn[segmentProperties.getNumberOfNoDictionaryDimension() + 
segmentProperties
+.getComplexDimensions().size()];
+int noDic = 0;
+int noDicAndComp = 0;
 for (CarbonDimension dim : allDimensions) {
   if (!dim.isComplex() && !dim.hasEncoding(Encoding.DICTIONARY) &&
   dim.getDataType() == DataTypes.VARCHAR) {
 // ordinal is set in 
CarbonTable.fillDimensionsAndMeasuresForTables()
 varcharDimIdxInNoDict.add(dim.getOrdinal() - dictDimCount);
   }
+  if (!dim.hasEncoding(Encoding.DICTIONARY) || dim.isComplex() || null 
!= dim
+  .getComplexParentDimension()) {
+noDicAndComplexColumns[noDicAndComp++] =
--- End diff --

Modify the if loop check to check only for dictionary encoding
if (!dim.hasEncoding(Encoding.DICTIONARY))


---