From d687986a8d15baa12bae62e5aaf8d1290654bb14 Mon Sep 17 00:00:00 2001 From: dhatchayani Date: Wed, 22 Aug 2018 12:45:44 +0530 Subject: [PATCH] [CARBONDATA-2896][Refactor] Adaptive Encoding for Primitive data types Loading configurations and settings (1) Parse data as like that of measure, so change in FieldEncoderFactory to take up measure flow (2) While creating loading configurations, no dictionary, sort columns should be taken care in all the needed flows Sort rows preparation (1) Prepare the row to be sorted with original data for no dictionary columns (2) Use data type based comparators for the no dictionary sort columns in all the flows like Intermediate Sort, Final sort, Unsafe sort (3) Handle read write of row with no dictionary primitive data types to intermediate files and in the final file merger, as we will be reading and writing as original data (4) Get the no dictionary sort data types from the load configurations what we set in LOAD step Adding to Column page and apply adaptive encoding (1) Add the no dictionary primitive datatypes data as original data (2) Apply adaptive encoding to the page (3) Reuse the adaptive encoding techniques existing for measure column Writing inverted index to adaptive encoded page (1) Prepare in the inverted inverted list based on the datatype based comparison (2) Apply RLE on the inverted index (3) Write the inverted index to the encoded page Create decoder while querying (1) Create proper decoder for the no dictionary column pages (2) Uncompress the column page and also the inverted index Filter flow changes (1) FilterValues will be in bytes, so convert the data to bytes for comparison (2) Change the isScanRequired to compare min/max values based on the data type Fill output row in case of queries (1) Change the noDictionaryKeys to Object, now it can be datatypes based data for no dictionary primitive data types Bloom filter changes (1) Change bloom filter load (2) While rebuilding the data map, the load expects the data to original data. Therefore a conversion is used (3) Fill the no dictionary primitive data as original data Compaction Changes Compaction will get the rows from the result collectors. But the result collectors will give bytes as no dictionary columns. So a conversion is needed to convert the bytes to original data based on the data type. --- .../carbondata/core/datastore/TableSpec.java | 17 + ...ressedDimensionChunkFileBasedReaderV3.java | 42 +- .../chunk/store/ColumnPageWrapper.java | 106 ++++- ...VariableLengthDimensionDataChunkStore.java | 28 +- .../columnar/BlockIndexerStorage.java | 104 +++++ .../BlockIndexerStorageForNoDictionary.java | 116 +++++ ...exerStorageForNoInvertedIndexForShort.java | 17 +- .../columnar/BlockIndexerStorageForShort.java | 71 +-- .../ColumnWithRowIdForNoDictionary.java | 72 +++ .../core/datastore/columnar/IndexStorage.java | 35 -- .../page/encoding/ColumnPageEncoder.java | 48 +- .../page/encoding/DefaultEncodingFactory.java | 70 ++- .../page/encoding/EncodingFactory.java | 18 +- .../page/encoding/adaptive/AdaptiveCodec.java | 195 ++++++++- .../adaptive/AdaptiveDeltaFloatingCodec.java | 31 +- .../adaptive/AdaptiveDeltaIntegralCodec.java | 30 +- .../adaptive/AdaptiveFloatingCodec.java | 30 +- .../adaptive/AdaptiveIntegralCodec.java | 30 +- .../legacy/ComplexDimensionIndexCodec.java | 4 +- .../legacy/DictDimensionIndexCodec.java | 4 +- .../legacy/DirectDictDimensionIndexCodec.java | 4 +- .../HighCardDictDimensionIndexCodec.java | 4 +- .../dimension/legacy/IndexStorageEncoder.java | 8 +- .../core/datastore/page/key/TablePageKey.java | 3 +- .../page/statistics/TablePageStatistics.java | 14 +- .../core/datastore/row/WriteStepRowUtil.java | 28 +- .../core/scan/executor/util/QueryUtil.java | 36 ++ .../core/scan/filter/FilterUtil.java | 42 +- .../executer/ExcludeFilterExecuterImpl.java | 2 +- .../executer/IncludeFilterExecuterImpl.java | 58 ++- .../RangeValueFilterExecuterImpl.java | 39 +- .../executer/RestructureEvaluatorImpl.java | 4 +- .../executer/RowLevelFilterExecuterImpl.java | 10 +- ...RowLevelRangeGrtThanFiterExecuterImpl.java | 51 ++- ...RangeGrtrThanEquaToFilterExecuterImpl.java | 53 ++- ...lRangeLessThanEqualFilterExecuterImpl.java | 51 ++- ...wLevelRangeLessThanFilterExecuterImpl.java | 51 ++- .../RowLevelRangeFilterResolverImpl.java | 2 +- .../scan/result/BlockletScannedResult.java | 5 +- .../result/impl/FilterQueryScannedResult.java | 9 +- .../core/util/CarbonUnsafeUtil.java | 95 ++++ .../carbondata/core/util/CarbonUtil.java | 22 + .../carbondata/core/util/DataTypeUtil.java | 58 ++- .../core/util/NonDictionaryUtil.java | 2 +- .../page/encoding/TestEncodingFactory.java | 17 +- .../core/util/DataTypeUtilTest.java | 6 +- .../core/util/RangeFilterProcessorTest.java | 23 + .../bloom/AbstractBloomDataMapWriter.java | 4 +- .../bloom/BloomCoarseGrainDataMap.java | 15 +- .../datamap/bloom/BloomDataMapBuilder.java | 9 +- .../datamap/bloom/BloomDataMapWriter.java | 10 +- .../hadoop/testutil/StoreCreator.java | 15 +- .../load/DataLoadProcessBuilderOnSpark.scala | 4 +- .../datamap/IndexDataMapRebuildRDD.scala | 13 +- .../CarbonGetTableDetailComandTestCase.scala | 6 +- .../loading/CarbonDataLoadConfiguration.java | 40 ++ .../converter/impl/FieldEncoderFactory.java | 6 + .../impl/MeasureFieldConverterImpl.java | 40 +- .../partition/impl/RawRowComparator.java | 30 +- .../loading/row/IntermediateSortTempRow.java | 8 +- .../loading/sort/SortStepRowHandler.java | 411 +++++++++++++----- .../comparator/UnsafeRowComparator.java | 58 ++- .../holder/UnsafeFinalMergePageHolder.java | 7 +- .../unsafe/holder/UnsafeInmemoryHolder.java | 3 +- .../holder/UnsafeSortTempFileChunkHolder.java | 3 +- .../CarbonRowDataWriterProcessorStepImpl.java | 6 +- .../steps/DataConverterProcessorStepImpl.java | 9 +- ...InputProcessorStepWithNoConverterImpl.java | 9 +- .../merger/CompactionResultSortProcessor.java | 30 +- .../merger/RowResultMergerProcessor.java | 6 +- .../partition/spliter/RowResultProcessor.java | 7 +- .../IntermediateSortTempRowComparator.java | 34 +- .../sort/sortdata/NewRowComparator.java | 48 +- .../sort/sortdata/SortDataRows.java | 7 +- .../sort/sortdata/SortParameters.java | 94 +++- .../sortdata/SortTempFileChunkHolder.java | 2 +- .../sort/sortdata/TableFieldStat.java | 34 +- .../store/CarbonFactDataHandlerColumnar.java | 13 +- .../store/CarbonFactDataHandlerModel.java | 25 ++ .../processing/store/TablePage.java | 62 ++- .../util/CarbonDataProcessorUtil.java | 80 +++- 81 files changed, 2298 insertions(+), 615 deletions(-) create mode 100644 core/src/main/java/org/apache/carbondata/core/datastore/columnar/BlockIndexerStorage.java create mode 100644 core/src/main/java/org/apache/carbondata/core/datastore/columnar/BlockIndexerStorageForNoDictionary.java create mode 100644 core/src/main/java/org/apache/carbondata/core/datastore/columnar/ColumnWithRowIdForNoDictionary.java delete mode 100644 core/src/main/java/org/apache/carbondata/core/datastore/columnar/IndexStorage.java create mode 100644 core/src/main/java/org/apache/carbondata/core/util/CarbonUnsafeUtil.java diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java b/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java index bded430f9fa..a26d6aedbfc 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java @@ -104,6 +104,23 @@ private void addMeasures(List measures) { } } + /** + * No dictionary and complex dimensions of the table + * + * @return + */ + public DimensionSpec[] getNoDictAndComplexDimensions() { + List noDictAndComplexDimensions = new ArrayList<>(); + 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) { + noDictAndComplexDimensions.add(dimensionSpec[i]); + } + } + return noDictAndComplexDimensions.toArray(new DimensionSpec[noDictAndComplexDimensions.size()]); + } + public DimensionSpec getDimensionSpec(int dimensionIndex) { return dimensionSpec[dimensionIndex]; } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java index dc0f17127cb..52779186972 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java @@ -227,41 +227,39 @@ private ColumnPage decodeDimensionByMeta(DataChunk2 pageMetadata, .decode(pageData.array(), offset, pageMetadata.data_page_length, isLocalDictEncodedPage); } - private boolean isEncodedWithMeta(DataChunk2 pageMetadata) { - List encodings = pageMetadata.getEncoders(); - if (encodings != null && encodings.size() == 1) { - Encoding encoding = encodings.get(0); - switch (encoding) { - case DIRECT_COMPRESS: - case DIRECT_STRING: - case ADAPTIVE_INTEGRAL: - case ADAPTIVE_DELTA_INTEGRAL: - case ADAPTIVE_FLOATING: - case ADAPTIVE_DELTA_FLOATING: - return true; - } - } - return false; - } - protected DimensionColumnPage decodeDimension(DimensionRawColumnChunk rawColumnPage, ByteBuffer pageData, DataChunk2 pageMetadata, int offset) throws IOException, MemoryException { - if (isEncodedWithMeta(pageMetadata)) { + List encodings = pageMetadata.getEncoders(); + if (CarbonUtil.isEncodedWithMeta(encodings)) { ColumnPage decodedPage = decodeDimensionByMeta(pageMetadata, pageData, offset, null != rawColumnPage.getLocalDictionary()); decodedPage.setNullBits(QueryUtil.getNullBitSet(pageMetadata.presence, this.compressor)); - return new ColumnPageWrapper(decodedPage, rawColumnPage.getLocalDictionary(), - isEncodedWithAdaptiveMeta(pageMetadata)); + int[] invertedIndexes = new int[0]; + int[] invertedIndexesReverse = new int[0]; + // in case of no dictionary measure data types, if it is included in sort columns + // then inverted index to be uncompressed + if (encodings.contains(Encoding.INVERTED_INDEX)) { + offset += pageMetadata.data_page_length; + if (CarbonUtil.hasEncoding(pageMetadata.encoders, Encoding.INVERTED_INDEX)) { + invertedIndexes = CarbonUtil + .getUnCompressColumnIndex(pageMetadata.rowid_page_length, pageData, offset); + // get the reverse index + invertedIndexesReverse = CarbonUtil.getInvertedReverseIndex(invertedIndexes); + } + } + return new ColumnPageWrapper(decodedPage, rawColumnPage.getLocalDictionary(), invertedIndexes, + invertedIndexesReverse, isEncodedWithAdaptiveMeta(pageMetadata), + CarbonUtil.hasEncoding(pageMetadata.encoders, Encoding.INVERTED_INDEX)); } else { // following code is for backward compatibility return decodeDimensionLegacy(rawColumnPage, pageData, pageMetadata, offset); } } - private boolean isEncodedWithAdaptiveMeta(DataChunk2 pageMetadata) { + public boolean isEncodedWithAdaptiveMeta(DataChunk2 pageMetadata) { List encodings = pageMetadata.getEncoders(); - if (encodings != null && encodings.size() == 1) { + if (encodings != null && !encodings.isEmpty()) { Encoding encoding = encodings.get(0); switch (encoding) { case ADAPTIVE_INTEGRAL: diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java index 65991a59d58..176a3e9ceaa 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java @@ -24,10 +24,13 @@ import org.apache.carbondata.core.datastore.page.ColumnPage; import org.apache.carbondata.core.metadata.datatype.DataType; import org.apache.carbondata.core.metadata.datatype.DataTypes; +import org.apache.carbondata.core.scan.executor.util.QueryUtil; +import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector; import org.apache.carbondata.core.scan.result.vector.CarbonDictionary; import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo; import org.apache.carbondata.core.util.ByteUtil; import org.apache.carbondata.core.util.CarbonUtil; +import org.apache.carbondata.core.util.DataTypeUtil; public class ColumnPageWrapper implements DimensionColumnPage { @@ -36,14 +39,23 @@ public class ColumnPageWrapper implements DimensionColumnPage { private CarbonDictionary localDictionary; - private boolean isAdaptiveComplexPrimitivePage; + private boolean isAdaptivePrimitivePage; + + private int[] invertedIndex; + + private int[] invertedReverseIndex; + + private boolean isExplicitSorted; public ColumnPageWrapper(ColumnPage columnPage, CarbonDictionary localDictionary, - boolean isAdaptiveComplexPrimitivePage) { + int[] invertedIndex, int[] invertedReverseIndex, boolean isAdaptivePrimitivePage, + boolean isExplicitSorted) { this.columnPage = columnPage; this.localDictionary = localDictionary; - this.isAdaptiveComplexPrimitivePage = isAdaptiveComplexPrimitivePage; - + this.invertedIndex = invertedIndex; + this.invertedReverseIndex = invertedReverseIndex; + this.isAdaptivePrimitivePage = isAdaptivePrimitivePage; + this.isExplicitSorted = isExplicitSorted; } @Override @@ -58,26 +70,79 @@ 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) { + if (columnPage.getNullBits().get(rowId) + && columnPage.getColumnSpec().getColumnType() == ColumnType.COMPLEX_PRIMITIVE) { + // if this row is null, return default null represent in byte array + byte[] value = CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY; + QueryUtil.putDataToVector(vector, value, vectorRow, value.length); + } else if (columnPage.getNullBits().get(rowId)) { + // if this row is null, return default null represent in byte array + byte[] value = CarbonCommonConstants.EMPTY_BYTE_ARRAY; + QueryUtil.putDataToVector(vector, value, vectorRow, value.length); + } else { + if (isExplicitSorted) { + rowId = invertedReverseIndex[rowId]; + } + byte[] value = getChunkData(rowId, true); + int length = value.length; + QueryUtil.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) { + return getChunkData(rowId, false); + } + + private byte[] getChunkData(int rowId, boolean isRowIdChanged) { ColumnType columnType = columnPage.getColumnSpec().getColumnType(); DataType srcDataType = columnPage.getColumnSpec().getSchemaDataType(); DataType targetDataType = columnPage.getDataType(); 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 && this.isAdaptivePrimitive()) || ( + columnType == ColumnType.PLAIN_VALUE && DataTypeUtil.isPrimitiveColumn(srcDataType))) { + if (!isRowIdChanged && columnPage.getNullBits().get(rowId) + && columnType == ColumnType.COMPLEX_PRIMITIVE) { // if this row is null, return default null represent in byte array return CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY; } + if (!isRowIdChanged && columnPage.getNullBits().get(rowId)) { + // if this row is null, return default null represent in byte array + return CarbonCommonConstants.EMPTY_BYTE_ARRAY; + } if (srcDataType == DataTypes.DOUBLE || srcDataType == DataTypes.FLOAT) { double doubleData = columnPage.getDouble(rowId); if (srcDataType == DataTypes.FLOAT) { @@ -118,15 +183,20 @@ public int fillVector(int[] filteredRowId, ColumnVectorInfo[] vectorInfo, int ch } else { throw new RuntimeException("unsupported type: " + targetDataType); } - } else if ((columnType == ColumnType.COMPLEX_PRIMITIVE) && !this.isAdaptiveComplexPrimitive()) { + } else if ((columnType == ColumnType.COMPLEX_PRIMITIVE && !this.isAdaptivePrimitive())) { + if (!isRowIdChanged && columnPage.getNullBits().get(rowId)) { + return CarbonCommonConstants.EMPTY_BYTE_ARRAY; + } if ((srcDataType == DataTypes.BYTE) || (srcDataType == DataTypes.BOOLEAN)) { byte[] out = new byte[1]; out[0] = (columnPage.getByte(rowId)); - return out; + return ByteUtil.toBytes(ByteUtil.toBoolean(out)); } else if (srcDataType == DataTypes.BYTE_ARRAY) { return columnPage.getBytes(rowId); - } else if (srcDataType == DataTypes.DOUBLE) { + } else if (srcDataType == DataTypes.DOUBLE) { return ByteUtil.toXorBytes(columnPage.getDouble(rowId)); + } else if (srcDataType == targetDataType) { + return columnPage.getBytes(rowId); } else { throw new RuntimeException("unsupported type: " + targetDataType); } @@ -135,15 +205,14 @@ public int fillVector(int[] filteredRowId, ColumnVectorInfo[] vectorInfo, int ch } } - @Override public int getInvertedIndex(int rowId) { - throw new UnsupportedOperationException("internal error"); + return invertedIndex[rowId]; } @Override public int getInvertedReverseIndex(int rowId) { - throw new UnsupportedOperationException("internal error"); + return invertedReverseIndex[rowId]; } @Override @@ -153,12 +222,13 @@ public boolean isNoDicitionaryColumn() { @Override public boolean isExplicitSorted() { - return false; + return isExplicitSorted; } @Override public int compareTo(int rowId, byte[] compareValue) { - throw new UnsupportedOperationException("internal error"); + byte[] chunkData = this.getChunkData((int) rowId); + return ByteUtil.UnsafeComparer.INSTANCE.compareTo(chunkData, compareValue); } @Override @@ -169,8 +239,8 @@ public void freeMemory() { } } - public boolean isAdaptiveComplexPrimitive() { - return isAdaptiveComplexPrimitivePage; + public boolean isAdaptivePrimitive() { + return isAdaptivePrimitivePage; } } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimensionDataChunkStore.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimensionDataChunkStore.java index 954cab2d117..15217b84ba8 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimensionDataChunkStore.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimensionDataChunkStore.java @@ -21,11 +21,8 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants; import org.apache.carbondata.core.memory.CarbonUnsafe; -import org.apache.carbondata.core.metadata.datatype.DataType; -import org.apache.carbondata.core.metadata.datatype.DataTypes; +import org.apache.carbondata.core.scan.executor.util.QueryUtil; import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector; -import org.apache.carbondata.core.util.ByteUtil; -import org.apache.carbondata.core.util.DataTypeUtil; /** * Below class is responsible to store variable length dimension data chunk in @@ -236,28 +233,7 @@ public void fillRow(int rowId, CarbonColumnVector vector, int vectorRow) { } // get the row from unsafe fillRowInternal(length, value, currentDataOffset); - DataType dt = vector.getType(); - if ((!(dt == DataTypes.STRING) && length == 0) || ByteUtil.UnsafeComparer.INSTANCE - .equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY, 0, - CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY.length, value, 0, length)) { - vector.putNull(vectorRow); - } else { - if (dt == DataTypes.STRING) { - vector.putBytes(vectorRow, 0, length, value); - } else if (dt == DataTypes.BOOLEAN) { - vector.putBoolean(vectorRow, ByteUtil.toBoolean(value[0])); - } else if (dt == DataTypes.SHORT) { - vector.putShort(vectorRow, ByteUtil.toXorShort(value, 0, length)); - } else if (dt == DataTypes.INT) { - vector.putInt(vectorRow, ByteUtil.toXorInt(value, 0, length)); - } else if (dt == DataTypes.LONG) { - vector.putLong(vectorRow, - DataTypeUtil.getDataBasedOnRestructuredDataType(value, vector.getBlockDataType(), 0, - length)); - } else if (dt == DataTypes.TIMESTAMP) { - vector.putLong(vectorRow, ByteUtil.toXorLong(value, 0, length) * 1000L); - } - } + QueryUtil.putDataToVector(vector, value, vectorRow, length); } /** diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/columnar/BlockIndexerStorage.java b/core/src/main/java/org/apache/carbondata/core/datastore/columnar/BlockIndexerStorage.java new file mode 100644 index 00000000000..6f3f139826b --- /dev/null +++ b/core/src/main/java/org/apache/carbondata/core/datastore/columnar/BlockIndexerStorage.java @@ -0,0 +1,104 @@ +/* + * 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 abstract class BlockIndexerStorage { + + public abstract short[] getRowIdPage(); + + public abstract int getRowIdPageLengthInBytes(); + + public abstract short[] getRowIdRlePage(); + + public abstract int getRowIdRlePageLengthInBytes(); + + public abstract T getDataPage(); + + public abstract short[] getDataRlePage(); + + public abstract int getDataRlePageLengthInBytes(); + + /** + * 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 + */ + protected 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; + } + + protected short[] convertToArray(List list) { + short[] shortArray = new short[list.size()]; + for (int i = 0; i < shortArray.length; i++) { + shortArray[i] = list.get(i); + } + return shortArray; + } +} diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/columnar/BlockIndexerStorageForNoDictionary.java b/core/src/main/java/org/apache/carbondata/core/datastore/columnar/BlockIndexerStorageForNoDictionary.java new file mode 100644 index 00000000000..b3e25d35297 --- /dev/null +++ b/core/src/main/java/org/apache/carbondata/core/datastore/columnar/BlockIndexerStorageForNoDictionary.java @@ -0,0 +1,116 @@ +/* + * 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.Arrays; +import java.util.Map; + +import org.apache.carbondata.core.metadata.datatype.DataType; + +public class BlockIndexerStorageForNoDictionary extends BlockIndexerStorage { + + private short[] rowIdPage; + + private short[] rowIdRlePage; + + private Object[] dataPage; + + private DataType dataType; + + public BlockIndexerStorageForNoDictionary(Object[] dataPage, DataType dataType, + boolean isSortRequired) { + this.dataType = dataType; + ColumnWithRowIdForNoDictionary[] dataWithRowId = createColumnWithRowId(dataPage); + if (isSortRequired) { + Arrays.sort(dataWithRowId); + } + short[] rowIds = extractDataAndReturnRowId(dataWithRowId, dataPage); + Map rowIdAndRleRowIdPages = + rleEncodeOnRowId(rowIds, getRowIdPage(), getRowIdRlePage()); + rowIdPage = rowIdAndRleRowIdPages.get("rowIdPage"); + rowIdRlePage = rowIdAndRleRowIdPages.get("rowRlePage"); + } + + /** + * Create an object with each column array and respective rowId + * + * @return + */ + private ColumnWithRowIdForNoDictionary[] createColumnWithRowId(Object[] dataPage) { + ColumnWithRowIdForNoDictionary[] columnWithIndexs = + new ColumnWithRowIdForNoDictionary[dataPage.length]; + 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(); + } + this.dataPage = dataPage; + return indexes; + } + + /** + * @return the rowIdPage + */ + @Override + public short[] getRowIdPage() { + return rowIdPage; + } + + @Override + public int getRowIdPageLengthInBytes() { + if (rowIdPage != null) { + return rowIdPage.length * 2; + } else { + return 0; + } + } + + @Override + public short[] getRowIdRlePage() { + return rowIdRlePage; + } + + @Override + public int getRowIdRlePageLengthInBytes() { + if (rowIdRlePage != null) { + return rowIdRlePage.length * 2; + } else { + return 0; + } + } + + @Override public Object[] getDataPage() { + return dataPage; + } + + @Override public short[] getDataRlePage() { + return new short[0]; + } + + @Override public int getDataRlePageLengthInBytes() { + return 0; + } + +} diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/columnar/BlockIndexerStorageForNoInvertedIndexForShort.java b/core/src/main/java/org/apache/carbondata/core/datastore/columnar/BlockIndexerStorageForNoInvertedIndexForShort.java index bbb3434ec03..66fefe09cca 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/columnar/BlockIndexerStorageForNoInvertedIndexForShort.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/columnar/BlockIndexerStorageForNoInvertedIndexForShort.java @@ -25,7 +25,7 @@ /** * Below class will be used to for no inverted index */ -public class BlockIndexerStorageForNoInvertedIndexForShort implements IndexStorage { +public class BlockIndexerStorageForNoInvertedIndexForShort extends BlockIndexerStorage { /** * column data @@ -78,14 +78,6 @@ private void rleEncodeOnData(List actualDataList) { } } - private short[] convertToArray(List list) { - short[] shortArray = new short[list.size()]; - for (int i = 0; i < shortArray.length; i++) { - shortArray[i] = list.get(i); - } - return shortArray; - } - private byte[][] convertToDataPage(List list) { byte[][] shortArray = new byte[list.size()][]; for (int i = 0; i < shortArray.length; i++) { @@ -98,7 +90,7 @@ public short[] getDataRlePage() { return dataRlePage; } - @Override public int getDataRlePageLengthInBytes() { + public int getDataRlePageLengthInBytes() { if (dataRlePage != null) { return dataRlePage.length * 2; } else { @@ -115,7 +107,7 @@ public short[] getRowIdPage() { return new short[0]; } - @Override public int getRowIdPageLengthInBytes() { + public int getRowIdPageLengthInBytes() { return 0; } @@ -128,7 +120,7 @@ public short[] getRowIdRlePage() { return new short[0]; } - @Override public int getRowIdRlePageLengthInBytes() { + public int getRowIdRlePageLengthInBytes() { return 0; } @@ -138,4 +130,5 @@ public short[] getRowIdRlePage() { public byte[][] getDataPage() { return dataPage; } + } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/columnar/BlockIndexerStorageForShort.java b/core/src/main/java/org/apache/carbondata/core/datastore/columnar/BlockIndexerStorageForShort.java index be6a1a778b8..f1b9af2f263 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/columnar/BlockIndexerStorageForShort.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/columnar/BlockIndexerStorageForShort.java @@ -19,11 +19,12 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Map; import org.apache.carbondata.core.constants.CarbonCommonConstants; import org.apache.carbondata.core.util.ByteUtil; -public class BlockIndexerStorageForShort implements IndexStorage { +public class BlockIndexerStorageForShort extends BlockIndexerStorage { private boolean alreadySorted; @@ -42,7 +43,10 @@ public BlockIndexerStorageForShort(byte[][] dataPage, boolean rleOnData, Arrays.sort(dataWithRowId); } short[] rowIds = extractDataAndReturnRowId(dataWithRowId, dataPage); - rleEncodeOnRowId(rowIds); + Map rowIdAndRleRowIdPages = + rleEncodeOnRowId(rowIds, getRowIdPage(), getRowIdRlePage()); + rowIdPage = rowIdAndRleRowIdPages.get("rowIdPage"); + rowIdRlePage = rowIdAndRleRowIdPages.get("rowRlePage"); if (rleOnData) { rleEncodeOnData(dataWithRowId); } @@ -79,66 +83,6 @@ private short[] extractDataAndReturnRowId(ColumnWithRowId[] dataWithRowId 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) { - 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); - } - if (rowIdPage.length == 2 && rowIdRlePage.length == 1) { - alreadySorted = true; - } - } - - private short[] convertToArray(List list) { - short[] shortArray = new short[list.size()]; - for (int i = 0; i < shortArray.length; i++) { - shortArray[i] = list.get(i); - } - return shortArray; - } - /** * @return the alreadySorted */ @@ -153,7 +97,6 @@ public short[] getRowIdPage() { return rowIdPage; } - @Override public int getRowIdPageLengthInBytes() { if (rowIdPage != null) { return rowIdPage.length * 2; @@ -169,7 +112,6 @@ public short[] getRowIdRlePage() { return rowIdRlePage; } - @Override public int getRowIdRlePageLengthInBytes() { if (rowIdRlePage != null) { return rowIdRlePage.length * 2; @@ -234,6 +176,7 @@ private byte[][] convertToDataPage(List list) { return shortArray; } + @Override public short[] getDataRlePage() { return dataRlePage; } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/columnar/ColumnWithRowIdForNoDictionary.java b/core/src/main/java/org/apache/carbondata/core/datastore/columnar/ColumnWithRowIdForNoDictionary.java new file mode 100644 index 00000000000..affef97a92f --- /dev/null +++ b/core/src/main/java/org/apache/carbondata/core/datastore/columnar/ColumnWithRowIdForNoDictionary.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.carbondata.core.datastore.columnar; + +import org.apache.carbondata.core.metadata.datatype.DataType; +import org.apache.carbondata.core.util.comparator.SerializableComparator; + +public class ColumnWithRowIdForNoDictionary + implements Comparable> { + + Object column; + + T index; + + DataType dataType; + + ColumnWithRowIdForNoDictionary(Object column, T index, DataType dataType) { + this.column = column; + this.index = index; + this.dataType = dataType; + } + + @Override public int compareTo(ColumnWithRowIdForNoDictionary o) { + // use the data type based comparator for the no dictionary encoded columns + SerializableComparator comparator = + org.apache.carbondata.core.util.comparator.Comparator.getComparator(dataType); + return comparator.compare(column, o.column); + } + + @Override public boolean equals(Object obj) { + if (obj == null || getClass() != obj.getClass()) { + return false; + } + ColumnWithRowIdForNoDictionary o = (ColumnWithRowIdForNoDictionary)obj; + return column.equals(o.column) && getIndex() == o.getIndex(); + } + + @Override public int hashCode() { + return getColumn().hashCode() + getIndex().hashCode(); + } + + /** + * @return the index + */ + public T getIndex() { + return index; + } + + + /** + * @return the column + */ + public Object getColumn() { + return column; + } + +} diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/columnar/IndexStorage.java b/core/src/main/java/org/apache/carbondata/core/datastore/columnar/IndexStorage.java deleted file mode 100644 index a30ea8865f4..00000000000 --- a/core/src/main/java/org/apache/carbondata/core/datastore/columnar/IndexStorage.java +++ /dev/null @@ -1,35 +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.columnar; - -public interface IndexStorage { - - T getRowIdPage(); - - int getRowIdPageLengthInBytes(); - - T getRowIdRlePage(); - - int getRowIdRlePageLengthInBytes(); - - byte[][] getDataPage(); - - T getDataRlePage(); - - int getDataRlePageLengthInBytes(); -} diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoder.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoder.java index 2ed12a0ac1d..ccf4c224ab9 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoder.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/ColumnPageEncoder.java @@ -34,9 +34,11 @@ import org.apache.carbondata.core.datastore.page.ComplexColumnPage; import org.apache.carbondata.core.datastore.page.encoding.compress.DirectCompressCodec; import org.apache.carbondata.core.memory.MemoryException; +import org.apache.carbondata.core.metadata.datatype.DataType; import org.apache.carbondata.core.metadata.datatype.DataTypes; import org.apache.carbondata.core.util.CarbonMetadataUtil; import org.apache.carbondata.core.util.CarbonUtil; +import org.apache.carbondata.core.util.DataTypeUtil; import org.apache.carbondata.format.BlockletMinMaxIndex; import org.apache.carbondata.format.DataChunk2; import org.apache.carbondata.format.Encoding; @@ -61,6 +63,21 @@ public abstract class ColumnPageEncoder { protected abstract ColumnPageEncoderMeta getEncoderMeta(ColumnPage inputPage); + /** + * Get the target data type of the page if encoded + * + * @param inputPage + * @return + */ + public DataType getTargetDataType(ColumnPage inputPage) { + ColumnPageEncoderMeta encoderMeta = getEncoderMeta(inputPage); + if (null != encoderMeta) { + return encoderMeta.getStoreDataType(); + } else { + return null; + } + } + /** * Return a encoded column page by encoding the input page * The encoded binary data and metadata are wrapped in encoding column page @@ -118,17 +135,28 @@ private List buildEncoderMeta(ColumnPage inputPage) throws IOExcepti } private void fillMinMaxIndex(ColumnPage inputPage, DataChunk2 dataChunk) { - dataChunk.setMin_max(buildMinMaxIndex(inputPage)); + dataChunk.setMin_max(buildMinMaxIndex(inputPage, dataChunk.encoders)); } - private BlockletMinMaxIndex buildMinMaxIndex(ColumnPage inputPage) { + private BlockletMinMaxIndex buildMinMaxIndex(ColumnPage inputPage, List encoders) { BlockletMinMaxIndex index = new BlockletMinMaxIndex(); - byte[] bytes = CarbonUtil.getValueAsBytes( - inputPage.getDataType(), inputPage.getStatistics().getMax()); - ByteBuffer max = ByteBuffer.wrap( - bytes); - ByteBuffer min = ByteBuffer.wrap( - CarbonUtil.getValueAsBytes(inputPage.getDataType(), inputPage.getStatistics().getMin())); + ByteBuffer max; + ByteBuffer min; + if (CarbonUtil.isEncodedWithMeta(encoders) + && inputPage.getColumnSpec().getColumnType() == ColumnType.PLAIN_VALUE) { + max = ByteBuffer.wrap(DataTypeUtil + .getMinMaxBytesBasedOnDataTypeForNoDictionaryColumn(inputPage.getStatistics().getMax(), + inputPage.getDataType())); + min = ByteBuffer.wrap(DataTypeUtil + .getMinMaxBytesBasedOnDataTypeForNoDictionaryColumn(inputPage.getStatistics().getMin(), + inputPage.getDataType())); + } else { + byte[] bytes = + CarbonUtil.getValueAsBytes(inputPage.getDataType(), inputPage.getStatistics().getMax()); + max = ByteBuffer.wrap(bytes); + min = ByteBuffer.wrap( + CarbonUtil.getValueAsBytes(inputPage.getDataType(), inputPage.getStatistics().getMin())); + } index.addToMax_values(max); index.addToMin_values(min); return index; @@ -186,11 +214,11 @@ private static ColumnPageEncoder createCodecForDimension(ColumnPage inputPage) { } else if ((inputPage.getDataType() == DataTypes.BYTE) || (inputPage.getDataType() == DataTypes.SHORT) || (inputPage.getDataType() == DataTypes.INT) || ( inputPage.getDataType() == DataTypes.LONG)) { - return selectCodecByAlgorithmForIntegral(inputPage.getStatistics(), true) + return selectCodecByAlgorithmForIntegral(inputPage.getStatistics(), true, columnSpec) .createEncoder(null); } else if ((inputPage.getDataType() == DataTypes.FLOAT) || (inputPage.getDataType() == DataTypes.DOUBLE)) { - return selectCodecByAlgorithmForFloating(inputPage.getStatistics(), true) + return selectCodecByAlgorithmForFloating(inputPage.getStatistics(), true, columnSpec) .createEncoder(null); } } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/DefaultEncodingFactory.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/DefaultEncodingFactory.java index 29772d11c8d..993b6b85052 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/DefaultEncodingFactory.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/DefaultEncodingFactory.java @@ -19,6 +19,7 @@ import java.math.BigDecimal; +import org.apache.carbondata.core.datastore.ColumnType; import org.apache.carbondata.core.datastore.TableSpec; import org.apache.carbondata.core.datastore.page.ColumnPage; import org.apache.carbondata.core.datastore.page.DecimalColumnPage; @@ -36,6 +37,7 @@ import org.apache.carbondata.core.metadata.datatype.DataType; import org.apache.carbondata.core.metadata.datatype.DataTypes; import org.apache.carbondata.core.metadata.datatype.DecimalConverterFactory; +import org.apache.carbondata.core.util.DataTypeUtil; /** * Default factory will select encoding base on column page data type and statistics @@ -57,8 +59,11 @@ public static EncodingFactory getInstance() { @Override public ColumnPageEncoder createEncoder(TableSpec.ColumnSpec columnSpec, ColumnPage inputPage) { // TODO: add log - if (columnSpec instanceof TableSpec.MeasureSpec) { - return createEncoderForMeasure(inputPage); + // choose the encoding type for measure type and no dictionary primitive type columns + if (columnSpec instanceof TableSpec.MeasureSpec || ( + DataTypeUtil.isPrimitiveColumn(columnSpec.getSchemaDataType()) + && columnSpec.getColumnType() == ColumnType.PLAIN_VALUE)) { + return createEncoderForMeasureOrNoDictionaryPrimitive(inputPage, columnSpec); } else { if (newWay) { return createEncoderForDimension((TableSpec.DimensionSpec) columnSpec, inputPage); @@ -107,7 +112,8 @@ private ColumnPageEncoder createEncoderForDimensionLegacy(TableSpec.DimensionSpe } } - private ColumnPageEncoder createEncoderForMeasure(ColumnPage columnPage) { + private ColumnPageEncoder createEncoderForMeasureOrNoDictionaryPrimitive(ColumnPage columnPage, + TableSpec.ColumnSpec columnSpec) { SimpleStatsResult stats = columnPage.getStatistics(); DataType dataType = stats.getDataType(); if (dataType == DataTypes.BOOLEAN) { @@ -116,11 +122,11 @@ private ColumnPageEncoder createEncoderForMeasure(ColumnPage columnPage) { dataType == DataTypes.SHORT || dataType == DataTypes.INT || dataType == DataTypes.LONG) { - return selectCodecByAlgorithmForIntegral(stats, false).createEncoder(null); + return selectCodecByAlgorithmForIntegral(stats, false, columnSpec).createEncoder(null); } else if (DataTypes.isDecimal(dataType)) { - return createEncoderForDecimalDataTypeMeasure(columnPage); + return createEncoderForDecimalDataTypeMeasure(columnPage, columnSpec); } else if (dataType == DataTypes.FLOAT || dataType == DataTypes.DOUBLE) { - return selectCodecByAlgorithmForFloating(stats, false).createEncoder(null); + return selectCodecByAlgorithmForFloating(stats, false, columnSpec).createEncoder(null); } else if (dataType == DataTypes.BYTE_ARRAY) { return new DirectCompressCodec(columnPage.getDataType()).createEncoder(null); } else { @@ -128,13 +134,15 @@ private ColumnPageEncoder createEncoderForMeasure(ColumnPage columnPage) { } } - private ColumnPageEncoder createEncoderForDecimalDataTypeMeasure(ColumnPage columnPage) { + private ColumnPageEncoder createEncoderForDecimalDataTypeMeasure(ColumnPage columnPage, + TableSpec.ColumnSpec columnSpec) { DecimalConverterFactory.DecimalConverterType decimalConverterType = ((DecimalColumnPage) columnPage).getDecimalConverter().getDecimalConverterType(); switch (decimalConverterType) { case DECIMAL_INT: case DECIMAL_LONG: - return selectCodecByAlgorithmForDecimal(columnPage.getStatistics(), decimalConverterType) + return selectCodecByAlgorithmForDecimal(columnPage.getStatistics(), decimalConverterType, + columnSpec) .createEncoder(null); default: return new DirectCompressCodec(columnPage.getDataType()).createEncoder(null); @@ -245,7 +253,7 @@ private static DataType compareMinMaxAndSelectDataType(long value) { * size is smaller */ static ColumnPageCodec selectCodecByAlgorithmForIntegral(SimpleStatsResult stats, - boolean isComplexPrimitive) { + boolean isComplexPrimitive, TableSpec.ColumnSpec columnSpec) { DataType srcDataType = stats.getDataType(); DataType adaptiveDataType = fitMinMax(stats.getDataType(), stats.getMax(), stats.getMin()); @@ -260,19 +268,40 @@ static ColumnPageCodec selectCodecByAlgorithmForIntegral(SimpleStatsResult stats return new DirectCompressCodec(stats.getDataType()); } } + boolean isInvertedIndex = isInvertedIndex(isComplexPrimitive, columnSpec); if (adaptiveDataType.getSizeInBytes() <= deltaDataType.getSizeInBytes()) { // choose adaptive encoding - return new AdaptiveIntegralCodec(stats.getDataType(), adaptiveDataType, stats); + return new AdaptiveIntegralCodec(stats.getDataType(), adaptiveDataType, stats, + isInvertedIndex); } else { // choose delta adaptive encoding - return new AdaptiveDeltaIntegralCodec(stats.getDataType(), deltaDataType, stats); + return new AdaptiveDeltaIntegralCodec(stats.getDataType(), deltaDataType, stats, + isInvertedIndex); } } + /** + * Check whether the column is sort column and inverted index column + * + * @param isComplexPrimitive + * @param columnSpec + * @return + */ + private static boolean isInvertedIndex(boolean isComplexPrimitive, + TableSpec.ColumnSpec columnSpec) { + boolean isSort; + boolean isInvertedIndex = false; + if (columnSpec instanceof TableSpec.DimensionSpec && !isComplexPrimitive) { + isSort = ((TableSpec.DimensionSpec) columnSpec).isInSortColumns(); + isInvertedIndex = isSort && ((TableSpec.DimensionSpec) columnSpec).isDoInvertedIndex(); + } + return isInvertedIndex; + } + // choose between upscale adaptive encoder or upscale delta adaptive encoder, // based on whose target data type size is smaller static ColumnPageCodec selectCodecByAlgorithmForFloating(SimpleStatsResult stats, - boolean isComplexPrimitive) { + boolean isComplexPrimitive, TableSpec.ColumnSpec columnSpec) { DataType srcDataType = stats.getDataType(); double maxValue = (double) stats.getMax(); double minValue = (double) stats.getMin(); @@ -290,7 +319,7 @@ static ColumnPageCodec selectCodecByAlgorithmForFloating(SimpleStatsResult stats double absMaxValue = Math.max(Math.abs(maxValue), Math.abs(minValue)); if (decimalCount == 0) { // short, int, long - return selectCodecByAlgorithmForIntegral(stats, false); + return selectCodecByAlgorithmForIntegral(stats, false, columnSpec); } else if (decimalCount < 0 && !isComplexPrimitive) { return new DirectCompressCodec(DataTypes.DOUBLE); } else { @@ -304,11 +333,13 @@ static ColumnPageCodec selectCodecByAlgorithmForFloating(SimpleStatsResult stats DataType deltaDataType = compareMinMaxAndSelectDataType( (long) (Math.pow(10, decimalCount) * (maxValue - minValue))); if (adaptiveDataType.getSizeInBytes() > deltaDataType.getSizeInBytes()) { - return new AdaptiveDeltaFloatingCodec(srcDataType, deltaDataType, stats); + return new AdaptiveDeltaFloatingCodec(srcDataType, deltaDataType, stats, + isInvertedIndex(isComplexPrimitive, columnSpec)); } else if (adaptiveDataType.getSizeInBytes() < DataTypes.DOUBLE.getSizeInBytes() || ( (isComplexPrimitive) && (adaptiveDataType.getSizeInBytes() == DataTypes.DOUBLE .getSizeInBytes()))) { - return new AdaptiveFloatingCodec(srcDataType, adaptiveDataType, stats); + return new AdaptiveFloatingCodec(srcDataType, adaptiveDataType, stats, + isInvertedIndex(isComplexPrimitive, columnSpec)); } else { return new DirectCompressCodec(DataTypes.DOUBLE); } @@ -321,7 +352,8 @@ static ColumnPageCodec selectCodecByAlgorithmForFloating(SimpleStatsResult stats * size is smaller for decimal data type */ static ColumnPageCodec selectCodecByAlgorithmForDecimal(SimpleStatsResult stats, - DecimalConverterFactory.DecimalConverterType decimalConverterType) { + DecimalConverterFactory.DecimalConverterType decimalConverterType, + TableSpec.ColumnSpec columnSpec) { DataType srcDataType = stats.getDataType(); DataType adaptiveDataType = fitMinMaxForDecimalType(stats.getDataType(), stats.getMax(), stats.getMin(), @@ -343,10 +375,12 @@ static ColumnPageCodec selectCodecByAlgorithmForDecimal(SimpleStatsResult stats, } if (adaptiveDataType.getSizeInBytes() <= deltaDataType.getSizeInBytes()) { // choose adaptive encoding - return new AdaptiveIntegralCodec(stats.getDataType(), adaptiveDataType, stats); + return new AdaptiveIntegralCodec(stats.getDataType(), adaptiveDataType, stats, + isInvertedIndex(columnSpec.getColumnType() == ColumnType.COMPLEX_PRIMITIVE, columnSpec)); } else { // choose delta adaptive encoding - return new AdaptiveDeltaIntegralCodec(stats.getDataType(), deltaDataType, stats); + return new AdaptiveDeltaIntegralCodec(stats.getDataType(), deltaDataType, stats, + isInvertedIndex(columnSpec.getColumnType() == ColumnType.COMPLEX_PRIMITIVE, columnSpec)); } } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java index d119c8fc484..920a51626ca 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java @@ -66,7 +66,7 @@ public abstract ColumnPageEncoder createEncoder(TableSpec.ColumnSpec columnSpec, */ public ColumnPageDecoder createDecoder(List encodings, List encoderMetas, String compressor) throws IOException { - assert (encodings.size() == 1); + assert (encodings.size() >= 1); assert (encoderMetas.size() == 1); Encoding encoding = encodings.get(0); byte[] encoderMeta = encoderMetas.get(0).array(); @@ -81,25 +81,27 @@ public ColumnPageDecoder createDecoder(List encodings, List indexStorage; + + protected ColumnPage encodedPage; + protected AdaptiveCodec(DataType srcDataType, DataType targetDataType, - SimpleStatsResult stats) { + SimpleStatsResult stats, boolean isInvertedIndex) { this.stats = stats; this.srcDataType = srcDataType; this.targetDataType = targetDataType; + this.isInvertedIndex = isInvertedIndex; } public DataType getTargetDataType() { return targetDataType; } + /** + * Convert the data of the page based on the data type for each row + * While preparing the inverted index for the page, + * we need the data based on data type for no dict measure column if adaptive encoding is applied + * This is similar to page.getByteArrayPage() + * + * @param input + * @return + */ + public Object[] getPageBasedOnDataType(ColumnPage input) { + Object[] data = new Object[input.getActualRowCount()]; + if (srcDataType == DataTypes.BYTE || srcDataType == DataTypes.BOOLEAN) { + for (int i = 0; i < input.getActualRowCount(); i++) { + data[i] = input.getByte(i); + } + } else if (srcDataType == DataTypes.SHORT) { + for (int i = 0; i < input.getActualRowCount(); i++) { + data[i] = input.getShort(i); + } + } else if (srcDataType == DataTypes.SHORT_INT) { + for (int i = 0; i < input.getActualRowCount(); i++) { + data[i] = input.getShortInt(i); + } + } else if (srcDataType == DataTypes.INT) { + for (int i = 0; i < input.getActualRowCount(); i++) { + data[i] = input.getInt(i); + } + } else if (srcDataType == DataTypes.LONG) { + for (int i = 0; i < input.getActualRowCount(); i++) { + data[i] = input.getLong(i); + } + } else if (srcDataType == DataTypes.FLOAT) { + for (int i = 0; i < input.getActualRowCount(); i++) { + data[i] = input.getFloat(i); + } + } else if (srcDataType == DataTypes.DOUBLE) { + for (int i = 0; i < input.getActualRowCount(); i++) { + data[i] = input.getDouble(i); + } + } + return data; + } + + /** + * Put the data to the page based on the data type for each row + * + * @param page + * @return + */ + public void putDataToPage(ColumnPage page, Object[] dataPage) { + if (srcDataType == DataTypes.BYTE || srcDataType == DataTypes.BOOLEAN) { + for (int i = 0; i < dataPage.length; i++) { + page.putByte(i, (byte) dataPage[i]); + } + } else if (srcDataType == DataTypes.SHORT) { + for (int i = 0; i < dataPage.length; i++) { + page.putShort(i, (short) dataPage[i]); + } + } else if (srcDataType == DataTypes.SHORT_INT) { + for (int i = 0; i < dataPage.length; i++) { + page.putShortInt(i, (int) dataPage[i]); + } + } else if (srcDataType == DataTypes.INT) { + for (int i = 0; i < dataPage.length; i++) { + page.putInt(i, (int) dataPage[i]); + } + } else if (srcDataType == DataTypes.LONG) { + for (int i = 0; i < dataPage.length; i++) { + page.putLong(i, (long) dataPage[i]); + } + } else if (srcDataType == DataTypes.DOUBLE) { + for (int i = 0; i < dataPage.length; i++) { + page.putDouble(i, (double) dataPage[i]); + } + } + } + + /** + * Write the inverted index to the page if required + * + * @param result + * @throws IOException + */ + public byte[] writeInvertedIndexIfRequired(byte[] result) throws IOException { + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(stream); + if (null != indexStorage) { + out.write(result); + if (indexStorage.getRowIdPageLengthInBytes() > 0) { + out.writeInt(indexStorage.getRowIdPageLengthInBytes()); + short[] rowIdPage = (short[]) indexStorage.getRowIdPage(); + for (short rowId : rowIdPage) { + out.writeShort(rowId); + } + if (indexStorage.getRowIdRlePageLengthInBytes() > 0) { + short[] rowIdRlePage = (short[]) indexStorage.getRowIdRlePage(); + for (short rowIdRle : rowIdRlePage) { + out.writeShort(rowIdRle); + } + } + } + } + byte[] bytes = stream.toByteArray(); + stream.close(); + return bytes; + } + + /** + * Fill legacy fields if required + * + * @param dataChunk + * @param result + */ + public void fillLegacyFieldsIfRequired(DataChunk2 dataChunk, byte[] result) { + if (null != indexStorage) { + SortState sort = (indexStorage.getRowIdPageLengthInBytes() > 0) ? + SortState.SORT_EXPLICIT : + SortState.SORT_NATIVE; + dataChunk.setSort_state(sort); + if (indexStorage.getRowIdPageLengthInBytes() > 0) { + int rowIdPageLength = + CarbonCommonConstants.INT_SIZE_IN_BYTE + indexStorage.getRowIdPageLengthInBytes() + + indexStorage.getRowIdRlePageLengthInBytes(); + dataChunk.setRowid_page_length(rowIdPageLength); + } + } else { + dataChunk.setRowid_page_length(0); + } + if (null != result) { + dataChunk.setData_page_length(result.length); + } + } + + /** + * Get the new column page based on the sorted data + * + * @param input + * @return + * @throws MemoryException + */ + public ColumnPage getSortedColumnPageIfRequired(ColumnPage input) throws MemoryException { + if (null != indexStorage) { + Object[] dataPage = indexStorage.getDataPage(); + ColumnPageEncoderMeta columnPageEncoderMeta = + new ColumnPageEncoderMeta(input.getColumnSpec(), input.getDataType(), + input.getColumnPageEncoderMeta().getCompressorName()); + ColumnPage columnPage = ColumnPage.newPage(columnPageEncoderMeta, input.getPageSize()); + putDataToPage(columnPage, dataPage); + return columnPage; + } else { + return input; + } + } + + public byte[] encodeAndCompressPage(ColumnPage input, ColumnPageValueConverter converter, + Compressor compressor) throws MemoryException, IOException { + encodedPage = ColumnPage.newPage( + new ColumnPageEncoderMeta(input.getColumnPageEncoderMeta().getColumnSpec(), targetDataType, + input.getColumnPageEncoderMeta().getCompressorName()), input.getPageSize()); + if (isInvertedIndex) { + indexStorage = + new BlockIndexerStorageForNoDictionary(getPageBasedOnDataType(input), input.getDataType(), + isInvertedIndex); + } + ColumnPage columnPage = getSortedColumnPageIfRequired(input); + columnPage.convertValue(converter); + byte[] result = encodedPage.compress(compressor); + return result; + } + @Override public String toString() { return String.format("%s[src type: %s, target type: %s, stats(%s)]", @@ -58,4 +250,5 @@ public String toString() { protected String debugInfo() { return this.toString(); } + } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaFloatingCodec.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaFloatingCodec.java index bb928c24894..6d0a8d1cc0e 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaFloatingCodec.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaFloatingCodec.java @@ -35,6 +35,7 @@ import org.apache.carbondata.core.memory.MemoryException; import org.apache.carbondata.core.metadata.datatype.DataType; import org.apache.carbondata.core.metadata.datatype.DataTypes; +import org.apache.carbondata.format.DataChunk2; import org.apache.carbondata.format.Encoding; /** @@ -44,18 +45,18 @@ */ public class AdaptiveDeltaFloatingCodec extends AdaptiveCodec { - private ColumnPage encodedPage; private Double factor; private long max; public static ColumnPageCodec newInstance(DataType srcDataType, DataType targetDataType, - SimpleStatsResult stats) { - return new AdaptiveDeltaFloatingCodec(srcDataType, targetDataType, stats); + SimpleStatsResult stats, boolean isInvertedIndex) { + return new AdaptiveDeltaFloatingCodec(srcDataType, targetDataType, stats, + isInvertedIndex); } public AdaptiveDeltaFloatingCodec(DataType srcDataType, DataType targetDataType, - SimpleStatsResult stats) { - super(srcDataType, targetDataType, stats); + SimpleStatsResult stats, boolean isInvertedIndex) { + super(srcDataType, targetDataType, stats, isInvertedIndex); this.factor = Math.pow(10, stats.getDecimalCount()); this.max = (long) (Math.pow(10, stats.getDecimalCount()) * (double) stats.getMax()); } @@ -68,20 +69,20 @@ public String getName() { @Override public ColumnPageEncoder createEncoder(Map parameter) { return new ColumnPageEncoder() { + byte[] result = null; @Override protected byte[] encodeData(ColumnPage input) throws MemoryException, IOException { if (encodedPage != null) { throw new IllegalStateException("already encoded"); } - encodedPage = ColumnPage.newPage( - new ColumnPageEncoderMeta(input.getColumnPageEncoderMeta().getColumnSpec(), - targetDataType, input.getColumnPageEncoderMeta().getCompressorName()), - input.getPageSize()); - input.convertValue(converter); Compressor compressor = CompressorFactory.getInstance().getCompressor( input.getColumnCompressorName()); - byte[] result = encodedPage.compress(compressor); + result = encodeAndCompressPage(input, converter, compressor); + byte[] bytes = writeInvertedIndexIfRequired(result); encodedPage.freeMemory(); + if (bytes.length != 0) { + return bytes; + } return result; } @@ -89,6 +90,9 @@ protected byte[] encodeData(ColumnPage input) throws MemoryException, IOExceptio protected List getEncodingList() { List encodings = new ArrayList(); encodings.add(Encoding.ADAPTIVE_DELTA_FLOATING); + if (null != indexStorage && indexStorage.getRowIdPageLengthInBytes() > 0) { + encodings.add(Encoding.INVERTED_INDEX); + } return encodings; } @@ -98,6 +102,11 @@ protected ColumnPageEncoderMeta getEncoderMeta(ColumnPage inputPage) { inputPage.getColumnCompressorName()); } + @Override + protected void fillLegacyFields(DataChunk2 dataChunk) throws IOException { + fillLegacyFieldsIfRequired(dataChunk, result); + } + }; } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaIntegralCodec.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaIntegralCodec.java index ac9693dafd1..9ada0bb93f8 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaIntegralCodec.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveDeltaIntegralCodec.java @@ -35,6 +35,7 @@ import org.apache.carbondata.core.memory.MemoryException; import org.apache.carbondata.core.metadata.datatype.DataType; import org.apache.carbondata.core.metadata.datatype.DataTypes; +import org.apache.carbondata.format.DataChunk2; import org.apache.carbondata.format.Encoding; /** @@ -45,12 +46,11 @@ */ public class AdaptiveDeltaIntegralCodec extends AdaptiveCodec { - private ColumnPage encodedPage; private long max; public AdaptiveDeltaIntegralCodec(DataType srcDataType, DataType targetDataType, - SimpleStatsResult stats) { - super(srcDataType, targetDataType, stats); + SimpleStatsResult stats, boolean isInvertedIndex) { + super(srcDataType, targetDataType, stats, isInvertedIndex); if (srcDataType == DataTypes.BYTE) { this.max = (byte) stats.getMax(); } else if (srcDataType == DataTypes.SHORT) { @@ -78,21 +78,19 @@ public String getName() { @Override public ColumnPageEncoder createEncoder(Map parameter) { return new ColumnPageEncoder() { - + byte[] result = null; + final Compressor compressor = CompressorFactory.getInstance().getCompressor(); @Override protected byte[] encodeData(ColumnPage input) throws MemoryException, IOException { if (encodedPage != null) { throw new IllegalStateException("already encoded"); } - encodedPage = ColumnPage.newPage( - new ColumnPageEncoderMeta(input.getColumnPageEncoderMeta().getColumnSpec(), - targetDataType, input.getColumnPageEncoderMeta().getCompressorName()), - input.getPageSize()); - input.convertValue(converter); - Compressor compressor = CompressorFactory.getInstance().getCompressor( - input.getColumnCompressorName()); - byte[] result = encodedPage.compress(compressor); + result = encodeAndCompressPage(input, converter, compressor); + byte[] bytes = writeInvertedIndexIfRequired(result); encodedPage.freeMemory(); + if (bytes.length != 0) { + return bytes; + } return result; } @@ -106,9 +104,17 @@ protected ColumnPageEncoderMeta getEncoderMeta(ColumnPage inputPage) { protected List getEncodingList() { List encodings = new ArrayList<>(); encodings.add(Encoding.ADAPTIVE_DELTA_INTEGRAL); + if (null != indexStorage && indexStorage.getRowIdPageLengthInBytes() > 0) { + encodings.add(Encoding.INVERTED_INDEX); + } return encodings; } + @Override + protected void fillLegacyFields(DataChunk2 dataChunk) throws IOException { + fillLegacyFieldsIfRequired(dataChunk, result); + } + }; } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveFloatingCodec.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveFloatingCodec.java index 028fa712eec..af1e9ec8780 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveFloatingCodec.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveFloatingCodec.java @@ -34,6 +34,7 @@ import org.apache.carbondata.core.memory.MemoryException; import org.apache.carbondata.core.metadata.datatype.DataType; import org.apache.carbondata.core.metadata.datatype.DataTypes; +import org.apache.carbondata.format.DataChunk2; import org.apache.carbondata.format.Encoding; /** @@ -43,12 +44,11 @@ */ public class AdaptiveFloatingCodec extends AdaptiveCodec { - private ColumnPage encodedPage; private Double factor; public AdaptiveFloatingCodec(DataType srcDataType, DataType targetDataType, - SimpleStatsResult stats) { - super(srcDataType, targetDataType, stats); + SimpleStatsResult stats, boolean isInvertedIndex) { + super(srcDataType, targetDataType, stats, isInvertedIndex); this.factor = Math.pow(10, stats.getDecimalCount()); } @@ -60,20 +60,20 @@ public String getName() { @Override public ColumnPageEncoder createEncoder(Map parameter) { return new ColumnPageEncoder() { + byte[] result = null; @Override protected byte[] encodeData(ColumnPage input) throws MemoryException, IOException { if (encodedPage != null) { throw new IllegalStateException("already encoded"); } - encodedPage = ColumnPage.newPage( - new ColumnPageEncoderMeta(input.getColumnPageEncoderMeta().getColumnSpec(), - targetDataType, input.getColumnPageEncoderMeta().getCompressorName()), - input.getPageSize()); - Compressor compressor = CompressorFactory.getInstance().getCompressor( - input.getColumnCompressorName()); - input.convertValue(converter); - byte[] result = encodedPage.compress(compressor); + Compressor compressor = + CompressorFactory.getInstance().getCompressor(input.getColumnCompressorName()); + result = encodeAndCompressPage(input, converter, compressor); + byte[] bytes = writeInvertedIndexIfRequired(result); encodedPage.freeMemory(); + if (bytes.length != 0) { + return bytes; + } return result; } @@ -81,6 +81,9 @@ protected byte[] encodeData(ColumnPage input) throws MemoryException, IOExceptio protected List getEncodingList() { List encodings = new ArrayList(); encodings.add(Encoding.ADAPTIVE_FLOATING); + if (null != indexStorage && indexStorage.getRowIdPageLengthInBytes() > 0) { + encodings.add(Encoding.INVERTED_INDEX); + } return encodings; } @@ -90,6 +93,11 @@ protected ColumnPageEncoderMeta getEncoderMeta(ColumnPage inputPage) { inputPage.getColumnCompressorName()); } + @Override + protected void fillLegacyFields(DataChunk2 dataChunk) throws IOException { + fillLegacyFieldsIfRequired(dataChunk, result); + } + }; } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveIntegralCodec.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveIntegralCodec.java index a9cf742aad9..f1c0ea0e69a 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveIntegralCodec.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveIntegralCodec.java @@ -34,6 +34,7 @@ import org.apache.carbondata.core.memory.MemoryException; import org.apache.carbondata.core.metadata.datatype.DataType; import org.apache.carbondata.core.metadata.datatype.DataTypes; +import org.apache.carbondata.format.DataChunk2; import org.apache.carbondata.format.Encoding; /** @@ -42,11 +43,9 @@ */ public class AdaptiveIntegralCodec extends AdaptiveCodec { - private ColumnPage encodedPage; - public AdaptiveIntegralCodec(DataType srcDataType, DataType targetDataType, - SimpleStatsResult stats) { - super(srcDataType, targetDataType, stats); + SimpleStatsResult stats, boolean isInvertedIndex) { + super(srcDataType, targetDataType, stats, isInvertedIndex); } @Override @@ -57,20 +56,20 @@ public String getName() { @Override public ColumnPageEncoder createEncoder(Map parameter) { return new ColumnPageEncoder() { + byte[] result = null; @Override protected byte[] encodeData(ColumnPage input) throws MemoryException, IOException { if (encodedPage != null) { throw new IllegalStateException("already encoded"); } - encodedPage = ColumnPage.newPage( - new ColumnPageEncoderMeta(input.getColumnPageEncoderMeta().getColumnSpec(), - targetDataType, input.getColumnPageEncoderMeta().getCompressorName()), - input.getPageSize()); - Compressor compressor = CompressorFactory.getInstance().getCompressor( - input.getColumnCompressorName()); - input.convertValue(converter); - byte[] result = encodedPage.compress(compressor); + Compressor compressor = + CompressorFactory.getInstance().getCompressor(input.getColumnCompressorName()); + result = encodeAndCompressPage(input, converter, compressor); + byte[] bytes = writeInvertedIndexIfRequired(result); encodedPage.freeMemory(); + if (bytes.length != 0) { + return bytes; + } return result; } @@ -78,6 +77,9 @@ protected byte[] encodeData(ColumnPage input) throws MemoryException, IOExceptio protected List getEncodingList() { List encodings = new ArrayList(); encodings.add(Encoding.ADAPTIVE_INTEGRAL); + if (null != indexStorage && indexStorage.getRowIdPageLengthInBytes() > 0) { + encodings.add(Encoding.INVERTED_INDEX); + } return encodings; } @@ -87,6 +89,10 @@ protected ColumnPageEncoderMeta getEncoderMeta(ColumnPage inputPage) { inputPage.getColumnCompressorName()); } + @Override + protected void fillLegacyFields(DataChunk2 dataChunk) throws IOException { + fillLegacyFieldsIfRequired(dataChunk, result); + } }; } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/dimension/legacy/ComplexDimensionIndexCodec.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/dimension/legacy/ComplexDimensionIndexCodec.java index cc044cc9356..f232652c59b 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/dimension/legacy/ComplexDimensionIndexCodec.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/dimension/legacy/ComplexDimensionIndexCodec.java @@ -21,8 +21,8 @@ import java.util.List; import java.util.Map; +import org.apache.carbondata.core.datastore.columnar.BlockIndexerStorage; import org.apache.carbondata.core.datastore.columnar.BlockIndexerStorageForShort; -import org.apache.carbondata.core.datastore.columnar.IndexStorage; import org.apache.carbondata.core.datastore.compression.Compressor; import org.apache.carbondata.core.datastore.compression.CompressorFactory; import org.apache.carbondata.core.datastore.page.ColumnPage; @@ -46,7 +46,7 @@ public ColumnPageEncoder createEncoder(Map parameter) { return new IndexStorageEncoder() { @Override void encodeIndexStorage(ColumnPage inputPage) { - IndexStorage indexStorage = + BlockIndexerStorage indexStorage = new BlockIndexerStorageForShort(inputPage.getByteArrayPage(), false, false, false); byte[] flattened = ByteUtil.flatten(indexStorage.getDataPage()); Compressor compressor = CompressorFactory.getInstance().getCompressor( diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/dimension/legacy/DictDimensionIndexCodec.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/dimension/legacy/DictDimensionIndexCodec.java index 66f5f1d1ba4..f3475fd35f4 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/dimension/legacy/DictDimensionIndexCodec.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/dimension/legacy/DictDimensionIndexCodec.java @@ -21,9 +21,9 @@ import java.util.List; import java.util.Map; +import org.apache.carbondata.core.datastore.columnar.BlockIndexerStorage; import org.apache.carbondata.core.datastore.columnar.BlockIndexerStorageForNoInvertedIndexForShort; import org.apache.carbondata.core.datastore.columnar.BlockIndexerStorageForShort; -import org.apache.carbondata.core.datastore.columnar.IndexStorage; import org.apache.carbondata.core.datastore.compression.Compressor; import org.apache.carbondata.core.datastore.compression.CompressorFactory; import org.apache.carbondata.core.datastore.page.ColumnPage; @@ -47,7 +47,7 @@ public ColumnPageEncoder createEncoder(Map parameter) { return new IndexStorageEncoder() { @Override void encodeIndexStorage(ColumnPage inputPage) { - IndexStorage indexStorage; + BlockIndexerStorage indexStorage; byte[][] data = inputPage.getByteArrayPage(); if (isInvertedIndex) { indexStorage = new BlockIndexerStorageForShort(data, true, false, isSort); diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/dimension/legacy/DirectDictDimensionIndexCodec.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/dimension/legacy/DirectDictDimensionIndexCodec.java index a130cbdacc2..15827f8c0cd 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/dimension/legacy/DirectDictDimensionIndexCodec.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/dimension/legacy/DirectDictDimensionIndexCodec.java @@ -21,9 +21,9 @@ import java.util.List; import java.util.Map; +import org.apache.carbondata.core.datastore.columnar.BlockIndexerStorage; import org.apache.carbondata.core.datastore.columnar.BlockIndexerStorageForNoInvertedIndexForShort; import org.apache.carbondata.core.datastore.columnar.BlockIndexerStorageForShort; -import org.apache.carbondata.core.datastore.columnar.IndexStorage; import org.apache.carbondata.core.datastore.compression.Compressor; import org.apache.carbondata.core.datastore.compression.CompressorFactory; import org.apache.carbondata.core.datastore.page.ColumnPage; @@ -47,7 +47,7 @@ public ColumnPageEncoder createEncoder(Map parameter) { return new IndexStorageEncoder() { @Override void encodeIndexStorage(ColumnPage inputPage) { - IndexStorage indexStorage; + BlockIndexerStorage indexStorage; byte[][] data = inputPage.getByteArrayPage(); if (isInvertedIndex) { indexStorage = new BlockIndexerStorageForShort(data, false, false, isSort); diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/dimension/legacy/HighCardDictDimensionIndexCodec.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/dimension/legacy/HighCardDictDimensionIndexCodec.java index bce85238c6d..7a1627cb279 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/dimension/legacy/HighCardDictDimensionIndexCodec.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/dimension/legacy/HighCardDictDimensionIndexCodec.java @@ -21,9 +21,9 @@ import java.util.List; import java.util.Map; +import org.apache.carbondata.core.datastore.columnar.BlockIndexerStorage; import org.apache.carbondata.core.datastore.columnar.BlockIndexerStorageForNoInvertedIndexForShort; import org.apache.carbondata.core.datastore.columnar.BlockIndexerStorageForShort; -import org.apache.carbondata.core.datastore.columnar.IndexStorage; import org.apache.carbondata.core.datastore.compression.Compressor; import org.apache.carbondata.core.datastore.compression.CompressorFactory; import org.apache.carbondata.core.datastore.page.ColumnPage; @@ -54,7 +54,7 @@ public ColumnPageEncoder createEncoder(Map parameter) { @Override protected void encodeIndexStorage(ColumnPage input) { - IndexStorage indexStorage; + BlockIndexerStorage indexStorage; byte[][] data = input.getByteArrayPage(); boolean isDictionary = input.isLocalDictGeneratedPage(); if (isInvertedIndex) { diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/dimension/legacy/IndexStorageEncoder.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/dimension/legacy/IndexStorageEncoder.java index 37d90522506..96fcc0847b7 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/dimension/legacy/IndexStorageEncoder.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/dimension/legacy/IndexStorageEncoder.java @@ -22,7 +22,7 @@ import java.io.IOException; import org.apache.carbondata.core.constants.CarbonCommonConstants; -import org.apache.carbondata.core.datastore.columnar.IndexStorage; +import org.apache.carbondata.core.datastore.columnar.BlockIndexerStorage; import org.apache.carbondata.core.datastore.page.ColumnPage; import org.apache.carbondata.core.datastore.page.encoding.ColumnPageEncoder; import org.apache.carbondata.core.datastore.page.encoding.ColumnPageEncoderMeta; @@ -31,7 +31,7 @@ import org.apache.carbondata.format.SortState; public abstract class IndexStorageEncoder extends ColumnPageEncoder { - IndexStorage indexStorage; + BlockIndexerStorage indexStorage; byte[] compressedDataPage; abstract void encodeIndexStorage(ColumnPage inputPage); @@ -61,7 +61,9 @@ protected byte[] encodeData(ColumnPage input) throws MemoryException, IOExceptio out.writeShort(dataRle); } } - return stream.toByteArray(); + byte[] result = stream.toByteArray(); + stream.close(); + return result; } @Override diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/key/TablePageKey.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/key/TablePageKey.java index c2b722bbaa8..609f17ede59 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/page/key/TablePageKey.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/key/TablePageKey.java @@ -61,7 +61,8 @@ public TablePageKey(int pageSize, SegmentProperties segmentProperties, /** update all keys based on the input row */ public void update(int rowId, CarbonRow row, byte[] mdk) { if (hasNoDictionary) { - currentNoDictionaryKey = WriteStepRowUtil.getNoDictAndComplexDimension(row); + Object[] noDictAndComplexDimension = WriteStepRowUtil.getNoDictAndComplexDimension(row); + currentNoDictionaryKey = new byte[noDictAndComplexDimension.length][0]; } if (rowId == 0) { startKey = mdk; diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/TablePageStatistics.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/TablePageStatistics.java index 46ad09c1f05..1a68fa23013 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/TablePageStatistics.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/statistics/TablePageStatistics.java @@ -19,6 +19,7 @@ import org.apache.carbondata.core.datastore.page.encoding.EncodedColumnPage; import org.apache.carbondata.core.util.CarbonUtil; +import org.apache.carbondata.core.util.DataTypeUtil; // Statistics of dimension and measure column in a TablePage public class TablePageStatistics { @@ -50,8 +51,17 @@ public TablePageStatistics(EncodedColumnPage[] dimensions, private void updateDimensionMinMax(EncodedColumnPage[] dimensions) { for (int i = 0; i < dimensions.length; i++) { SimpleStatsResult stats = dimensions[i].getStats(); - dimensionMaxValue[i] = CarbonUtil.getValueAsBytes(stats.getDataType(), stats.getMax()); - dimensionMinValue[i] = CarbonUtil.getValueAsBytes(stats.getDataType(), stats.getMin()); + Object min = stats.getMin(); + Object max = stats.getMax(); + if (CarbonUtil.isEncodedWithMeta(dimensions[i].getPageMetadata().getEncoders())) { + dimensionMaxValue[i] = DataTypeUtil + .getMinMaxBytesBasedOnDataTypeForNoDictionaryColumn(max, stats.getDataType()); + dimensionMinValue[i] = DataTypeUtil + .getMinMaxBytesBasedOnDataTypeForNoDictionaryColumn(min, stats.getDataType()); + } else { + dimensionMaxValue[i] = CarbonUtil.getValueAsBytes(stats.getDataType(), max); + dimensionMinValue[i] = CarbonUtil.getValueAsBytes(stats.getDataType(), min); + } } } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/row/WriteStepRowUtil.java b/core/src/main/java/org/apache/carbondata/core/datastore/row/WriteStepRowUtil.java index 08dd800b9ec..3d9de56d3cb 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/row/WriteStepRowUtil.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/row/WriteStepRowUtil.java @@ -20,7 +20,9 @@ import org.apache.carbondata.core.datastore.block.SegmentProperties; import org.apache.carbondata.core.keygenerator.KeyGenException; import org.apache.carbondata.core.keygenerator.KeyGenerator; +import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn; import org.apache.carbondata.core.scan.wrappers.ByteArrayWrapper; +import org.apache.carbondata.core.util.DataTypeUtil; // Utility to create and retrieve data from CarbonRow in write step. public class WriteStepRowUtil { @@ -34,7 +36,7 @@ public class WriteStepRowUtil { public static final int NO_DICTIONARY_AND_COMPLEX = 1; public static final int MEASURE = 2; - public static CarbonRow fromColumnCategory(int[] dictDimensions, byte[][] noDictAndComplex, + public static CarbonRow fromColumnCategory(int[] dictDimensions, Object[] noDictAndComplex, Object[] measures) { Object[] row = new Object[3]; row[DICTIONARY_DIMENSION] = dictDimensions; @@ -43,7 +45,8 @@ public static CarbonRow fromColumnCategory(int[] dictDimensions, byte[][] noDict return new CarbonRow(row); } - public static CarbonRow fromMergerRow(Object[] row, SegmentProperties segmentProperties) { + public static CarbonRow fromMergerRow(Object[] row, SegmentProperties segmentProperties, + CarbonColumn[] noDicAndComplexColumns) { Object[] converted = new Object[3]; // dictionary dimension @@ -55,8 +58,23 @@ public static CarbonRow fromMergerRow(Object[] row, SegmentProperties segmentPro } converted[DICTIONARY_DIMENSION] = dictDimensions; + byte[][] noDictionaryKeys = ((ByteArrayWrapper) row[0]).getNoDictionaryKeys(); + Object[] noDictKeys = new Object[noDictionaryKeys.length]; + for (int i = 0; i < noDictionaryKeys.length; i++) { + // in case of compaction rows are collected from result collector and are in byte[]. + // Convert the no dictionary columns to original data, + // as load expects the no dictionary column with original data. + if (DataTypeUtil.isPrimitiveColumn(noDicAndComplexColumns[i].getDataType())) { + noDictKeys[i] = DataTypeUtil + .getDataBasedOnDataTypeForNoDictionaryColumn(noDictionaryKeys[i], + noDicAndComplexColumns[i].getDataType()); + } else { + noDictKeys[i] = noDictionaryKeys[i]; + } + } + // no dictionary and complex dimension - converted[NO_DICTIONARY_AND_COMPLEX] = ((ByteArrayWrapper) row[0]).getNoDictionaryKeys(); + converted[NO_DICTIONARY_AND_COMPLEX] = noDictKeys; // measure int measureCount = row.length - 1; @@ -75,8 +93,8 @@ public static byte[] getMdk(CarbonRow row, KeyGenerator keyGenerator) throws Key return keyGenerator.generateKey(getDictDimension(row)); } - public static byte[][] getNoDictAndComplexDimension(CarbonRow row) { - return (byte[][]) row.getData()[NO_DICTIONARY_AND_COMPLEX]; + public static Object[] getNoDictAndComplexDimension(CarbonRow row) { + return (Object[]) row.getData()[NO_DICTIONARY_AND_COMPLEX]; } public static Object[] getMeasure(CarbonRow row) { diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java index 7cc2b093415..219e7a2c075 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java @@ -58,7 +58,10 @@ import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf; import org.apache.carbondata.core.scan.model.ProjectionDimension; import org.apache.carbondata.core.scan.model.ProjectionMeasure; +import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector; +import org.apache.carbondata.core.util.ByteUtil; import org.apache.carbondata.core.util.CarbonUtil; +import org.apache.carbondata.core.util.DataTypeUtil; import org.apache.commons.lang3.ArrayUtils; @@ -745,4 +748,37 @@ public static void updateColumnUniqueIdForNonTransactionTable(List } } } + + /** + * Put the data to vector + * + * @param vector + * @param value + * @param vectorRow + * @param length + */ + public static void putDataToVector(CarbonColumnVector vector, byte[] value, int vectorRow, + int length) { + DataType dt = vector.getType(); + if ((!(dt == DataTypes.STRING) && length == 0) || ByteUtil.UnsafeComparer.INSTANCE + .equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY, 0, + CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY.length, value, 0, length)) { + vector.putNull(vectorRow); + } else { + if (dt == DataTypes.STRING) { + vector.putBytes(vectorRow, 0, length, value); + } else if (dt == DataTypes.BOOLEAN) { + vector.putBoolean(vectorRow, ByteUtil.toBoolean(value[0])); + } else if (dt == DataTypes.SHORT) { + vector.putShort(vectorRow, ByteUtil.toXorShort(value, 0, length)); + } else if (dt == DataTypes.INT) { + vector.putInt(vectorRow, ByteUtil.toXorInt(value, 0, length)); + } else if (dt == DataTypes.LONG) { + vector.putLong(vectorRow, DataTypeUtil + .getDataBasedOnRestructuredDataType(value, vector.getBlockDataType(), 0, length)); + } else if (dt == DataTypes.TIMESTAMP) { + vector.putLong(vectorRow, ByteUtil.toXorLong(value, 0, length) * 1000L); + } + } + } } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java index 410f67a44d8..8c0ea561d01 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java @@ -645,8 +645,9 @@ public static ColumnFilterInfo getMeasureValKeyMemberForFilter( continue; } - filterValuesList - .add(DataTypeUtil.getMeasureValueBasedOnDataType(result, dataType, carbonMeasure)); + filterValuesList.add(DataTypeUtil + .getMeasureValueBasedOnDataType(result, dataType, carbonMeasure.getScale(), + carbonMeasure.getPrecision())); } } catch (Throwable ex) { @@ -2178,4 +2179,41 @@ public static FilterExecuter getFilterExecutorForRangeFilters( } return filterExecuter; } + + /** + * This method is used to compare the filter value with min and max values. + * This is used in case of filter queries on no dictionary column. + * + * @param filterValue + * @param minMaxBytes + * @param carbonDimension + * @param isMin + * @return + */ + public static int compareValues(byte[] filterValue, byte[] minMaxBytes, + CarbonDimension carbonDimension, boolean isMin) { + DataType dataType = carbonDimension.getDataType(); + if (DataTypeUtil.isPrimitiveColumn(dataType) && !carbonDimension + .hasEncoding(Encoding.DICTIONARY)) { + Object value = + DataTypeUtil.getDataBasedOnDataTypeForNoDictionaryColumn(minMaxBytes, dataType); + // filter value should be in range of max and min value i.e + // max>filtervalue>min + // so filter-max should be negative + Object data = DataTypeUtil.getDataBasedOnDataTypeForNoDictionaryColumn(filterValue, dataType); + SerializableComparator comparator = Comparator.getComparator(dataType); + if (isMin) { + return comparator.compare(value, data); + } else { + return comparator.compare(data, value); + } + } else { + if (isMin) { + return ByteUtil.UnsafeComparer.INSTANCE.compareTo(minMaxBytes, filterValue); + } else { + return ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValue, minMaxBytes); + } + } + } + } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java index 71646c90067..5eb5d7c6df0 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java @@ -397,7 +397,7 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage, return bitSet; } // binary search can only be applied if column is sorted - if (isNaturalSorted) { + if (isNaturalSorted && dimensionColumnPage.isExplicitSorted()) { int startIndex = 0; for (int i = 0; i < filterValues.length; i++) { if (startIndex >= numerOfRows) { diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java index 974830d546f..0884a49847d 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.util.BitSet; +import org.apache.carbondata.core.constants.CarbonCommonConstants; import org.apache.carbondata.core.datastore.block.SegmentProperties; import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage; import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; @@ -26,6 +27,7 @@ import org.apache.carbondata.core.datastore.page.ColumnPage; import org.apache.carbondata.core.metadata.datatype.DataType; import org.apache.carbondata.core.metadata.datatype.DataTypes; +import org.apache.carbondata.core.metadata.encoder.Encoding; import org.apache.carbondata.core.scan.filter.FilterUtil; import org.apache.carbondata.core.scan.filter.intf.RowIntf; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo; @@ -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], + dimensionRawColumnChunk.getMinValues()[i], dimColumnExecuterInfo.getFilterKeys(), + dimColumnEvaluatorInfo.getDimension().getDataType()); + } else { + scanRequired = isScanRequired(dimensionRawColumnChunk.getMaxValues()[i], + dimensionRawColumnChunk.getMinValues()[i], dimColumnExecuterInfo.getFilterKeys()); + } + if (scanRequired) { DimensionColumnPage dimensionColumnPage = dimensionRawColumnChunk.decodeColumnPage(i); if (!isDecoded) { filterValues = FilterUtil @@ -412,7 +425,7 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage, } // binary search can only be applied if column is sorted and // inverted index exists for that column - if (isNaturalSorted) { + if (isNaturalSorted && dimensionColumnPage.isExplicitSorted()) { int startIndex = 0; for (int i = 0; i < filterValues.length; i++) { if (startIndex >= numerOfRows) { @@ -458,7 +471,16 @@ public BitSet isScanRequired(byte[][] blkMaxVal, byte[][] blkMinVal) { if (isDimensionPresentInCurrentBlock) { filterValues = dimColumnExecuterInfo.getFilterKeys(); chunkIndex = dimColumnEvaluatorInfo.getColumnIndexInMinMaxByteArray(); - isScanRequired = isScanRequired(blkMaxVal[chunkIndex], blkMinVal[chunkIndex], filterValues); + // 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)) { + isScanRequired = isScanRequired(blkMaxVal[chunkIndex], blkMinVal[chunkIndex], filterValues, + dimColumnEvaluatorInfo.getDimension().getDataType()); + } else { + isScanRequired = isScanRequired(blkMaxVal[chunkIndex], blkMinVal[chunkIndex], filterValues); + } } else if (isMeasurePresentInCurrentBlock) { chunkIndex = msrColumnEvaluatorInfo.getColumnIndexInMinMaxByteArray(); isScanRequired = isScanRequired(blkMaxVal[chunkIndex], blkMinVal[chunkIndex], @@ -494,6 +516,34 @@ private boolean isScanRequired(byte[] blkMaxVal, byte[] blkMinVal, byte[][] filt return isScanRequired; } + private boolean isScanRequired(byte[] blkMaxVal, byte[] blkMinVal, byte[][] filterValues, + DataType dataType) { + boolean isScanRequired = false; + Object minValue = DataTypeUtil.getDataBasedOnDataTypeForNoDictionaryColumn(blkMinVal, dataType); + Object maxValue = DataTypeUtil.getDataBasedOnDataTypeForNoDictionaryColumn(blkMaxVal, dataType); + for (int k = 0; k < filterValues.length; k++) { + if (ByteUtil.UnsafeComparer.INSTANCE + .compareTo(filterValues[k], CarbonCommonConstants.EMPTY_BYTE_ARRAY) == 0) { + return true; + } + // filter value should be in range of max and min value i.e + // max>filtervalue>min + // so filter-max should be negative + Object data = + DataTypeUtil.getDataBasedOnDataTypeForNoDictionaryColumn(filterValues[k], dataType); + SerializableComparator comparator = Comparator.getComparator(dataType); + int maxCompare = comparator.compare(data, maxValue); + int minCompare = comparator.compare(data, minValue); + // if any filter value is in range than this block needs to be + // scanned + if (maxCompare <= 0 && minCompare >= 0) { + isScanRequired = true; + break; + } + } + return isScanRequired; + } + private boolean isScanRequired(byte[] maxValue, byte[] minValue, Object[] filterValue, DataType dataType) { Object maxObject = DataTypeUtil.getMeasureObjectFromDataType(maxValue, dataType); diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java index 9718dcfa2aa..89abad012f5 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java @@ -121,13 +121,13 @@ private void ifDefaultValueMatchesFilter() { CarbonDimension dimension = this.dimColEvaluatorInfo.getDimension(); byte[] defaultValue = dimension.getDefaultValue(); if (null != defaultValue) { - int maxCompare = - ByteUtil.UnsafeComparer.INSTANCE.compareTo(defaultValue, filterRangesValues[0]); int minCompare = - ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterRangesValues[1], defaultValue); + FilterUtil.compareValues(filterRangesValues[0], defaultValue, dimension, true); + int maxCompare = + FilterUtil.compareValues(filterRangesValues[1], defaultValue, dimension, false); - if (((greaterThanExp && maxCompare > 0) || (greaterThanEqualExp && maxCompare >= 0)) - && ((lessThanExp && minCompare > 0) || (lessThanEqualExp && minCompare >= 0))) { + if (((greaterThanExp && maxCompare > 0) || (greaterThanEqualExp && maxCompare >= 0)) && ( + (lessThanExp && minCompare > 0) || (lessThanEqualExp && minCompare >= 0))) { isDefaultValuePresentInFilter = true; } } @@ -270,30 +270,35 @@ public boolean isScanRequired(byte[] blockMinValue, byte[] blockMaxValue, byte[] // Case D: Filter Values Completely overlaps Block Min and Max then all bits are set. // Block Min <-----------------------> Block Max // Filter Min <-----------------------------------------------> Filter Max + // for no dictionary measure column comparison can be done + // on the original data as like measure column if (isDimensionPresentInCurrentBlock) { + CarbonDimension carbonDimension = dimColEvaluatorInfo.getDimension(); if (((lessThanExp) && ( - ByteUtil.UnsafeComparer.INSTANCE.compareTo(blockMinValue, filterValues[1]) >= 0)) || ( - (lessThanEqualExp) && ( - ByteUtil.UnsafeComparer.INSTANCE.compareTo(blockMinValue, filterValues[1]) > 0)) || ( + FilterUtil.compareValues(filterValues[1], blockMinValue, carbonDimension, true) >= 0)) + || ((lessThanEqualExp) && ( + FilterUtil.compareValues(filterValues[1], blockMinValue, carbonDimension, true) > 0)) || ( (greaterThanExp) && ( - ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[0], blockMaxValue) >= 0)) || ( - (greaterThanEqualExp) && ( - ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[0], blockMaxValue) > 0))) { + FilterUtil.compareValues(filterValues[0], blockMaxValue, carbonDimension, false) + >= 0)) || ((greaterThanEqualExp) && ( + FilterUtil.compareValues(filterValues[0], blockMaxValue, carbonDimension, false) > 0))) { // completely out of block boundary isScanRequired = false; } else { if (((greaterThanExp) && ( - ByteUtil.UnsafeComparer.INSTANCE.compareTo(blockMinValue, filterValues[0]) > 0)) || ( - (greaterThanEqualExp) && ( - ByteUtil.UnsafeComparer.INSTANCE.compareTo(blockMinValue, filterValues[0]) >= 0))) { + FilterUtil.compareValues(filterValues[0], blockMinValue, carbonDimension, true) > 0)) + || ((greaterThanEqualExp) && ( + FilterUtil.compareValues(filterValues[0], blockMinValue, carbonDimension, true) + >= 0))) { startBlockMinIsDefaultStart = true; } if (((lessThanExp) && ( - ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[1], blockMaxValue) > 0)) || ( - (lessThanEqualExp) && ( - ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[1], blockMaxValue) >= 0))) { + FilterUtil.compareValues(filterValues[1], blockMaxValue, carbonDimension, false) > 0)) + || ((lessThanEqualExp) && ( + FilterUtil.compareValues(filterValues[1], blockMaxValue, carbonDimension, false) + >= 0))) { endBlockMaxisDefaultEnd = true; } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureEvaluatorImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureEvaluatorImpl.java index f90123812d0..c403846c10c 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureEvaluatorImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureEvaluatorImpl.java @@ -29,9 +29,9 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure; import org.apache.carbondata.core.scan.executor.util.RestructureUtil; import org.apache.carbondata.core.scan.filter.ColumnFilterInfo; +import org.apache.carbondata.core.scan.filter.FilterUtil; 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.util.ByteUtil; import org.apache.carbondata.core.util.comparator.Comparator; import org.apache.carbondata.core.util.comparator.SerializableComparator; @@ -65,7 +65,7 @@ protected boolean isDimensionDefaultValuePresentInFilterValues( } List noDictionaryFilterValuesList = filterValues.getNoDictionaryFilterValuesList(); for (byte[] filterValue : noDictionaryFilterValuesList) { - int compare = ByteUtil.UnsafeComparer.INSTANCE.compareTo(defaultValue, filterValue); + int compare = FilterUtil.compareValues(filterValue, defaultValue, dimension, true); if (compare == 0) { isDefaultValuePresentInFilterValues = true; break; 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 7e5826c1447..bb010f536b4 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 @@ -36,6 +36,7 @@ import org.apache.carbondata.core.datastore.block.SegmentProperties; import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage; import org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionColumnPage; +import org.apache.carbondata.core.datastore.chunk.store.ColumnPageWrapper; import org.apache.carbondata.core.datastore.page.ColumnPage; import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator; import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory; @@ -406,12 +407,11 @@ private void createRow(RawBlockletColumnChunks blockChunkHolder, RowIntf row, in DimensionColumnPage columnDataChunk = blockChunkHolder.getDimensionRawColumnChunks()[dimensionChunkIndex[i]] .decodeColumnPage(pageIndex); - if (!dimColumnEvaluatorInfo.getDimension().hasEncoding(Encoding.DICTIONARY) - && columnDataChunk instanceof VariableLengthDimensionColumnPage) { + if (!dimColumnEvaluatorInfo.getDimension().hasEncoding(Encoding.DICTIONARY) && ( + columnDataChunk instanceof VariableLengthDimensionColumnPage + || columnDataChunk instanceof ColumnPageWrapper)) { - VariableLengthDimensionColumnPage dimensionColumnDataChunk = - (VariableLengthDimensionColumnPage) columnDataChunk; - byte[] memberBytes = dimensionColumnDataChunk.getChunkData(index); + byte[] memberBytes = columnDataChunk.getChunkData(index); if (null != memberBytes) { if (Arrays.equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY, memberBytes)) { memberBytes = null; diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java index e561f969c53..7d9d4d182c2 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java @@ -126,7 +126,15 @@ public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) { isScanRequired(maxValue, msrFilterRangeValues, msrColEvalutorInfoList.get(0).getType()); } else { maxValue = blockMaxValue[dimensionChunkIndex[0]]; - isScanRequired = isScanRequired(maxValue, filterRangeValues); + DataType dataType = dimColEvaluatorInfoList.get(0).getDimension().getDataType(); + // for no dictionary measure column comparison can be done + // on the original data as like measure column + if (DataTypeUtil.isPrimitiveColumn(dataType) && !dimColEvaluatorInfoList.get(0) + .getDimension().hasEncoding(Encoding.DICTIONARY)) { + isScanRequired = isScanRequired(maxValue, filterRangeValues, dataType); + } else { + isScanRequired = isScanRequired(maxValue, filterRangeValues); + } } } else { isScanRequired = isDefaultValuePresentInFilter; @@ -156,6 +164,32 @@ private boolean isScanRequired(byte[] blockMaxValue, byte[][] filterValues) { return isScanRequired; } + private boolean isScanRequired(byte[] blockMaxValue, byte[][] filterValues, DataType dataType) { + boolean isScanRequired = false; + Object maxValue = + DataTypeUtil.getDataBasedOnDataTypeForNoDictionaryColumn(blockMaxValue, dataType); + for (int k = 0; k < filterValues.length; k++) { + if (ByteUtil.UnsafeComparer.INSTANCE + .compareTo(filterValues[k], CarbonCommonConstants.EMPTY_BYTE_ARRAY) == 0) { + return true; + } + // filter value should be in range of max and min value i.e + // max>filtervalue>min + // so filter-max should be negative + Object data = + DataTypeUtil.getDataBasedOnDataTypeForNoDictionaryColumn(filterValues[k], dataType); + SerializableComparator comparator = Comparator.getComparator(dataType); + int maxCompare = comparator.compare(data, maxValue); + // if any filter value is in range than this block needs to be + // scanned less than equal to max range. + if (maxCompare <= 0) { + isScanRequired = true; + break; + } + } + return isScanRequired; + } + private boolean isScanRequired(byte[] maxValue, Object[] filterValue, DataType dataType) { Object value = DataTypeUtil.getMeasureObjectFromDataType(maxValue, dataType); @@ -196,7 +230,18 @@ public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks, boolean isExclude = false; for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) { if (rawColumnChunk.getMaxValues() != null) { - if (isScanRequired(rawColumnChunk.getMaxValues()[i], this.filterRangeValues)) { + boolean scanRequired; + DataType dataType = dimColEvaluatorInfoList.get(0).getDimension().getDataType(); + // for no dictionary measure column comparison can be done + // on the original data as like measure column + if (DataTypeUtil.isPrimitiveColumn(dataType) && !dimColEvaluatorInfoList.get(0) + .getDimension().hasEncoding(Encoding.DICTIONARY)) { + scanRequired = + isScanRequired(rawColumnChunk.getMaxValues()[i], this.filterRangeValues, dataType); + } else { + scanRequired = isScanRequired(rawColumnChunk.getMaxValues()[i], this.filterRangeValues); + } + if (scanRequired) { int compare = ByteUtil.UnsafeComparer.INSTANCE .compareTo(filterRangeValues[0], rawColumnChunk.getMinValues()[i]); if (compare < 0) { @@ -417,7 +462,7 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage, BitSet bitSet = new BitSet(numerOfRows); byte[][] filterValues = this.filterRangeValues; // binary search can only be applied if column is sorted - if (isNaturalSorted) { + if (isNaturalSorted && dimensionColumnPage.isExplicitSorted()) { int start = 0; int last = 0; int startIndex = 0; diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java index 53c2d76a062..5d135d2200e 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java @@ -86,7 +86,7 @@ private void ifDefaultValueMatchesFilter() { if (null != defaultValue) { for (int k = 0; k < filterRangeValues.length; k++) { int maxCompare = - ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterRangeValues[k], defaultValue); + FilterUtil.compareValues(filterRangeValues[k], defaultValue, dimension, false); if (maxCompare <= 0) { isDefaultValuePresentInFilter = true; break; @@ -123,7 +123,15 @@ private void ifDefaultValueMatchesFilter() { isScanRequired(maxValue, msrFilterRangeValues, msrColEvalutorInfoList.get(0).getType()); } else { maxValue = blockMaxValue[dimensionChunkIndex[0]]; - isScanRequired = isScanRequired(maxValue, filterRangeValues); + DataType dataType = dimColEvaluatorInfoList.get(0).getDimension().getDataType(); + // for no dictionary measure column comparison can be done + // on the original data as like measure column + if (DataTypeUtil.isPrimitiveColumn(dataType) && !dimColEvaluatorInfoList.get(0) + .getDimension().hasEncoding(Encoding.DICTIONARY)) { + isScanRequired = isScanRequired(maxValue, filterRangeValues, dataType); + } else { + isScanRequired = isScanRequired(maxValue, filterRangeValues); + } } } else { isScanRequired = isDefaultValuePresentInFilter; @@ -152,6 +160,32 @@ private boolean isScanRequired(byte[] blockMaxValue, byte[][] filterValues) { return isScanRequired; } + private boolean isScanRequired(byte[] blockMaxValue, byte[][] filterValues, DataType dataType) { + boolean isScanRequired = false; + Object maxValue = + DataTypeUtil.getDataBasedOnDataTypeForNoDictionaryColumn(blockMaxValue, dataType); + for (int k = 0; k < filterValues.length; k++) { + if (ByteUtil.UnsafeComparer.INSTANCE + .compareTo(filterValues[k], CarbonCommonConstants.EMPTY_BYTE_ARRAY) == 0) { + return true; + } + // filter value should be in range of max and min value i.e + // max>filtervalue>min + // so filter-max should be negative + Object data = + DataTypeUtil.getDataBasedOnDataTypeForNoDictionaryColumn(filterValues[k], dataType); + SerializableComparator comparator = Comparator.getComparator(dataType); + int maxCompare = comparator.compare(data, maxValue); + // if any filter value is in range than this block needs to be + // scanned less than equal to max range. + if (maxCompare <= 0) { + isScanRequired = true; + break; + } + } + return isScanRequired; + } + private boolean isScanRequired(byte[] maxValue, Object[] filterValue, DataType dataType) { Object value = DataTypeUtil.getMeasureObjectFromDataType(maxValue, dataType); @@ -193,7 +227,18 @@ public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks, boolean isExclude = false; for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) { if (rawColumnChunk.getMaxValues() != null) { - if (isScanRequired(rawColumnChunk.getMaxValues()[i], this.filterRangeValues)) { + boolean scanRequired; + DataType dataType = dimColEvaluatorInfoList.get(0).getDimension().getDataType(); + // for no dictionary measure column comparison can be done + // on the original data as like measure column + if (DataTypeUtil.isPrimitiveColumn(dataType) && !dimColEvaluatorInfoList.get(0) + .getDimension().hasEncoding(Encoding.DICTIONARY)) { + scanRequired = + isScanRequired(rawColumnChunk.getMaxValues()[i], this.filterRangeValues, dataType); + } else { + scanRequired = isScanRequired(rawColumnChunk.getMaxValues()[i], this.filterRangeValues); + } + if (scanRequired) { int compare = ByteUtil.UnsafeComparer.INSTANCE .compareTo(filterRangeValues[0], rawColumnChunk.getMinValues()[i]); if (compare <= 0) { @@ -404,7 +449,7 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage, BitSet bitSet = new BitSet(numerOfRows); byte[][] filterValues = this.filterRangeValues; // binary search can only be applied if column is sorted - if (isNaturalSorted) { + if (isNaturalSorted && dimensionColumnPage.isExplicitSorted()) { int start = 0; int last = 0; int startIndex = 0; diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java index f410322df6f..6b37163baff 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java @@ -126,7 +126,15 @@ private void ifDefaultValueMatchesFilter() { isScanRequired(minValue, msrFilterRangeValues, msrColEvalutorInfoList.get(0).getType()); } else { minValue = blockMinValue[dimensionChunkIndex[0]]; - isScanRequired = isScanRequired(minValue, filterRangeValues); + DataType dataType = dimColEvaluatorInfoList.get(0).getDimension().getDataType(); + // for no dictionary measure column comparison can be done + // on the original data as like measure column + if (DataTypeUtil.isPrimitiveColumn(dataType) && !dimColEvaluatorInfoList.get(0) + .getDimension().hasEncoding(Encoding.DICTIONARY)) { + isScanRequired = isScanRequired(minValue, filterRangeValues, dataType); + } else { + isScanRequired = isScanRequired(minValue, filterRangeValues); + } } } else { isScanRequired = isDefaultValuePresentInFilter; @@ -154,6 +162,32 @@ private boolean isScanRequired(byte[] blockMinValue, byte[][] filterValues) { return isScanRequired; } + private boolean isScanRequired(byte[] blockMinValue, byte[][] filterValues, DataType dataType) { + boolean isScanRequired = false; + Object minValue = + DataTypeUtil.getDataBasedOnDataTypeForNoDictionaryColumn(blockMinValue, dataType); + for (int k = 0; k < filterValues.length; k++) { + if (ByteUtil.UnsafeComparer.INSTANCE + .compareTo(filterValues[k], CarbonCommonConstants.EMPTY_BYTE_ARRAY) == 0) { + return true; + } + // filter value should be in range of max and min value i.e + // max>filtervalue>min + // so filter-max should be negative + Object data = + DataTypeUtil.getDataBasedOnDataTypeForNoDictionaryColumn(filterValues[k], dataType); + SerializableComparator comparator = Comparator.getComparator(dataType); + int minCompare = comparator.compare(data, minValue); + // if any filter value is in range than this block needs to be + // scanned less than equal to max range. + if (minCompare >= 0) { + isScanRequired = true; + break; + } + } + return isScanRequired; + } + private boolean isScanRequired(byte[] minValue, Object[] filterValue, DataType dataType) { Object value = @@ -195,7 +229,18 @@ public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks, boolean isExclude = false; for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) { if (rawColumnChunk.getMinValues() != null) { - if (isScanRequired(rawColumnChunk.getMinValues()[i], this.filterRangeValues)) { + boolean scanRequired; + DataType dataType = dimColEvaluatorInfoList.get(0).getDimension().getDataType(); + // for no dictionary measure column comparison can be done + // on the original data as like measure column + if (DataTypeUtil.isPrimitiveColumn(dataType) && !dimColEvaluatorInfoList.get(0) + .getDimension().hasEncoding(Encoding.DICTIONARY)) { + scanRequired = + isScanRequired(rawColumnChunk.getMinValues()[i], this.filterRangeValues, dataType); + } else { + scanRequired = isScanRequired(rawColumnChunk.getMinValues()[i], this.filterRangeValues); + } + if (scanRequired) { BitSet bitSet; DimensionColumnPage dimensionColumnPage = rawColumnChunk.decodeColumnPage(i); if (null != rawColumnChunk.getLocalDictionary()) { @@ -426,7 +471,7 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage, BitSet bitSet = new BitSet(numerOfRows); byte[][] filterValues = this.filterRangeValues; // binary search can only be applied if column is sorted - if (isNaturalSorted) { + if (isNaturalSorted && dimensionColumnPage.isExplicitSorted()) { int start = 0; int last = 0; int startIndex = 0; diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFilterExecuterImpl.java index 1f3b9a427db..d9afabd2e7e 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFilterExecuterImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFilterExecuterImpl.java @@ -126,7 +126,15 @@ private void ifDefaultValueMatchesFilter() { isScanRequired(minValue, msrFilterRangeValues, msrColEvalutorInfoList.get(0).getType()); } else { minValue = blockMinValue[dimensionChunkIndex[0]]; - isScanRequired = isScanRequired(minValue, filterRangeValues); + DataType dataType = dimColEvaluatorInfoList.get(0).getDimension().getDataType(); + // for no dictionary measure column comparison can be done + // on the original data as like measure column + if (DataTypeUtil.isPrimitiveColumn(dataType) && !dimColEvaluatorInfoList.get(0) + .getDimension().hasEncoding(Encoding.DICTIONARY)) { + isScanRequired = isScanRequired(minValue, filterRangeValues, dataType); + } else { + isScanRequired = isScanRequired(minValue, filterRangeValues); + } } } else { isScanRequired = isDefaultValuePresentInFilter; @@ -155,6 +163,32 @@ private boolean isScanRequired(byte[] blockMinValue, byte[][] filterValues) { return isScanRequired; } + private boolean isScanRequired(byte[] blockMinValue, byte[][] filterValues, DataType dataType) { + boolean isScanRequired = false; + Object minValue = + DataTypeUtil.getDataBasedOnDataTypeForNoDictionaryColumn(blockMinValue, dataType); + for (int k = 0; k < filterValues.length; k++) { + if (ByteUtil.UnsafeComparer.INSTANCE + .compareTo(filterValues[k], CarbonCommonConstants.EMPTY_BYTE_ARRAY) == 0) { + return true; + } + // filter value should be in range of max and min value i.e + // max>filtervalue>min + // so filter-max should be negative + Object data = + DataTypeUtil.getDataBasedOnDataTypeForNoDictionaryColumn(filterValues[k], dataType); + SerializableComparator comparator = Comparator.getComparator(dataType); + int minCompare = comparator.compare(data, minValue); + // if any filter value is in range than this block needs to be + // scanned less than equal to max range. + if (minCompare >= 0) { + isScanRequired = true; + break; + } + } + return isScanRequired; + } + private boolean isScanRequired(byte[] minValue, Object[] filterValue, DataType dataType) { Object value = DataTypeUtil.getMeasureObjectFromDataType(minValue, dataType); @@ -195,7 +229,18 @@ public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks, boolean isExclude = false; for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) { if (rawColumnChunk.getMinValues() != null) { - if (isScanRequired(rawColumnChunk.getMinValues()[i], this.filterRangeValues)) { + boolean scanRequired; + DataType dataType = dimColEvaluatorInfoList.get(0).getDimension().getDataType(); + // for no dictionary measure column comparison can be done + // on the original data as like measure column + if (DataTypeUtil.isPrimitiveColumn(dataType) && !dimColEvaluatorInfoList.get(0) + .getDimension().hasEncoding(Encoding.DICTIONARY)) { + scanRequired = + isScanRequired(rawColumnChunk.getMinValues()[i], this.filterRangeValues, dataType); + } else { + scanRequired = isScanRequired(rawColumnChunk.getMinValues()[i], this.filterRangeValues); + } + if (scanRequired) { BitSet bitSet; DimensionColumnPage dimensionColumnPage = rawColumnChunk.decodeColumnPage(i); if (null != rawColumnChunk.getLocalDictionary()) { @@ -434,7 +479,7 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage, BitSet bitSet = new BitSet(numerOfRows); byte[][] filterValues = this.filterRangeValues; // binary search can only be applied if column is sorted - if (isNaturalSorted) { + if (isNaturalSorted && dimensionColumnPage.isExplicitSorted()) { int start = 0; int last = 0; int startIndex = 0; diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java index 2bc73c5e084..4a713d52eeb 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java @@ -210,7 +210,7 @@ private List getMeasureRangeValues(CarbonMeasure carbonMeasure) { continue; } filterValuesList.add(DataTypeUtil.getMeasureValueBasedOnDataType(result.getString(), - result.getDataType(), carbonMeasure)); + result.getDataType(), carbonMeasure.getScale(), carbonMeasure.getPrecision())); } catch (FilterIllegalMemberException e) { // Any invalid member while evaluation shall be ignored, system will log the // error only once since all rows the evaluation happens so inorder to avoid diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/BlockletScannedResult.java b/core/src/main/java/org/apache/carbondata/core/scan/result/BlockletScannedResult.java index daf9a915ef9..0f9ba223ce6 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/result/BlockletScannedResult.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/result/BlockletScannedResult.java @@ -240,10 +240,9 @@ public void fillColumnarDictionaryBatch(ColumnVectorInfo[] vectorInfo) { * Fill the column data to vector */ public void fillColumnarNoDictionaryBatch(ColumnVectorInfo[] vectorInfo) { - int column = 0; for (int i = 0; i < this.noDictionaryColumnChunkIndexes.length; i++) { - column = dimensionColumnPages[noDictionaryColumnChunkIndexes[i]][pageCounter] - .fillVector(vectorInfo, column); + dimensionColumnPages[noDictionaryColumnChunkIndexes[i]][pageCounter] + .fillVector(vectorInfo, i); } } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/impl/FilterQueryScannedResult.java b/core/src/main/java/org/apache/carbondata/core/scan/result/impl/FilterQueryScannedResult.java index 26b1135862a..1b831103926 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/result/impl/FilterQueryScannedResult.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/result/impl/FilterQueryScannedResult.java @@ -110,12 +110,9 @@ public void fillColumnarDictionaryBatch(ColumnVectorInfo[] vectorInfo) { * Fill the column data to vector */ public void fillColumnarNoDictionaryBatch(ColumnVectorInfo[] vectorInfo) { - int column = 0; - for (int chunkIndex : this.noDictionaryColumnChunkIndexes) { - column = dimensionColumnPages[chunkIndex][pageCounter].fillVector( - pageFilteredRowId[pageCounter], - vectorInfo, - column); + for (int index = 0; index < this.noDictionaryColumnChunkIndexes.length; index++) { + dimensionColumnPages[noDictionaryColumnChunkIndexes[index]][pageCounter] + .fillVector(pageFilteredRowId[pageCounter], vectorInfo, index); } } diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUnsafeUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUnsafeUtil.java new file mode 100644 index 00000000000..00e7dee6848 --- /dev/null +++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUnsafeUtil.java @@ -0,0 +1,95 @@ +/* + * 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; + +import org.apache.carbondata.core.memory.CarbonUnsafe; +import org.apache.carbondata.core.metadata.datatype.DataType; +import org.apache.carbondata.core.metadata.datatype.DataTypes; + +public class CarbonUnsafeUtil { + + /** + * 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; + } +} 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 dc039448b2a..9ab875c86e6 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 @@ -3242,6 +3242,28 @@ public static ColumnarFormatVersion getFormatVersion(CarbonTable carbonTable) th return version; } + /** + * Check if the page is adaptive encoded + * + * @param encodings + * @return + */ + public static boolean isEncodedWithMeta(List encodings) { + if (encodings != null && !encodings.isEmpty()) { + org.apache.carbondata.format.Encoding encoding = encodings.get(0); + switch (encoding) { + case DIRECT_COMPRESS: + case DIRECT_STRING: + case ADAPTIVE_INTEGRAL: + case ADAPTIVE_DELTA_INTEGRAL: + case ADAPTIVE_FLOATING: + case ADAPTIVE_DELTA_FLOATING: + return true; + } + } + return false; + } + /** * Check whether it is standard table means tablepath has Fact/Part0/Segment_ tail present with * all carbon files. In other cases carbon files present directly under tablepath or diff --git a/core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java b/core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java index fa08df9b719..612e17cd7e6 100644 --- a/core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java +++ b/core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java @@ -82,8 +82,8 @@ public final class DataTypeUtil { * @return */ public static Object getMeasureValueBasedOnDataType(String msrValue, DataType dataType, - CarbonMeasure carbonMeasure) { - return getMeasureValueBasedOnDataType(msrValue, dataType,carbonMeasure, false); + int scale, int precision) { + return getMeasureValueBasedOnDataType(msrValue, dataType, scale, precision, false); } /** @@ -95,13 +95,13 @@ public static Object getMeasureValueBasedOnDataType(String msrValue, DataType da * @return */ public static Object getMeasureValueBasedOnDataType(String msrValue, DataType dataType, - CarbonMeasure carbonMeasure, boolean useConverter) { + int scale, int precision, boolean useConverter) { if (dataType == DataTypes.BOOLEAN) { return BooleanConvert.parseBoolean(msrValue); } else if (DataTypes.isDecimal(dataType)) { BigDecimal bigDecimal = - new BigDecimal(msrValue).setScale(carbonMeasure.getScale(), RoundingMode.HALF_UP); - BigDecimal decimal = normalizeDecimalValue(bigDecimal, carbonMeasure.getPrecision()); + new BigDecimal(msrValue).setScale(scale, RoundingMode.HALF_UP); + BigDecimal decimal = normalizeDecimalValue(bigDecimal, precision); if (useConverter) { return converter.convertFromBigDecimalToDecimal(decimal); } else { @@ -414,6 +414,38 @@ public static byte[] getBytesDataDataTypeForNoDictionaryColumn(Object dimensionV } } + /** + * Convert the min/max values to bytes for no dictionary column + * + * @param dimensionValue + * @param actualDataType + * @return + */ + public static byte[] getMinMaxBytesBasedOnDataTypeForNoDictionaryColumn(Object dimensionValue, + DataType actualDataType) { + if (dimensionValue == null) { + if (actualDataType == DataTypes.STRING) { + return CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY; + } else { + return new byte[0]; + } + } + if (actualDataType == DataTypes.BOOLEAN) { + return ByteUtil.toBytes(Boolean.valueOf(ByteUtil.toBoolean((byte) dimensionValue))); + } else if (actualDataType == DataTypes.SHORT) { + return ByteUtil.toXorBytes((Short) dimensionValue); + } else if (actualDataType == DataTypes.INT) { + return ByteUtil.toXorBytes((Integer) dimensionValue); + } else if (actualDataType == DataTypes.LONG) { + return ByteUtil.toXorBytes((Long) dimensionValue); + } else if (actualDataType == DataTypes.TIMESTAMP) { + return ByteUtil.toXorBytes((Long)dimensionValue); + } else { + // Default action for String/Varchar + return ByteUtil.toBytes(dimensionValue.toString()); + } + } + /** * Returns true for fixed length DataTypes. * @param dataType @@ -976,4 +1008,20 @@ 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; + } + } 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 fca1244e3be..a51d6d1eab0 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 @@ -116,7 +116,7 @@ public static Object getMeasure(int index, Object[] row) { return measures[index]; } - public static void prepareOutObj(Object[] out, int[] dimArray, byte[][] byteBufferArr, + public static void prepareOutObj(Object[] out, int[] dimArray, Object[] byteBufferArr, Object[] measureArray) { out[WriteStepRowUtil.DICTIONARY_DIMENSION] = dimArray; out[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX] = byteBufferArr; diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/page/encoding/TestEncodingFactory.java b/core/src/test/java/org/apache/carbondata/core/datastore/page/encoding/TestEncodingFactory.java index 0b8bcc71210..3f1c819d0f3 100644 --- a/core/src/test/java/org/apache/carbondata/core/datastore/page/encoding/TestEncodingFactory.java +++ b/core/src/test/java/org/apache/carbondata/core/datastore/page/encoding/TestEncodingFactory.java @@ -17,6 +17,7 @@ package org.apache.carbondata.core.datastore.page.encoding; +import org.apache.carbondata.core.datastore.TableSpec; import org.apache.carbondata.core.datastore.page.encoding.adaptive.AdaptiveDeltaIntegralCodec; import org.apache.carbondata.core.datastore.page.encoding.adaptive.AdaptiveIntegralCodec; import org.apache.carbondata.core.datastore.page.encoding.compress.DirectCompressCodec; @@ -38,25 +39,25 @@ public class TestEncodingFactory extends TestCase { // for Byte primitivePageStatsCollector.update((long) Byte.MAX_VALUE); ColumnPageCodec columnPageCodec = - DefaultEncodingFactory.selectCodecByAlgorithmForIntegral(primitivePageStatsCollector, false); + DefaultEncodingFactory.selectCodecByAlgorithmForIntegral(primitivePageStatsCollector, false, null); assert (columnPageCodec instanceof AdaptiveIntegralCodec); assert (DataTypes.BYTE == ((AdaptiveIntegralCodec) columnPageCodec).getTargetDataType()); // for Short primitivePageStatsCollector.update((long) Short.MAX_VALUE); columnPageCodec = - DefaultEncodingFactory.selectCodecByAlgorithmForIntegral(primitivePageStatsCollector, false); + DefaultEncodingFactory.selectCodecByAlgorithmForIntegral(primitivePageStatsCollector, false, null); assert (columnPageCodec instanceof AdaptiveIntegralCodec); assert (DataTypes.SHORT == ((AdaptiveIntegralCodec) columnPageCodec).getTargetDataType()); // for int primitivePageStatsCollector.update((long) Integer.MAX_VALUE); columnPageCodec = - DefaultEncodingFactory.selectCodecByAlgorithmForIntegral(primitivePageStatsCollector, false); + DefaultEncodingFactory.selectCodecByAlgorithmForIntegral(primitivePageStatsCollector, false, null); assert (columnPageCodec instanceof AdaptiveIntegralCodec); assert (DataTypes.INT == ((AdaptiveIntegralCodec) columnPageCodec).getTargetDataType()); // for long primitivePageStatsCollector.update(Long.MAX_VALUE); columnPageCodec = - DefaultEncodingFactory.selectCodecByAlgorithmForIntegral(primitivePageStatsCollector, false); + DefaultEncodingFactory.selectCodecByAlgorithmForIntegral(primitivePageStatsCollector, false, null); assert (columnPageCodec instanceof DirectCompressCodec); assert ("DirectCompressCodec".equals(columnPageCodec.getName())); } @@ -67,25 +68,25 @@ public class TestEncodingFactory extends TestCase { // for Byte primitivePageStatsCollector.update((long) 200); ColumnPageCodec columnPageCodec = - DefaultEncodingFactory.selectCodecByAlgorithmForIntegral(primitivePageStatsCollector, false); + DefaultEncodingFactory.selectCodecByAlgorithmForIntegral(primitivePageStatsCollector, false, null); assert (columnPageCodec instanceof AdaptiveDeltaIntegralCodec); assert (DataTypes.BYTE == ((AdaptiveDeltaIntegralCodec) columnPageCodec).getTargetDataType()); // for Short primitivePageStatsCollector.update((long) 634767); columnPageCodec = - DefaultEncodingFactory.selectCodecByAlgorithmForIntegral(primitivePageStatsCollector, false); + DefaultEncodingFactory.selectCodecByAlgorithmForIntegral(primitivePageStatsCollector, false, null); assert (columnPageCodec instanceof AdaptiveIntegralCodec); assert (DataTypes.SHORT_INT == ((AdaptiveIntegralCodec) columnPageCodec).getTargetDataType()); // for int primitivePageStatsCollector.update((long) (Integer.MAX_VALUE + 200)); columnPageCodec = - DefaultEncodingFactory.selectCodecByAlgorithmForIntegral(primitivePageStatsCollector, false); + DefaultEncodingFactory.selectCodecByAlgorithmForIntegral(primitivePageStatsCollector, false, null); assert (columnPageCodec instanceof AdaptiveIntegralCodec); assert (DataTypes.INT == ((AdaptiveIntegralCodec) columnPageCodec).getTargetDataType()); // for int primitivePageStatsCollector.update(Long.MAX_VALUE); columnPageCodec = - DefaultEncodingFactory.selectCodecByAlgorithmForIntegral(primitivePageStatsCollector, false); + DefaultEncodingFactory.selectCodecByAlgorithmForIntegral(primitivePageStatsCollector, false, null); assert (columnPageCodec instanceof DirectCompressCodec); assert ("DirectCompressCodec".equals(columnPageCodec.getName())); } diff --git a/core/src/test/java/org/apache/carbondata/core/util/DataTypeUtilTest.java b/core/src/test/java/org/apache/carbondata/core/util/DataTypeUtilTest.java index 720e954bf7f..e69fa9e5488 100644 --- a/core/src/test/java/org/apache/carbondata/core/util/DataTypeUtilTest.java +++ b/core/src/test/java/org/apache/carbondata/core/util/DataTypeUtilTest.java @@ -62,13 +62,13 @@ public class DataTypeUtilTest { @Test public void testGetMeasureValueBasedOnDataType() { ColumnSchema columnSchema = new ColumnSchema(); CarbonMeasure carbonMeasure = new CarbonMeasure(columnSchema, 1); - Object resultInt = getMeasureValueBasedOnDataType("1", DataTypes.INT, carbonMeasure); + Object resultInt = getMeasureValueBasedOnDataType("1", DataTypes.INT, carbonMeasure.getScale(), carbonMeasure.getPrecision()); Object expectedInt = Double.valueOf(1).intValue(); assertEquals(expectedInt, resultInt); - Object resultLong = getMeasureValueBasedOnDataType("1", DataTypes.LONG, carbonMeasure); + Object resultLong = getMeasureValueBasedOnDataType("1", DataTypes.LONG, carbonMeasure.getScale(), carbonMeasure.getPrecision()); Object expectedLong = Long.valueOf(1); assertEquals(expectedLong, resultLong); - Object resultDefault = getMeasureValueBasedOnDataType("1", DataTypes.DOUBLE, carbonMeasure); + Object resultDefault = getMeasureValueBasedOnDataType("1", DataTypes.DOUBLE, carbonMeasure.getScale(), carbonMeasure.getPrecision()); Double expectedDefault = Double.valueOf(1); assertEquals(expectedDefault, resultDefault); diff --git a/core/src/test/java/org/apache/carbondata/core/util/RangeFilterProcessorTest.java b/core/src/test/java/org/apache/carbondata/core/util/RangeFilterProcessorTest.java index 9b8be7905cb..6527fef3317 100644 --- a/core/src/test/java/org/apache/carbondata/core/util/RangeFilterProcessorTest.java +++ b/core/src/test/java/org/apache/carbondata/core/util/RangeFilterProcessorTest.java @@ -17,10 +17,13 @@ package org.apache.carbondata.core.util; +import java.io.IOException; import java.util.Arrays; +import org.apache.carbondata.core.metadata.datatype.DataType; import org.apache.carbondata.core.metadata.datatype.DataTypes; import org.apache.carbondata.core.metadata.encoder.Encoding; +import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn; import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension; import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema; import org.apache.carbondata.core.scan.expression.ColumnExpression; @@ -37,9 +40,14 @@ import org.apache.carbondata.core.scan.filter.executer.RangeValueFilterExecuterImpl; import org.apache.carbondata.core.scan.filter.intf.FilterOptimizer; import org.apache.carbondata.core.scan.filter.optimizer.RangeFilterOptmizer; +import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo; import mockit.Deencapsulation; +import mockit.Mock; import mockit.MockUp; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -47,7 +55,15 @@ /* Test Cases for Range Filter */ public class RangeFilterProcessorTest { + + public static DimColumnResolvedFilterInfo dimColumnResolvedFilterInfo = + new DimColumnResolvedFilterInfo(); + @BeforeClass public static void setUp() throws Exception { + ColumnSchema columnSchema = new ColumnSchema(); + columnSchema.setDataType(DataTypes.STRING); + CarbonDimension carbonDimension = new CarbonDimension(columnSchema, 0, 0, 0); + dimColumnResolvedFilterInfo.setDimension(carbonDimension); } public boolean checkBothTrees(Expression a, Expression b) { @@ -320,6 +336,8 @@ public boolean checkBothTrees(Expression a, Expression b) { Deencapsulation.setField(range, "isDimensionPresentInCurrentBlock", true); Deencapsulation.setField(range, "lessThanExp", true); Deencapsulation.setField(range, "greaterThanExp", true); + Deencapsulation.setField(range, "dimColEvaluatorInfo", dimColumnResolvedFilterInfo); + result = range.isScanRequired(BlockMin, BlockMax, filterMinMax); Assert.assertFalse(result); } @@ -336,6 +354,7 @@ public boolean checkBothTrees(Expression a, Expression b) { Deencapsulation.setField(range, "isDimensionPresentInCurrentBlock", true); Deencapsulation.setField(range, "lessThanExp", true); Deencapsulation.setField(range, "greaterThanExp", true); + Deencapsulation.setField(range, "dimColEvaluatorInfo", dimColumnResolvedFilterInfo); result = range.isScanRequired(BlockMin, BlockMax, filterMinMax); Assert.assertFalse(result); } @@ -352,6 +371,7 @@ public boolean checkBothTrees(Expression a, Expression b) { Deencapsulation.setField(range, "isDimensionPresentInCurrentBlock", true); Deencapsulation.setField(range, "lessThanExp", true); Deencapsulation.setField(range, "greaterThanExp", true); + Deencapsulation.setField(range, "dimColEvaluatorInfo", dimColumnResolvedFilterInfo); result = range.isScanRequired(BlockMin, BlockMax, filterMinMax); Assert.assertTrue(result); } @@ -369,6 +389,7 @@ public boolean checkBothTrees(Expression a, Expression b) { Deencapsulation.setField(range, "isDimensionPresentInCurrentBlock", true); Deencapsulation.setField(range, "lessThanExp", true); Deencapsulation.setField(range, "greaterThanExp", true); + Deencapsulation.setField(range, "dimColEvaluatorInfo", dimColumnResolvedFilterInfo); result = range.isScanRequired(BlockMin, BlockMax, filterMinMax); rangeCovered = Deencapsulation.getField(range, "isRangeFullyCoverBlock"); @@ -389,6 +410,7 @@ public boolean checkBothTrees(Expression a, Expression b) { Deencapsulation.setField(range, "isDimensionPresentInCurrentBlock", true); Deencapsulation.setField(range, "lessThanExp", true); Deencapsulation.setField(range, "greaterThanExp", true); + Deencapsulation.setField(range, "dimColEvaluatorInfo", dimColumnResolvedFilterInfo); result = range.isScanRequired(BlockMin, BlockMax, filterMinMax); startBlockMinIsDefaultStart = Deencapsulation.getField(range, "startBlockMinIsDefaultStart"); @@ -409,6 +431,7 @@ public boolean checkBothTrees(Expression a, Expression b) { Deencapsulation.setField(range, "isDimensionPresentInCurrentBlock", true); Deencapsulation.setField(range, "lessThanExp", true); Deencapsulation.setField(range, "greaterThanExp", true); + Deencapsulation.setField(range, "dimColEvaluatorInfo", dimColumnResolvedFilterInfo); result = range.isScanRequired(BlockMin, BlockMax, filterMinMax); endBlockMaxisDefaultEnd = Deencapsulation.getField(range, "endBlockMaxisDefaultEnd"); diff --git a/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/AbstractBloomDataMapWriter.java b/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/AbstractBloomDataMapWriter.java index 38234605db5..4734abd220e 100644 --- a/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/AbstractBloomDataMapWriter.java +++ b/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/AbstractBloomDataMapWriter.java @@ -144,7 +144,7 @@ protected void addValue2BloomIndex(int indexColIdx, Object value) { || indexColumns.get(indexColIdx).hasEncoding(Encoding.DIRECT_DICTIONARY)) { indexValue = convertDictionaryValue(indexColIdx, value); } else { - indexValue = convertNonDictionaryValue(indexColIdx, (byte[]) value); + indexValue = convertNonDictionaryValue(indexColIdx, value); } } if (indexValue.length == 0) { @@ -155,7 +155,7 @@ protected void addValue2BloomIndex(int indexColIdx, Object value) { protected abstract byte[] convertDictionaryValue(int indexColIdx, Object value); - protected abstract byte[] convertNonDictionaryValue(int indexColIdx, byte[] value); + protected abstract byte[] convertNonDictionaryValue(int indexColIdx, Object value); private void initDataMapFile() throws IOException { if (!FileFactory.isFileExist(dataMapPath)) { diff --git a/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java b/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java index 27911cabbd3..2e2d94b6b41 100644 --- a/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java +++ b/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java @@ -62,6 +62,7 @@ import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf; 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.DataField; import org.apache.carbondata.processing.loading.converter.BadRecordLogHolder; import org.apache.carbondata.processing.loading.converter.FieldConverter; @@ -343,8 +344,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); + } else { + internalFilterValue = (byte[]) convertedValue; + } } if (internalFilterValue.length == 0) { internalFilterValue = CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY; diff --git a/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomDataMapBuilder.java b/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomDataMapBuilder.java index 29e3060e41d..29a40982774 100644 --- a/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomDataMapBuilder.java +++ b/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomDataMapBuilder.java @@ -27,6 +27,7 @@ import org.apache.carbondata.core.metadata.datatype.DataTypes; import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn; import org.apache.carbondata.core.util.CarbonUtil; +import org.apache.carbondata.core.util.DataTypeUtil; /** * Implementation for BloomFilter DataMap to rebuild the datamap for main table with existing data @@ -61,8 +62,12 @@ public void addRow(int blockletId, int pageId, int rowId, Object[] values) { } @Override - protected byte[] convertNonDictionaryValue(int indexColIdx, byte[] value) { - return value; + protected byte[] convertNonDictionaryValue(int indexColIdx, Object value) { + // no dictionary measure columns will be of original data, so convert it to bytes + if (DataTypeUtil.isPrimitiveColumn(indexColumns.get(indexColIdx).getDataType())) { + return CarbonUtil.getValueAsBytes(indexColumns.get(indexColIdx).getDataType(), value); + } + return (byte[]) value; } @Override diff --git a/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomDataMapWriter.java b/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomDataMapWriter.java index cad9787df62..61bd0366d1b 100644 --- a/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomDataMapWriter.java +++ b/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomDataMapWriter.java @@ -29,6 +29,7 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn; import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension; import org.apache.carbondata.core.util.CarbonUtil; +import org.apache.carbondata.core.util.DataTypeUtil; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.collections.Predicate; @@ -73,11 +74,14 @@ public class BloomDataMapWriter extends AbstractBloomDataMapWriter { } } - protected byte[] convertNonDictionaryValue(int indexColIdx, byte[] value) { + protected byte[] convertNonDictionaryValue(int indexColIdx, Object value) { if (DataTypes.VARCHAR == indexColumns.get(indexColIdx).getDataType()) { - return DataConvertUtil.getRawBytesForVarchar(value); + return DataConvertUtil.getRawBytesForVarchar((byte[]) value); + } else if (DataTypeUtil.isPrimitiveColumn(indexColumns.get(indexColIdx).getDataType())) { + // get bytes for the original value of the no dictionary column + return CarbonUtil.getValueAsBytes(indexColumns.get(indexColIdx).getDataType(), value); } else { - return DataConvertUtil.getRawBytes(value); + return DataConvertUtil.getRawBytes((byte[]) value); } } diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/testutil/StoreCreator.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/testutil/StoreCreator.java index 7cd241af79b..cee58a16d3b 100644 --- a/hadoop/src/main/java/org/apache/carbondata/hadoop/testutil/StoreCreator.java +++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/testutil/StoreCreator.java @@ -232,10 +232,11 @@ public CarbonTable createTable( id.setEncodingList(encodings); id.setColumnUniqueId(UUID.randomUUID().toString()); id.setColumnReferenceId(id.getColumnUniqueId()); - id.setDimensionColumn(true); + id.setDimensionColumn(false); id.setSchemaOrdinal(schemaOrdinal++); if (sortColumns.contains(id.getColumnName())) { id.setSortColumn(true); + id.setDimensionColumn(true); } columnSchemas.add(id); @@ -245,7 +246,7 @@ public CarbonTable createTable( date.setEncodingList(encodings); date.setColumnUniqueId(UUID.randomUUID().toString()); date.setDimensionColumn(true); - date.setColumnReferenceId(id.getColumnUniqueId()); + date.setColumnReferenceId(date.getColumnUniqueId()); date.setSchemaOrdinal(schemaOrdinal++); if (sortColumns.contains(date.getColumnName())) { date.setSortColumn(true); @@ -263,7 +264,7 @@ public CarbonTable createTable( if (sortColumns.contains(country.getColumnName())) { country.setSortColumn(true); } - country.setColumnReferenceId(id.getColumnUniqueId()); + country.setColumnReferenceId(country.getColumnUniqueId()); columnSchemas.add(country); ColumnSchema name = new ColumnSchema(); @@ -276,7 +277,7 @@ public CarbonTable createTable( if (sortColumns.contains(name.getColumnName())) { name.setSortColumn(true); } - name.setColumnReferenceId(id.getColumnUniqueId()); + name.setColumnReferenceId(name.getColumnUniqueId()); columnSchemas.add(name); ColumnSchema phonetype = new ColumnSchema(); @@ -289,7 +290,7 @@ public CarbonTable createTable( if (sortColumns.contains(phonetype.getColumnName())) { phonetype.setSortColumn(true); } - phonetype.setColumnReferenceId(id.getColumnUniqueId()); + phonetype.setColumnReferenceId(phonetype.getColumnUniqueId()); columnSchemas.add(phonetype); ColumnSchema serialname = new ColumnSchema(); @@ -302,7 +303,7 @@ public CarbonTable createTable( if (sortColumns.contains(serialname.getColumnName())) { serialname.setSortColumn(true); } - serialname.setColumnReferenceId(id.getColumnUniqueId()); + serialname.setColumnReferenceId(serialname.getColumnUniqueId()); columnSchemas.add(serialname); ColumnSchema salary = new ColumnSchema(); salary.setColumnName("salary"); @@ -310,7 +311,7 @@ public CarbonTable createTable( salary.setEncodingList(new ArrayList()); salary.setColumnUniqueId(UUID.randomUUID().toString()); salary.setDimensionColumn(false); - salary.setColumnReferenceId(id.getColumnUniqueId()); + salary.setColumnReferenceId(salary.getColumnUniqueId()); salary.setSchemaOrdinal(schemaOrdinal++); columnSchemas.add(salary); diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessBuilderOnSpark.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessBuilderOnSpark.scala index be40b13ad69..e810829fd05 100644 --- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessBuilderOnSpark.scala +++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessBuilderOnSpark.scala @@ -86,7 +86,9 @@ object DataLoadProcessBuilderOnSpark { val sortParameters = SortParameters.createSortParameters(configuration) val rowComparator: Comparator[Array[AnyRef]] = if (sortParameters.getNoDictionaryCount > 0) { - new NewRowComparator(sortParameters.getNoDictionaryDimnesionColumn) + new NewRowComparator(sortParameters.getNoDictionaryDimnesionColumn, + sortParameters.getNoDictionarySortColumn, + sortParameters.getNoDictDataType) } else { new NewRowComparatorForNormalDims(sortParameters.getDimColCount) } diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/datamap/IndexDataMapRebuildRDD.scala b/integration/spark2/src/main/scala/org/apache/carbondata/datamap/IndexDataMapRebuildRDD.scala index 0584fb17d4f..1897c874ec4 100644 --- a/integration/spark2/src/main/scala/org/apache/carbondata/datamap/IndexDataMapRebuildRDD.scala +++ b/integration/spark2/src/main/scala/org/apache/carbondata/datamap/IndexDataMapRebuildRDD.scala @@ -52,7 +52,7 @@ import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, DataMapSch import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn import org.apache.carbondata.core.scan.wrappers.ByteArrayWrapper import org.apache.carbondata.core.statusmanager.SegmentStatusManager -import org.apache.carbondata.core.util.{CarbonUtil, TaskMetricsMap} +import org.apache.carbondata.core.util.{CarbonUtil, DataTypeUtil, TaskMetricsMap} import org.apache.carbondata.core.util.path.CarbonTablePath import org.apache.carbondata.datamap.bloom.DataConvertUtil import org.apache.carbondata.events.{BuildDataMapPostExecutionEvent, BuildDataMapPreExecutionEvent, OperationContext, OperationListenerBus} @@ -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 + } else { + bytes + } } else { // measures start from 1 val value = data(1 + indexCol2IdxInMeasureArray(col.getColName)) diff --git a/integration/spark2/src/test/scala/org/apache/spark/sql/CarbonGetTableDetailComandTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/sql/CarbonGetTableDetailComandTestCase.scala index a49d5bbc40b..030d69f7a5a 100644 --- a/integration/spark2/src/test/scala/org/apache/spark/sql/CarbonGetTableDetailComandTestCase.scala +++ b/integration/spark2/src/test/scala/org/apache/spark/sql/CarbonGetTableDetailComandTestCase.scala @@ -42,10 +42,10 @@ class CarbonGetTableDetailCommandTestCase extends QueryTest with BeforeAndAfterA assertResult(2)(result.length) assertResult("table_info1")(result(0).getString(0)) - // 2087 is the size of carbon table. Note that since 1.5.0, we add additional compressor name in metadata - assertResult(2187)(result(0).getLong(1)) + // 2191 is the size of carbon table + assertResult(2191)(result(0).getLong(1)) assertResult("table_info2")(result(1).getString(0)) - assertResult(2187)(result(1).getLong(1)) + assertResult(2191)(result(1).getLong(1)) } override def afterAll: Unit = { diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/CarbonDataLoadConfiguration.java b/processing/src/main/java/org/apache/carbondata/processing/loading/CarbonDataLoadConfiguration.java index 4d852968e16..616edebd1ed 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/loading/CarbonDataLoadConfiguration.java +++ b/processing/src/main/java/org/apache/carbondata/processing/loading/CarbonDataLoadConfiguration.java @@ -28,6 +28,7 @@ import org.apache.carbondata.core.keygenerator.factory.KeyGeneratorFactory; import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier; import org.apache.carbondata.core.metadata.datatype.DataType; +import org.apache.carbondata.core.metadata.encoder.Encoding; import org.apache.carbondata.core.metadata.schema.BucketingInfo; import org.apache.carbondata.core.metadata.schema.SortColumnRangeInfo; import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn; @@ -339,6 +340,45 @@ public DataType[] getMeasureDataType() { return type; } + /** + * Get the data types of the no dictionary and the complex dimensions of the table + * + * @return + */ + public CarbonColumn[] getNoDictAndComplexDimensions() { + List noDicOrCompIndexes = new ArrayList<>(dataFields.length); + int noDicCount = 0; + for (int i = 0; i < dataFields.length; i++) { + if (dataFields[i].getColumn().isDimension() && ( + !(dataFields[i].getColumn().hasEncoding(Encoding.DICTIONARY)) || dataFields[i].getColumn() + .isComplex())) { + noDicOrCompIndexes.add(i); + noDicCount++; + } + } + + CarbonColumn[] dims = new CarbonColumn[noDicCount]; + for (int i = 0; i < dims.length; i++) { + dims[i] = dataFields[noDicOrCompIndexes.get(i)].getColumn(); + } + return dims; + } + + /** + * Get the sort column mapping of the table + * + * @return + */ + public boolean[] getSortColumnMapping() { + boolean[] sortColumnMapping = new boolean[dataFields.length]; + for (int i = 0; i < sortColumnMapping.length; i++) { + if (dataFields[i].getColumn().getColumnSchema().isSortColumn()) { + sortColumnMapping[i] = true; + } + } + return sortColumnMapping; + } + public int[] calcDimensionLengths() { int[] dimLensWithComplex = getCardinalityFinder().getCardinality(); if (!isSortTable()) { diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java index 86f273db40e..7dfe95fdd29 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java +++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java @@ -32,6 +32,7 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn; import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension; import org.apache.carbondata.core.metadata.schema.table.column.ParentColumnTableRelation; +import org.apache.carbondata.core.util.DataTypeUtil; import org.apache.carbondata.processing.datatypes.ArrayDataType; import org.apache.carbondata.processing.datatypes.GenericDataType; import org.apache.carbondata.processing.datatypes.PrimitiveDataType; @@ -111,6 +112,11 @@ public FieldConverter createFieldEncoder(DataField dataField, createComplexDataType(dataField, absoluteTableIdentifier, client, useOnePass, localCache, index, nullFormat, isEmptyBadRecord), index); } else { + // if the no dictionary column is a numeric column then treat is as measure col + // so that the adaptive encoding can be applied on it easily + if (DataTypeUtil.isPrimitiveColumn(dataField.getColumn().getDataType())) { + return new MeasureFieldConverterImpl(dataField, nullFormat, index, isEmptyBadRecord); + } return new NonDictionaryFieldConverterImpl(dataField, nullFormat, index, isEmptyBadRecord); } } else { diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/MeasureFieldConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/MeasureFieldConverterImpl.java index 9cbd607926c..20278e40bb3 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/MeasureFieldConverterImpl.java +++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/MeasureFieldConverterImpl.java @@ -20,8 +20,6 @@ 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.metadata.datatype.DataType; -import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure; import org.apache.carbondata.core.util.DataTypeUtil; import org.apache.carbondata.processing.loading.DataField; import org.apache.carbondata.processing.loading.converter.BadRecordLogHolder; @@ -39,10 +37,6 @@ public class MeasureFieldConverterImpl implements FieldConverter { private int index; - private DataType dataType; - - private CarbonMeasure measure; - private String nullformat; private boolean isEmptyBadRecord; @@ -51,8 +45,6 @@ public class MeasureFieldConverterImpl implements FieldConverter { public MeasureFieldConverterImpl(DataField dataField, String nullformat, int index, boolean isEmptyBadRecord) { - this.dataType = dataField.getColumn().getDataType(); - this.measure = (CarbonMeasure) dataField.getColumn(); this.nullformat = nullformat; this.index = index; this.isEmptyBadRecord = isEmptyBadRecord; @@ -73,20 +65,20 @@ public Object convert(Object value, BadRecordLogHolder logHolder) Object output; boolean isNull = CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(literalValue); if (literalValue == null || isNull) { - String message = logHolder.getColumnMessageMap().get(measure.getColName()); + String message = logHolder.getColumnMessageMap().get(dataField.getColumn().getColName()); if (null == message) { - message = CarbonDataProcessorUtil - .prepareFailureReason(measure.getColName(), measure.getDataType()); - logHolder.getColumnMessageMap().put(measure.getColName(), message); + message = CarbonDataProcessorUtil.prepareFailureReason(dataField.getColumn().getColName(), + dataField.getColumn().getDataType()); + logHolder.getColumnMessageMap().put(dataField.getColumn().getColName(), message); } return null; } else if (literalValue.length() == 0) { if (isEmptyBadRecord) { - String message = logHolder.getColumnMessageMap().get(measure.getColName()); + String message = logHolder.getColumnMessageMap().get(dataField.getColumn().getColName()); if (null == message) { - message = CarbonDataProcessorUtil - .prepareFailureReason(measure.getColName(), measure.getDataType()); - logHolder.getColumnMessageMap().put(measure.getColName(), message); + message = CarbonDataProcessorUtil.prepareFailureReason(dataField.getColumn().getColName(), + dataField.getColumn().getDataType()); + logHolder.getColumnMessageMap().put(dataField.getColumn().getColName(), message); } logHolder.setReason(message); } @@ -96,18 +88,24 @@ public Object convert(Object value, BadRecordLogHolder logHolder) } else { try { if (dataField.isUseActualData()) { - output = - DataTypeUtil.getMeasureValueBasedOnDataType(literalValue, dataType, measure, true); + output = DataTypeUtil + .getMeasureValueBasedOnDataType(literalValue, dataField.getColumn().getDataType(), + dataField.getColumn().getColumnSchema().getScale(), + dataField.getColumn().getColumnSchema().getPrecision(), true); } else { - output = DataTypeUtil.getMeasureValueBasedOnDataType(literalValue, dataType, measure); + output = DataTypeUtil + .getMeasureValueBasedOnDataType(literalValue, dataField.getColumn().getDataType(), + dataField.getColumn().getColumnSchema().getScale(), + dataField.getColumn().getColumnSchema().getPrecision()); } return output; } catch (NumberFormatException e) { if (LOGGER.isDebugEnabled()) { LOGGER.debug("Can not convert value to Numeric type value. Value considered as null."); } - logHolder.setReason( - CarbonDataProcessorUtil.prepareFailureReason(measure.getColName(), dataType)); + logHolder.setReason(CarbonDataProcessorUtil + .prepareFailureReason(dataField.getColumn().getColName(), + dataField.getColumn().getDataType())); return null; } } diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/RawRowComparator.java b/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/RawRowComparator.java index 64b64f53b1b..3a325a4091e 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/RawRowComparator.java +++ b/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/RawRowComparator.java @@ -21,7 +21,10 @@ import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.core.datastore.row.CarbonRow; +import org.apache.carbondata.core.metadata.datatype.DataType; import org.apache.carbondata.core.util.ByteUtil.UnsafeComparer; +import org.apache.carbondata.core.util.DataTypeUtil; +import org.apache.carbondata.core.util.comparator.SerializableComparator; /** * comparator for the converted row. The row has not been rearranged as 3-parted yet. @@ -30,23 +33,38 @@ 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 + .getComparator(noDicDataTypes[noDicIdx]); + int difference = comparator.compare(o1.getObject(colIdx), o2.getObject(colIdx)); + if (difference != 0) { + return difference; + } + noDicIdx++; + } else { + byte[] colA = (byte[]) o1.getObject(colIdx); + byte[] colB = (byte[]) o2.getObject(colIdx); + diff = UnsafeComparer.INSTANCE.compareTo(colA, colB); + if (diff != 0) { + return diff; + } } } else { int colA = (int) o1.getObject(colIdx); diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/row/IntermediateSortTempRow.java b/processing/src/main/java/org/apache/carbondata/processing/loading/row/IntermediateSortTempRow.java index 1ad7879da9f..844e45e0a91 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/loading/row/IntermediateSortTempRow.java +++ b/processing/src/main/java/org/apache/carbondata/processing/loading/row/IntermediateSortTempRow.java @@ -23,7 +23,7 @@ */ public class IntermediateSortTempRow { private int[] dictSortDims; - private byte[][] noDictSortDims; + private Object[] noDictSortDims; /** * this will be used for intermediate merger when * no sort field and measure field will not be @@ -35,14 +35,14 @@ public class IntermediateSortTempRow { */ private Object[] measures; - public IntermediateSortTempRow(int[] dictSortDims, byte[][] noDictSortDims, + public IntermediateSortTempRow(int[] dictSortDims, Object[] noDictSortDims, byte[] noSortDimsAndMeasures) { this.dictSortDims = dictSortDims; this.noDictSortDims = noDictSortDims; this.noSortDimsAndMeasures = noSortDimsAndMeasures; } - public IntermediateSortTempRow(int[] dictSortDims, byte[][] noDictSortDims, + public IntermediateSortTempRow(int[] dictSortDims, Object[] noDictSortDims, Object[] measures) { this.dictSortDims = dictSortDims; this.noDictSortDims = noDictSortDims; @@ -57,7 +57,7 @@ public Object[] getMeasures() { return measures; } - public byte[][] getNoDictSortDims() { + public Object[] getNoDictSortDims() { return noDictSortDims; } diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowHandler.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowHandler.java index 697f590047f..edfd317ff44 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowHandler.java +++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowHandler.java @@ -23,10 +23,13 @@ import java.io.Serializable; import java.math.BigDecimal; import java.nio.ByteBuffer; +import java.nio.charset.Charset; +import org.apache.carbondata.core.constants.CarbonCommonConstants; import org.apache.carbondata.core.memory.CarbonUnsafe; import org.apache.carbondata.core.metadata.datatype.DataType; import org.apache.carbondata.core.metadata.datatype.DataTypes; +import org.apache.carbondata.core.util.CarbonUnsafeUtil; import org.apache.carbondata.core.util.DataTypeUtil; import org.apache.carbondata.core.util.NonDictionaryUtil; import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow; @@ -65,6 +68,14 @@ public class SortStepRowHandler implements Serializable { private DataType[] dataTypes; + private DataType[] noDictSortDataTypes; + + private boolean[] noDictSortColMapping; + + private DataType[] noDictNoSortDataTypes; + + private boolean[] noDictNoSortColMapping; + /** * constructor * @param tableFieldStat table field stat @@ -85,6 +96,16 @@ public SortStepRowHandler(TableFieldStat tableFieldStat) { this.complexDimIdx = tableFieldStat.getComplexDimIdx(); this.measureIdx = tableFieldStat.getMeasureIdx(); this.dataTypes = tableFieldStat.getMeasureDataType(); + this.noDictSortDataTypes = tableFieldStat.getNoDictSortDataType(); + noDictSortColMapping = new boolean[noDictSortDataTypes.length]; + for (int i = 0; i < noDictSortDataTypes.length; i++) { + noDictSortColMapping[i] = DataTypeUtil.isPrimitiveColumn(noDictSortDataTypes[i]); + } + this.noDictNoSortDataTypes = tableFieldStat.getNoDictNoSortDataType(); + noDictNoSortColMapping = new boolean[noDictNoSortDataTypes.length]; + for (int i = 0; i < noDictNoSortDataTypes.length; i++) { + noDictNoSortColMapping[i] = DataTypeUtil.isPrimitiveColumn(noDictNoSortDataTypes[i]); + } } /** @@ -108,8 +129,8 @@ public Object[] convertRawRowTo3Parts(Object[] row) { try { int[] dictDims = new int[this.dictSortDimCnt + this.dictNoSortDimCnt]; - byte[][] nonDictArray = new byte[this.noDictSortDimCnt + this.noDictNoSortDimCnt - + this.varcharDimCnt + this.complexDimCnt ][]; + Object[] nonDictArray = new Object[this.noDictSortDimCnt + this.noDictNoSortDimCnt + + this.varcharDimCnt + this.complexDimCnt]; Object[] measures = new Object[this.measureCnt]; // convert dict & data @@ -125,19 +146,19 @@ public Object[] convertRawRowTo3Parts(Object[] row) { // convert no-dict & sort idxAcc = 0; for (int idx = 0; idx < this.noDictSortDimCnt; idx++) { - nonDictArray[idxAcc++] = (byte[]) row[this.noDictSortDimIdx[idx]]; + nonDictArray[idxAcc++] = row[this.noDictSortDimIdx[idx]]; } // convert no-dict & no-sort for (int idx = 0; idx < this.noDictNoSortDimCnt; idx++) { - nonDictArray[idxAcc++] = (byte[]) row[this.noDictNoSortDimIdx[idx]]; + nonDictArray[idxAcc++] = row[this.noDictNoSortDimIdx[idx]]; } // convert varchar dims for (int idx = 0; idx < this.varcharDimCnt; idx++) { - nonDictArray[idxAcc++] = (byte[]) row[this.varcharDimIdx[idx]]; + nonDictArray[idxAcc++] = row[this.varcharDimIdx[idx]]; } // convert complex dims for (int idx = 0; idx < this.complexDimCnt; idx++) { - nonDictArray[idxAcc++] = (byte[]) row[this.complexDimIdx[idx]]; + nonDictArray[idxAcc++] = row[this.complexDimIdx[idx]]; } // convert measure data @@ -178,7 +199,7 @@ public Object[] convertIntermediateSortTempRowTo3Parted(IntermediateSortTempRow public IntermediateSortTempRow readWithoutNoSortFieldConvert( DataInputStream inputStream) throws IOException { int[] dictSortDims = new int[this.dictSortDimCnt]; - byte[][] noDictSortDims = new byte[this.noDictSortDimCnt][]; + Object[] noDictSortDims = new Object[this.noDictSortDimCnt]; // read dict & sort dim data for (int idx = 0; idx < this.dictSortDimCnt; idx++) { @@ -187,10 +208,8 @@ public IntermediateSortTempRow readWithoutNoSortFieldConvert( // 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 + noDictSortDims[idx] = getDataForNoDictSortColumn(inputStream, idx); } // read no-dict dims & measures @@ -213,9 +232,9 @@ public IntermediateSortTempRow readWithoutNoSortFieldConvert( public IntermediateSortTempRow readWithNoSortFieldConvert( DataInputStream inputStream) throws IOException { int[] dictSortDims = new int[this.dictSortDimCnt + this.dictNoSortDimCnt]; - byte[][] noDictSortDims = - new byte[this.noDictSortDimCnt + this.noDictNoSortDimCnt + this.varcharDimCnt - + this.complexDimCnt][]; + Object[] noDictSortDims = + new Object[this.noDictSortDimCnt + this.noDictNoSortDimCnt + this.varcharDimCnt + + this.complexDimCnt]; // read dict & sort dim data for (int idx = 0; idx < this.dictSortDimCnt; idx++) { @@ -224,10 +243,8 @@ 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 + noDictSortDims[idx] = getDataForNoDictSortColumn(inputStream, idx); } // read no-dict dims & measures @@ -240,8 +257,63 @@ public IntermediateSortTempRow readWithNoSortFieldConvert( return new IntermediateSortTempRow(dictSortDims, noDictSortDims,measure); } + /** + * Return the data from the stream according to the column type + * + * @param inputStream + * @param idx + * @throws IOException + */ + private Object getDataForNoDictSortColumn(DataInputStream inputStream, int idx) + throws IOException { + if (this.noDictSortColMapping[idx]) { + return readDataFromStream(inputStream, idx); + } else { + short len = inputStream.readShort(); + byte[] bytes = new byte[len]; + inputStream.readFully(bytes); + return bytes; + } + } + + /** + * Read the data from the stream + * + * @param inputStream + * @param idx + * @return + * @throws IOException + */ + private Object readDataFromStream(DataInputStream inputStream, int idx) throws IOException { + DataType dataType = noDictSortDataTypes[idx]; + Object data = null; + if (!inputStream.readBoolean()) { + return null; + } + if (dataType == DataTypes.BOOLEAN) { + data = inputStream.readBoolean(); + } else if (dataType == DataTypes.BYTE) { + data = inputStream.readByte(); + } else if (dataType == DataTypes.SHORT) { + data = inputStream.readShort(); + } else if (dataType == DataTypes.INT) { + data = inputStream.readInt(); + } else if (dataType == DataTypes.LONG) { + data = inputStream.readLong(); + } else if (dataType == DataTypes.DOUBLE) { + data = inputStream.readDouble(); + } else if (dataType == DataTypes.FLOAT) { + data = inputStream.readFloat(); + } else if (dataType == DataTypes.BYTE_ARRAY || DataTypes.isDecimal(dataType)) { + byte[] bytes = + inputStream.readUTF().getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)); + data = bytes; + } + return data; + } + private void unpackNoSortFromBytes(byte[] noSortDimsAndMeasures, int[] dictDims, - byte[][] noDictDims, Object[] measures) { + Object[] noDictDims, Object[] measures) { ByteBuffer rowBuffer = ByteBuffer.wrap(noSortDimsAndMeasures); // read dict_no_sort for (int i = dictSortDimCnt; i < dictDims.length; i++) { @@ -251,10 +323,15 @@ private void unpackNoSortFromBytes(byte[] noSortDimsAndMeasures, int[] dictDims, int noDictIndex = noDictSortDimCnt; // read no_dict_no_sort for (int i = 0; i < noDictNoSortDimCnt; i++) { - short len = rowBuffer.getShort(); - byte[] bytes = new byte[len]; - rowBuffer.get(bytes); - noDictDims[noDictIndex++] = bytes; + // for no dict measure column get the original data + if (this.noDictNoSortColMapping[i]) { + noDictDims[noDictIndex++] = getDataFromRowBuffer(noDictNoSortDataTypes[i], rowBuffer); + } else { + short len = rowBuffer.getShort(); + byte[] bytes = new byte[len]; + rowBuffer.get(bytes); + noDictDims[noDictIndex++] = bytes; + } } // read varchar dims @@ -275,39 +352,49 @@ private void unpackNoSortFromBytes(byte[] noSortDimsAndMeasures, int[] dictDims, // read measure int measureCnt = measures.length; - DataType tmpDataType; Object tmpContent; for (short idx = 0 ; idx < measureCnt; idx++) { - if ((byte) 0 == rowBuffer.get()) { - measures[idx] = null; - continue; - } + tmpContent = getDataFromRowBuffer(dataTypes[idx], rowBuffer); + measures[idx] = tmpContent; + } + } - tmpDataType = dataTypes[idx]; - if (DataTypes.BOOLEAN == tmpDataType) { - if ((byte) 1 == rowBuffer.get()) { - tmpContent = true; - } else { - tmpContent = false; - } - } else if (DataTypes.SHORT == tmpDataType) { - tmpContent = rowBuffer.getShort(); - } else if (DataTypes.INT == tmpDataType) { - tmpContent = rowBuffer.getInt(); - } else if (DataTypes.LONG == tmpDataType) { - tmpContent = rowBuffer.getLong(); - } else if (DataTypes.DOUBLE == tmpDataType) { - tmpContent = rowBuffer.getDouble(); - } else if (DataTypes.isDecimal(tmpDataType)) { - short len = rowBuffer.getShort(); - byte[] decimalBytes = new byte[len]; - rowBuffer.get(decimalBytes); - tmpContent = DataTypeUtil.byteToBigDecimal(decimalBytes); + /** + * Retrieve/Get the data from the row buffer. + * + * @param tmpDataType + * @param rowBuffer + * @return + */ + private Object getDataFromRowBuffer(DataType tmpDataType, ByteBuffer rowBuffer) { + Object tmpContent; + if ((byte) 0 == rowBuffer.get()) { + return null; + } + + if (DataTypes.BOOLEAN == tmpDataType) { + if ((byte) 1 == rowBuffer.get()) { + tmpContent = true; } else { - throw new IllegalArgumentException("Unsupported data type: " + tmpDataType); + tmpContent = false; } - measures[idx] = tmpContent; + } else if (DataTypes.SHORT == tmpDataType) { + tmpContent = rowBuffer.getShort(); + } else if (DataTypes.INT == tmpDataType) { + tmpContent = rowBuffer.getInt(); + } else if (DataTypes.LONG == tmpDataType) { + tmpContent = rowBuffer.getLong(); + } else if (DataTypes.DOUBLE == tmpDataType) { + tmpContent = rowBuffer.getDouble(); + } else if (DataTypes.isDecimal(tmpDataType)) { + short len = rowBuffer.getShort(); + byte[] decimalBytes = new byte[len]; + rowBuffer.get(decimalBytes); + tmpContent = DataTypeUtil.byteToBigDecimal(decimalBytes); + } else { + throw new IllegalArgumentException("Unsupported data type: " + tmpDataType); } + return tmpContent; } /** @@ -327,9 +414,14 @@ public void writeIntermediateSortTempRowToOutputStream(IntermediateSortTempRow s // write no-dict & sort dim for (int idx = 0; idx < this.noDictSortDimCnt; idx++) { - byte[] bytes = sortTempRow.getNoDictSortDims()[idx]; - outputStream.writeShort(bytes.length); - outputStream.write(bytes); + if (this.noDictSortColMapping[idx]) { + // write the original data to the stream + writeDataToStream(sortTempRow.getNoDictSortDims()[idx], outputStream, idx); + } else { + byte[] bytes = (byte[]) sortTempRow.getNoDictSortDims()[idx]; + outputStream.writeShort(bytes.length); + outputStream.write(bytes); + } } // write packed no-sort dim & measure @@ -359,9 +451,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 (this.noDictSortColMapping[idx]) { + // write the original data to the stream + writeDataToStream(row[this.noDictSortDimIdx[idx]], outputStream, idx); + } else { + byte[] bytes = (byte[]) row[this.noDictSortDimIdx[idx]]; + outputStream.writeShort(bytes.length); + outputStream.write(bytes); + } } // pack no-sort @@ -375,6 +472,46 @@ 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 = noDictSortDataTypes[idx]; + if (null == data) { + outputStream.writeBoolean(false); + } else { + outputStream.writeBoolean(true); + if (dataType == DataTypes.BOOLEAN) { + outputStream.writeBoolean((boolean) data); + } else if (dataType == DataTypes.BYTE) { + outputStream.writeByte((byte) data); + } else if (dataType == DataTypes.SHORT) { + outputStream.writeShort((short) data); + } else if (dataType == DataTypes.INT) { + outputStream.writeInt((int) data); + } else if (dataType == DataTypes.LONG) { + outputStream.writeLong((long) data); + } else if (dataType == DataTypes.DOUBLE) { + outputStream.writeDouble((double) data); + } else if (DataTypes.isDecimal(dataType)) { + BigDecimal val = (BigDecimal) data; + byte[] bigDecimalInBytes = DataTypeUtil.bigDecimalToByte(val); + outputStream.writeShort(bigDecimalInBytes.length); + outputStream.write(bigDecimalInBytes); + } else if (dataType == DataTypes.FLOAT) { + outputStream.writeFloat((float) data); + } else if (dataType == DataTypes.BYTE_ARRAY) { + outputStream.writeUTF(data.toString()); + } + } + } + /** * Read intermediate sort temp row from unsafe memory. * This method is used during merge sort phase for off-heap sort. @@ -430,9 +567,9 @@ public IntermediateSortTempRow readRowFromMemoryWithNoSortFieldConvert(Object ba int size = 0; int[] dictSortDims = new int[this.dictSortDimCnt + this.dictNoSortDimCnt]; - byte[][] noDictSortDims = - new byte[this.noDictSortDimCnt + this.noDictNoSortDimCnt + this.varcharDimCnt - + this.complexDimCnt][]; + Object[] noDictSortDims = + new Object[this.noDictSortDimCnt + this.noDictNoSortDimCnt + this.varcharDimCnt + + this.complexDimCnt]; // read dict & sort dim for (int idx = 0; idx < dictSortDimCnt; idx++) { @@ -444,11 +581,24 @@ public IntermediateSortTempRow readRowFromMemoryWithNoSortFieldConvert(Object ba for (int idx = 0; idx < this.noDictSortDimCnt; idx++) { short length = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size); size += 2; - byte[] bytes = new byte[length]; - CarbonUnsafe.getUnsafe().copyMemory(baseObject, address + size, - bytes, CarbonUnsafe.BYTE_ARRAY_OFFSET, length); - size += length; - noDictSortDims[idx] = bytes; + if (this.noDictSortColMapping[idx]) { + // get the original data from the unsafe memory + if (0 == length) { + // if the length is 0, the the data is null + noDictSortDims[idx] = null; + } else { + Object data = CarbonUnsafeUtil + .getDataFromUnsafe(noDictSortDataTypes[idx], baseObject, address, size, length); + size += length; + noDictSortDims[idx] = data; + } + } else { + byte[] bytes = new byte[length]; + CarbonUnsafe.getUnsafe() + .copyMemory(baseObject, address + size, bytes, CarbonUnsafe.BYTE_ARRAY_OFFSET, length); + size += length; + noDictSortDims[idx] = bytes; + } } // read no-sort dims & measures @@ -487,13 +637,26 @@ public void writeIntermediateSortTempRowFromUnsafeMemoryToStream(Object baseObje for (int idx = 0; idx < noDictSortDimCnt; idx++) { short length = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size); size += 2; - byte[] bytes = new byte[length]; - CarbonUnsafe.getUnsafe().copyMemory(baseObject, address + size, - bytes, CarbonUnsafe.BYTE_ARRAY_OFFSET, length); - size += length; + if (this.noDictSortColMapping[idx]) { + // get the original data from unsafe memory + if (0 == length) { + // if the length is 0, then the data is null + writeDataToStream(null, outputStream, idx); + } else { + Object data = CarbonUnsafeUtil + .getDataFromUnsafe(noDictSortDataTypes[idx], baseObject, address, size, length); + size += length; + writeDataToStream(data, outputStream, idx); + } + } else { + byte[] bytes = new byte[length]; + CarbonUnsafe.getUnsafe() + .copyMemory(baseObject, address + size, bytes, CarbonUnsafe.BYTE_ARRAY_OFFSET, length); + size += length; - outputStream.writeShort(length); - outputStream.write(bytes); + outputStream.writeShort(length); + outputStream.write(bytes); + } } // packed no-sort & measure @@ -534,13 +697,31 @@ public int writeRawRowAsIntermediateSortTempRowToUnsafeMemory(Object[] row, // write no-dict & sort for (int idx = 0; idx < this.noDictSortDimCnt; idx++) { - byte[] bytes = (byte[]) row[this.noDictSortDimIdx[idx]]; - CarbonUnsafe.getUnsafe().putShort(baseObject, address + size, (short) bytes.length); - size += 2; - CarbonUnsafe.getUnsafe() - .copyMemory(bytes, CarbonUnsafe.BYTE_ARRAY_OFFSET, baseObject, address + size, - bytes.length); - size += bytes.length; + if (this.noDictSortColMapping[idx]) { + Object data = row[this.noDictSortDimIdx[idx]]; + if (null == data) { + // if the data is null, then write only the length as 0. + CarbonUnsafe.getUnsafe().putShort(baseObject, address + size, (short) 0); + size += 2; + } else { + int sizeInBytes = this.noDictSortDataTypes[idx].getSizeInBytes(); + CarbonUnsafe.getUnsafe().putShort(baseObject, address + size, (short) sizeInBytes); + size += 2; + // put data to unsafe according to the data types + CarbonUnsafeUtil + .putDataToUnsafe(noDictSortDataTypes[idx], data, baseObject, address, size, + sizeInBytes); + size += sizeInBytes; + } + } else { + byte[] bytes = (byte[]) row[this.noDictSortDimIdx[idx]]; + CarbonUnsafe.getUnsafe().putShort(baseObject, address + size, (short) bytes.length); + size += 2; + CarbonUnsafe.getUnsafe() + .copyMemory(bytes, CarbonUnsafe.BYTE_ARRAY_OFFSET, baseObject, address + size, + bytes.length); + size += bytes.length; + } } // convert pack no-sort @@ -574,9 +755,15 @@ private void packNoSortFieldsToBytes(Object[] row, ByteBuffer rowBuffer) { } // convert no-dict & no-sort for (int idx = 0; idx < this.noDictNoSortDimCnt; idx++) { - byte[] bytes = (byte[]) row[this.noDictNoSortDimIdx[idx]]; - rowBuffer.putShort((short) bytes.length); - rowBuffer.put(bytes); + if (this.noDictNoSortColMapping[idx]) { + // put the original data to buffer + putDataToRowBuffer(this.noDictNoSortDataTypes[idx], row[this.noDictNoSortDimIdx[idx]], + rowBuffer); + } else { + byte[] bytes = (byte[]) row[this.noDictNoSortDimIdx[idx]]; + rowBuffer.putShort((short) bytes.length); + rowBuffer.put(bytes); + } } // convert varchar dims for (int idx = 0; idx < this.varcharDimCnt; idx++) { @@ -592,37 +779,45 @@ private void packNoSortFieldsToBytes(Object[] row, ByteBuffer rowBuffer) { } // convert measure - Object tmpValue; - DataType tmpDataType; for (int idx = 0; idx < this.measureCnt; idx++) { - tmpValue = row[this.measureIdx[idx]]; - tmpDataType = this.dataTypes[idx]; - if (null == tmpValue) { - rowBuffer.put((byte) 0); - continue; - } - rowBuffer.put((byte) 1); - if (DataTypes.BOOLEAN == tmpDataType) { - if ((boolean) tmpValue) { - rowBuffer.put((byte) 1); - } else { - rowBuffer.put((byte) 0); - } - } else if (DataTypes.SHORT == tmpDataType) { - rowBuffer.putShort((Short) tmpValue); - } else if (DataTypes.INT == tmpDataType) { - rowBuffer.putInt((Integer) tmpValue); - } else if (DataTypes.LONG == tmpDataType) { - rowBuffer.putLong((Long) tmpValue); - } else if (DataTypes.DOUBLE == tmpDataType) { - rowBuffer.putDouble((Double) tmpValue); - } else if (DataTypes.isDecimal(tmpDataType)) { - byte[] decimalBytes = DataTypeUtil.bigDecimalToByte((BigDecimal) tmpValue); - rowBuffer.putShort((short) decimalBytes.length); - rowBuffer.put(decimalBytes); + putDataToRowBuffer(this.dataTypes[idx], row[this.measureIdx[idx]], rowBuffer); + } + } + + /** + * Put the data to the row buffer + * + * @param tmpDataType + * @param tmpValue + * @param rowBuffer + */ + private void putDataToRowBuffer(DataType tmpDataType, Object tmpValue, ByteBuffer rowBuffer) { + if (null == tmpValue) { + rowBuffer.put((byte) 0); + return; + } + rowBuffer.put((byte) 1); + if (DataTypes.BOOLEAN == tmpDataType) { + if ((boolean) tmpValue) { + rowBuffer.put((byte) 1); } else { - throw new IllegalArgumentException("Unsupported data type: " + tmpDataType); + rowBuffer.put((byte) 0); } + } else if (DataTypes.SHORT == tmpDataType) { + rowBuffer.putShort((Short) tmpValue); + } else if (DataTypes.INT == tmpDataType) { + rowBuffer.putInt((Integer) tmpValue); + } else if (DataTypes.LONG == tmpDataType) { + rowBuffer.putLong((Long) tmpValue); + } else if (DataTypes.DOUBLE == tmpDataType) { + rowBuffer.putDouble((Double) tmpValue); + } else if (DataTypes.isDecimal(tmpDataType)) { + byte[] decimalBytes = DataTypeUtil.bigDecimalToByte((BigDecimal) tmpValue); + rowBuffer.putShort((short) decimalBytes.length); + rowBuffer.put(decimalBytes); + } else { + throw new IllegalArgumentException("Unsupported data type: " + tmpDataType); } } + } diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparator.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparator.java index 8f29ceed32d..b0109fa6d16 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparator.java +++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparator.java @@ -20,7 +20,11 @@ import java.util.Comparator; import org.apache.carbondata.core.memory.CarbonUnsafe; +import org.apache.carbondata.core.metadata.datatype.DataType; import org.apache.carbondata.core.util.ByteUtil.UnsafeComparer; +import org.apache.carbondata.core.util.CarbonUnsafeUtil; +import org.apache.carbondata.core.util.DataTypeUtil; +import org.apache.carbondata.core.util.comparator.SerializableComparator; import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage; import org.apache.carbondata.processing.loading.sort.unsafe.holder.UnsafeCarbonRow; import org.apache.carbondata.processing.sort.sortdata.TableFieldStat; @@ -52,6 +56,7 @@ public int compare(UnsafeCarbonRow rowL, Object baseObjectL, UnsafeCarbonRow row long rowA = rowL.address; long rowB = rowR.address; int sizeInDictPartA = 0; + int noDicSortIdx = 0; int sizeInNonDictPartA = 0; int sizeInDictPartB = 0; @@ -60,25 +65,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.getNoDictDataType()[noDicSortIdx++]; + if (DataTypeUtil.isPrimitiveColumn(dataType)) { + Object data1 = null; + if (0 != lengthA) { + data1 = CarbonUnsafeUtil + .getDataFromUnsafe(dataType, baseObjectL, rowA + dictSizeInMemory, + sizeInNonDictPartA, lengthA); + sizeInNonDictPartA += lengthA; + } + Object data2 = null; + if (0 != lengthB) { + data2 = CarbonUnsafeUtil + .getDataFromUnsafe(dataType, baseObjectR, rowB + dictSizeInMemory, + sizeInNonDictPartB, lengthB); + sizeInNonDictPartB += lengthB; + } + // use the data type based comparator for the no dictionary encoded columns + SerializableComparator comparator = + org.apache.carbondata.core.util.comparator.Comparator.getComparator(dataType); + int difference = comparator.compare(data1, data2); + if (difference != 0) { + return difference; + } + } else { + byte[] byteArr1 = new byte[lengthA]; + CarbonUnsafe.getUnsafe() + .copyMemory(baseObjectL, rowA + dictSizeInMemory + sizeInNonDictPartA, byteArr1, + CarbonUnsafe.BYTE_ARRAY_OFFSET, lengthA); + sizeInNonDictPartA += lengthA; + + byte[] byteArr2 = new byte[lengthB]; + CarbonUnsafe.getUnsafe() + .copyMemory(baseObjectR, rowB + dictSizeInMemory + sizeInNonDictPartB, byteArr2, + CarbonUnsafe.BYTE_ARRAY_OFFSET, lengthB); + sizeInNonDictPartB += lengthB; - int difference = UnsafeComparer.INSTANCE.compareTo(byteArr1, byteArr2); - if (difference != 0) { - return difference; + int difference = UnsafeComparer.INSTANCE.compareTo(byteArr1, byteArr2); + if (difference != 0) { + return difference; + } } } else { int dimFieldA = CarbonUnsafe.getUnsafe().getInt(baseObjectL, rowA + sizeInDictPartA); diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeFinalMergePageHolder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeFinalMergePageHolder.java index 102b0572fe6..b805d37a8d3 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeFinalMergePageHolder.java +++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeFinalMergePageHolder.java @@ -19,6 +19,7 @@ import org.apache.carbondata.common.logging.LogService; import org.apache.carbondata.common.logging.LogServiceFactory; +import org.apache.carbondata.core.metadata.datatype.DataType; import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow; import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage; import org.apache.carbondata.processing.loading.sort.unsafe.merger.UnsafeInMemoryIntermediateDataMerger; @@ -43,6 +44,8 @@ public class UnsafeFinalMergePageHolder implements SortTempChunkHolder { private IntermediateSortTempRow currentRow; + private DataType[] noDictDataType; + public UnsafeFinalMergePageHolder(UnsafeInMemoryIntermediateDataMerger merger, boolean[] noDictSortColumnMapping) { this.actualSize = merger.getEntryCount(); @@ -52,8 +55,10 @@ public UnsafeFinalMergePageHolder(UnsafeInMemoryIntermediateDataMerger merger, for (UnsafeCarbonRowPage rowPage: rowPages) { rowPage.setReadConvertedNoSortField(); } + this.noDictDataType = rowPages[0].getTableFieldStat().getNoDictDataType(); LOGGER.info("Processing unsafe inmemory rows page with size : " + actualSize); - this.comparator = new IntermediateSortTempRowComparator(noDictSortColumnMapping); + this.comparator = + new IntermediateSortTempRowComparator(noDictSortColumnMapping, noDictDataType); } public boolean hasNext() { diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeInmemoryHolder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeInmemoryHolder.java index 02ffd6881af..baa9e711297 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeInmemoryHolder.java +++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeInmemoryHolder.java @@ -45,7 +45,8 @@ public UnsafeInmemoryHolder(UnsafeCarbonRowPage rowPage) { this.rowPage = rowPage; LOGGER.info("Processing unsafe inmemory rows page with size : " + actualSize); this.comparator = new IntermediateSortTempRowComparator( - rowPage.getTableFieldStat().getIsSortColNoDictFlags()); + rowPage.getTableFieldStat().getIsSortColNoDictFlags(), + rowPage.getTableFieldStat().getNoDictDataType()); this.rowPage.setReadConvertedNoSortField(); } 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 7c3c056e23e..a991d4cf570 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 @@ -109,7 +109,8 @@ public UnsafeSortTempFileChunkHolder(File tempFile, SortParameters parameters, this.tableFieldStat = new TableFieldStat(parameters); this.sortStepRowHandler = new SortStepRowHandler(tableFieldStat); this.executorService = Executors.newFixedThreadPool(1); - comparator = new IntermediateSortTempRowComparator(parameters.getNoDictionarySortColumn()); + comparator = new IntermediateSortTempRowComparator(parameters.getNoDictionarySortColumn(), + parameters.getNoDictDataType()); this.convertNoSortFields = convertNoSortFields; initialize(); } diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/CarbonRowDataWriterProcessorStepImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/CarbonRowDataWriterProcessorStepImpl.java index ac13d248c18..7683bbceeeb 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/CarbonRowDataWriterProcessorStepImpl.java +++ b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/CarbonRowDataWriterProcessorStepImpl.java @@ -273,19 +273,19 @@ private CarbonRow convertRow(CarbonRow row) throws KeyGenException { int dictIndex = 0; int nonDicIndex = 0; int[] dim = new int[this.dimensionCount]; - byte[][] nonDicArray = new byte[this.noDictWithComplextCount][]; + Object[] nonDicArray = new Object[this.noDictWithComplextCount]; // read dimension values int dimCount = 0; for (; dimCount < isNoDictionaryDimensionColumn.length; dimCount++) { if (isNoDictionaryDimensionColumn[dimCount]) { - nonDicArray[nonDicIndex++] = (byte[]) row.getObject(dimCount); + nonDicArray[nonDicIndex++] = row.getObject(dimCount); } else { dim[dictIndex++] = (int) row.getObject(dimCount); } } for (; dimCount < this.dimensionWithComplexCount; dimCount++) { - nonDicArray[nonDicIndex++] = (byte[]) row.getObject(dimCount); + nonDicArray[nonDicIndex++] = row.getObject(dimCount); } Object[] measures = new Object[measureCount]; diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataConverterProcessorStepImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataConverterProcessorStepImpl.java index e3bc97f2b59..ae9ec3d432d 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataConverterProcessorStepImpl.java +++ b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataConverterProcessorStepImpl.java @@ -44,6 +44,7 @@ import org.apache.carbondata.processing.loading.partition.impl.RawRowComparator; import org.apache.carbondata.processing.loading.row.CarbonRowBatch; import org.apache.carbondata.processing.util.CarbonBadRecordUtil; +import org.apache.carbondata.processing.util.CarbonDataProcessorUtil; import org.apache.commons.lang3.StringUtils; @@ -134,12 +135,16 @@ private void initializeSortColumnRangesPartitioner() { // sort the range bounds (sort in carbon is a little different from what we think) Arrays.sort(convertedSortColumnRanges, new RawRowComparator(sortColumnRangeInfo.getSortColumnIndex(), - sortColumnRangeInfo.getIsSortColumnNoDict())); + sortColumnRangeInfo.getIsSortColumnNoDict(), CarbonDataProcessorUtil + .getNoDictDataTypes(configuration.getTableIdentifier().getDatabaseName(), + configuration.getTableIdentifier().getTableName()))); // range partitioner to dispatch rows by sort columns this.partitioner = new RangePartitionerImpl(convertedSortColumnRanges, new RawRowComparator(sortColumnRangeInfo.getSortColumnIndex(), - sortColumnRangeInfo.getIsSortColumnNoDict())); + sortColumnRangeInfo.getIsSortColumnNoDict(), CarbonDataProcessorUtil + .getNoDictDataTypes(configuration.getTableIdentifier().getDatabaseName(), + configuration.getTableIdentifier().getTableName()))); } // only convert sort column fields diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/InputProcessorStepWithNoConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/InputProcessorStepWithNoConverterImpl.java index ce8b62f1e32..b9216758718 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/InputProcessorStepWithNoConverterImpl.java +++ b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/InputProcessorStepWithNoConverterImpl.java @@ -266,8 +266,13 @@ private Object[] convertToNoDictionaryToBytes(Object[] data, DataField[] dataFie Object[] newData = new Object[data.length]; for (int i = 0; i < data.length; i++) { if (i < noDictionaryMapping.length && noDictionaryMapping[i]) { - newData[i] = DataTypeUtil - .getBytesDataDataTypeForNoDictionaryColumn(data[orderOfData[i]], dataTypes[i]); + if (DataTypeUtil.isPrimitiveColumn(dataTypes[i])) { + // keep the no dictionary measure column as original data + newData[i] = data[orderOfData[i]]; + } else { + newData[i] = DataTypeUtil + .getBytesDataDataTypeForNoDictionaryColumn(data[orderOfData[i]], dataTypes[i]); + } } else { // if this is a complex column then recursively comver the data into Byte Array. if (dataTypes[i].isComplexType()) { 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 0fc229a470c..1aa6da830c2 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 @@ -91,6 +91,8 @@ public class CompactionResultSortProcessor extends AbstractResultProcessor { * boolean mapping for no dictionary columns in schema */ private boolean[] noDictionaryColMapping; + + private boolean[] sortColumnMapping; /** * boolean mapping for long string dimension */ @@ -275,7 +277,15 @@ private Object[] prepareRowObjectForSorting(Object[] row) { preparedRow[i] = dictionaryValues[dictionaryIndex++]; } else { // no dictionary dims - preparedRow[i] = wrapper.getNoDictionaryKeyByIndex(noDictionaryIndex++); + byte[] noDictionaryKeyByIndex = wrapper.getNoDictionaryKeyByIndex(noDictionaryIndex++); + if (DataTypeUtil.isPrimitiveColumn(dims.getDataType())) { + // no dictionary measure columns are expected as original data + preparedRow[i] = DataTypeUtil + .getDataBasedOnDataTypeForNoDictionaryColumn(noDictionaryKeyByIndex, + dims.getDataType()); + } else { + preparedRow[i] = noDictionaryKeyByIndex; + } } } // fill all the measures @@ -357,6 +367,7 @@ private void initSortDataRows() throws Exception { measureCount = carbonTable.getMeasureByTableName(tableName).size(); List dimensions = carbonTable.getDimensionByTableName(tableName); noDictionaryColMapping = new boolean[dimensions.size()]; + sortColumnMapping = new boolean[dimensions.size()]; isVarcharDimMapping = new boolean[dimensions.size()]; int i = 0; for (CarbonDimension dimension : dimensions) { @@ -364,6 +375,9 @@ private void initSortDataRows() throws Exception { i++; continue; } + if (dimension.isSortColumn()) { + sortColumnMapping[i] = true; + } noDictionaryColMapping[i] = true; if (dimension.getColumnSchema().getDataType() == DataTypes.VARCHAR) { isVarcharDimMapping[i] = true; @@ -395,8 +409,8 @@ private SortParameters createSortParameters() { return SortParameters .createSortParameters(carbonTable, carbonLoadModel.getDatabaseName(), tableName, dimensionColumnCount, segmentProperties.getComplexDimensions().size(), measureCount, - noDictionaryCount, segmentId, - carbonLoadModel.getTaskNo(), noDictionaryColMapping, isVarcharDimMapping, true); + noDictionaryCount, segmentId, carbonLoadModel.getTaskNo(), noDictionaryColMapping, + sortColumnMapping, isVarcharDimMapping, true); } /** @@ -404,14 +418,8 @@ private SortParameters createSortParameters() { * sort temp files */ private void initializeFinalThreadMergerForMergeSort() { - boolean[] noDictionarySortColumnMapping = null; - if (noDictionaryColMapping.length == this.segmentProperties.getNumberOfSortColumns()) { - noDictionarySortColumnMapping = noDictionaryColMapping; - } else { - noDictionarySortColumnMapping = new boolean[this.segmentProperties.getNumberOfSortColumns()]; - System.arraycopy(noDictionaryColMapping, 0, - noDictionarySortColumnMapping, 0, noDictionarySortColumnMapping.length); - } + boolean[] noDictionarySortColumnMapping = CarbonDataProcessorUtil + .getNoDictSortColMapping(carbonTable.getDatabaseName(), carbonTable.getTableName()); sortParameters.setNoDictionarySortColumn(noDictionarySortColumnMapping); String[] sortTempFileLocation = CarbonDataProcessorUtil.arrayAppend(tempStoreLocation, CarbonCommonConstants.FILE_SEPARATOR, CarbonCommonConstants.SORT_TEMP_FILE_LOCATION); 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 b877d52389d..2911c05456d 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 @@ -34,6 +34,7 @@ import org.apache.carbondata.core.metadata.CarbonMetadata; import org.apache.carbondata.core.metadata.SegmentFileStore; import org.apache.carbondata.core.metadata.schema.table.CarbonTable; +import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn; import org.apache.carbondata.core.scan.result.iterator.RawResultIterator; import org.apache.carbondata.core.scan.wrappers.ByteArrayWrapper; import org.apache.carbondata.core.util.ByteUtil; @@ -53,6 +54,8 @@ public class RowResultMergerProcessor extends AbstractResultProcessor { private SegmentProperties segprop; private CarbonLoadModel loadModel; private PartitionSpec partitionSpec; + + CarbonColumn[] noDicAndComplexColumns; /** * record holder heap */ @@ -86,6 +89,7 @@ public RowResultMergerProcessor(String databaseName, setDataFileAttributesInModel(loadModel, compactionType, carbonFactDataHandlerModel); carbonFactDataHandlerModel.setCompactionFlow(true); carbonFactDataHandlerModel.setSegmentId(loadModel.getSegmentId()); + this.noDicAndComplexColumns = carbonFactDataHandlerModel.getNoDictAndComplexColumns(); dataHandler = new CarbonFactDataHandlerColumnar(carbonFactDataHandlerModel); } @@ -200,7 +204,7 @@ public void close() { * @throws SliceMergerException */ private void addRow(Object[] carbonTuple) throws SliceMergerException { - CarbonRow row = WriteStepRowUtil.fromMergerRow(carbonTuple, segprop); + CarbonRow row = WriteStepRowUtil.fromMergerRow(carbonTuple, segprop, noDicAndComplexColumns); try { this.dataHandler.addDataToStore(row); } catch (CarbonDataWriterException e) { diff --git a/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/RowResultProcessor.java b/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/RowResultProcessor.java index 2dc79a399ea..00fbc7a40e3 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/RowResultProcessor.java +++ b/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/RowResultProcessor.java @@ -25,6 +25,7 @@ import org.apache.carbondata.core.datastore.row.CarbonRow; import org.apache.carbondata.core.datastore.row.WriteStepRowUtil; import org.apache.carbondata.core.metadata.schema.table.CarbonTable; +import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn; import org.apache.carbondata.processing.loading.model.CarbonLoadModel; import org.apache.carbondata.processing.store.CarbonDataFileAttributes; import org.apache.carbondata.processing.store.CarbonFactDataHandlerColumnar; @@ -37,6 +38,8 @@ public class RowResultProcessor { private CarbonFactHandler dataHandler; private SegmentProperties segmentProperties; + private CarbonColumn[] noDicAndComplexColumns; + private static final LogService LOGGER = LogServiceFactory.getLogService(RowResultProcessor.class.getName()); @@ -59,6 +62,7 @@ public RowResultProcessor(CarbonTable carbonTable, CarbonLoadModel loadModel, //Note: set compaction flow just to convert decimal type carbonFactDataHandlerModel.setCompactionFlow(true); carbonFactDataHandlerModel.setSegmentId(loadModel.getSegmentId()); + noDicAndComplexColumns = carbonFactDataHandlerModel.getNoDictAndComplexColumns(); dataHandler = new CarbonFactDataHandlerColumnar(carbonFactDataHandlerModel); } @@ -97,7 +101,8 @@ public boolean execute(List resultList) { } private void addRow(Object[] carbonTuple) throws CarbonDataWriterException { - CarbonRow row = WriteStepRowUtil.fromMergerRow(carbonTuple, segmentProperties); + CarbonRow row = WriteStepRowUtil.fromMergerRow(carbonTuple, segmentProperties, + noDicAndComplexColumns); try { this.dataHandler.addDataToStore(row); } catch (CarbonDataWriterException e) { diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateSortTempRowComparator.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateSortTempRowComparator.java index 9b6d1e825a9..54fa99ec827 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateSortTempRowComparator.java +++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateSortTempRowComparator.java @@ -19,7 +19,10 @@ import java.util.Comparator; +import org.apache.carbondata.core.metadata.datatype.DataType; import org.apache.carbondata.core.util.ByteUtil.UnsafeComparer; +import org.apache.carbondata.core.util.DataTypeUtil; +import org.apache.carbondata.core.util.comparator.SerializableComparator; import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow; /** @@ -31,11 +34,15 @@ public class IntermediateSortTempRowComparator implements Comparator, Serializable { private static final long serialVersionUID = -1739874611112709436L; @@ -28,13 +31,20 @@ public class NewRowComparator implements Comparator, Serializable { /** * mapping of dictionary dimensions and no dictionary of sort_column. */ - private boolean[] noDictionarySortColumnMaping; + private boolean[] noDicDimColMapping; + + private DataType[] noDicDataTypes; + + private boolean[] noDicSortColumnMapping; /** - * @param noDictionarySortColumnMaping + * @param noDicDimColMapping */ - public NewRowComparator(boolean[] noDictionarySortColumnMaping) { - this.noDictionarySortColumnMaping = noDictionarySortColumnMaping; + public NewRowComparator(boolean[] noDicDimColMapping, boolean[] noDicSortColumnMapping, + DataType[] noDicDataTypes) { + this.noDicDimColMapping = noDicDimColMapping; + this.noDicSortColumnMapping = noDicSortColumnMapping; + this.noDicDataTypes = noDicDataTypes; } /** @@ -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 = + org.apache.carbondata.core.util.comparator.Comparator + .getComparator(noDicDataTypes[dataTypeIdx]); + int difference = comparator.compare(rowA[index], rowB[index]); + if (difference != 0) { + return difference; + } + dataTypeIdx++; + } else { + byte[] byteArr1 = (byte[]) rowA[index]; + byte[] byteArr2 = (byte[]) rowB[index]; - int difference = UnsafeComparer.INSTANCE.compareTo(byteArr1, byteArr2); - if (difference != 0) { - return difference; + int difference = UnsafeComparer.INSTANCE.compareTo(byteArr1, byteArr2); + if (difference != 0) { + return difference; + } + } } } else { int dimFieldA = (int) rowA[index]; diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java index a5caf7b6823..730c729d20f 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java +++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java @@ -203,7 +203,9 @@ public void startSorting() throws CarbonSortKeyAndGroupByException { toSort = new Object[entryCount][]; System.arraycopy(recordHolderList, 0, toSort, 0, entryCount); if (parameters.getNumberOfNoDictSortColumns() > 0) { - Arrays.sort(toSort, new NewRowComparator(parameters.getNoDictionarySortColumn())); + Arrays.sort(toSort, new NewRowComparator(parameters.getNoDictionaryDimnesionColumn(), + parameters.getNoDictionarySortColumn(), + parameters.getNoDictDataType())); } else { Arrays.sort(toSort, new NewRowComparatorForNormalDims(parameters.getNumberOfSortColumns())); } @@ -315,7 +317,8 @@ public void run() { long startTime = System.currentTimeMillis(); if (parameters.getNumberOfNoDictSortColumns() > 0) { Arrays.sort(recordHolderArray, - new NewRowComparator(parameters.getNoDictionarySortColumn())); + new NewRowComparator(parameters.getNoDictionaryDimnesionColumn(), + parameters.getNoDictionarySortColumn(), parameters.getNoDictDataType())); } else { Arrays.sort(recordHolderArray, new NewRowComparatorForNormalDims(parameters.getNumberOfSortColumns())); diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortParameters.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortParameters.java index d3d538a8172..c4416d5236d 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortParameters.java +++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortParameters.java @@ -18,6 +18,7 @@ import java.io.File; import java.io.Serializable; +import java.util.Map; import org.apache.carbondata.common.logging.LogService; import org.apache.carbondata.common.logging.LogServiceFactory; @@ -88,6 +89,17 @@ public class SortParameters implements Serializable { private DataType[] measureDataType; + // no dictionary data types of the table + private DataType[] noDictDataType; + + // no dictionary columns data types participating in sort + // used while writing the row to sort temp file where sort no dict columns are handled seperately + private DataType[] noDictSortDataType; + + // no dictionary columns data types not participating in sort + // used while writing the row to sort temp file where nosort nodict columns are handled seperately + private DataType[] noDictNoSortDataType; + /** * To know how many columns are of high cardinality. */ @@ -111,6 +123,8 @@ public class SortParameters implements Serializable { private boolean[] noDictionaryDimnesionColumn; private boolean[] noDictionarySortColumn; + + private boolean[] sortColumn; /** * whether dimension is varchar data type. * since all dimensions are string, we use an array of boolean instead of datatypes @@ -142,11 +156,15 @@ public SortParameters getCopy() { parameters.databaseName = databaseName; parameters.tableName = tableName; parameters.measureDataType = measureDataType; + parameters.noDictDataType = noDictDataType; + parameters.noDictSortDataType = noDictSortDataType; + parameters.noDictNoSortDataType = noDictNoSortDataType; parameters.noDictionaryCount = noDictionaryCount; parameters.partitionID = partitionID; parameters.segmentId = segmentId; parameters.taskNo = taskNo; parameters.noDictionaryDimnesionColumn = noDictionaryDimnesionColumn; + parameters.sortColumn = sortColumn; parameters.isVarcharDimensionColumn = isVarcharDimensionColumn; parameters.noDictionarySortColumn = noDictionarySortColumn; parameters.numberOfSortColumns = numberOfSortColumns; @@ -382,7 +400,10 @@ public static SortParameters createSortParameters(CarbonDataLoadConfiguration co parameters.setNumberOfSortColumns(configuration.getNumberOfSortColumns()); parameters.setNumberOfNoDictSortColumns(configuration.getNumberOfNoDictSortColumns()); - setNoDictionarySortColumnMapping(parameters); + parameters.setNoDictionarySortColumn(CarbonDataProcessorUtil + .getNoDictSortColMapping(configuration.getTableIdentifier().getDatabaseName(), + configuration.getTableIdentifier().getTableName())); + parameters.setSortColumn(configuration.getSortColumnMapping()); parameters.setObserver(new SortObserver()); // get sort buffer size parameters.setSortBufferSize(Integer.parseInt(carbonProperties @@ -431,6 +452,14 @@ public static SortParameters createSortParameters(CarbonDataLoadConfiguration co DataType[] measureDataType = configuration.getMeasureDataType(); parameters.setMeasureDataType(measureDataType); + parameters.setNoDictDataType(CarbonDataProcessorUtil + .getNoDictDataTypes(configuration.getTableIdentifier().getDatabaseName(), + configuration.getTableIdentifier().getTableName())); + Map noDictSortAndNoSortDataTypes = CarbonDataProcessorUtil + .getNoDictSortAndNoSortDataTypes(configuration.getTableIdentifier().getDatabaseName(), + configuration.getTableIdentifier().getTableName()); + parameters.setNoDictSortDataType(noDictSortAndNoSortDataTypes.get("noDictSortDataTypes")); + parameters.setNoDictNoSortDataType(noDictSortAndNoSortDataTypes.get("noDictNoSortDataTypes")); return parameters; } @@ -442,28 +471,10 @@ public void setRangeId(int rangeId) { this.rangeId = rangeId; } - /** - * this method will set the boolean mapping for no dictionary sort columns - * - * @param parameters - */ - private static void setNoDictionarySortColumnMapping(SortParameters parameters) { - if (parameters.getNumberOfSortColumns() == parameters.getNoDictionaryDimnesionColumn().length) { - parameters.setNoDictionarySortColumn(parameters.getNoDictionaryDimnesionColumn()); - } else { - boolean[] noDictionarySortColumnTemp = new boolean[parameters.getNumberOfSortColumns()]; - System - .arraycopy(parameters.getNoDictionaryDimnesionColumn(), 0, noDictionarySortColumnTemp, 0, - parameters.getNumberOfSortColumns()); - parameters.setNoDictionarySortColumn(noDictionarySortColumnTemp); - } - } - public static SortParameters createSortParameters(CarbonTable carbonTable, String databaseName, String tableName, int dimColCount, int complexDimColCount, int measureColCount, - int noDictionaryCount, String segmentId, String taskNo, - boolean[] noDictionaryColMaping, boolean[] isVarcharDimensionColumn, - boolean isCompactionFlow) { + int noDictionaryCount, String segmentId, String taskNo, boolean[] noDictionaryColMaping, + boolean[] sortColumnMapping, boolean[] isVarcharDimensionColumn, boolean isCompactionFlow) { SortParameters parameters = new SortParameters(); CarbonProperties carbonProperties = CarbonProperties.getInstance(); parameters.setDatabaseName(databaseName); @@ -478,6 +489,7 @@ public static SortParameters createSortParameters(CarbonTable carbonTable, Strin parameters.setNumberOfNoDictSortColumns(carbonTable.getNumberOfNoDictSortColumns()); parameters.setComplexDimColCount(complexDimColCount); parameters.setNoDictionaryDimnesionColumn(noDictionaryColMaping); + parameters.setSortColumn(sortColumnMapping); parameters.setIsVarcharDimensionColumn(isVarcharDimensionColumn); parameters.setObserver(new SortObserver()); // get sort buffer size @@ -523,8 +535,46 @@ public static SortParameters createSortParameters(CarbonTable carbonTable, Strin .getMeasureDataType(parameters.getMeasureColCount(), parameters.getDatabaseName(), parameters.getTableName()); parameters.setMeasureDataType(type); - setNoDictionarySortColumnMapping(parameters); + parameters.setNoDictDataType(CarbonDataProcessorUtil + .getNoDictDataTypes(parameters.getDatabaseName(), parameters.getTableName())); + Map noDictSortAndNoSortDataTypes = CarbonDataProcessorUtil + .getNoDictSortAndNoSortDataTypes(parameters.getDatabaseName(), parameters.getTableName()); + parameters.setNoDictSortDataType(noDictSortAndNoSortDataTypes.get("noDictSortDataTypes")); + parameters.setNoDictNoSortDataType(noDictSortAndNoSortDataTypes.get("noDictNoSortDataTypes")); + parameters.setNoDictionarySortColumn(CarbonDataProcessorUtil + .getNoDictSortColMapping(parameters.getDatabaseName(), parameters.getTableName())); return parameters; } + public DataType[] getNoDictSortDataType() { + return noDictSortDataType; + } + + public void setNoDictSortDataType(DataType[] noDictSortDataType) { + this.noDictSortDataType = noDictSortDataType; + } + + public DataType[] getNoDictNoSortDataType() { + return noDictNoSortDataType; + } + + public DataType[] getNoDictDataType() { + return noDictDataType; + } + + public void setNoDictNoSortDataType(DataType[] noDictNoSortDataType) { + this.noDictNoSortDataType = noDictNoSortDataType; + } + + public void setNoDictDataType(DataType[] noDictDataType) { + this.noDictDataType = noDictDataType; + } + + public boolean[] getSortColumn() { + return sortColumn; + } + + public void setSortColumn(boolean[] sortColumn) { + this.sortColumn = sortColumn; + } } diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java index e39fe1d3dde..a1ef04eb260 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java +++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java @@ -115,7 +115,7 @@ public SortTempFileChunkHolder(File tempFile, SortParameters sortParameters, Str this.tableFieldStat = new TableFieldStat(sortParameters); this.sortStepRowHandler = new SortStepRowHandler(tableFieldStat); this.comparator = new IntermediateSortTempRowComparator( - tableFieldStat.getIsSortColNoDictFlags()); + tableFieldStat.getIsSortColNoDictFlags(), tableFieldStat.getNoDictDataType()); this.executorService = Executors .newFixedThreadPool(1, new CarbonThreadFactory("SafeSortTempChunkHolderPool:" + tableName)); this.convertToActualField = convertToActualField; diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TableFieldStat.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TableFieldStat.java index 353ddb41f86..e9ed6f3ad6f 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TableFieldStat.java +++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TableFieldStat.java @@ -44,6 +44,9 @@ public class TableFieldStat implements Serializable { private boolean[] isVarcharDimFlags; private int measureCnt; private DataType[] measureDataType; + private DataType[] noDictDataType; + private DataType[] noDictSortDataType; + private DataType[] noDictNoSortDataType; // indices for dict & sort dimension columns private int[] dictSortDimIdx; @@ -66,17 +69,20 @@ public TableFieldStat(SortParameters sortParameters) { this.complexDimCnt = sortParameters.getComplexDimColCount(); this.isSortColNoDictFlags = sortParameters.getNoDictionarySortColumn(); this.isVarcharDimFlags = sortParameters.getIsVarcharDimensionColumn(); - int sortColCnt = isSortColNoDictFlags.length; - for (boolean flag : isSortColNoDictFlags) { - if (flag) { + boolean[] isDimNoDictFlags = sortParameters.getNoDictionaryDimnesionColumn(); + boolean[] sortColumn = sortParameters.getSortColumn(); + for (int i = 0; i < isDimNoDictFlags.length; i++) { + if (isDimNoDictFlags[i] && sortColumn[i]) { noDictSortDimCnt++; - } else { + } else if (!isDimNoDictFlags[i] && sortColumn[i]) { dictSortDimCnt++; } } this.measureCnt = sortParameters.getMeasureColCount(); this.measureDataType = sortParameters.getMeasureDataType(); - + this.noDictDataType = sortParameters.getNoDictDataType(); + this.noDictSortDataType = sortParameters.getNoDictSortDataType(); + this.noDictNoSortDataType = sortParameters.getNoDictNoSortDataType(); for (boolean flag : isVarcharDimFlags) { if (flag) { varcharDimCnt++; @@ -97,19 +103,18 @@ public TableFieldStat(SortParameters sortParameters) { int tmpDictSortCnt = 0; int tmpDictNoSortCnt = 0; int tmpVarcharCnt = 0; - boolean[] isDimNoDictFlags = sortParameters.getNoDictionaryDimnesionColumn(); for (int i = 0; i < isDimNoDictFlags.length; i++) { if (isDimNoDictFlags[i]) { if (isVarcharDimFlags[i]) { varcharDimIdx[tmpVarcharCnt++] = i; - } else if (i < sortColCnt && isSortColNoDictFlags[i]) { + } else if (sortColumn[i]) { noDictSortDimIdx[tmpNoDictSortCnt++] = i; } else { noDictNoSortDimIdx[tmpNoDictNoSortCnt++] = i; } } else { - if (i < sortColCnt && !isSortColNoDictFlags[i]) { + if (sortColumn[i]) { dictSortDimIdx[tmpDictSortCnt++] = i; } else { dictNoSortDimIdx[tmpDictNoSortCnt++] = i; @@ -217,4 +222,17 @@ public int[] getMeasureIdx() { return Objects.hash(dictSortDimCnt, dictNoSortDimCnt, noDictSortDimCnt, noDictNoSortDimCnt, complexDimCnt, varcharDimCnt, measureCnt); } + + public DataType[] getNoDictSortDataType() { + return noDictSortDataType; + } + + public DataType[] getNoDictNoSortDataType() { + return noDictNoSortDataType; + } + + + public DataType[] getNoDictDataType() { + return noDictDataType; + } } \ No newline at end of file diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java index 7151b47b794..c23b071c25f 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java +++ b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java @@ -48,6 +48,7 @@ import org.apache.carbondata.core.util.CarbonProperties; import org.apache.carbondata.core.util.CarbonThreadFactory; import org.apache.carbondata.core.util.CarbonUtil; +import org.apache.carbondata.core.util.DataTypeUtil; import org.apache.carbondata.processing.datatypes.GenericDataType; import org.apache.carbondata.processing.store.writer.CarbonFactDataWriter; @@ -240,9 +241,17 @@ public void addDataToStore(CarbonRow row) throws CarbonDataWriterException { */ private boolean isVarcharColumnFull(CarbonRow row) { if (model.getVarcharDimIdxInNoDict().size() > 0) { - byte[][] nonDictArray = WriteStepRowUtil.getNoDictAndComplexDimension(row); + Object[] nonDictArray = WriteStepRowUtil.getNoDictAndComplexDimension(row); for (int i = 0; i < model.getVarcharDimIdxInNoDict().size(); i++) { - varcharColumnSizeInByte[i] += nonDictArray[model.getVarcharDimIdxInNoDict().get(i)].length; + if (DataTypeUtil + .isPrimitiveColumn(model.getNoDictAndComplexColumns()[i].getDataType())) { + // get the size from the data type + varcharColumnSizeInByte[i] += + model.getNoDictAndComplexColumns()[i].getDataType().getSizeInBytes(); + } else { + varcharColumnSizeInByte[i] += + ((byte[]) nonDictArray[model.getVarcharDimIdxInNoDict().get(i)]).length; + } if (SnappyCompressor.MAX_BYTE_TO_COMPRESS - (varcharColumnSizeInByte[i] + dataRows.size() * 4) < (2 << 20)) { LOGGER.info("Limited by varchar column, page size is " + dataRows.size()); diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java index 1a38de6efa1..4b42bfc1a71 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java +++ b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java @@ -125,6 +125,12 @@ public void setBlockSizeInMB(int blockSize) { * data type of all measures in the table */ private DataType[] measureDataType; + + /** + * no dictionary and complex columns in the table + */ + private CarbonColumn[] noDictAndComplexColumns; + /** * carbon data file attributes like task id, file stamp */ @@ -276,6 +282,8 @@ public static CarbonFactDataHandlerModel createCarbonFactDataHandlerModel( carbonFactDataHandlerModel.setSegmentProperties(segmentProperties); carbonFactDataHandlerModel.setColCardinality(colCardinality); carbonFactDataHandlerModel.setMeasureDataType(configuration.getMeasureDataType()); + carbonFactDataHandlerModel + .setNoDictAndComplexColumns(configuration.getNoDictAndComplexDimensions()); carbonFactDataHandlerModel.setWrapperColumnSchema(wrapperColumnSchema); carbonFactDataHandlerModel.setPrimitiveDimLens(simpleDimsLen); carbonFactDataHandlerModel.setCarbonDataFileAttributes(carbonDataFileAttributes); @@ -325,12 +333,20 @@ 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 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)) { + noDicAndComplexColumns[noDicAndComp++] = + new CarbonColumn(dim.getColumnSchema(), dim.getOrdinal(), dim.getSchemaOrdinal()); + } } CarbonFactDataHandlerModel carbonFactDataHandlerModel = new CarbonFactDataHandlerModel(); @@ -365,6 +381,7 @@ public static CarbonFactDataHandlerModel getCarbonFactDataHandlerModel(CarbonLoa measureDataTypes[i++] = msr.getDataType(); } carbonFactDataHandlerModel.setMeasureDataType(measureDataTypes); + carbonFactDataHandlerModel.setNoDictAndComplexColumns(noDicAndComplexColumns); CarbonUtil.checkAndCreateFolderWithPermission(carbonDataDirectoryPath); carbonFactDataHandlerModel.setCarbonDataDirectoryPath(carbonDataDirectoryPath); carbonFactDataHandlerModel.setPrimitiveDimLens(segmentProperties.getDimColumnsCardinality()); @@ -713,5 +730,13 @@ public String getColumnCompressor() { public void setColumnCompressor(String columnCompressor) { this.columnCompressor = columnCompressor; } + + public CarbonColumn[] getNoDictAndComplexColumns() { + return noDictAndComplexColumns; + } + + public void setNoDictAndComplexColumns(CarbonColumn[] noDictAndComplexColumns) { + this.noDictAndComplexColumns = noDictAndComplexColumns; + } } diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/TablePage.java b/processing/src/main/java/org/apache/carbondata/processing/store/TablePage.java index a311483d74b..2f49ef28591 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/store/TablePage.java +++ b/processing/src/main/java/org/apache/carbondata/processing/store/TablePage.java @@ -27,6 +27,8 @@ import java.util.List; import java.util.Map; +import org.apache.carbondata.common.logging.LogService; +import org.apache.carbondata.common.logging.LogServiceFactory; import org.apache.carbondata.core.datastore.ColumnType; import org.apache.carbondata.core.datastore.TableSpec; import org.apache.carbondata.core.datastore.exception.CarbonDataWriterException; @@ -60,6 +62,9 @@ */ public class TablePage { + private static final LogService LOGGER = + LogServiceFactory.getLogService(TablePage.class.getName()); + // For all dimension and measure columns, we store the column data directly in the page, // the length of the page is the number of rows. @@ -125,10 +130,24 @@ public class TablePage { page = ColumnPage.newLocalDictPage( columnPageEncoderMeta, pageSize, localDictionaryGenerator, false); } else { - page = ColumnPage.newPage(columnPageEncoderMeta, pageSize); + if (DataTypeUtil.isPrimitiveColumn(spec.getSchemaDataType())) { + columnPageEncoderMeta = + new ColumnPageEncoderMeta(spec, spec.getSchemaDataType(), columnCompressor); + // create the column page according to the data type for no dictionary numeric columns + if (DataTypes.isDecimal(spec.getSchemaDataType())) { + page = ColumnPage.newDecimalPage(columnPageEncoderMeta, pageSize); + } else { + page = ColumnPage.newPage(columnPageEncoderMeta, pageSize); + } + } else { + page = ColumnPage.newPage(columnPageEncoderMeta, pageSize); + } } + // set the stats collector according to the data type of the columns if (DataTypes.VARCHAR == dataType) { page.setStatsCollector(LVLongStringStatsCollector.newInstance()); + } else if (DataTypeUtil.isPrimitiveColumn(spec.getSchemaDataType())) { + page.setStatsCollector(PrimitivePageStatsCollector.newInstance(spec.getSchemaDataType())); } else { page.setStatsCollector(LVShortStringStatsCollector.newInstance()); } @@ -194,22 +213,35 @@ private void convertToColumnarAndAddToPages(int rowId, CarbonRow row, byte[] mdk int complexColumnCount = complexDimensionPages.length; if (noDictionaryCount > 0 || complexColumnCount > 0) { TableSpec tableSpec = model.getTableSpec(); - byte[][] noDictAndComplex = WriteStepRowUtil.getNoDictAndComplexDimension(row); List noDictionaryDimensionSpec = tableSpec.getNoDictionaryDimensionSpec(); + Object[] noDictAndComplex = WriteStepRowUtil.getNoDictAndComplexDimension(row); for (int i = 0; i < noDictAndComplex.length; i++) { if (noDictionaryDimensionSpec.get(i).getSchemaDataType() == DataTypes.VARCHAR) { - byte[] valueWithLength = addIntLengthToByteArray(noDictAndComplex[i]); + byte[] valueWithLength = addIntLengthToByteArray((byte[]) noDictAndComplex[i]); noDictDimensionPages[i].putData(rowId, valueWithLength); } else if (i < noDictionaryCount) { - // noDictionary columns, since it is variable length, we need to prepare each - // element as LV result byte array (first two bytes are the length of the array) - byte[] valueWithLength = addShortLengthToByteArray(noDictAndComplex[i]); - noDictDimensionPages[i].putData(rowId, valueWithLength); + if (DataTypeUtil + .isPrimitiveColumn(noDictDimensionPages[i].getColumnSpec().getSchemaDataType())) { + // put the actual data to the row + Object value = noDictAndComplex[i]; + // in compaction flow the measure with decimal type will come as Spark decimal. + // need to convert it to byte array. + if (DataTypes.isDecimal(noDictDimensionPages[i].getDataType()) && model + .isCompactionFlow() && value != null) { + value = DataTypeUtil.getDataTypeConverter().convertFromDecimalToBigDecimal(value); + } + noDictDimensionPages[i].putData(rowId, value); + } else { + // noDictionary columns, since it is variable length, we need to prepare each + // element as LV result byte array (first two bytes are the length of the array) + byte[] valueWithLength = addShortLengthToByteArray((byte[]) noDictAndComplex[i]); + noDictDimensionPages[i].putData(rowId, valueWithLength); + } } else { // complex columns - addComplexColumn(i - noDictionaryCount, rowId, noDictAndComplex[i]); + addComplexColumn(i - noDictionaryCount, rowId, (byte[]) noDictAndComplex[i]); } } } @@ -373,7 +405,19 @@ private EncodedColumnPage[] encodeAndCompressDimensions() columnPageEncoder = encodingFactory.createEncoder( spec, noDictDimensionPages[noDictIndex]); - encodedPage = columnPageEncoder.encode(noDictDimensionPages[noDictIndex++]); + encodedPage = columnPageEncoder.encode(noDictDimensionPages[noDictIndex]); + if (LOGGER.isDebugEnabled()) { + DataType targetDataType = + columnPageEncoder.getTargetDataType(noDictDimensionPages[noDictIndex]); + if (null != targetDataType) { + LOGGER.debug( + "Encoder result ---> Source data type: " + noDictDimensionPages[noDictIndex] + .getDataType().getName() + " Destination data type: " + targetDataType + .getName() + " for the column: " + noDictDimensionPages[noDictIndex] + .getColumnSpec().getFieldName()); + } + } + noDictIndex++; encodedDimensions.add(encodedPage); break; case COMPLEX: diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java index c2b21a6767c..3ba1e1db59c 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java +++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.LinkedHashMap; @@ -426,12 +427,81 @@ public static DataType[] getMeasureDataType(int measureCount, String databaseNam return type; } - public static DataType[] getMeasureDataType(int measureCount, DataField[] measureFields) { - DataType[] type = new DataType[measureCount]; - for (int i = 0; i < type.length; i++) { - type[i] = measureFields[i].getColumn().getDataType(); + /** + * Get the no dictionary data types on the table + * + * @param databaseName + * @param tableName + * @return + */ + public static DataType[] getNoDictDataTypes(String databaseName, String tableName) { + CarbonTable carbonTable = CarbonMetadata.getInstance().getCarbonTable(databaseName, tableName); + List dimensions = carbonTable.getDimensionByTableName(tableName); + List type = new ArrayList<>(); + for (int i = 0; i < dimensions.size(); i++) { + if (!dimensions.get(i).hasEncoding(Encoding.DICTIONARY)) { + type.add(dimensions.get(i).getDataType()); + } } - return type; + return type.toArray(new DataType[type.size()]); + } + + /** + * Get the no dictionary sort column mapping of the table + * + * @param databaseName + * @param tableName + * @return + */ + public static boolean[] getNoDictSortColMapping(String databaseName, String tableName) { + CarbonTable carbonTable = CarbonMetadata.getInstance().getCarbonTable(databaseName, tableName); + List dimensions = carbonTable.getDimensionByTableName(tableName); + List noDicSortColMap = new ArrayList<>(); + for (int i = 0; i < dimensions.size(); i++) { + if (!dimensions.get(i).hasEncoding(Encoding.DICTIONARY)) { + if (dimensions.get(i).isSortColumn()) { + noDicSortColMap.add(true); + } else { + noDicSortColMap.add(false); + } + } + } + Boolean[] mapping = noDicSortColMap.toArray(new Boolean[noDicSortColMap.size()]); + boolean[] noDicSortColMapping = new boolean[mapping.length]; + for (int i = 0; i < mapping.length; i++) { + noDicSortColMapping[i] = mapping[i].booleanValue(); + } + return noDicSortColMapping; + } + + /** + * Get the data types of the no dictionary sort columns + * + * @param databaseName + * @param tableName + * @return + */ + public static Map getNoDictSortAndNoSortDataTypes(String databaseName, + String tableName) { + CarbonTable carbonTable = CarbonMetadata.getInstance().getCarbonTable(databaseName, tableName); + List dimensions = carbonTable.getDimensionByTableName(tableName); + List noDictSortType = new ArrayList<>(); + List noDictNoSortType = new ArrayList<>(); + for (int i = 0; i < dimensions.size(); i++) { + if (!dimensions.get(i).hasEncoding(Encoding.DICTIONARY)) { + if (dimensions.get(i).isSortColumn()) { + noDictSortType.add(dimensions.get(i).getDataType()); + } else { + noDictNoSortType.add(dimensions.get(i).getDataType()); + } + } + } + DataType[] noDictSortTypes = noDictSortType.toArray(new DataType[noDictSortType.size()]); + DataType[] noDictNoSortTypes = noDictNoSortType.toArray(new DataType[noDictNoSortType.size()]); + Map noDictSortAndNoSortTypes = new HashMap<>(2); + noDictSortAndNoSortTypes.put("noDictSortDataTypes", noDictSortTypes); + noDictSortAndNoSortTypes.put("noDictNoSortDataTypes", noDictNoSortTypes); + return noDictSortAndNoSortTypes; } /**