Repository: carbondata Updated Branches: refs/heads/master c8f706304 -> 077dd58e3
[CARBONDATA-2818] upgrade presto version from 0.187 to 0.208 upgrade presto version from 0.187 to 0.208 This closes #2607 Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/077dd58e Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/077dd58e Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/077dd58e Branch: refs/heads/master Commit: 077dd58e3b84a84cea23b97bae937f13c5e8fa2f Parents: c8f7063 Author: Bhavya <[email protected]> Authored: Fri Aug 3 13:40:14 2018 +0530 Committer: chenliang613 <[email protected]> Committed: Tue Sep 18 22:56:29 2018 +0800 ---------------------------------------------------------------------- integration/presto/pom.xml | 2 +- .../carbondata/presto/CarbonVectorBatch.java | 8 +-- .../presto/CarbondataSplitManager.java | 3 +- .../presto/readers/BooleanStreamReader.java | 5 +- .../readers/DecimalSliceStreamReader.java | 7 ++- .../presto/readers/DoubleStreamReader.java | 5 +- .../presto/readers/IntegerStreamReader.java | 5 +- .../presto/readers/LongStreamReader.java | 5 +- .../presto/readers/ObjectStreamReader.java | 5 +- .../presto/readers/ShortStreamReader.java | 5 +- .../presto/readers/SliceStreamReader.java | 22 +++---- .../presto/readers/TimestampStreamReader.java | 5 +- .../CarbonDictionaryDecodeReadSupport.scala | 62 +++++++++++++------- 13 files changed, 76 insertions(+), 63 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/carbondata/blob/077dd58e/integration/presto/pom.xml ---------------------------------------------------------------------- diff --git a/integration/presto/pom.xml b/integration/presto/pom.xml index bf8c64d..9a9bc55 100644 --- a/integration/presto/pom.xml +++ b/integration/presto/pom.xml @@ -31,7 +31,7 @@ <packaging>presto-plugin</packaging> <properties> - <presto.version>0.187</presto.version> + <presto.version>0.208</presto.version> <dev.path>${basedir}/../../dev</dev.path> <jacoco.append>true</jacoco.append> </properties> http://git-wip-us.apache.org/repos/asf/carbondata/blob/077dd58e/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorBatch.java ---------------------------------------------------------------------- diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorBatch.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorBatch.java index 6a4cc0d..2a0ab75 100644 --- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorBatch.java +++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorBatch.java @@ -36,7 +36,7 @@ import org.apache.carbondata.presto.readers.ShortStreamReader; import org.apache.carbondata.presto.readers.SliceStreamReader; import org.apache.carbondata.presto.readers.TimestampStreamReader; -import com.facebook.presto.spi.block.SliceArrayBlock; +import com.facebook.presto.spi.block.Block; public class CarbonVectorBatch { @@ -63,7 +63,7 @@ public class CarbonVectorBatch { for (int i = 0; i < schema.length; ++i) { columns[i] = createDirectStreamReader(maxRows, dataTypes[i], schema[i], dictionaries[i], - readSupport.getSliceArrayBlock(i)); + readSupport.getDictionaryBlock(i)); } } @@ -73,7 +73,7 @@ public class CarbonVectorBatch { } private CarbonColumnVectorImpl createDirectStreamReader(int batchSize, DataType dataType, - StructField field, Dictionary dictionary, SliceArrayBlock dictionarySliceArrayBlock) { + StructField field, Dictionary dictionary, Block dictionaryBlock) { if (dataType == DataTypes.BOOLEAN) { return new BooleanStreamReader(batchSize, field.getDataType(), dictionary); } else if (dataType == DataTypes.SHORT) { @@ -87,7 +87,7 @@ public class CarbonVectorBatch { } else if (dataType == DataTypes.DOUBLE) { return new DoubleStreamReader(batchSize, field.getDataType(), dictionary); } else if (dataType == DataTypes.STRING) { - return new SliceStreamReader(batchSize, field.getDataType(), dictionarySliceArrayBlock); + return new SliceStreamReader(batchSize, field.getDataType(), dictionaryBlock); } else if (DataTypes.isDecimal(dataType)) { return new DecimalSliceStreamReader(batchSize, (DecimalType) field.getDataType(), dictionary); } else { http://git-wip-us.apache.org/repos/asf/carbondata/blob/077dd58e/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataSplitManager.java ---------------------------------------------------------------------- diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataSplitManager.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataSplitManager.java index e8e1f03..eeb16f7 100755 --- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataSplitManager.java +++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataSplitManager.java @@ -64,7 +64,8 @@ public class CarbondataSplitManager implements ConnectorSplitManager { } public ConnectorSplitSource getSplits(ConnectorTransactionHandle transactionHandle, - ConnectorSession session, ConnectorTableLayoutHandle layout) { + ConnectorSession session, ConnectorTableLayoutHandle layout, + SplitSchedulingStrategy splitSchedulingStrategy) { CarbondataTableLayoutHandle layoutHandle = (CarbondataTableLayoutHandle) layout; CarbondataTableHandle tableHandle = layoutHandle.getTable(); SchemaTableName key = tableHandle.getSchemaTableName(); http://git-wip-us.apache.org/repos/asf/carbondata/blob/077dd58e/integration/presto/src/main/java/org/apache/carbondata/presto/readers/BooleanStreamReader.java ---------------------------------------------------------------------- diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/readers/BooleanStreamReader.java b/integration/presto/src/main/java/org/apache/carbondata/presto/readers/BooleanStreamReader.java index 17578d7..b523064 100644 --- a/integration/presto/src/main/java/org/apache/carbondata/presto/readers/BooleanStreamReader.java +++ b/integration/presto/src/main/java/org/apache/carbondata/presto/readers/BooleanStreamReader.java @@ -25,7 +25,6 @@ import org.apache.carbondata.core.util.DataTypeUtil; import com.facebook.presto.spi.block.Block; import com.facebook.presto.spi.block.BlockBuilder; -import com.facebook.presto.spi.block.BlockBuilderStatus; import com.facebook.presto.spi.type.BooleanType; import com.facebook.presto.spi.type.Type; @@ -43,7 +42,7 @@ public class BooleanStreamReader extends CarbonColumnVectorImpl public BooleanStreamReader(int batchSize, DataType dataType, Dictionary dictionary) { super(batchSize, dataType); this.batchSize = batchSize; - this.builder = type.createBlockBuilder(new BlockBuilderStatus(), batchSize); + this.builder = type.createBlockBuilder(null, batchSize); this.dictionary = dictionary; } @@ -74,7 +73,7 @@ public class BooleanStreamReader extends CarbonColumnVectorImpl } @Override public void reset() { - builder = type.createBlockBuilder(new BlockBuilderStatus(), batchSize); + builder = type.createBlockBuilder(null, batchSize); } } \ No newline at end of file http://git-wip-us.apache.org/repos/asf/carbondata/blob/077dd58e/integration/presto/src/main/java/org/apache/carbondata/presto/readers/DecimalSliceStreamReader.java ---------------------------------------------------------------------- diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/readers/DecimalSliceStreamReader.java b/integration/presto/src/main/java/org/apache/carbondata/presto/readers/DecimalSliceStreamReader.java index 6860ce0..7bbf1ca 100644 --- a/integration/presto/src/main/java/org/apache/carbondata/presto/readers/DecimalSliceStreamReader.java +++ b/integration/presto/src/main/java/org/apache/carbondata/presto/readers/DecimalSliceStreamReader.java @@ -30,7 +30,6 @@ import org.apache.carbondata.core.util.DataTypeUtil; import com.facebook.presto.spi.block.Block; import com.facebook.presto.spi.block.BlockBuilder; -import com.facebook.presto.spi.block.BlockBuilderStatus; import com.facebook.presto.spi.type.DecimalType; import com.facebook.presto.spi.type.Decimals; import com.facebook.presto.spi.type.Type; @@ -44,6 +43,8 @@ import static com.google.common.base.Preconditions.checkState; import static io.airlift.slice.Slices.utf8Slice; + + /** * Reader for DecimalValues */ @@ -61,7 +62,7 @@ public class DecimalSliceStreamReader extends CarbonColumnVectorImpl super(batchSize, dataType); this.type = DecimalType.createDecimalType(dataType.getPrecision(), dataType.getScale()); this.batchSize = batchSize; - this.builder = type.createBlockBuilder(new BlockBuilderStatus(), batchSize); + this.builder = type.createBlockBuilder(null, batchSize); this.dictionary = dictionary; } @@ -93,7 +94,7 @@ public class DecimalSliceStreamReader extends CarbonColumnVectorImpl } @Override public void reset() { - builder = type.createBlockBuilder(new BlockBuilderStatus(), batchSize); + builder = type.createBlockBuilder(null, batchSize); } private void decimalBlockWriter(BigDecimal value) { http://git-wip-us.apache.org/repos/asf/carbondata/blob/077dd58e/integration/presto/src/main/java/org/apache/carbondata/presto/readers/DoubleStreamReader.java ---------------------------------------------------------------------- diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/readers/DoubleStreamReader.java b/integration/presto/src/main/java/org/apache/carbondata/presto/readers/DoubleStreamReader.java index 384112f..563f1b7 100644 --- a/integration/presto/src/main/java/org/apache/carbondata/presto/readers/DoubleStreamReader.java +++ b/integration/presto/src/main/java/org/apache/carbondata/presto/readers/DoubleStreamReader.java @@ -25,7 +25,6 @@ import org.apache.carbondata.core.util.DataTypeUtil; import com.facebook.presto.spi.block.Block; import com.facebook.presto.spi.block.BlockBuilder; -import com.facebook.presto.spi.block.BlockBuilderStatus; import com.facebook.presto.spi.type.DoubleType; import com.facebook.presto.spi.type.Type; @@ -45,7 +44,7 @@ public class DoubleStreamReader extends CarbonColumnVectorImpl implements Presto public DoubleStreamReader(int batchSize, DataType dataType, Dictionary dictionary) { super(batchSize, dataType); this.batchSize = batchSize; - this.builder = type.createBlockBuilder(new BlockBuilderStatus(), batchSize); + this.builder = type.createBlockBuilder(null, batchSize); this.dictionary = dictionary; } @@ -76,6 +75,6 @@ public class DoubleStreamReader extends CarbonColumnVectorImpl implements Presto } @Override public void reset() { - builder = type.createBlockBuilder(new BlockBuilderStatus(), batchSize); + builder = type.createBlockBuilder(null, batchSize); } } http://git-wip-us.apache.org/repos/asf/carbondata/blob/077dd58e/integration/presto/src/main/java/org/apache/carbondata/presto/readers/IntegerStreamReader.java ---------------------------------------------------------------------- diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/readers/IntegerStreamReader.java b/integration/presto/src/main/java/org/apache/carbondata/presto/readers/IntegerStreamReader.java index a3ce908..6e15da6 100644 --- a/integration/presto/src/main/java/org/apache/carbondata/presto/readers/IntegerStreamReader.java +++ b/integration/presto/src/main/java/org/apache/carbondata/presto/readers/IntegerStreamReader.java @@ -25,7 +25,6 @@ import org.apache.carbondata.core.util.DataTypeUtil; import com.facebook.presto.spi.block.Block; import com.facebook.presto.spi.block.BlockBuilder; -import com.facebook.presto.spi.block.BlockBuilderStatus; import com.facebook.presto.spi.type.IntegerType; import com.facebook.presto.spi.type.Type; @@ -43,7 +42,7 @@ public class IntegerStreamReader extends CarbonColumnVectorImpl public IntegerStreamReader(int batchSize, DataType dataType, Dictionary dictionary) { super(batchSize, dataType); this.batchSize = batchSize; - this.builder = type.createBlockBuilder(new BlockBuilderStatus(), batchSize); + this.builder = type.createBlockBuilder(null, batchSize); this.dictionary = dictionary; } @@ -74,7 +73,7 @@ public class IntegerStreamReader extends CarbonColumnVectorImpl } @Override public void reset() { - builder = type.createBlockBuilder(new BlockBuilderStatus(), batchSize); + builder = type.createBlockBuilder(null, batchSize); } } http://git-wip-us.apache.org/repos/asf/carbondata/blob/077dd58e/integration/presto/src/main/java/org/apache/carbondata/presto/readers/LongStreamReader.java ---------------------------------------------------------------------- diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/readers/LongStreamReader.java b/integration/presto/src/main/java/org/apache/carbondata/presto/readers/LongStreamReader.java index 892614d..494344c 100644 --- a/integration/presto/src/main/java/org/apache/carbondata/presto/readers/LongStreamReader.java +++ b/integration/presto/src/main/java/org/apache/carbondata/presto/readers/LongStreamReader.java @@ -25,7 +25,6 @@ import org.apache.carbondata.core.util.DataTypeUtil; import com.facebook.presto.spi.block.Block; import com.facebook.presto.spi.block.BlockBuilder; -import com.facebook.presto.spi.block.BlockBuilderStatus; import com.facebook.presto.spi.type.BigintType; import com.facebook.presto.spi.type.Type; @@ -42,7 +41,7 @@ public class LongStreamReader extends CarbonColumnVectorImpl implements PrestoVe public LongStreamReader(int batchSize, DataType dataType, Dictionary dictionary) { super(batchSize, dataType); this.batchSize = batchSize; - this.builder = type.createBlockBuilder(new BlockBuilderStatus(), batchSize); + this.builder = type.createBlockBuilder(null, batchSize); this.dictionary = dictionary; } @@ -73,6 +72,6 @@ public class LongStreamReader extends CarbonColumnVectorImpl implements PrestoVe } @Override public void reset() { - builder = type.createBlockBuilder(new BlockBuilderStatus(), batchSize); + builder = type.createBlockBuilder(null, batchSize); } } http://git-wip-us.apache.org/repos/asf/carbondata/blob/077dd58e/integration/presto/src/main/java/org/apache/carbondata/presto/readers/ObjectStreamReader.java ---------------------------------------------------------------------- diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/readers/ObjectStreamReader.java b/integration/presto/src/main/java/org/apache/carbondata/presto/readers/ObjectStreamReader.java index e4c9775..cdba4bf 100644 --- a/integration/presto/src/main/java/org/apache/carbondata/presto/readers/ObjectStreamReader.java +++ b/integration/presto/src/main/java/org/apache/carbondata/presto/readers/ObjectStreamReader.java @@ -22,7 +22,6 @@ import org.apache.carbondata.core.scan.result.vector.impl.CarbonColumnVectorImpl import com.facebook.presto.spi.block.Block; import com.facebook.presto.spi.block.BlockBuilder; -import com.facebook.presto.spi.block.BlockBuilderStatus; import com.facebook.presto.spi.type.IntegerType; import com.facebook.presto.spi.type.Type; @@ -40,7 +39,7 @@ public class ObjectStreamReader extends CarbonColumnVectorImpl implements Presto public ObjectStreamReader(int batchSize, DataType dataType) { super(batchSize, dataType); this.batchSize = batchSize; - this.builder = type.createBlockBuilder(new BlockBuilderStatus(), batchSize); + this.builder = type.createBlockBuilder(null, batchSize); } @Override public Block buildBlock() { @@ -60,7 +59,7 @@ public class ObjectStreamReader extends CarbonColumnVectorImpl implements Presto } @Override public void reset() { - builder = type.createBlockBuilder(new BlockBuilderStatus(), batchSize); + builder = type.createBlockBuilder(null, batchSize); } } http://git-wip-us.apache.org/repos/asf/carbondata/blob/077dd58e/integration/presto/src/main/java/org/apache/carbondata/presto/readers/ShortStreamReader.java ---------------------------------------------------------------------- diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/readers/ShortStreamReader.java b/integration/presto/src/main/java/org/apache/carbondata/presto/readers/ShortStreamReader.java index d207fd9..198f82a 100644 --- a/integration/presto/src/main/java/org/apache/carbondata/presto/readers/ShortStreamReader.java +++ b/integration/presto/src/main/java/org/apache/carbondata/presto/readers/ShortStreamReader.java @@ -25,7 +25,6 @@ import org.apache.carbondata.core.util.DataTypeUtil; import com.facebook.presto.spi.block.Block; import com.facebook.presto.spi.block.BlockBuilder; -import com.facebook.presto.spi.block.BlockBuilderStatus; import com.facebook.presto.spi.type.SmallintType; import com.facebook.presto.spi.type.Type; @@ -42,7 +41,7 @@ public class ShortStreamReader extends CarbonColumnVectorImpl implements PrestoV public ShortStreamReader(int batchSize, DataType dataType, Dictionary dictionary) { super(batchSize, dataType); this.batchSize = batchSize; - this.builder = type.createBlockBuilder(new BlockBuilderStatus(), batchSize); + this.builder = type.createBlockBuilder(null, batchSize); this.dictionary = dictionary; } @@ -73,6 +72,6 @@ public class ShortStreamReader extends CarbonColumnVectorImpl implements PrestoV } @Override public void reset() { - builder = type.createBlockBuilder(new BlockBuilderStatus(), batchSize); + builder = type.createBlockBuilder(null, batchSize); } } http://git-wip-us.apache.org/repos/asf/carbondata/blob/077dd58e/integration/presto/src/main/java/org/apache/carbondata/presto/readers/SliceStreamReader.java ---------------------------------------------------------------------- diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/readers/SliceStreamReader.java b/integration/presto/src/main/java/org/apache/carbondata/presto/readers/SliceStreamReader.java index 53ece0b..39fd19a 100644 --- a/integration/presto/src/main/java/org/apache/carbondata/presto/readers/SliceStreamReader.java +++ b/integration/presto/src/main/java/org/apache/carbondata/presto/readers/SliceStreamReader.java @@ -22,9 +22,7 @@ import org.apache.carbondata.core.scan.result.vector.impl.CarbonColumnVectorImpl import com.facebook.presto.spi.block.Block; import com.facebook.presto.spi.block.BlockBuilder; -import com.facebook.presto.spi.block.BlockBuilderStatus; import com.facebook.presto.spi.block.DictionaryBlock; -import com.facebook.presto.spi.block.SliceArrayBlock; import com.facebook.presto.spi.type.Type; import com.facebook.presto.spi.type.VarcharType; @@ -40,26 +38,28 @@ public class SliceStreamReader extends CarbonColumnVectorImpl implements PrestoV protected Type type = VarcharType.VARCHAR; protected BlockBuilder builder; + int[] values; - private SliceArrayBlock dictionarySliceArrayBlock; + + private Block dictionaryBlock; public SliceStreamReader(int batchSize, DataType dataType, - SliceArrayBlock dictionarySliceArrayBlock) { + Block dictionaryBlock) { super(batchSize, dataType); this.batchSize = batchSize; - if (dictionarySliceArrayBlock == null) { - this.builder = type.createBlockBuilder(new BlockBuilderStatus(), batchSize); + if (dictionaryBlock == null) { + this.builder = type.createBlockBuilder(null, batchSize); } else { - this.dictionarySliceArrayBlock = dictionarySliceArrayBlock; + this.dictionaryBlock = dictionaryBlock; this.values = new int[batchSize]; } } @Override public Block buildBlock() { - if (dictionarySliceArrayBlock == null) { + if (dictionaryBlock == null) { return builder.build(); } else { - return new DictionaryBlock(batchSize, dictionarySliceArrayBlock, values); + return new DictionaryBlock(batchSize, dictionaryBlock, values); } } @@ -82,12 +82,12 @@ public class SliceStreamReader extends CarbonColumnVectorImpl implements PrestoV } @Override public void putNull(int rowId) { - if (dictionarySliceArrayBlock == null) { + if (dictionaryBlock == null) { builder.appendNull(); } } @Override public void reset() { - builder = type.createBlockBuilder(new BlockBuilderStatus(), batchSize); + builder = type.createBlockBuilder(null, batchSize); } } http://git-wip-us.apache.org/repos/asf/carbondata/blob/077dd58e/integration/presto/src/main/java/org/apache/carbondata/presto/readers/TimestampStreamReader.java ---------------------------------------------------------------------- diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/readers/TimestampStreamReader.java b/integration/presto/src/main/java/org/apache/carbondata/presto/readers/TimestampStreamReader.java index f52916c..43c471a 100644 --- a/integration/presto/src/main/java/org/apache/carbondata/presto/readers/TimestampStreamReader.java +++ b/integration/presto/src/main/java/org/apache/carbondata/presto/readers/TimestampStreamReader.java @@ -25,7 +25,6 @@ import org.apache.carbondata.core.util.DataTypeUtil; import com.facebook.presto.spi.block.Block; import com.facebook.presto.spi.block.BlockBuilder; -import com.facebook.presto.spi.block.BlockBuilderStatus; import com.facebook.presto.spi.type.TimestampType; import com.facebook.presto.spi.type.Type; @@ -43,7 +42,7 @@ public class TimestampStreamReader extends CarbonColumnVectorImpl public TimestampStreamReader(int batchSize, DataType dataType, Dictionary dictionary) { super(batchSize, dataType); this.batchSize = batchSize; - this.builder = type.createBlockBuilder(new BlockBuilderStatus(), batchSize); + this.builder = type.createBlockBuilder(null, batchSize); this.dictionary = dictionary; } @@ -74,6 +73,6 @@ public class TimestampStreamReader extends CarbonColumnVectorImpl } @Override public void reset() { - builder = type.createBlockBuilder(new BlockBuilderStatus(), batchSize); + builder = type.createBlockBuilder(null, batchSize); } } http://git-wip-us.apache.org/repos/asf/carbondata/blob/077dd58e/integration/presto/src/main/scala/org/apache/carbondata/presto/CarbonDictionaryDecodeReadSupport.scala ---------------------------------------------------------------------- diff --git a/integration/presto/src/main/scala/org/apache/carbondata/presto/CarbonDictionaryDecodeReadSupport.scala b/integration/presto/src/main/scala/org/apache/carbondata/presto/CarbonDictionaryDecodeReadSupport.scala index 42d7c93..97deb6f 100644 --- a/integration/presto/src/main/scala/org/apache/carbondata/presto/CarbonDictionaryDecodeReadSupport.scala +++ b/integration/presto/src/main/scala/org/apache/carbondata/presto/CarbonDictionaryDecodeReadSupport.scala @@ -16,8 +16,9 @@ */ package org.apache.carbondata.presto -import com.facebook.presto.spi.block.SliceArrayBlock -import io.airlift.slice.{Slice, Slices} +import java.util.Optional + +import com.facebook.presto.spi.block.{Block, VariableWidthBlock} import io.airlift.slice.Slices._ import org.apache.carbondata.core.cache.{Cache, CacheProvider, CacheType} @@ -36,7 +37,7 @@ import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport class CarbonDictionaryDecodeReadSupport[T] extends CarbonReadSupport[T] { private var dictionaries: Array[Dictionary] = _ private var dataTypes: Array[DataType] = _ - private var dictionarySliceArray: Array[SliceArrayBlock] = _ + private var dictionaryBlock: Array[Block] = _ /** * This initialization is done inside executor task @@ -49,7 +50,7 @@ class CarbonDictionaryDecodeReadSupport[T] extends CarbonReadSupport[T] { dictionaries = new Array[Dictionary](carbonColumns.length) dataTypes = new Array[DataType](carbonColumns.length) - dictionarySliceArray = new Array[SliceArrayBlock](carbonColumns.length) + dictionaryBlock = new Array[Block](carbonColumns.length) carbonColumns.zipWithIndex.foreach { case (carbonColumn, index) => if (carbonColumn.hasEncoding(Encoding.DICTIONARY) && @@ -67,7 +68,7 @@ class CarbonDictionaryDecodeReadSupport[T] extends CarbonReadSupport[T] { carbonColumn.getColumnIdentifier, dataTypes(index), dictionaryPath)) // in case of string data type create dictionarySliceArray same as that of presto code if (dataTypes(index).equals(DataTypes.STRING)) { - dictionarySliceArray(index) = createSliceArrayBlock(dictionaries(index)) + dictionaryBlock(index) = createDictionaryBlock(dictionaries(index)) } } @@ -84,25 +85,42 @@ class CarbonDictionaryDecodeReadSupport[T] extends CarbonReadSupport[T] { * @param dictionaryData * @return */ - private def createSliceArrayBlock(dictionaryData: Dictionary): SliceArrayBlock = { + private def createDictionaryBlock(dictionaryData: Dictionary): Block = { val chunks: DictionaryChunksWrapper = dictionaryData.getDictionaryChunks - val sliceArray = new Array[Slice](chunks.getSize + 1) - // Initialize Slice Array with Empty Slice as per Presto's code - sliceArray(0) = Slices.EMPTY_SLICE - var count = 1 + val positionCount = chunks.getSize; + + // In dictionary there will be only one null and the key value will be 1 by default in carbon, + // hence the isNullVector will be populated only once with null value it has no bearing on + // actual data. + + val offsetVector : Array[Int] = new Array[Int](positionCount + 2 ) + val isNullVector: Array[Boolean] = new Array[Boolean](positionCount + 1) + // the first value is just a filler as we always start with index 1 in carbon + isNullVector(0) = true + isNullVector(1) = true + var count = 0 + var byteArray = new Array[Byte](0) + // The Carbondata key starts from 1 so we need a filler at 0th position hence adding filler to + // offset, hence 0th Position -> 0 + offsetVector(0) = 0 while (chunks.hasNext) { - { - val value: Array[Byte] = chunks.next - if (count == 1) { - sliceArray(count) = null - } - else { - sliceArray(count) = wrappedBuffer(value, 0, value.length) - } - count += 1 + val value: Array[Byte] = chunks.next + if (count == 0) { + // 1 index is actually Null to map to carbondata null values . + // 1st Position -> 0 (For actual Null) + offsetVector(count + 1) = 0 + // 2nd Postion -> 0 as the byte[] is still null so starting point will be 0 only + offsetVector(count + 2) = 0 + } else { + byteArray = byteArray ++ value + offsetVector(count + 2) = byteArray.length } + count += 1 } - new SliceArrayBlock(sliceArray.length, sliceArray, true) + new VariableWidthBlock(positionCount + 1, + wrappedBuffer(byteArray, 0, byteArray.length), + offsetVector, + Optional.ofNullable(isNullVector)) } override def readRow(data: Array[AnyRef]): T = { @@ -115,8 +133,8 @@ class CarbonDictionaryDecodeReadSupport[T] extends CarbonReadSupport[T] { * @param columnNo * @return */ - def getSliceArrayBlock(columnNo: Int): SliceArrayBlock = { - dictionarySliceArray(columnNo) + def getDictionaryBlock(columnNo: Int): Block = { + dictionaryBlock(columnNo) } def getDictionaries: Array[Dictionary] = {
