diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java index 598d00e18d1..fb67208783a 100644 --- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java +++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java @@ -260,8 +260,7 @@ private void loadDictionaryData(DictionaryInfo dictionaryInfo, DictionaryCacheLoader dictionaryCacheLoader = new DictionaryCacheLoaderImpl(dictionaryColumnUniqueIdentifier); dictionaryCacheLoader - .load(dictionaryInfo, dictionaryColumnUniqueIdentifier.getColumnIdentifier(), - dictionaryChunkStartOffset, dictionaryChunkEndOffset, loadSortIndex); + .load(dictionaryInfo, dictionaryChunkStartOffset, dictionaryChunkEndOffset, loadSortIndex); } /** diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoader.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoader.java index 01c277d4bf4..8df15392435 100644 --- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoader.java +++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoader.java @@ -19,8 +19,6 @@ import java.io.IOException; -import org.apache.carbondata.core.metadata.ColumnIdentifier; - public interface DictionaryCacheLoader { /** @@ -28,7 +26,6 @@ public interface DictionaryCacheLoader { * * @param dictionaryInfo dictionary info object which will hold the required data * for a given column - * @param columnIdentifier column unique identifier * @param dictionaryChunkStartOffset start offset from where dictionary file has to * be read * @param dictionaryChunkEndOffset end offset till where dictionary file has to @@ -37,7 +34,7 @@ public interface DictionaryCacheLoader { * read in memory after dictionary loading * @throws IOException */ - void load(DictionaryInfo dictionaryInfo, ColumnIdentifier columnIdentifier, - long dictionaryChunkStartOffset, long dictionaryChunkEndOffset, boolean loadSortIndex) + void load(DictionaryInfo dictionaryInfo, long dictionaryChunkStartOffset, + long dictionaryChunkEndOffset, boolean loadSortIndex) throws IOException; } diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java index a603c01dac2..899abf50014 100644 --- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java @@ -23,7 +23,6 @@ import java.util.List; import org.apache.carbondata.core.constants.CarbonCommonConstants; -import org.apache.carbondata.core.metadata.ColumnIdentifier; import org.apache.carbondata.core.reader.CarbonDictionaryReader; import org.apache.carbondata.core.reader.sortindex.CarbonDictionarySortIndexReader; import org.apache.carbondata.core.service.CarbonCommonFactory; @@ -43,8 +42,7 @@ public class DictionaryCacheLoaderImpl implements DictionaryCacheLoader { /** * @param dictionaryColumnUniqueIdentifier dictionary column identifier */ - public DictionaryCacheLoaderImpl( - DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) { + DictionaryCacheLoaderImpl(DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) { this.dictionaryColumnUniqueIdentifier = dictionaryColumnUniqueIdentifier; } @@ -53,7 +51,6 @@ public DictionaryCacheLoaderImpl( * * @param dictionaryInfo dictionary info object which will hold the required data * for a given column - * @param columnIdentifier column unique identifier * @param dictionaryChunkStartOffset start offset from where dictionary file has to * be read * @param dictionaryChunkEndOffset end offset till where dictionary file has to @@ -62,9 +59,9 @@ public DictionaryCacheLoaderImpl( * read in memory after dictionary loading * @throws IOException */ - @Override public void load(DictionaryInfo dictionaryInfo, ColumnIdentifier columnIdentifier, - long dictionaryChunkStartOffset, long dictionaryChunkEndOffset, boolean loadSortIndex) - throws IOException { + @Override + public void load(DictionaryInfo dictionaryInfo, long dictionaryChunkStartOffset, + long dictionaryChunkEndOffset, boolean loadSortIndex) throws IOException { Iterator columnDictionaryChunkWrapper = load(dictionaryColumnUniqueIdentifier, dictionaryChunkStartOffset, dictionaryChunkEndOffset); diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/BTreeBuilderInfo.java b/core/src/main/java/org/apache/carbondata/core/datastore/BTreeBuilderInfo.java index 3bfbc451b0f..381e7640e6f 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/BTreeBuilderInfo.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/BTreeBuilderInfo.java @@ -43,16 +43,10 @@ public BTreeBuilderInfo(List footerList, this.footerList = footerList; } - /** - * @return the eachDimensionBlockSize - */ public int[] getDimensionColumnValueSize() { return dimensionColumnValueSize; } - /** - * @return the footerList - */ public List getFooterList() { return footerList; } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/DataRefNode.java b/core/src/main/java/org/apache/carbondata/core/datastore/DataRefNode.java index 13d5f698bae..273f8334ecf 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/DataRefNode.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/DataRefNode.java @@ -18,7 +18,6 @@ import java.io.IOException; -import org.apache.carbondata.core.cache.update.BlockletLevelDeleteDeltaDataCache; import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk; @@ -28,46 +27,47 @@ public interface DataRefNode { /** - * Method to get the next block this can be used while scanning when + * Return the next data block in the tree, this can be used while scanning when * iterator of this class can be used iterate over blocks - * - * @return next block */ DataRefNode getNextDataRefNode(); /** - * to get the number of keys tuples present in the block - * - * @return number of keys in the block + * Return the number of rows in the data block */ - int nodeSize(); + int numRows(); /** - * Method can be used to get the block index .This can be used when multiple - * thread can be used scan group of blocks in that can we can assign the + * Return the block index. This can be used when multiple + * thread can be used scan group of blocks in that can we can assign * some of the blocks to one thread and some to other - * - * @return block number */ - long nodeNumber(); + long nodeIndex(); + + /** + * Return the blocklet index in the node + */ + short blockletIndex(); /** - * Method is used for retreiving the BlockletId. - * @return the blockletid related to the data block. + * Return the number of pages */ - String blockletId(); + int numberOfPages(); /** - * This method will be used to get the max value of all the columns this can + * Return the number of rows for a give page + */ + int getPageRowCount(int pageNumber); + + /** + * Return the max value of all the columns, this can * be used in case of filter query - * */ byte[][] getColumnsMaxValue(); /** - * This method will be used to get the min value of all the columns this can + * Return the min value of all the columns, this can * be used in case of filter query - * */ byte[][] getColumnsMinValue(); @@ -75,15 +75,15 @@ public interface DataRefNode { * Below method will be used to get the dimension chunks * * @param fileReader file reader to read the chunks from file - * @param blockIndexes range indexes of the blocks need to be read + * @param columnIndexRange range indexes of the blocks need to be read * value can be {{0,10},{11,12},{13,13}} * here 0 to 10 and 11 to 12 column blocks will be read in one * IO operation 13th column block will be read separately * This will be helpful to reduce IO by reading bigger chunk of - * data in On IO + * data in one IO operation * @return dimension data chunks */ - DimensionRawColumnChunk[] getDimensionChunks(FileHolder fileReader, int[][] blockIndexes) + DimensionRawColumnChunk[] readDimensionChunks(FileReader fileReader, int[][] columnIndexRange) throws IOException; /** @@ -92,54 +92,31 @@ DimensionRawColumnChunk[] getDimensionChunks(FileHolder fileReader, int[][] bloc * @param fileReader file reader to read the chunk from file * @return dimension data chunk */ - DimensionRawColumnChunk getDimensionChunk(FileHolder fileReader, int blockIndexes) + DimensionRawColumnChunk readDimensionChunk(FileReader fileReader, int columnIndex) throws IOException; /** * Below method will be used to get the measure chunk * * @param fileReader file reader to read the chunk from file - * @param blockIndexes range indexes of the blocks need to be read + * @param columnIndexRange range indexes of the blocks need to be read * value can be {{0,10},{11,12},{13,13}} * here 0 to 10 and 11 to 12 column blocks will be read in one * IO operation 13th column block will be read separately * This will be helpful to reduce IO by reading bigger chunk of - * data in On IO + * data in one IO operation * @return measure column data chunk */ - MeasureRawColumnChunk[] getMeasureChunks(FileHolder fileReader, int[][] blockIndexes) + MeasureRawColumnChunk[] readMeasureChunks(FileReader fileReader, int[][] columnIndexRange) throws IOException; /** * Below method will be used to read the measure chunk * * @param fileReader file read to read the file chunk - * @param blockIndex block index to be read from file + * @param columnIndex block index to be read from file * @return measure data chunk */ - MeasureRawColumnChunk getMeasureChunk(FileHolder fileReader, int blockIndex) throws IOException; + MeasureRawColumnChunk readMeasureChunk(FileReader fileReader, int columnIndex) throws IOException; - /** - * @param deleteDeltaDataCache - */ - void setDeleteDeltaDataCache(BlockletLevelDeleteDeltaDataCache deleteDeltaDataCache); - - /** - * @return - */ - BlockletLevelDeleteDeltaDataCache getDeleteDeltaDataCache(); - - /** - * number of pages in blocklet - * @return - */ - int numberOfPages(); - - /** - * Return the number of rows for a give page - * - * @param pageNumber - * @return - */ - int getPageRowCount(int pageNumber); } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/FileHolder.java b/core/src/main/java/org/apache/carbondata/core/datastore/FileReader.java similarity index 97% rename from core/src/main/java/org/apache/carbondata/core/datastore/FileHolder.java rename to core/src/main/java/org/apache/carbondata/core/datastore/FileReader.java index b4130a010fd..df0d745bc05 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/FileHolder.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/FileReader.java @@ -20,7 +20,7 @@ import java.io.IOException; import java.nio.ByteBuffer; -public interface FileHolder { +public interface FileReader { /** * This method will be used to reads the data to byteBuffer from file based on offset @@ -98,10 +98,6 @@ ByteBuffer readByteBuffer(String filePath, long offset, int length) */ void finish() throws IOException; - void setQueryId(String queryId); - - String getQueryId(); - /** * Set the flag to read data page by page instead of whole blocklet. * diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java b/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java index 5f82e87aee4..c134db989e5 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java @@ -94,10 +94,9 @@ public class SegmentProperties { private int[] complexDimColumnCardinality; /** - * mapping of dimension column to block in a file this will be used for - * reading the blocks from file + * mapping of dimension ordinal in schema to column chunk index in the data file */ - private Map dimensionOrdinalToBlockMapping; + private Map dimensionOrdinalToChunkMapping; /** * a block can have multiple columns. This will have block index as key @@ -106,10 +105,9 @@ public class SegmentProperties { private Map> blockTodimensionOrdinalMapping; /** - * mapping of measure column to block to in file this will be used while - * reading the block in a file + * mapping of measure ordinal in schema to column chunk index in the data file */ - private Map measuresOrdinalToBlockMapping; + private Map measuresOrdinalToChunkMapping; /** * size of the each dimension column value in a block this can be used when @@ -172,15 +170,15 @@ public SegmentProperties(List columnsInTable, int[] columnCardinal new ArrayList(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); measures = new ArrayList(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); fillDimensionAndMeasureDetails(columnsInTable, columnCardinality); - dimensionOrdinalToBlockMapping = + dimensionOrdinalToChunkMapping = new HashMap(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); blockTodimensionOrdinalMapping = new HashMap>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - measuresOrdinalToBlockMapping = + measuresOrdinalToChunkMapping = new HashMap(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); intialiseColGroups(); fillOrdinalToBlockMappingForDimension(); - fillOrdinalToBlockIndexMappingForMeasureColumns(); + fillOrdinalToChunkIndexMappingForMeasureColumns(); fillColumnGroupAndItsCardinality(columnCardinality); fillKeyGeneratorDetails(); } @@ -237,7 +235,7 @@ private void fillOrdinalToBlockMappingForDimension() { if (dimension.isColumnar() || dimension.columnGroupId() != prvcolumnGroupId) { blockOrdinal++; } - dimensionOrdinalToBlockMapping.put(dimension.getOrdinal(), blockOrdinal); + dimensionOrdinalToChunkMapping.put(dimension.getOrdinal(), blockOrdinal); prvcolumnGroupId = dimension.columnGroupId(); index++; } @@ -245,7 +243,7 @@ private void fillOrdinalToBlockMappingForDimension() { // complex dimension will be stored at last while (index < complexDimensions.size()) { dimension = complexDimensions.get(index); - dimensionOrdinalToBlockMapping.put(dimension.getOrdinal(), ++blockOrdinal); + dimensionOrdinalToChunkMapping.put(dimension.getOrdinal(), ++blockOrdinal); blockOrdinal = fillComplexDimensionChildBlockIndex(blockOrdinal, dimension); index++; } @@ -256,7 +254,7 @@ private void fillOrdinalToBlockMappingForDimension() { * */ private void fillBlockToDimensionOrdinalMapping() { - Set> blocks = dimensionOrdinalToBlockMapping.entrySet(); + Set> blocks = dimensionOrdinalToChunkMapping.entrySet(); Iterator> blockItr = blocks.iterator(); while (blockItr.hasNext()) { Entry block = blockItr.next(); @@ -280,7 +278,7 @@ private void fillBlockToDimensionOrdinalMapping() { */ private int fillComplexDimensionChildBlockIndex(int blockOrdinal, CarbonDimension dimension) { for (int i = 0; i < dimension.getNumberOfChild(); i++) { - dimensionOrdinalToBlockMapping + dimensionOrdinalToChunkMapping .put(dimension.getListOfChildDimensions().get(i).getOrdinal(), ++blockOrdinal); if (dimension.getListOfChildDimensions().get(i).getNumberOfChild() > 0) { blockOrdinal = fillComplexDimensionChildBlockIndex(blockOrdinal, @@ -295,11 +293,11 @@ private int fillComplexDimensionChildBlockIndex(int blockOrdinal, CarbonDimensio * of measure ordinal to its block index mapping in * file */ - private void fillOrdinalToBlockIndexMappingForMeasureColumns() { + private void fillOrdinalToChunkIndexMappingForMeasureColumns() { int blockOrdinal = 0; int index = 0; while (index < measures.size()) { - measuresOrdinalToBlockMapping.put(measures.get(index).getOrdinal(), blockOrdinal); + measuresOrdinalToChunkMapping.put(measures.get(index).getOrdinal(), blockOrdinal); blockOrdinal++; index++; } @@ -731,17 +729,17 @@ public int[] getComplexDimColumnCardinality() { } /** - * @return the dimensionOrdinalToBlockMapping + * @return the dimensionOrdinalToChunkMapping */ - public Map getDimensionOrdinalToBlockMapping() { - return dimensionOrdinalToBlockMapping; + public Map getDimensionOrdinalToChunkMapping() { + return dimensionOrdinalToChunkMapping; } /** - * @return the measuresOrdinalToBlockMapping + * @return the measuresOrdinalToChunkMapping */ - public Map getMeasuresOrdinalToBlockMapping() { - return measuresOrdinalToBlockMapping; + public Map getMeasuresOrdinalToChunkMapping() { + return measuresOrdinalToChunkMapping; } /** @@ -804,16 +802,6 @@ public int getColumnGroupMdKeyOrdinal(int colGrpId, int ordinal) { return columnGroupOrdinalToMdkeymapping.get(colGrpId).get(ordinal); } - /** - * It returns no of column availble in given column group - * - * @param colGrpId - * @return no of column in given column group - */ - public int getNoOfColumnsInColumnGroup(int colGrpId) { - return columnGroupOrdinalToMdkeymapping.get(colGrpId).size(); - } - /** * @param blockIndex * @return It returns all dimension present in given block index diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/DimensionColumnDataChunk.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/DimensionColumnPage.java similarity index 74% rename from core/src/main/java/org/apache/carbondata/core/datastore/chunk/DimensionColumnDataChunk.java rename to core/src/main/java/org/apache/carbondata/core/datastore/chunk/DimensionColumnPage.java index 3791314b2e5..15840bce5c9 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/DimensionColumnDataChunk.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/DimensionColumnPage.java @@ -22,47 +22,47 @@ /** * Interface for dimension column chunk. */ -public interface DimensionColumnDataChunk { +public interface DimensionColumnPage { /** * Below method will be used to fill the data based on offset and row id * - * @param data data to filed * @param offset offset from which data need to be filed + * @param data data to filed * @return how many bytes was copied */ - int fillChunkData(byte[] data, int offset, int columnIndex, KeyStructureInfo restructuringInfo); + int fillRawData(int rowId, int offset, byte[] data, KeyStructureInfo restructuringInfo); /** * It uses to convert column data to dictionary integer value * * @param rowId - * @param columnIndex - * @param row + * @param chunkIndex + * @param outputSurrogateKey * @param restructuringInfo @return */ - int fillConvertedChunkData(int rowId, int columnIndex, int[] row, + int fillSurrogateKey(int rowId, int chunkIndex, int[] outputSurrogateKey, KeyStructureInfo restructuringInfo); /** * Fill the data to vector * @param vectorInfo - * @param column + * @param chunkIndex * @param restructuringInfo * @return next column index */ - int fillConvertedChunkData(ColumnVectorInfo[] vectorInfo, int column, + int fillVector(ColumnVectorInfo[] vectorInfo, int chunkIndex, KeyStructureInfo restructuringInfo); /** * Fill the data to vector - * @param rowMapping + * @param filteredRowId * @param vectorInfo - * @param column + * @param chunkIndex * @param restructuringInfo * @return next column index */ - int fillConvertedChunkData(int[] rowMapping, ColumnVectorInfo[] vectorInfo, int column, + int fillVector(int[] filteredRowId, ColumnVectorInfo[] vectorInfo, int chunkIndex, KeyStructureInfo restructuringInfo); /** @@ -70,30 +70,25 @@ int fillConvertedChunkData(int[] rowMapping, ColumnVectorInfo[] vectorInfo, int * * @return chunk */ - byte[] getChunkData(int columnIndex); + byte[] getChunkData(int rowId); /** * @return inverted index */ - int getInvertedIndex(int index); + int getInvertedIndex(int rowId); /** * - * @param invertedIndex + * @param rowId * @return index reverse index */ - int getInvertedReverseIndex(int invertedIndex); + int getInvertedReverseIndex(int rowId); /** * @return whether column is dictionary column or not */ boolean isNoDicitionaryColumn(); - /** - * @return length of each column - */ - int getColumnValueSize(); - /** * @return whether columns where explictly sorted or not */ @@ -102,11 +97,11 @@ int fillConvertedChunkData(int[] rowMapping, ColumnVectorInfo[] vectorInfo, int /** * to compare the data * - * @param index row index to be compared + * @param rowId row index to be compared * @param compareValue value to compare * @return compare result */ - int compareTo(int index, byte[] compareValue); + int compareTo(int rowId, byte[] compareValue); /** * below method will be used to free the allocated memory diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/AbstractDimensionDataChunk.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/AbstractDimensionColumnPage.java similarity index 69% rename from core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/AbstractDimensionDataChunk.java rename to core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/AbstractDimensionColumnPage.java index eac062fd8fa..6f316c50639 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/AbstractDimensionDataChunk.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/AbstractDimensionColumnPage.java @@ -16,18 +16,18 @@ */ package org.apache.carbondata.core.datastore.chunk.impl; -import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk; +import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage; import org.apache.carbondata.core.datastore.chunk.store.DimensionDataChunkStore; /** * Class responsibility is to give access to dimension column data chunk store */ -public abstract class AbstractDimensionDataChunk implements DimensionColumnDataChunk { +public abstract class AbstractDimensionColumnPage implements DimensionColumnPage { /** * data chunks */ - protected DimensionDataChunkStore dataChunkStore; + DimensionDataChunkStore dataChunkStore; /** * @return whether columns where explicitly sorted or not @@ -39,44 +39,38 @@ public abstract class AbstractDimensionDataChunk implements DimensionColumnDataC /** * Below method to get the data based in row id * - * @param index row id of the data + * @param rowId row id of the data * @return chunk */ - @Override public byte[] getChunkData(int index) { - return dataChunkStore.getRow(index); + @Override public byte[] getChunkData(int rowId) { + return dataChunkStore.getRow(rowId); } /** * @return inverted index */ - @Override public int getInvertedIndex(int index) { - return dataChunkStore.getInvertedIndex(index); + @Override public int getInvertedIndex(int rowId) { + return dataChunkStore.getInvertedIndex(rowId); } /** - * @param invertedIndex + * @param rowId * @return inverted index reverse */ - @Override public int getInvertedReverseIndex(int invertedIndex) { - return dataChunkStore.getInvertedReverseIndex(invertedIndex); - } - /** - * @return length of each column - */ - @Override public int getColumnValueSize() { - return dataChunkStore.getColumnValueSize(); + @Override public int getInvertedReverseIndex(int rowId) { + return dataChunkStore.getInvertedReverseIndex(rowId); } /** * To compare the data * - * @param index row index to be compared + * @param rowId row index to be compared * @param compareValue value to compare * @return compare result */ - @Override public int compareTo(int index, byte[] compareValue) { + @Override public int compareTo(int rowId, byte[] compareValue) { // TODO Auto-generated method stub - return dataChunkStore.compareTo(index, compareValue); + return dataChunkStore.compareTo(rowId, compareValue); } /** diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunk.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionColumnPage.java similarity index 80% rename from core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunk.java rename to core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionColumnPage.java index b76ae534cd6..741c13df5cc 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunk.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionColumnPage.java @@ -24,7 +24,7 @@ /** * This class is gives access to column group dimension data chunk store */ -public class ColumnGroupDimensionDataChunk extends AbstractDimensionDataChunk { +public class ColumnGroupDimensionColumnPage extends AbstractDimensionColumnPage { /** * Constructor for this class @@ -33,7 +33,7 @@ public class ColumnGroupDimensionDataChunk extends AbstractDimensionDataChunk { * @param columnValueSize chunk attributes * @param numberOfRows */ - public ColumnGroupDimensionDataChunk(byte[] dataChunk, int columnValueSize, int numberOfRows) { + public ColumnGroupDimensionColumnPage(byte[] dataChunk, int columnValueSize, int numberOfRows) { this.dataChunkStore = DimensionChunkStoreFactory.INSTANCE .getDimensionChunkStore(columnValueSize, false, numberOfRows, dataChunk.length, DimensionStoreType.FIXEDLENGTH); @@ -43,13 +43,13 @@ public ColumnGroupDimensionDataChunk(byte[] dataChunk, int columnValueSize, int /** * Below method will be used to fill the data based on offset and row id * - * @param data data to filed - * @param offset offset from which data need to be filed * @param rowId row id of the chunk + * @param offset offset from which data need to be filed + * @param data data to filed * @param restructuringInfo define the structure of the key * @return how many bytes was copied */ - @Override public int fillChunkData(byte[] data, int offset, int rowId, + @Override public int fillRawData(int rowId, int offset, byte[] data, KeyStructureInfo restructuringInfo) { byte[] row = dataChunkStore.getRow(rowId); byte[] maskedKey = getMaskedKey(row, restructuringInfo); @@ -61,20 +61,20 @@ public ColumnGroupDimensionDataChunk(byte[] dataChunk, int columnValueSize, int * Converts to column dictionary integer value * * @param rowId - * @param columnIndex - * @param row + * @param chunkIndex + * @param outputSurrogateKey * @param info KeyStructureInfo * @return */ - @Override public int fillConvertedChunkData(int rowId, int columnIndex, int[] row, + @Override public int fillSurrogateKey(int rowId, int chunkIndex, int[] outputSurrogateKey, KeyStructureInfo info) { byte[] data = dataChunkStore.getRow(rowId); long[] keyArray = info.getKeyGenerator().getKeyArray(data); int[] ordinal = info.getMdkeyQueryDimensionOrdinal(); for (int i = 0; i < ordinal.length; i++) { - row[columnIndex++] = (int) keyArray[ordinal[i]]; + outputSurrogateKey[chunkIndex++] = (int) keyArray[ordinal[i]]; } - return columnIndex; + return chunkIndex; } /** @@ -98,15 +98,15 @@ private byte[] getMaskedKey(byte[] data, KeyStructureInfo info) { /** * @return inverted index */ - @Override public int getInvertedIndex(int index) { + @Override public int getInvertedIndex(int rowId) { throw new UnsupportedOperationException("Operation not supported in case of cloumn group"); } /** - * @param invertedIndex + * @param rowId * @return inverted index reverse */ - @Override public int getInvertedReverseIndex(int invertedIndex) { + @Override public int getInvertedReverseIndex(int rowId) { throw new UnsupportedOperationException("Operation not supported in case of cloumn group"); } @@ -120,11 +120,11 @@ private byte[] getMaskedKey(byte[] data, KeyStructureInfo info) { /** * to compare the data * - * @param index row index to be compared + * @param rowId row index to be compared * @param compareValue value to compare * @return compare result */ - @Override public int compareTo(int index, byte[] compareValue) { + @Override public int compareTo(int rowId, byte[] compareValue) { throw new UnsupportedOperationException("Operation not supported in case of cloumn group"); } @@ -132,13 +132,13 @@ private byte[] getMaskedKey(byte[] data, KeyStructureInfo info) { * Fill the data to vector * * @param vectorInfo - * @param column + * @param chunkIndex * @param restructuringInfo * @return next column index */ - @Override public int fillConvertedChunkData(ColumnVectorInfo[] vectorInfo, int column, + @Override public int fillVector(ColumnVectorInfo[] vectorInfo, int chunkIndex, KeyStructureInfo restructuringInfo) { - ColumnVectorInfo columnVectorInfo = vectorInfo[column]; + ColumnVectorInfo columnVectorInfo = vectorInfo[chunkIndex]; int offset = columnVectorInfo.offset; int vectorOffset = columnVectorInfo.vectorOffset; int len = offset + columnVectorInfo.size; @@ -146,7 +146,7 @@ private byte[] getMaskedKey(byte[] data, KeyStructureInfo info) { for (int k = offset; k < len; k++) { long[] keyArray = restructuringInfo.getKeyGenerator().getKeyArray(dataChunkStore.getRow(k)); int index = 0; - for (int i = column; i < column + ordinal.length; i++) { + for (int i = chunkIndex; i < chunkIndex + ordinal.length; i++) { if (vectorInfo[i].directDictionaryGenerator == null) { vectorInfo[i].vector.putInt(vectorOffset, (int) keyArray[ordinal[index++]]); } else { @@ -156,30 +156,30 @@ private byte[] getMaskedKey(byte[] data, KeyStructureInfo info) { } vectorOffset++; } - return column + ordinal.length; + return chunkIndex + ordinal.length; } /** * Fill the data to vector * - * @param rowMapping + * @param filteredRowId * @param vectorInfo - * @param column + * @param chunkIndex * @param restructuringInfo * @return next column index */ - @Override public int fillConvertedChunkData(int[] rowMapping, ColumnVectorInfo[] vectorInfo, - int column, KeyStructureInfo restructuringInfo) { - ColumnVectorInfo columnVectorInfo = vectorInfo[column]; + @Override public int fillVector(int[] filteredRowId, ColumnVectorInfo[] vectorInfo, + int chunkIndex, KeyStructureInfo restructuringInfo) { + ColumnVectorInfo columnVectorInfo = vectorInfo[chunkIndex]; int offset = columnVectorInfo.offset; int vectorOffset = columnVectorInfo.vectorOffset; int len = offset + columnVectorInfo.size; int[] ordinal = restructuringInfo.getMdkeyQueryDimensionOrdinal(); for (int k = offset; k < len; k++) { long[] keyArray = - restructuringInfo.getKeyGenerator().getKeyArray(dataChunkStore.getRow(rowMapping[k])); + restructuringInfo.getKeyGenerator().getKeyArray(dataChunkStore.getRow(filteredRowId[k])); int index = 0; - for (int i = column; i < column + ordinal.length; i++) { + for (int i = chunkIndex; i < chunkIndex + ordinal.length; i++) { if (vectorInfo[i].directDictionaryGenerator == null) { vectorInfo[i].vector.putInt(vectorOffset, (int) keyArray[ordinal[index++]]); } else { @@ -189,6 +189,6 @@ private byte[] getMaskedKey(byte[] data, KeyStructureInfo info) { } vectorOffset++; } - return column + ordinal.length; + return chunkIndex + ordinal.length; } } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/DimensionRawColumnChunk.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/DimensionRawColumnChunk.java index 705c13cc39e..f9bb5901c7c 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/DimensionRawColumnChunk.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/DimensionRawColumnChunk.java @@ -19,25 +19,25 @@ import java.io.IOException; import java.nio.ByteBuffer; -import org.apache.carbondata.core.datastore.FileHolder; +import org.apache.carbondata.core.datastore.FileReader; import org.apache.carbondata.core.datastore.chunk.AbstractRawColumnChunk; -import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk; +import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage; import org.apache.carbondata.core.datastore.chunk.reader.DimensionColumnChunkReader; import org.apache.carbondata.core.memory.MemoryException; /** * Contains raw dimension data, * 1. The read uncompressed raw data of column chunk with all pages is stored in this instance. - * 2. The raw data can be converted to processed chunk using convertToDimColDataChunk method + * 2. The raw data can be converted to processed chunk using decodeColumnPage method * by specifying page number. */ public class DimensionRawColumnChunk extends AbstractRawColumnChunk { - private DimensionColumnDataChunk[] dataChunks; + private DimensionColumnPage[] dataChunks; private DimensionColumnChunkReader chunkReader; - private FileHolder fileHolder; + private FileReader fileReader; public DimensionRawColumnChunk(int columnIndex, ByteBuffer rawData, long offSet, int length, DimensionColumnChunkReader columnChunkReader) { @@ -46,17 +46,17 @@ public DimensionRawColumnChunk(int columnIndex, ByteBuffer rawData, long offSet, } /** - * Convert all raw data with all pages to processed DimensionColumnDataChunk's + * Convert all raw data with all pages to processed DimensionColumnPage's * @return */ - public DimensionColumnDataChunk[] convertToDimColDataChunks() { + public DimensionColumnPage[] decodeAllColumnPages() { if (dataChunks == null) { - dataChunks = new DimensionColumnDataChunk[pagesCount]; + dataChunks = new DimensionColumnPage[pagesCount]; } for (int i = 0; i < pagesCount; i++) { try { if (dataChunks[i] == null) { - dataChunks[i] = chunkReader.convertToDimensionChunk(this, i); + dataChunks[i] = chunkReader.decodeColumnPage(this, i); } } catch (IOException | MemoryException e) { throw new RuntimeException(e); @@ -66,24 +66,24 @@ public DimensionColumnDataChunk[] convertToDimColDataChunks() { } /** - * Convert raw data with specified page number processed to DimensionColumnDataChunk - * @param index + * Convert raw data with specified page number processed to DimensionColumnPage + * @param pageNumber * @return */ - public DimensionColumnDataChunk convertToDimColDataChunk(int index) { - assert index < pagesCount; + public DimensionColumnPage decodeColumnPage(int pageNumber) { + assert pageNumber < pagesCount; if (dataChunks == null) { - dataChunks = new DimensionColumnDataChunk[pagesCount]; + dataChunks = new DimensionColumnPage[pagesCount]; } - if (dataChunks[index] == null) { + if (dataChunks[pageNumber] == null) { try { - dataChunks[index] = chunkReader.convertToDimensionChunk(this, index); + dataChunks[pageNumber] = chunkReader.decodeColumnPage(this, pageNumber); } catch (IOException | MemoryException e) { throw new RuntimeException(e); } } - return dataChunks[index]; + return dataChunks[pageNumber]; } /** @@ -92,10 +92,10 @@ public DimensionColumnDataChunk convertToDimColDataChunk(int index) { * @param index * @return */ - public DimensionColumnDataChunk convertToDimColDataChunkWithOutCache(int index) { + public DimensionColumnPage convertToDimColDataChunkWithOutCache(int index) { assert index < pagesCount; try { - return chunkReader.convertToDimensionChunk(this, index); + return chunkReader.decodeColumnPage(this, index); } catch (Exception e) { throw new RuntimeException(e); } @@ -111,11 +111,11 @@ public DimensionColumnDataChunk convertToDimColDataChunkWithOutCache(int index) } } - public void setFileHolder(FileHolder fileHolder) { - this.fileHolder = fileHolder; + public void setFileReader(FileReader fileReader) { + this.fileReader = fileReader; } - public FileHolder getFileReader() { - return fileHolder; + public FileReader getFileReader() { + return fileReader; } } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunk.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionColumnPage.java similarity index 83% rename from core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunk.java rename to core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionColumnPage.java index 6629d318072..ff54b12274a 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunk.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionColumnPage.java @@ -28,7 +28,7 @@ /** * This class is gives access to fixed length dimension data chunk store */ -public class FixedLengthDimensionDataChunk extends AbstractDimensionDataChunk { +public class FixedLengthDimensionColumnPage extends AbstractDimensionColumnPage { /** * Constructor @@ -39,7 +39,7 @@ public class FixedLengthDimensionDataChunk extends AbstractDimensionDataChunk { * @param numberOfRows number of rows * @param columnValueSize size of each column value */ - public FixedLengthDimensionDataChunk(byte[] dataChunk, int[] invertedIndex, + public FixedLengthDimensionColumnPage(byte[] dataChunk, int[] invertedIndex, int[] invertedIndexReverse, int numberOfRows, int columnValueSize) { long totalSize = null != invertedIndex ? dataChunk.length + (2 * numberOfRows * CarbonCommonConstants.INT_SIZE_IN_BYTE) : @@ -53,15 +53,15 @@ public FixedLengthDimensionDataChunk(byte[] dataChunk, int[] invertedIndex, /** * Below method will be used to fill the data based on offset and row id * - * @param data data to filed + * @param rowId row id of the chunk * @param offset offset from which data need to be filed - * @param index row id of the chunk + * @param data data to filed * @param keyStructureInfo define the structure of the key * @return how many bytes was copied */ - @Override public int fillChunkData(byte[] data, int offset, int index, + @Override public int fillRawData(int rowId, int offset, byte[] data, KeyStructureInfo keyStructureInfo) { - dataChunkStore.fillRow(index, data, offset); + dataChunkStore.fillRow(rowId, data, offset); return dataChunkStore.getColumnValueSize(); } @@ -69,28 +69,28 @@ public FixedLengthDimensionDataChunk(byte[] dataChunk, int[] invertedIndex, * Converts to column dictionary integer value * * @param rowId - * @param columnIndex - * @param row + * @param chunkIndex + * @param outputSurrogateKey * @param restructuringInfo * @return */ - @Override public int fillConvertedChunkData(int rowId, int columnIndex, int[] row, + @Override public int fillSurrogateKey(int rowId, int chunkIndex, int[] outputSurrogateKey, KeyStructureInfo restructuringInfo) { - row[columnIndex] = dataChunkStore.getSurrogate(rowId); - return columnIndex + 1; + outputSurrogateKey[chunkIndex] = dataChunkStore.getSurrogate(rowId); + return chunkIndex + 1; } /** * Fill the data to vector * * @param vectorInfo - * @param column + * @param chunkIndex * @param restructuringInfo * @return next column index */ - @Override public int fillConvertedChunkData(ColumnVectorInfo[] vectorInfo, int column, + @Override public int fillVector(ColumnVectorInfo[] vectorInfo, int chunkIndex, KeyStructureInfo restructuringInfo) { - ColumnVectorInfo columnVectorInfo = vectorInfo[column]; + ColumnVectorInfo columnVectorInfo = vectorInfo[chunkIndex]; int offset = columnVectorInfo.offset; int vectorOffset = columnVectorInfo.vectorOffset; int len = columnVectorInfo.size + offset; @@ -117,27 +117,27 @@ public FixedLengthDimensionDataChunk(byte[] dataChunk, int[] invertedIndex, } } } - return column + 1; + return chunkIndex + 1; } /** * Fill the data to vector * - * @param rowMapping + * @param filteredRowId * @param vectorInfo - * @param column + * @param chunkIndex * @param restructuringInfo * @return next column index */ - @Override public int fillConvertedChunkData(int[] rowMapping, ColumnVectorInfo[] vectorInfo, - int column, KeyStructureInfo restructuringInfo) { - ColumnVectorInfo columnVectorInfo = vectorInfo[column]; + @Override public int fillVector(int[] filteredRowId, ColumnVectorInfo[] vectorInfo, + int chunkIndex, KeyStructureInfo restructuringInfo) { + ColumnVectorInfo columnVectorInfo = vectorInfo[chunkIndex]; int offset = columnVectorInfo.offset; int vectorOffset = columnVectorInfo.vectorOffset; int len = columnVectorInfo.size + offset; CarbonColumnVector vector = columnVectorInfo.vector; for (int j = offset; j < len; j++) { - int dict = dataChunkStore.getSurrogate(rowMapping[j]); + int dict = dataChunkStore.getSurrogate(filteredRowId[j]); if (columnVectorInfo.directDictionaryGenerator == null) { vector.putInt(vectorOffset++, dict); } else { @@ -158,6 +158,6 @@ public FixedLengthDimensionDataChunk(byte[] dataChunk, int[] invertedIndex, } } } - return column + 1; + return chunkIndex + 1; } } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/MeasureRawColumnChunk.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/MeasureRawColumnChunk.java index fa0777bcf93..5e8618bee4e 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/MeasureRawColumnChunk.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/MeasureRawColumnChunk.java @@ -19,7 +19,7 @@ import java.io.IOException; import java.nio.ByteBuffer; -import org.apache.carbondata.core.datastore.FileHolder; +import org.apache.carbondata.core.datastore.FileReader; import org.apache.carbondata.core.datastore.chunk.AbstractRawColumnChunk; import org.apache.carbondata.core.datastore.chunk.reader.MeasureColumnChunkReader; import org.apache.carbondata.core.datastore.page.ColumnPage; @@ -28,7 +28,7 @@ /** * Contains raw measure data * 1. The read uncompressed raw data of column chunk with all pages is stored in this instance. - * 2. The raw data can be converted to processed chunk using convertToColumnPage method + * 2. The raw data can be converted to processed chunk using decodeColumnPage method * by specifying page number. */ public class MeasureRawColumnChunk extends AbstractRawColumnChunk { @@ -37,7 +37,7 @@ public class MeasureRawColumnChunk extends AbstractRawColumnChunk { private MeasureColumnChunkReader chunkReader; - private FileHolder fileReader; + private FileReader fileReader; public MeasureRawColumnChunk(int columnIndex, ByteBuffer rawData, long offSet, int length, MeasureColumnChunkReader chunkReader) { @@ -48,14 +48,14 @@ public MeasureRawColumnChunk(int columnIndex, ByteBuffer rawData, long offSet, i /** * Convert all raw data with all pages to processed ColumnPage */ - public ColumnPage[] convertToColumnPage() { + public ColumnPage[] decodeAllColumnPages() { if (columnPages == null) { columnPages = new ColumnPage[pagesCount]; } for (int i = 0; i < pagesCount; i++) { try { if (columnPages[i] == null) { - columnPages[i] = chunkReader.convertToColumnPage(this, i); + columnPages[i] = chunkReader.decodeColumnPage(this, i); } } catch (Exception e) { throw new RuntimeException(e); @@ -68,21 +68,21 @@ public ColumnPage[] convertToColumnPage() { /** * Convert raw data with specified `columnIndex` processed to ColumnPage */ - public ColumnPage convertToColumnPage(int columnIndex) { - assert columnIndex < pagesCount; + public ColumnPage decodeColumnPage(int pageNumber) { + assert pageNumber < pagesCount; if (columnPages == null) { columnPages = new ColumnPage[pagesCount]; } try { - if (columnPages[columnIndex] == null) { - columnPages[columnIndex] = chunkReader.convertToColumnPage(this, columnIndex); + if (columnPages[pageNumber] == null) { + columnPages[pageNumber] = chunkReader.decodeColumnPage(this, pageNumber); } } catch (IOException | MemoryException e) { throw new RuntimeException(e); } - return columnPages[columnIndex]; + return columnPages[pageNumber]; } /** @@ -95,7 +95,7 @@ public ColumnPage convertToColumnPageWithOutCache(int index) { assert index < pagesCount; try { - return chunkReader.convertToColumnPage(this, index); + return chunkReader.decodeColumnPage(this, index); } catch (IOException | MemoryException e) { throw new RuntimeException(e); } @@ -111,11 +111,11 @@ public ColumnPage convertToColumnPageWithOutCache(int index) { } } - public void setFileReader(FileHolder fileReader) { + public void setFileReader(FileReader fileReader) { this.fileReader = fileReader; } - public FileHolder getFileReader() { + public FileReader getFileReader() { return fileReader; } } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/VariableLengthDimensionDataChunk.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/VariableLengthDimensionColumnPage.java similarity index 78% rename from core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/VariableLengthDimensionDataChunk.java rename to core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/VariableLengthDimensionColumnPage.java index 6c47bf5365d..d03b2deb13d 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/VariableLengthDimensionDataChunk.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/VariableLengthDimensionColumnPage.java @@ -26,7 +26,7 @@ /** * This class is gives access to variable length dimension data chunk store */ -public class VariableLengthDimensionDataChunk extends AbstractDimensionDataChunk { +public class VariableLengthDimensionColumnPage extends AbstractDimensionColumnPage { /** * Constructor for this class @@ -35,7 +35,7 @@ public class VariableLengthDimensionDataChunk extends AbstractDimensionDataChunk * @param invertedIndexReverse * @param numberOfRows */ - public VariableLengthDimensionDataChunk(byte[] dataChunks, int[] invertedIndex, + public VariableLengthDimensionColumnPage(byte[] dataChunks, int[] invertedIndex, int[] invertedIndexReverse, int numberOfRows) { long totalSize = null != invertedIndex ? (dataChunks.length + (2 * numberOfRows * CarbonCommonConstants.INT_SIZE_IN_BYTE) + ( @@ -50,13 +50,13 @@ public VariableLengthDimensionDataChunk(byte[] dataChunks, int[] invertedIndex, /** * Below method will be used to fill the data based on offset and row id * - * @param data data to filed + * @param rowId row id of the chunk * @param offset offset from which data need to be filed - * @param index row id of the chunk + * @param data data to filed * @param restructuringInfo define the structure of the key * @return how many bytes was copied */ - @Override public int fillChunkData(byte[] data, int offset, int index, + @Override public int fillRawData(int rowId, int offset, byte[] data, KeyStructureInfo restructuringInfo) { // no required in this case because this column chunk is not the part if // mdkey @@ -67,14 +67,14 @@ public VariableLengthDimensionDataChunk(byte[] dataChunks, int[] invertedIndex, * Converts to column dictionary integer value * * @param rowId - * @param columnIndex - * @param row + * @param chunkIndex + * @param outputSurrogateKey * @param restructuringInfo * @return */ - @Override public int fillConvertedChunkData(int rowId, int columnIndex, int[] row, + @Override public int fillSurrogateKey(int rowId, int chunkIndex, int[] outputSurrogateKey, KeyStructureInfo restructuringInfo) { - return columnIndex + 1; + return chunkIndex + 1; } /** @@ -84,24 +84,17 @@ public VariableLengthDimensionDataChunk(byte[] dataChunks, int[] invertedIndex, return true; } - /** - * @return length of each column - */ - @Override public int getColumnValueSize() { - return -1; - } - /** * Fill the data to vector * * @param vectorInfo - * @param column + * @param chunkIndex * @param restructuringInfo * @return next column index */ - @Override public int fillConvertedChunkData(ColumnVectorInfo[] vectorInfo, int column, + @Override public int fillVector(ColumnVectorInfo[] vectorInfo, int chunkIndex, KeyStructureInfo restructuringInfo) { - ColumnVectorInfo columnVectorInfo = vectorInfo[column]; + ColumnVectorInfo columnVectorInfo = vectorInfo[chunkIndex]; CarbonColumnVector vector = columnVectorInfo.vector; int offset = columnVectorInfo.offset; int vectorOffset = columnVectorInfo.vectorOffset; @@ -111,21 +104,21 @@ public VariableLengthDimensionDataChunk(byte[] dataChunks, int[] invertedIndex, // string in no dictionary case at present. dataChunkStore.fillRow(i, vector, vectorOffset++); } - return column + 1; + return chunkIndex + 1; } /** * Fill the data to vector * - * @param rowMapping + * @param filteredRowId * @param vectorInfo - * @param column + * @param chunkIndex * @param restructuringInfo * @return next column index */ - @Override public int fillConvertedChunkData(int[] rowMapping, ColumnVectorInfo[] vectorInfo, - int column, KeyStructureInfo restructuringInfo) { - ColumnVectorInfo columnVectorInfo = vectorInfo[column]; + @Override public int fillVector(int[] filteredRowId, ColumnVectorInfo[] vectorInfo, + int chunkIndex, KeyStructureInfo restructuringInfo) { + ColumnVectorInfo columnVectorInfo = vectorInfo[chunkIndex]; CarbonColumnVector vector = columnVectorInfo.vector; int offset = columnVectorInfo.offset; int vectorOffset = columnVectorInfo.vectorOffset; @@ -133,8 +126,8 @@ public VariableLengthDimensionDataChunk(byte[] dataChunks, int[] invertedIndex, for (int i = offset; i < len; i++) { // Considering only String case now as we support only // string in no dictionary case at present. - dataChunkStore.fillRow(rowMapping[i], vector, vectorOffset++); + dataChunkStore.fillRow(filteredRowId[i], vector, vectorOffset++); } - return column + 1; + return chunkIndex + 1; } } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/DimensionColumnChunkReader.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/DimensionColumnChunkReader.java index 7b5b9c87213..fd81973a821 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/DimensionColumnChunkReader.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/DimensionColumnChunkReader.java @@ -18,8 +18,8 @@ import java.io.IOException; -import org.apache.carbondata.core.datastore.FileHolder; -import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk; +import org.apache.carbondata.core.datastore.FileReader; +import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage; import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; import org.apache.carbondata.core.memory.MemoryException; @@ -34,20 +34,20 @@ public interface DimensionColumnChunkReader { * Below method will be used to read the chunk based on block indexes * * @param fileReader file reader to read the blocks from file - * @param blockletIndexes blocklets to be read + * @param columnIndexRange blocklets to be read * @return dimension column chunks */ - DimensionRawColumnChunk[] readRawDimensionChunks(FileHolder fileReader, int[][] blockletIndexes) + DimensionRawColumnChunk[] readRawDimensionChunks(FileReader fileReader, int[][] columnIndexRange) throws IOException; /** * Below method will be used to read the chunk based on block index * * @param fileReader file reader to read the blocks from file - * @param blockletIndex block to be read + * @param columnIndex column to be read * @return dimension column chunk */ - DimensionRawColumnChunk readRawDimensionChunk(FileHolder fileReader, int blockletIndex) + DimensionRawColumnChunk readRawDimensionChunk(FileReader fileReader, int columnIndex) throws IOException; /** @@ -58,6 +58,6 @@ DimensionRawColumnChunk readRawDimensionChunk(FileHolder fileReader, int blockle * @return * @throws IOException */ - DimensionColumnDataChunk convertToDimensionChunk(DimensionRawColumnChunk dimensionRawColumnChunk, + DimensionColumnPage decodeColumnPage(DimensionRawColumnChunk dimensionRawColumnChunk, int pageNumber) throws IOException, MemoryException; } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/MeasureColumnChunkReader.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/MeasureColumnChunkReader.java index 02dc6a2933d..bf76025a780 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/MeasureColumnChunkReader.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/MeasureColumnChunkReader.java @@ -18,7 +18,7 @@ import java.io.IOException; -import org.apache.carbondata.core.datastore.FileHolder; +import org.apache.carbondata.core.datastore.FileReader; import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk; import org.apache.carbondata.core.datastore.page.ColumnPage; import org.apache.carbondata.core.memory.MemoryException; @@ -32,20 +32,20 @@ public interface MeasureColumnChunkReader { * Method to read the blocks data based on block indexes * * @param fileReader file reader to read the blocks - * @param blockIndexes blocks to be read + * @param columnIndexRange blocks to be read * @return measure data chunks */ - MeasureRawColumnChunk[] readRawMeasureChunks(FileHolder fileReader, int[][] blockIndexes) + MeasureRawColumnChunk[] readRawMeasureChunks(FileReader fileReader, int[][] columnIndexRange) throws IOException; /** * Method to read the blocks data based on block index * * @param fileReader file reader to read the blocks - * @param blockIndex block to be read + * @param columnIndex block to be read * @return measure data chunk */ - MeasureRawColumnChunk readRawMeasureChunk(FileHolder fileReader, int blockIndex) + MeasureRawColumnChunk readRawMeasureChunk(FileReader fileReader, int columnIndex) throws IOException; /** @@ -55,7 +55,7 @@ MeasureRawColumnChunk readRawMeasureChunk(FileHolder fileReader, int blockIndex) * @return * @throws IOException */ - ColumnPage convertToColumnPage(MeasureRawColumnChunk measureRawColumnChunk, + ColumnPage decodeColumnPage(MeasureRawColumnChunk measureRawColumnChunk, int pageNumber) throws IOException, MemoryException; } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractChunkReaderV2V3Format.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractChunkReaderV2V3Format.java index f083612ac54..60950c93b8f 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractChunkReaderV2V3Format.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractChunkReaderV2V3Format.java @@ -19,7 +19,7 @@ import java.io.IOException; import java.util.List; -import org.apache.carbondata.core.datastore.FileHolder; +import org.apache.carbondata.core.datastore.FileReader; import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; import org.apache.carbondata.core.metadata.blocklet.BlockletInfo; import org.apache.carbondata.format.Encoding; @@ -55,43 +55,43 @@ public AbstractChunkReaderV2V3Format(final BlockletInfo blockletInfo, * For last column read is separately and process * * @param fileReader file reader to read the blocks from file - * @param blockletIndexes blocks range to be read + * @param columnIndexRange column index range to be read * @return dimension column chunks */ - @Override public DimensionRawColumnChunk[] readRawDimensionChunks(final FileHolder fileReader, - final int[][] blockletIndexes) throws IOException { + @Override public DimensionRawColumnChunk[] readRawDimensionChunks(final FileReader fileReader, + final int[][] columnIndexRange) throws IOException { // read the column chunk based on block index and add DimensionRawColumnChunk[] dataChunks = new DimensionRawColumnChunk[dimensionChunksOffset.size()]; // if blocklet index is empty then return empry data chunk - if (blockletIndexes.length == 0) { + if (columnIndexRange.length == 0) { return dataChunks; } DimensionRawColumnChunk[] groupChunk = null; int index = 0; // iterate till block indexes -1 as block index will be in sorted order, so to avoid // the last column reading in group - for (int i = 0; i < blockletIndexes.length - 1; i++) { + for (int i = 0; i < columnIndexRange.length - 1; i++) { index = 0; groupChunk = - readRawDimensionChunksInGroup(fileReader, blockletIndexes[i][0], blockletIndexes[i][1]); - for (int j = blockletIndexes[i][0]; j <= blockletIndexes[i][1]; j++) { + readRawDimensionChunksInGroup(fileReader, columnIndexRange[i][0], columnIndexRange[i][1]); + for (int j = columnIndexRange[i][0]; j <= columnIndexRange[i][1]; j++) { dataChunks[j] = groupChunk[index++]; } } // check last index is present in block index, if it is present then read separately - if (blockletIndexes[blockletIndexes.length - 1][0] == dimensionChunksOffset.size() - 1) { - dataChunks[blockletIndexes[blockletIndexes.length - 1][0]] = - readRawDimensionChunk(fileReader, blockletIndexes[blockletIndexes.length - 1][0]); + if (columnIndexRange[columnIndexRange.length - 1][0] == dimensionChunksOffset.size() - 1) { + dataChunks[columnIndexRange[columnIndexRange.length - 1][0]] = + readRawDimensionChunk(fileReader, columnIndexRange[columnIndexRange.length - 1][0]); } // otherwise read the data in group else { - groupChunk = - readRawDimensionChunksInGroup(fileReader, blockletIndexes[blockletIndexes.length - 1][0], - blockletIndexes[blockletIndexes.length - 1][1]); + groupChunk = readRawDimensionChunksInGroup( + fileReader, columnIndexRange[columnIndexRange.length - 1][0], + columnIndexRange[columnIndexRange.length - 1][1]); index = 0; - for (int j = blockletIndexes[blockletIndexes.length - 1][0]; - j <= blockletIndexes[blockletIndexes.length - 1][1]; j++) { + for (int j = columnIndexRange[columnIndexRange.length - 1][0]; + j <= columnIndexRange[columnIndexRange.length - 1][1]; j++) { dataChunks[j] = groupChunk[index++]; } } @@ -109,7 +109,7 @@ public AbstractChunkReaderV2V3Format(final BlockletInfo blockletInfo, * @return measure raw chunkArray * @throws IOException */ - protected abstract DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileHolder fileReader, + protected abstract DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileReader fileReader, int startColumnBlockletIndex, int endColumnBlockletIndex) throws IOException; /** diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java index 27a4d89e43e..0dc1c1b969a 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java @@ -20,12 +20,12 @@ import java.nio.ByteBuffer; import java.util.List; -import org.apache.carbondata.core.datastore.FileHolder; -import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk; -import org.apache.carbondata.core.datastore.chunk.impl.ColumnGroupDimensionDataChunk; +import org.apache.carbondata.core.datastore.FileReader; +import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage; +import org.apache.carbondata.core.datastore.chunk.impl.ColumnGroupDimensionColumnPage; import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; -import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionDataChunk; -import org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionDataChunk; +import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionColumnPage; +import org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionColumnPage; import org.apache.carbondata.core.datastore.chunk.reader.dimension.AbstractChunkReader; import org.apache.carbondata.core.datastore.columnar.UnBlockIndexer; import org.apache.carbondata.core.metadata.blocklet.BlockletInfo; @@ -61,14 +61,14 @@ public CompressedDimensionChunkFileBasedReaderV1(final BlockletInfo blockletInfo * Below method will be used to read the raw chunk based on block indexes * * @param fileReader file reader to read the blocks from file - * @param blockletIndexes blocks to be read + * @param columnIndexRange blocks to be read * @return dimension column chunks */ - @Override public DimensionRawColumnChunk[] readRawDimensionChunks(FileHolder fileReader, - int[][] blockletIndexes) throws IOException { + @Override public DimensionRawColumnChunk[] readRawDimensionChunks(FileReader fileReader, + int[][] columnIndexRange) throws IOException { DimensionRawColumnChunk[] dataChunks = new DimensionRawColumnChunk[dimensionColumnChunk.size()]; - for (int i = 0; i < blockletIndexes.length; i++) { - for (int j = blockletIndexes[i][0]; j <= blockletIndexes[i][1]; j++) { + for (int i = 0; i < columnIndexRange.length; i++) { + for (int j = columnIndexRange[i][0]; j <= columnIndexRange[i][1]; j++) { dataChunks[j] = readRawDimensionChunk(fileReader, j); } } @@ -82,7 +82,7 @@ public CompressedDimensionChunkFileBasedReaderV1(final BlockletInfo blockletInfo * @param columnIndex column to be read * @return dimension column chunk */ - @Override public DimensionRawColumnChunk readRawDimensionChunk(FileHolder fileReader, + @Override public DimensionRawColumnChunk readRawDimensionChunk(FileReader fileReader, int columnIndex) throws IOException { DataChunk dataChunk = dimensionColumnChunk.get(columnIndex); ByteBuffer buffer = null; @@ -92,20 +92,20 @@ public CompressedDimensionChunkFileBasedReaderV1(final BlockletInfo blockletInfo } DimensionRawColumnChunk rawColumnChunk = new DimensionRawColumnChunk(columnIndex, buffer, 0, dataChunk.getDataPageLength(), this); - rawColumnChunk.setFileHolder(fileReader); + rawColumnChunk.setFileReader(fileReader); rawColumnChunk.setPagesCount(1); rawColumnChunk.setRowCount(new int[] { numberOfRows }); return rawColumnChunk; } - @Override public DimensionColumnDataChunk convertToDimensionChunk( + @Override public DimensionColumnPage decodeColumnPage( DimensionRawColumnChunk dimensionRawColumnChunk, int pageNumber) throws IOException { int blockIndex = dimensionRawColumnChunk.getColumnIndex(); byte[] dataPage = null; int[] invertedIndexes = null; int[] invertedIndexesReverse = null; int[] rlePage = null; - FileHolder fileReader = dimensionRawColumnChunk.getFileReader(); + FileReader fileReader = dimensionRawColumnChunk.getFileReader(); ByteBuffer rawData = dimensionRawColumnChunk.getRawData(); dataPage = COMPRESSOR.unCompressByte(rawData.array(), (int) dimensionRawColumnChunk.getOffSet(), @@ -145,23 +145,23 @@ public CompressedDimensionChunkFileBasedReaderV1(final BlockletInfo blockletInfo rlePage = null; } // fill chunk attributes - DimensionColumnDataChunk columnDataChunk = null; + DimensionColumnPage columnDataChunk = null; if (dataChunk.isRowMajor()) { // to store fixed length column chunk values - columnDataChunk = new ColumnGroupDimensionDataChunk(dataPage, eachColumnValueSize[blockIndex], - numberOfRows); + columnDataChunk = new ColumnGroupDimensionColumnPage( + dataPage, eachColumnValueSize[blockIndex], numberOfRows); } // if no dictionary column then first create a no dictionary column chunk // and set to data chunk instance else if (!CarbonUtil .hasEncoding(dataChunk.getEncodingList(), Encoding.DICTIONARY)) { columnDataChunk = - new VariableLengthDimensionDataChunk(dataPage, invertedIndexes, invertedIndexesReverse, + new VariableLengthDimensionColumnPage(dataPage, invertedIndexes, invertedIndexesReverse, numberOfRows); } else { // to store fixed length column chunk values columnDataChunk = - new FixedLengthDimensionDataChunk(dataPage, invertedIndexes, invertedIndexesReverse, + new FixedLengthDimensionColumnPage(dataPage, invertedIndexes, invertedIndexesReverse, numberOfRows, eachColumnValueSize[blockIndex]); } return columnDataChunk; diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java index b43f89c63d6..31fa81947ec 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java @@ -19,12 +19,12 @@ import java.io.IOException; import java.nio.ByteBuffer; -import org.apache.carbondata.core.datastore.FileHolder; -import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk; -import org.apache.carbondata.core.datastore.chunk.impl.ColumnGroupDimensionDataChunk; +import org.apache.carbondata.core.datastore.FileReader; +import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage; +import org.apache.carbondata.core.datastore.chunk.impl.ColumnGroupDimensionColumnPage; import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; -import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionDataChunk; -import org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionDataChunk; +import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionColumnPage; +import org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionColumnPage; import org.apache.carbondata.core.datastore.chunk.reader.dimension.AbstractChunkReaderV2V3Format; import org.apache.carbondata.core.datastore.columnar.UnBlockIndexer; import org.apache.carbondata.core.metadata.blocklet.BlockletInfo; @@ -56,7 +56,7 @@ public CompressedDimensionChunkFileBasedReaderV2(final BlockletInfo blockletInfo * @param columnIndex column to be read * @return dimension column chunk */ - public DimensionRawColumnChunk readRawDimensionChunk(FileHolder fileReader, int columnIndex) + public DimensionRawColumnChunk readRawDimensionChunk(FileReader fileReader, int columnIndex) throws IOException { int length = 0; if (dimensionChunksOffset.size() - 1 == columnIndex) { @@ -73,7 +73,7 @@ public DimensionRawColumnChunk readRawDimensionChunk(FileHolder fileReader, int } DimensionRawColumnChunk rawColumnChunk = new DimensionRawColumnChunk(columnIndex, buffer, 0, length, this); - rawColumnChunk.setFileHolder(fileReader); + rawColumnChunk.setFileReader(fileReader); rawColumnChunk.setPagesCount(1); rawColumnChunk.setRowCount(new int[] { numberOfRows }); return rawColumnChunk; @@ -90,7 +90,7 @@ public DimensionRawColumnChunk readRawDimensionChunk(FileHolder fileReader, int * @return measure raw chunkArray * @throws IOException */ - protected DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileHolder fileReader, + protected DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileReader fileReader, int startColumnBlockletIndex, int endColumnBlockletIndex) throws IOException { long currentDimensionOffset = dimensionChunksOffset.get(startColumnBlockletIndex); ByteBuffer buffer = null; @@ -106,7 +106,7 @@ protected DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileHolder fil int currentLength = (int) (dimensionChunksOffset.get(i + 1) - dimensionChunksOffset.get(i)); dataChunks[index] = new DimensionRawColumnChunk(i, buffer, runningLength, currentLength, this); - dataChunks[index].setFileHolder(fileReader); + dataChunks[index].setFileReader(fileReader); dataChunks[index].setPagesCount(1); dataChunks[index].setRowCount(new int[] { numberOfRows }); runningLength += currentLength; @@ -115,7 +115,7 @@ protected DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileHolder fil return dataChunks; } - public DimensionColumnDataChunk convertToDimensionChunk( + public DimensionColumnPage decodeColumnPage( DimensionRawColumnChunk dimensionRawColumnChunk, int pageNumber) throws IOException { byte[] dataPage = null; int[] invertedIndexes = null; @@ -169,23 +169,23 @@ public DimensionColumnDataChunk convertToDimensionChunk( dataPage = UnBlockIndexer.uncompressData(dataPage, rlePage, eachColumnValueSize[blockIndex]); } // fill chunk attributes - DimensionColumnDataChunk columnDataChunk = null; + DimensionColumnPage columnDataChunk = null; if (dimensionColumnChunk.isRowMajor()) { // to store fixed length column chunk values - columnDataChunk = new ColumnGroupDimensionDataChunk(dataPage, eachColumnValueSize[blockIndex], - numberOfRows); + columnDataChunk = new ColumnGroupDimensionColumnPage( + dataPage, eachColumnValueSize[blockIndex], numberOfRows); } // if no dictionary column then first create a no dictionary column chunk // and set to data chunk instance else if (!hasEncoding(dimensionColumnChunk.encoders, Encoding.DICTIONARY)) { columnDataChunk = - new VariableLengthDimensionDataChunk(dataPage, invertedIndexes, invertedIndexesReverse, + new VariableLengthDimensionColumnPage(dataPage, invertedIndexes, invertedIndexesReverse, numberOfRows); } else { // to store fixed length column chunk values columnDataChunk = - new FixedLengthDimensionDataChunk(dataPage, invertedIndexes, invertedIndexesReverse, + new FixedLengthDimensionColumnPage(dataPage, invertedIndexes, invertedIndexesReverse, numberOfRows, eachColumnValueSize[blockIndex]); } return columnDataChunk; diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimChunkFileBasedPageLevelReaderV3.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimChunkFileBasedPageLevelReaderV3.java index 1edfd090215..60f0b6717e2 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimChunkFileBasedPageLevelReaderV3.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimChunkFileBasedPageLevelReaderV3.java @@ -20,8 +20,8 @@ import java.io.IOException; import java.nio.ByteBuffer; -import org.apache.carbondata.core.datastore.FileHolder; -import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk; +import org.apache.carbondata.core.datastore.FileReader; +import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage; import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; import org.apache.carbondata.core.memory.MemoryException; import org.apache.carbondata.core.metadata.blocklet.BlockletInfo; @@ -70,7 +70,8 @@ public CompressedDimChunkFileBasedPageLevelReaderV3(BlockletInfo blockletInfo, * @param blockletColumnIndex blocklet index of the column in carbon data file * @return dimension raw chunk */ - public DimensionRawColumnChunk readRawDimensionChunk(FileHolder fileReader, + @Override + public DimensionRawColumnChunk readRawDimensionChunk(FileReader fileReader, int blockletColumnIndex) throws IOException { // get the current dimension offset long currentDimensionOffset = dimensionChunksOffset.get(blockletColumnIndex); @@ -116,7 +117,7 @@ public DimensionRawColumnChunk readRawDimensionChunk(FileHolder fileReader, * @param endBlockletColumnIndex blocklet index of the last dimension column * @ DimensionRawColumnChunk array */ - protected DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileHolder fileReader, + protected DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileReader fileReader, int startBlockletColumnIndex, int endBlockletColumnIndex) throws IOException { // create raw chunk for each dimension column DimensionRawColumnChunk[] dimensionDataChunks = @@ -136,7 +137,7 @@ protected DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileHolder fil * @param pageNumber number * @return DimensionColumnDataChunk */ - @Override public DimensionColumnDataChunk convertToDimensionChunk( + @Override public DimensionColumnPage decodeColumnPage( DimensionRawColumnChunk dimensionRawColumnChunk, int pageNumber) throws IOException, MemoryException { // data chunk of page 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 566e9b7a883..0fdc515f07d 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 @@ -20,11 +20,11 @@ import java.nio.ByteBuffer; import java.util.List; -import org.apache.carbondata.core.datastore.FileHolder; -import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk; +import org.apache.carbondata.core.datastore.FileReader; +import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage; import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; -import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionDataChunk; -import org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionDataChunk; +import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionColumnPage; +import org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionColumnPage; import org.apache.carbondata.core.datastore.chunk.reader.dimension.AbstractChunkReaderV2V3Format; import org.apache.carbondata.core.datastore.chunk.store.ColumnPageWrapper; import org.apache.carbondata.core.datastore.columnar.UnBlockIndexer; @@ -78,23 +78,23 @@ public CompressedDimensionChunkFileBasedReaderV3(BlockletInfo blockletInfo, * 5. Create the raw chunk object and fill the details * * @param fileReader reader for reading the column from carbon data file - * @param blockletColumnIndex blocklet index of the column in carbon data file + * @param columnIndex blocklet index of the column in carbon data file * @return dimension raw chunk */ - public DimensionRawColumnChunk readRawDimensionChunk(FileHolder fileReader, - int blockletColumnIndex) throws IOException { + public DimensionRawColumnChunk readRawDimensionChunk(FileReader fileReader, + int columnIndex) throws IOException { // get the current dimension offset - long currentDimensionOffset = dimensionChunksOffset.get(blockletColumnIndex); + long currentDimensionOffset = dimensionChunksOffset.get(columnIndex); int length = 0; // to calculate the length of the data to be read // column other than last column we can subtract the offset of current column with // next column and get the total length. // but for last column we need to use lastDimensionOffset which is the end position // of the last dimension, we can subtract current dimension offset from lastDimesionOffset - if (dimensionChunksOffset.size() - 1 == blockletColumnIndex) { + if (dimensionChunksOffset.size() - 1 == columnIndex) { length = (int) (lastDimensionOffsets - currentDimensionOffset); } else { - length = (int) (dimensionChunksOffset.get(blockletColumnIndex + 1) - currentDimensionOffset); + length = (int) (dimensionChunksOffset.get(columnIndex + 1) - currentDimensionOffset); } ByteBuffer buffer = null; // read the data from carbon data file @@ -103,15 +103,15 @@ public DimensionRawColumnChunk readRawDimensionChunk(FileHolder fileReader, } // get the data chunk which will have all the details about the data pages DataChunk3 dataChunk = CarbonUtil.readDataChunk3(buffer, 0, length); - return getDimensionRawColumnChunk(fileReader, blockletColumnIndex, 0, length, buffer, + return getDimensionRawColumnChunk(fileReader, columnIndex, 0, length, buffer, dataChunk); } - protected DimensionRawColumnChunk getDimensionRawColumnChunk(FileHolder fileReader, - int blockletColumnIndex, long offset, int length, ByteBuffer buffer, DataChunk3 dataChunk) { + protected DimensionRawColumnChunk getDimensionRawColumnChunk(FileReader fileReader, + int columnIndex, long offset, int length, ByteBuffer buffer, DataChunk3 dataChunk) { // creating a raw chunks instance and filling all the details DimensionRawColumnChunk rawColumnChunk = - new DimensionRawColumnChunk(blockletColumnIndex, buffer, offset, length, this); + new DimensionRawColumnChunk(columnIndex, buffer, offset, length, this); int numberOfPages = dataChunk.getPage_length().size(); byte[][] maxValueOfEachPage = new byte[numberOfPages][]; byte[][] minValueOfEachPage = new byte[numberOfPages][]; @@ -124,7 +124,7 @@ protected DimensionRawColumnChunk getDimensionRawColumnChunk(FileHolder fileRead eachPageLength[i] = dataChunk.getData_chunk_list().get(i).getNumberOfRowsInpage(); } rawColumnChunk.setDataChunkV3(dataChunk); - rawColumnChunk.setFileHolder(fileReader); + rawColumnChunk.setFileReader(fileReader); rawColumnChunk.setPagesCount(dataChunk.getPage_length().size()); rawColumnChunk.setMaxValues(maxValueOfEachPage); rawColumnChunk.setMinValues(minValueOfEachPage); @@ -153,7 +153,7 @@ protected DimensionRawColumnChunk getDimensionRawColumnChunk(FileHolder fileRead * blocklet index of the last dimension column * @ DimensionRawColumnChunk array */ - protected DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileHolder fileReader, + protected DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileReader fileReader, int startBlockletColumnIndex, int endBlockletColumnIndex) throws IOException { // to calculate the length of the data to be read // column we can subtract the offset of start column offset with @@ -188,9 +188,9 @@ protected DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileHolder fil * * @param rawColumnPage dimension raw chunk * @param pageNumber number - * @return DimensionColumnDataChunk + * @return DimensionColumnPage */ - @Override public DimensionColumnDataChunk convertToDimensionChunk( + @Override public DimensionColumnPage decodeColumnPage( DimensionRawColumnChunk rawColumnPage, int pageNumber) throws IOException, MemoryException { // data chunk of blocklet column DataChunk3 dataChunk3 = rawColumnPage.getDataChunkV3(); @@ -228,20 +228,19 @@ private boolean isEncodedWithMeta(DataChunk2 pageMetadata) { return false; } - protected DimensionColumnDataChunk decodeDimension(DimensionRawColumnChunk rawColumnPage, + protected DimensionColumnPage decodeDimension(DimensionRawColumnChunk rawColumnPage, ByteBuffer pageData, DataChunk2 pageMetadata, int offset) throws IOException, MemoryException { if (isEncodedWithMeta(pageMetadata)) { ColumnPage decodedPage = decodeDimensionByMeta(pageMetadata, pageData, offset); - return new ColumnPageWrapper(decodedPage, - eachColumnValueSize[rawColumnPage.getColumnIndex()]); + return new ColumnPageWrapper(decodedPage); } else { // following code is for backward compatibility return decodeDimensionLegacy(rawColumnPage, pageData, pageMetadata, offset); } } - private DimensionColumnDataChunk decodeDimensionLegacy(DimensionRawColumnChunk rawColumnPage, + private DimensionColumnPage decodeDimensionLegacy(DimensionRawColumnChunk rawColumnPage, ByteBuffer pageData, DataChunk2 pageMetadata, int offset) { byte[] dataPage; int[] rlePage; @@ -267,18 +266,18 @@ private DimensionColumnDataChunk decodeDimensionLegacy(DimensionRawColumnChunk r eachColumnValueSize[rawColumnPage.getColumnIndex()]); } - DimensionColumnDataChunk columnDataChunk = null; + DimensionColumnPage columnDataChunk = null; // if no dictionary column then first create a no dictionary column chunk // and set to data chunk instance if (!hasEncoding(pageMetadata.encoders, Encoding.DICTIONARY)) { columnDataChunk = - new VariableLengthDimensionDataChunk(dataPage, invertedIndexes, invertedIndexesReverse, + new VariableLengthDimensionColumnPage(dataPage, invertedIndexes, invertedIndexesReverse, pageMetadata.getNumberOfRowsInpage()); } else { // to store fixed length column chunk values columnDataChunk = - new FixedLengthDimensionDataChunk(dataPage, invertedIndexes, invertedIndexesReverse, + new FixedLengthDimensionColumnPage(dataPage, invertedIndexes, invertedIndexesReverse, pageMetadata.getNumberOfRowsInpage(), eachColumnValueSize[rawColumnPage.getColumnIndex()]); } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/AbstractMeasureChunkReaderV2V3Format.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/AbstractMeasureChunkReaderV2V3Format.java index 2239a2b2997..a3ed3391098 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/AbstractMeasureChunkReaderV2V3Format.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/AbstractMeasureChunkReaderV2V3Format.java @@ -20,7 +20,7 @@ import java.util.BitSet; import java.util.List; -import org.apache.carbondata.core.datastore.FileHolder; +import org.apache.carbondata.core.datastore.FileReader; import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk; import org.apache.carbondata.core.datastore.compression.Compressor; import org.apache.carbondata.core.datastore.compression.CompressorFactory; @@ -56,36 +56,40 @@ public AbstractMeasureChunkReaderV2V3Format(final BlockletInfo blockletInfo, * separately and process * * @param fileReader file reader to read the blocks from file - * @param blockIndexes blocks range to be read + * @param columnIndexRange blocks range to be read, columnIndexGroup[i] is one group, inside the + * group, columnIndexGroup[i][0] is start column index, + * and columnIndexGroup[i][1] is end column index * @return measure column chunks * @throws IOException */ - public MeasureRawColumnChunk[] readRawMeasureChunks(FileHolder fileReader, int[][] blockIndexes) - throws IOException { + public MeasureRawColumnChunk[] readRawMeasureChunks(FileReader fileReader, + int[][] columnIndexRange) throws IOException { // read the column chunk based on block index and add MeasureRawColumnChunk[] dataChunks = new MeasureRawColumnChunk[measureColumnChunkOffsets.size()]; - if (blockIndexes.length == 0) { + if (columnIndexRange.length == 0) { return dataChunks; } MeasureRawColumnChunk[] groupChunk = null; int index = 0; - for (int i = 0; i < blockIndexes.length - 1; i++) { + for (int i = 0; i < columnIndexRange.length - 1; i++) { index = 0; - groupChunk = readRawMeasureChunksInGroup(fileReader, blockIndexes[i][0], blockIndexes[i][1]); - for (int j = blockIndexes[i][0]; j <= blockIndexes[i][1]; j++) { + groupChunk = readRawMeasureChunksInGroup( + fileReader, columnIndexRange[i][0], columnIndexRange[i][1]); + for (int j = columnIndexRange[i][0]; j <= columnIndexRange[i][1]; j++) { dataChunks[j] = groupChunk[index++]; } } - if (blockIndexes[blockIndexes.length - 1][0] == measureColumnChunkOffsets.size() - 1) { - dataChunks[blockIndexes[blockIndexes.length - 1][0]] = - readRawMeasureChunk(fileReader, blockIndexes[blockIndexes.length - 1][0]); + if (columnIndexRange[columnIndexRange.length - 1][0] == measureColumnChunkOffsets.size() - 1) { + dataChunks[columnIndexRange[columnIndexRange.length - 1][0]] = + readRawMeasureChunk(fileReader, columnIndexRange[columnIndexRange.length - 1][0]); } else { - groupChunk = readRawMeasureChunksInGroup(fileReader, blockIndexes[blockIndexes.length - 1][0], - blockIndexes[blockIndexes.length - 1][1]); + groupChunk = readRawMeasureChunksInGroup( + fileReader, columnIndexRange[columnIndexRange.length - 1][0], + columnIndexRange[columnIndexRange.length - 1][1]); index = 0; - for (int j = blockIndexes[blockIndexes.length - 1][0]; - j <= blockIndexes[blockIndexes.length - 1][1]; j++) { + for (int j = columnIndexRange[columnIndexRange.length - 1][0]; + j <= columnIndexRange[columnIndexRange.length - 1][1]; j++) { dataChunks[j] = groupChunk[index++]; } } @@ -112,12 +116,12 @@ protected BitSet getNullBitSet( * data from * * @param fileReader file reader to read the data - * @param startColumnBlockletIndex first column blocklet index to be read - * @param endColumnBlockletIndex end column blocklet index to be read + * @param startColumnIndex first column index to be read + * @param endColumnIndex end column index to be read * @return measure raw chunkArray * @throws IOException */ - protected abstract MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileHolder fileReader, - int startColumnBlockletIndex, int endColumnBlockletIndex) throws IOException; + protected abstract MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileReader fileReader, + int startColumnIndex, int endColumnIndex) throws IOException; } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java index ae5537582ef..f0c1b754b7a 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java @@ -20,7 +20,7 @@ import java.nio.ByteBuffer; import java.util.List; -import org.apache.carbondata.core.datastore.FileHolder; +import org.apache.carbondata.core.datastore.FileReader; import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk; import org.apache.carbondata.core.datastore.chunk.reader.measure.AbstractMeasureChunkReader; import org.apache.carbondata.core.datastore.page.ColumnPage; @@ -56,14 +56,14 @@ public CompressedMeasureChunkFileBasedReaderV1(final BlockletInfo blockletInfo, * Method to read the blocks data based on block indexes * * @param fileReader file reader to read the blocks - * @param blockIndexes blocks to be read + * @param columnIndexRange blocks to be read * @return measure data chunks */ - @Override public MeasureRawColumnChunk[] readRawMeasureChunks(FileHolder fileReader, - int[][] blockIndexes) throws IOException { + @Override public MeasureRawColumnChunk[] readRawMeasureChunks(FileReader fileReader, + int[][] columnIndexRange) throws IOException { MeasureRawColumnChunk[] datChunk = new MeasureRawColumnChunk[measureColumnChunks.size()]; - for (int i = 0; i < blockIndexes.length; i++) { - for (int j = blockIndexes[i][0]; j <= blockIndexes[i][1]; j++) { + for (int i = 0; i < columnIndexRange.length; i++) { + for (int j = columnIndexRange[i][0]; j <= columnIndexRange[i][1]; j++) { datChunk[j] = readRawMeasureChunk(fileReader, j); } } @@ -77,7 +77,7 @@ public CompressedMeasureChunkFileBasedReaderV1(final BlockletInfo blockletInfo, * @param columnIndex column to be read * @return measure data chunk */ - @Override public MeasureRawColumnChunk readRawMeasureChunk(FileHolder fileReader, int columnIndex) + @Override public MeasureRawColumnChunk readRawMeasureChunk(FileReader fileReader, int columnIndex) throws IOException { DataChunk dataChunk = measureColumnChunks.get(columnIndex); ByteBuffer buffer = fileReader @@ -91,7 +91,7 @@ public CompressedMeasureChunkFileBasedReaderV1(final BlockletInfo blockletInfo, } @Override - public ColumnPage convertToColumnPage(MeasureRawColumnChunk measureRawColumnChunk, + public ColumnPage decodeColumnPage(MeasureRawColumnChunk measureRawColumnChunk, int pageNumber) throws IOException, MemoryException { int blockIndex = measureRawColumnChunk.getColumnIndex(); DataChunk dataChunk = measureColumnChunks.get(blockIndex); diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java index d61f98a52e7..04d6e2e1ca4 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java @@ -20,7 +20,7 @@ import java.nio.ByteBuffer; import java.util.List; -import org.apache.carbondata.core.datastore.FileHolder; +import org.apache.carbondata.core.datastore.FileReader; import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk; import org.apache.carbondata.core.datastore.chunk.reader.measure.AbstractMeasureChunkReaderV2V3Format; import org.apache.carbondata.core.datastore.page.ColumnPage; @@ -48,7 +48,7 @@ public CompressedMeasureChunkFileBasedReaderV2(final BlockletInfo blockletInfo, } @Override - public MeasureRawColumnChunk readRawMeasureChunk(FileHolder fileReader, int columnIndex) + public MeasureRawColumnChunk readRawMeasureChunk(FileReader fileReader, int columnIndex) throws IOException { int dataLength = 0; if (measureColumnChunkOffsets.size() - 1 == columnIndex) { @@ -83,30 +83,30 @@ public MeasureRawColumnChunk readRawMeasureChunk(FileHolder fileReader, int colu * data from * * @param fileReader file reader to read the data - * @param startColumnBlockletIndex first column blocklet index to be read - * @param endColumnBlockletIndex end column blocklet index to be read + * @param startColumnIndex first column blocklet index to be read + * @param endColumnIndex end column blocklet index to be read * @return measure raw chunkArray * @throws IOException */ - protected MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileHolder fileReader, - int startColumnBlockletIndex, int endColumnBlockletIndex) throws IOException { - long currentMeasureOffset = measureColumnChunkOffsets.get(startColumnBlockletIndex); + protected MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileReader fileReader, + int startColumnIndex, int endColumnIndex) throws IOException { + long currentMeasureOffset = measureColumnChunkOffsets.get(startColumnIndex); ByteBuffer buffer = null; synchronized (fileReader) { buffer = fileReader.readByteBuffer(filePath, currentMeasureOffset, - (int) (measureColumnChunkOffsets.get(endColumnBlockletIndex + 1) - currentMeasureOffset)); + (int) (measureColumnChunkOffsets.get(endColumnIndex + 1) - currentMeasureOffset)); } MeasureRawColumnChunk[] dataChunks = - new MeasureRawColumnChunk[endColumnBlockletIndex - startColumnBlockletIndex + 1]; + new MeasureRawColumnChunk[endColumnIndex - startColumnIndex + 1]; int runningLength = 0; int index = 0; - for (int i = startColumnBlockletIndex; i <= endColumnBlockletIndex; i++) { + for (int i = startColumnIndex; i <= endColumnIndex; i++) { int currentLength = (int) (measureColumnChunkOffsets.get(i + 1) - measureColumnChunkOffsets.get(i)); MeasureRawColumnChunk measureRawColumnChunk = new MeasureRawColumnChunk(i, buffer, runningLength, currentLength, this); - measureRawColumnChunk.setFileReader(fileReader); measureRawColumnChunk.setRowCount(new int[] { numberOfRows }); + measureRawColumnChunk.setFileReader(fileReader); measureRawColumnChunk.setPagesCount(1); dataChunks[index] = measureRawColumnChunk; runningLength += currentLength; @@ -115,7 +115,7 @@ protected MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileHolder fileRea return dataChunks; } - public ColumnPage convertToColumnPage(MeasureRawColumnChunk measureRawColumnChunk, + public ColumnPage decodeColumnPage(MeasureRawColumnChunk measureRawColumnChunk, int pageNumber) throws IOException, MemoryException { int copyPoint = (int) measureRawColumnChunk.getOffSet(); int blockIndex = measureRawColumnChunk.getColumnIndex(); diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMeasureChunkFileBasedReaderV3.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMeasureChunkFileBasedReaderV3.java index 4f6987b8d8a..6dc02a390e5 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMeasureChunkFileBasedReaderV3.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMeasureChunkFileBasedReaderV3.java @@ -20,7 +20,7 @@ import java.nio.ByteBuffer; import java.util.List; -import org.apache.carbondata.core.datastore.FileHolder; +import org.apache.carbondata.core.datastore.FileReader; import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk; import org.apache.carbondata.core.datastore.chunk.reader.measure.AbstractMeasureChunkReaderV2V3Format; import org.apache.carbondata.core.datastore.page.ColumnPage; @@ -70,7 +70,7 @@ public CompressedMeasureChunkFileBasedReaderV3(BlockletInfo blockletInfo, String * @param columnIndex column to be read * @return measure raw chunk */ - @Override public MeasureRawColumnChunk readRawMeasureChunk(FileHolder fileReader, + @Override public MeasureRawColumnChunk readRawMeasureChunk(FileReader fileReader, int columnIndex) throws IOException { int dataLength = 0; // to calculate the length of the data to be read @@ -99,9 +99,8 @@ public CompressedMeasureChunkFileBasedReaderV3(BlockletInfo blockletInfo, String dataChunk); } - protected MeasureRawColumnChunk getMeasureRawColumnChunk(FileHolder fileReader, - int columnIndex, long offset, int dataLength, ByteBuffer buffer, - DataChunk3 dataChunk) { + MeasureRawColumnChunk getMeasureRawColumnChunk(FileReader fileReader, int columnIndex, + long offset, int dataLength, ByteBuffer buffer, DataChunk3 dataChunk) { // creating a raw chunks instance and filling all the details MeasureRawColumnChunk rawColumnChunk = new MeasureRawColumnChunk(columnIndex, buffer, offset, dataLength, this); @@ -140,30 +139,30 @@ protected MeasureRawColumnChunk getMeasureRawColumnChunk(FileHolder fileReader, * * @param fileReader * reader which will be used to read the measure columns data from file - * @param startColumnBlockletIndex - * blocklet index of the first measure column - * @param endColumnBlockletIndex - * blocklet index of the last measure column + * @param startColumnIndex + * column index of the first measure column + * @param endColumnIndex + * column index of the last measure column * @return MeasureRawColumnChunk array */ - protected MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileHolder fileReader, - int startColumnBlockletIndex, int endColumnBlockletIndex) throws IOException { + protected MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileReader fileReader, + int startColumnIndex, int endColumnIndex) throws IOException { // to calculate the length of the data to be read // column we can subtract the offset of start column offset with // end column+1 offset and get the total length. - long currentMeasureOffset = measureColumnChunkOffsets.get(startColumnBlockletIndex); + long currentMeasureOffset = measureColumnChunkOffsets.get(startColumnIndex); ByteBuffer buffer = null; // read the data from carbon data file synchronized (fileReader) { buffer = fileReader.readByteBuffer(filePath, currentMeasureOffset, - (int) (measureColumnChunkOffsets.get(endColumnBlockletIndex + 1) - currentMeasureOffset)); + (int) (measureColumnChunkOffsets.get(endColumnIndex + 1) - currentMeasureOffset)); } // create raw chunk for each measure column MeasureRawColumnChunk[] measureDataChunk = - new MeasureRawColumnChunk[endColumnBlockletIndex - startColumnBlockletIndex + 1]; + new MeasureRawColumnChunk[endColumnIndex - startColumnIndex + 1]; int runningLength = 0; int index = 0; - for (int i = startColumnBlockletIndex; i <= endColumnBlockletIndex; i++) { + for (int i = startColumnIndex; i <= endColumnIndex; i++) { int currentLength = (int) (measureColumnChunkOffsets.get(i + 1) - measureColumnChunkOffsets.get(i)); DataChunk3 dataChunk = @@ -180,25 +179,25 @@ protected MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileHolder fileRea /** * Below method will be used to convert the compressed measure chunk raw data to actual data * - * @param rawColumnPage measure raw chunk + * @param rawColumnChunk measure raw chunk * @param pageNumber number - * @return DimensionColumnDataChunk + * @return DimensionColumnPage */ @Override - public ColumnPage convertToColumnPage( - MeasureRawColumnChunk rawColumnPage, int pageNumber) + public ColumnPage decodeColumnPage( + MeasureRawColumnChunk rawColumnChunk, int pageNumber) throws IOException, MemoryException { // data chunk of blocklet column - DataChunk3 dataChunk3 = rawColumnPage.getDataChunkV3(); + DataChunk3 dataChunk3 = rawColumnChunk.getDataChunkV3(); // data chunk of page DataChunk2 pageMetadata = dataChunk3.getData_chunk_list().get(pageNumber); // calculating the start point of data // as buffer can contain multiple column data, start point will be datachunkoffset + // data chunk length + page offset - int offset = (int) rawColumnPage.getOffSet() + - measureColumnChunkLength.get(rawColumnPage.getColumnIndex()) + + int offset = (int) rawColumnChunk.getOffSet() + + measureColumnChunkLength.get(rawColumnChunk.getColumnIndex()) + dataChunk3.getPage_offset().get(pageNumber); - ColumnPage decodedPage = decodeMeasure(pageMetadata, rawColumnPage.getRawData(), offset); + ColumnPage decodedPage = decodeMeasure(pageMetadata, rawColumnChunk.getRawData(), offset); decodedPage.setNullBits(getNullBitSet(pageMetadata.presence)); return decodedPage; } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMsrChunkFileBasedPageLevelReaderV3.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMsrChunkFileBasedPageLevelReaderV3.java index 31ff4c0d70d..6b3757513fd 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMsrChunkFileBasedPageLevelReaderV3.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMsrChunkFileBasedPageLevelReaderV3.java @@ -20,7 +20,7 @@ import java.io.IOException; import java.nio.ByteBuffer; -import org.apache.carbondata.core.datastore.FileHolder; +import org.apache.carbondata.core.datastore.FileReader; import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk; import org.apache.carbondata.core.datastore.page.ColumnPage; import org.apache.carbondata.core.memory.MemoryException; @@ -66,7 +66,7 @@ public CompressedMsrChunkFileBasedPageLevelReaderV3(BlockletInfo blockletInfo, S * @param blockletColumnIndex blocklet index of the column in carbon data file * @return measure raw chunk */ - @Override public MeasureRawColumnChunk readRawMeasureChunk(FileHolder fileReader, + @Override public MeasureRawColumnChunk readRawMeasureChunk(FileReader fileReader, int blockletColumnIndex) throws IOException { int dataLength = 0; // to calculate the length of the data to be read @@ -110,7 +110,7 @@ public CompressedMsrChunkFileBasedPageLevelReaderV3(BlockletInfo blockletInfo, S * @param endColumnBlockletIndex blocklet index of the last measure column * @return MeasureRawColumnChunk array */ - protected MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileHolder fileReader, + protected MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileReader fileReader, int startColumnBlockletIndex, int endColumnBlockletIndex) throws IOException { // create raw chunk for each measure column MeasureRawColumnChunk[] measureDataChunk = @@ -130,7 +130,7 @@ protected MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileHolder fileRea * @param pageNumber number * @return DimensionColumnDataChunk */ - @Override public ColumnPage convertToColumnPage( + @Override public ColumnPage decodeColumnPage( MeasureRawColumnChunk rawColumnPage, int pageNumber) throws IOException, MemoryException { // data chunk of blocklet column 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 fbdb4996a96..c89ecc3b8e8 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 @@ -17,48 +17,45 @@ package org.apache.carbondata.core.datastore.chunk.store; -import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk; +import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage; import org.apache.carbondata.core.datastore.page.ColumnPage; import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo; import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo; -public class ColumnPageWrapper implements DimensionColumnDataChunk { +public class ColumnPageWrapper implements DimensionColumnPage { private ColumnPage columnPage; - private int columnValueSize; - public ColumnPageWrapper(ColumnPage columnPage, int columnValueSize) { + public ColumnPageWrapper(ColumnPage columnPage) { this.columnPage = columnPage; - this.columnValueSize = columnValueSize; } @Override - public int fillChunkData(byte[] data, int offset, int columnIndex, - KeyStructureInfo restructuringInfo) { + public int fillRawData(int rowId, int offset, byte[] data, KeyStructureInfo restructuringInfo) { throw new UnsupportedOperationException("internal error"); } @Override - public int fillConvertedChunkData(int rowId, int columnIndex, int[] row, + public int fillSurrogateKey(int rowId, int chunkIndex, int[] outputSurrogateKey, KeyStructureInfo restructuringInfo) { throw new UnsupportedOperationException("internal error"); } @Override - public int fillConvertedChunkData(ColumnVectorInfo[] vectorInfo, int column, + public int fillVector(ColumnVectorInfo[] vectorInfo, int chunkIndex, KeyStructureInfo restructuringInfo) { throw new UnsupportedOperationException("internal error"); } @Override - public int fillConvertedChunkData(int[] rowMapping, ColumnVectorInfo[] vectorInfo, int column, + public int fillVector(int[] filteredRowId, ColumnVectorInfo[] vectorInfo, int chunkIndex, KeyStructureInfo restructuringInfo) { throw new UnsupportedOperationException("internal error"); } @Override - public byte[] getChunkData(int columnIndex) { - return columnPage.getBytes(columnIndex); + public byte[] getChunkData(int rowId) { + return columnPage.getBytes(rowId); } @Override @@ -66,7 +63,7 @@ public int getInvertedIndex(int rowId) { throw new UnsupportedOperationException("internal error"); } - @Override public int getInvertedReverseIndex(int invertedIndex) { + @Override public int getInvertedReverseIndex(int rowId) { throw new UnsupportedOperationException("internal error"); } @@ -75,18 +72,13 @@ public boolean isNoDicitionaryColumn() { return true; } - @Override - public int getColumnValueSize() { - return columnValueSize; - } - @Override public boolean isExplicitSorted() { return false; } @Override - public int compareTo(int index, byte[] compareValue) { + public int compareTo(int rowId, byte[] compareValue) { throw new UnsupportedOperationException("internal error"); } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/DimensionDataChunkStore.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/DimensionDataChunkStore.java index 5072c751110..28aed5b4d3c 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/DimensionDataChunkStore.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/DimensionDataChunkStore.java @@ -69,10 +69,10 @@ public interface DimensionDataChunkStore { /** * Below method will be used to get the reverse Inverted Index - * @param invertedIndex + * @param rowId * @return reverse Inverted Index */ - int getInvertedReverseIndex(int invertedIndex); + int getInvertedReverseIndex(int rowId); /** * Below method will be used to get the surrogate key of the @@ -102,9 +102,9 @@ public interface DimensionDataChunkStore { /** * to compare the two byte array * - * @param index index of first byte array + * @param rowId index of first byte array * @param compareValue value of to be compared * @return compare result */ - int compareTo(int index, byte[] compareValue); + int compareTo(int rowId, byte[] compareValue); } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeFixedLengthDimensionDataChunkStore.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeFixedLengthDimensionDataChunkStore.java index 8656878b2a5..41218d0d840 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeFixedLengthDimensionDataChunkStore.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeFixedLengthDimensionDataChunkStore.java @@ -97,13 +97,13 @@ public SafeFixedLengthDimensionDataChunkStore(boolean isInvertedIndex, int colum /** * to compare the two byte array * - * @param index index of first byte array + * @param rowId index of first byte array * @param compareValue value of to be compared * @return compare result */ - @Override public int compareTo(int index, byte[] compareValue) { + @Override public int compareTo(int rowId, byte[] compareValue) { return ByteUtil.UnsafeComparer.INSTANCE - .compareTo(data, index * columnValueSize, columnValueSize, compareValue, 0, + .compareTo(data, rowId * columnValueSize, columnValueSize, compareValue, 0, columnValueSize); } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeVariableLengthDimensionDataChunkStore.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeVariableLengthDimensionDataChunkStore.java index db831980e1c..f498c6ec0e1 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeVariableLengthDimensionDataChunkStore.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeVariableLengthDimensionDataChunkStore.java @@ -162,7 +162,7 @@ public SafeVariableLengthDimensionDataChunkStore(boolean isInvertedIndex, int nu } } - @Override public int compareTo(int index, byte[] compareValue) { + @Override public int compareTo(int rowId, byte[] compareValue) { // now to get the row from memory block we need to do following thing // 1. first get the current offset // 2. if it's not a last row- get the next row offset @@ -171,11 +171,11 @@ public SafeVariableLengthDimensionDataChunkStore(boolean isInvertedIndex, int nu // length // get the offset of set of data - int currentDataOffset = dataOffsets[index]; + int currentDataOffset = dataOffsets[rowId]; short length = 0; // calculating the length of data - if (index < numberOfRows - 1) { - length = (short) (dataOffsets[index + 1] - (currentDataOffset + if (rowId < numberOfRows - 1) { + length = (short) (dataOffsets[rowId + 1] - (currentDataOffset + CarbonCommonConstants.SHORT_SIZE_IN_BYTE)); } else { // for last record diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeFixedLengthDimensionDataChunkStore.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeFixedLengthDimensionDataChunkStore.java index 8c8d08f97fa..a689d8e3b66 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeFixedLengthDimensionDataChunkStore.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeFixedLengthDimensionDataChunkStore.java @@ -124,22 +124,22 @@ public UnsafeFixedLengthDimensionDataChunkStore(long totalDataSize, int columnVa /** * to compare the two byte array * - * @param index index of first byte array + * @param rowId index of first byte array * @param compareValue value of to be compared * @return compare result */ - @Override public int compareTo(int index, byte[] compareValue) { + @Override public int compareTo(int rowId, byte[] compareValue) { // based on index we need to calculate the actual position in memory block - index = index * columnValueSize; + rowId = rowId * columnValueSize; int compareResult = 0; for (int i = 0; i < compareValue.length; i++) { compareResult = (CarbonUnsafe.getUnsafe() - .getByte(dataPageMemoryBlock.getBaseObject(), dataPageMemoryBlock.getBaseOffset() + index) + .getByte(dataPageMemoryBlock.getBaseObject(), dataPageMemoryBlock.getBaseOffset() + rowId) & 0xff) - (compareValue[i] & 0xff); if (compareResult != 0) { break; } - index++; + rowId++; } return compareResult; } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimesionDataChunkStore.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimesionDataChunkStore.java index 36b2bd80329..e1eb378763a 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimesionDataChunkStore.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimesionDataChunkStore.java @@ -189,11 +189,11 @@ public UnsafeVariableLengthDimesionDataChunkStore(long totalSize, boolean isInve /** * to compare the two byte array * - * @param index index of first byte array + * @param rowId index of first byte array * @param compareValue value of to be compared * @return compare result */ - @Override public int compareTo(int index, byte[] compareValue) { + @Override public int compareTo(int rowId, byte[] compareValue) { // now to get the row from memory block we need to do following thing // 1. first get the current offset // 2. if it's not a last row- get the next row offset @@ -201,13 +201,13 @@ public UnsafeVariableLengthDimesionDataChunkStore(long totalSize, boolean isInve // else subtract the current row offset // with complete data length get the offset of set of data int currentDataOffset = CarbonUnsafe.getUnsafe().getInt(dataPageMemoryBlock.getBaseObject(), - dataPageMemoryBlock.getBaseOffset() + this.dataPointersOffsets + ((long)index + dataPageMemoryBlock.getBaseOffset() + this.dataPointersOffsets + ((long) rowId * CarbonCommonConstants.INT_SIZE_IN_BYTE * 1L)); short length = 0; // calculating the length of data - if (index < numberOfRows - 1) { + if (rowId < numberOfRows - 1) { int OffsetOfNextdata = CarbonUnsafe.getUnsafe().getInt(dataPageMemoryBlock.getBaseObject(), - dataPageMemoryBlock.getBaseOffset() + this.dataPointersOffsets + ((index + 1) + dataPageMemoryBlock.getBaseOffset() + this.dataPointersOffsets + ((rowId + 1) * CarbonCommonConstants.INT_SIZE_IN_BYTE)); length = (short) (OffsetOfNextdata - (currentDataOffset + CarbonCommonConstants.SHORT_SIZE_IN_BYTE)); diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/columnar/ColumnGroupModel.java b/core/src/main/java/org/apache/carbondata/core/datastore/columnar/ColumnGroupModel.java index 74d268afe61..e2a41613f7a 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/columnar/ColumnGroupModel.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/columnar/ColumnGroupModel.java @@ -28,13 +28,6 @@ public class ColumnGroupModel { */ private int noOfColumnsStore; - /** - * whether given index is columnar or not - * true: columnar - * false: row block - */ - private boolean[] columnarStore; - /** * column groups * e.g @@ -76,15 +69,6 @@ public void setNoOfColumnStore(int noOfColumnsStore) { this.noOfColumnsStore = noOfColumnsStore; } - /** - * it's an identifier for row block or single column block - * - * @param columnarStore - */ - public void setColumnarStore(boolean[] columnarStore) { - this.columnarStore = columnarStore; - } - /** * set column groups * @@ -94,16 +78,6 @@ public void setColumnGroup(int[][] columnGroups) { this.columnGroups = columnGroups; } - /** - * check if given column group is columnar - * - * @param colGroup - * @return true if given block is columnar - */ - public boolean isColumnar(int colGroup) { - return columnarStore[colGroup]; - } - /** * @return columngroups */ diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/DFSFileHolderImpl.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/DFSFileReaderImpl.java similarity index 92% rename from core/src/main/java/org/apache/carbondata/core/datastore/impl/DFSFileHolderImpl.java rename to core/src/main/java/org/apache/carbondata/core/datastore/impl/DFSFileReaderImpl.java index 182c8eb7f62..1a0cd4179cc 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/DFSFileHolderImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/DFSFileReaderImpl.java @@ -23,24 +23,21 @@ import java.util.Map.Entry; import org.apache.carbondata.core.constants.CarbonCommonConstants; -import org.apache.carbondata.core.datastore.FileHolder; +import org.apache.carbondata.core.datastore.FileReader; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -public class DFSFileHolderImpl implements FileHolder { +public class DFSFileReaderImpl implements FileReader { /** * cache to hold filename and its stream */ private Map fileNameAndStreamCache; - private String queryId; - private boolean readPageByPage; - - public DFSFileHolderImpl() { + public DFSFileReaderImpl() { this.fileNameAndStreamCache = new HashMap(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); } @@ -59,7 +56,7 @@ public DFSFileHolderImpl() { * @param filePath fully qualified file path * @return channel */ - public FSDataInputStream updateCache(String filePath) throws IOException { + private FSDataInputStream updateCache(String filePath) throws IOException { FSDataInputStream fileChannel = fileNameAndStreamCache.get(filePath); if (null == fileChannel) { Path pt = new Path(filePath); @@ -143,14 +140,6 @@ private byte[] read(FSDataInputStream channel, int size) throws IOException { return byteBuffer; } - @Override public void setQueryId(String queryId) { - this.queryId = queryId; - } - - @Override public String getQueryId() { - return queryId; - } - @Override public void setReadPageByPage(boolean isReadPageByPage) { this.readPageByPage = isReadPageByPage; } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/DefaultFileTypeProvider.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/DefaultFileTypeProvider.java index 67648fec03c..b58a473235d 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/DefaultFileTypeProvider.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/DefaultFileTypeProvider.java @@ -17,24 +17,28 @@ package org.apache.carbondata.core.datastore.impl; -import org.apache.carbondata.core.datastore.FileHolder; -import org.apache.carbondata.core.datastore.filesystem.*; +import org.apache.carbondata.core.datastore.FileReader; +import org.apache.carbondata.core.datastore.filesystem.AlluxioCarbonFile; +import org.apache.carbondata.core.datastore.filesystem.CarbonFile; +import org.apache.carbondata.core.datastore.filesystem.HDFSCarbonFile; +import org.apache.carbondata.core.datastore.filesystem.LocalCarbonFile; +import org.apache.carbondata.core.datastore.filesystem.ViewFSCarbonFile; import org.apache.hadoop.conf.Configuration; public class DefaultFileTypeProvider implements FileTypeInerface { - public FileHolder getFileHolder(FileFactory.FileType fileType) { + public FileReader getFileHolder(FileFactory.FileType fileType) { switch (fileType) { case LOCAL: - return new FileHolderImpl(); + return new FileReaderImpl(); case HDFS: case ALLUXIO: case VIEWFS: case S3: - return new DFSFileHolderImpl(); + return new DFSFileReaderImpl(); default: - return new FileHolderImpl(); + return new FileReaderImpl(); } } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java index f1419911178..9bcdfaedac7 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java @@ -28,7 +28,7 @@ import org.apache.carbondata.common.logging.LogService; import org.apache.carbondata.common.logging.LogServiceFactory; import org.apache.carbondata.core.constants.CarbonCommonConstants; -import org.apache.carbondata.core.datastore.FileHolder; +import org.apache.carbondata.core.datastore.FileReader; import org.apache.carbondata.core.datastore.filesystem.CarbonFile; import org.apache.commons.io.FileUtils; @@ -62,7 +62,7 @@ public static Configuration getConfiguration() { return configuration; } - public static FileHolder getFileHolder(FileType fileType) { + public static FileReader getFileHolder(FileType fileType) { return fileFileTypeInerface.getFileHolder(fileType); } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileHolderImpl.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileReaderImpl.java similarity index 94% rename from core/src/main/java/org/apache/carbondata/core/datastore/impl/FileHolderImpl.java rename to core/src/main/java/org/apache/carbondata/core/datastore/impl/FileReaderImpl.java index cc589b7dc6e..6fef27809b8 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileHolderImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileReaderImpl.java @@ -27,27 +27,26 @@ import java.util.Map.Entry; import org.apache.carbondata.core.constants.CarbonCommonConstants; -import org.apache.carbondata.core.datastore.FileHolder; +import org.apache.carbondata.core.datastore.FileReader; -public class FileHolderImpl implements FileHolder { +public class FileReaderImpl implements FileReader { /** * cache to hold filename and its stream */ private Map fileNameAndStreamCache; - private String queryId; private boolean readPageByPage; /** - * FileHolderImpl Constructor + * FileReaderImpl Constructor * It will create the cache */ - public FileHolderImpl() { + public FileReaderImpl() { this.fileNameAndStreamCache = new HashMap(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); } - public FileHolderImpl(int capacity) { + public FileReaderImpl(int capacity) { this.fileNameAndStreamCache = new HashMap(capacity); } @@ -206,14 +205,6 @@ private ByteBuffer read(FileChannel channel, int size) throws IOException { return byteBuffer; } - @Override public void setQueryId(String queryId) { - this.queryId = queryId; - } - - @Override public String getQueryId() { - return queryId; - } - @Override public void setReadPageByPage(boolean isReadPageByPage) { this.readPageByPage = isReadPageByPage; } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileTypeInerface.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileTypeInerface.java index 4676278911f..413261c7d53 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileTypeInerface.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileTypeInerface.java @@ -17,14 +17,14 @@ package org.apache.carbondata.core.datastore.impl; -import org.apache.carbondata.core.datastore.FileHolder; +import org.apache.carbondata.core.datastore.FileReader; import org.apache.carbondata.core.datastore.filesystem.CarbonFile; import org.apache.hadoop.conf.Configuration; public interface FileTypeInerface { - FileHolder getFileHolder(FileFactory.FileType fileType); + FileReader getFileHolder(FileFactory.FileType fileType); CarbonFile getCarbonFile(String path, FileFactory.FileType fileType); CarbonFile getCarbonFile(String path, FileFactory.FileType fileType, Configuration configuration); diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/AbstractBTreeLeafNode.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/AbstractBTreeLeafNode.java index 19b1f1c0116..fe4cf83c4bd 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/AbstractBTreeLeafNode.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/AbstractBTreeLeafNode.java @@ -18,9 +18,8 @@ import java.io.IOException; -import org.apache.carbondata.core.cache.update.BlockletLevelDeleteDeltaDataCache; import org.apache.carbondata.core.datastore.DataRefNode; -import org.apache.carbondata.core.datastore.FileHolder; +import org.apache.carbondata.core.datastore.FileReader; import org.apache.carbondata.core.datastore.IndexKey; import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk; @@ -30,39 +29,32 @@ */ public abstract class AbstractBTreeLeafNode implements BTreeNode { - /** - * Below method will be used to load the data block - * - * @param blockInfo block detail - */ - protected BlockletLevelDeleteDeltaDataCache deleteDeltaDataCache; - /** * number of keys in a btree */ - protected int numberOfKeys; + int numberOfKeys; /** * node number */ - protected long nodeNumber; + long nodeNumber; /** * Next node of the leaf */ - protected BTreeNode nextNode; + private BTreeNode nextNode; /** * max key of the column this will be used to check whether this leaf will * be used for scanning or not */ - protected byte[][] maxKeyOfColumns; + byte[][] maxKeyOfColumns; /** * min key of the column this will be used to check whether this leaf will * be used for scanning or not */ - protected byte[][] minKeyOfColumns; + byte[][] minKeyOfColumns; /** * Method to get the next block this can be used while scanning when @@ -70,7 +62,7 @@ public abstract class AbstractBTreeLeafNode implements BTreeNode { * * @return next block */ - @Override public int nodeSize() { + @Override public int numRows() { return this.numberOfKeys; } @@ -109,7 +101,7 @@ public abstract class AbstractBTreeLeafNode implements BTreeNode { * * @return block number */ - @Override public long nodeNumber() { + @Override public long nodeIndex() { return nodeNumber; } @@ -174,11 +166,11 @@ public abstract class AbstractBTreeLeafNode implements BTreeNode { * Below method will be used to get the dimension chunks * * @param fileReader file reader to read the chunks from file - * @param blockIndexes indexes of the blocks need to be read + * @param columnIndexRange indexes of the blocks need to be read * @return dimension data chunks */ - @Override public DimensionRawColumnChunk[] getDimensionChunks(FileHolder fileReader, - int[][] blockIndexes) throws IOException { + @Override public DimensionRawColumnChunk[] readDimensionChunks(FileReader fileReader, + int[][] columnIndexRange) throws IOException { // No required here as leaf which will will be use this class will implement its own get // dimension chunks return null; @@ -188,11 +180,11 @@ public abstract class AbstractBTreeLeafNode implements BTreeNode { * Below method will be used to get the dimension chunk * * @param fileReader file reader to read the chunk from file - * @param blockIndex block index to be read + * @param columnIndex block index to be read * @return dimension data chunk */ - @Override public DimensionRawColumnChunk getDimensionChunk(FileHolder fileReader, - int blockIndex) throws IOException { + @Override public DimensionRawColumnChunk readDimensionChunk(FileReader fileReader, + int columnIndex) throws IOException { // No required here as leaf which will will be use this class will implement // its own get dimension chunks return null; @@ -202,11 +194,11 @@ public abstract class AbstractBTreeLeafNode implements BTreeNode { * Below method will be used to get the measure chunk * * @param fileReader file reader to read the chunk from file - * @param blockIndexes block indexes to be read from file + * @param columnIndexRange block indexes to be read from file * @return measure column data chunk */ - @Override public MeasureRawColumnChunk[] getMeasureChunks(FileHolder fileReader, - int[][] blockIndexes) throws IOException { + @Override public MeasureRawColumnChunk[] readMeasureChunks(FileReader fileReader, + int[][] columnIndexRange) throws IOException { // No required here as leaf which will will be use this class will implement its own get // measure chunks return null; @@ -216,30 +208,16 @@ public abstract class AbstractBTreeLeafNode implements BTreeNode { * Below method will be used to read the measure chunk * * @param fileReader file read to read the file chunk - * @param blockIndex block index to be read from file + * @param columnIndex block index to be read from file * @return measure data chunk */ - @Override public MeasureRawColumnChunk getMeasureChunk(FileHolder fileReader, int blockIndex) + @Override public MeasureRawColumnChunk readMeasureChunk(FileReader fileReader, int columnIndex) throws IOException { // No required here as leaf which will will be use this class will implement its own get // measure chunks return null; } - /** - * @param deleteDeltaDataCache - */ - public void setDeleteDeltaDataCache(BlockletLevelDeleteDeltaDataCache deleteDeltaDataCache) { - - this.deleteDeltaDataCache = deleteDeltaDataCache; - } - /** - * @return the segmentProperties - */ - public BlockletLevelDeleteDeltaDataCache getDeleteDeltaDataCache() { - return deleteDeltaDataCache; - } - @Override public int getPageRowCount(int pageNumber) { throw new UnsupportedOperationException("Unsupported operation"); diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeDataRefNodeFinder.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeDataRefNodeFinder.java index 2f8aadf0492..688d56aece0 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeDataRefNodeFinder.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeDataRefNodeFinder.java @@ -106,7 +106,7 @@ public BTreeDataRefNodeFinder(int[] eachColumnValueSize, int numberOfSortColumns private BTreeNode findFirstLeafNode(IndexKey key, BTreeNode node) { int childNodeIndex; int low = 0; - int high = node.nodeSize() - 1; + int high = node.numRows() - 1; int mid = 0; int compareRes = -1; IndexKey[] nodeKeys = node.getNodeKeys(); @@ -156,7 +156,7 @@ private BTreeNode findFirstLeafNode(IndexKey key, BTreeNode node) { private BTreeNode findLastLeafNode(IndexKey key, BTreeNode node) { int childNodeIndex; int low = 0; - int high = node.nodeSize() - 1; + int high = node.numRows() - 1; int mid = 0; int compareRes = -1; IndexKey[] nodeKeys = node.getNodeKeys(); @@ -172,7 +172,7 @@ private BTreeNode findLastLeafNode(IndexKey key, BTreeNode node) { } else { int currentPos = mid; // if key is matched then get the first entry - while (currentPos + 1 < node.nodeSize() + while (currentPos + 1 < node.numRows() && compareIndexes(key, nodeKeys[currentPos + 1]) == 0) { currentPos++; } diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeNonLeafNode.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeNonLeafNode.java index ccc5e123304..c200f8db0bb 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeNonLeafNode.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeNonLeafNode.java @@ -19,10 +19,9 @@ import java.util.ArrayList; import java.util.List; -import org.apache.carbondata.core.cache.update.BlockletLevelDeleteDeltaDataCache; import org.apache.carbondata.core.constants.CarbonCommonConstants; import org.apache.carbondata.core.datastore.DataRefNode; -import org.apache.carbondata.core.datastore.FileHolder; +import org.apache.carbondata.core.datastore.FileReader; import org.apache.carbondata.core.datastore.IndexKey; import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk; @@ -33,13 +32,6 @@ */ public class BTreeNonLeafNode implements BTreeNode { - /** - * Below method will be used to load the data block - * - * @param blockInfo block detail - */ - protected BlockletLevelDeleteDeltaDataCache deleteDeltaDataCache; - /** * Child nodes */ @@ -50,7 +42,7 @@ public class BTreeNonLeafNode implements BTreeNode { */ private List listOfKeys; - public BTreeNonLeafNode() { + BTreeNonLeafNode() { // creating a list which will store all the indexes listOfKeys = new ArrayList(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); } @@ -120,7 +112,7 @@ public BTreeNonLeafNode() { * * @return number of keys in the block */ - @Override public int nodeSize() { + @Override public int numRows() { return listOfKeys.size(); } @@ -131,11 +123,11 @@ public BTreeNonLeafNode() { * * @return block number */ - @Override public long nodeNumber() { + @Override public long nodeIndex() { throw new UnsupportedOperationException("Unsupported operation"); } - @Override public String blockletId() { + @Override public short blockletIndex() { throw new UnsupportedOperationException("Unsupported operation"); } @@ -171,11 +163,11 @@ public BTreeNonLeafNode() { * Below method will be used to get the dimension chunks * * @param fileReader file reader to read the chunks from file - * @param blockIndexes indexes of the blocks need to be read + * @param columnIndexRange indexes of the blocks need to be read * @return dimension data chunks */ - @Override public DimensionRawColumnChunk[] getDimensionChunks(FileHolder fileReader, - int[][] blockIndexes) { + @Override public DimensionRawColumnChunk[] readDimensionChunks(FileReader fileReader, + int[][] columnIndexRange) { // operation of getting the dimension chunks is not supported as its a // non leaf node @@ -191,8 +183,8 @@ public BTreeNonLeafNode() { * @param fileReader file reader to read the chunk from file * @return dimension data chunk */ - @Override public DimensionRawColumnChunk getDimensionChunk(FileHolder fileReader, - int blockIndexes) { + @Override public DimensionRawColumnChunk readDimensionChunk(FileReader fileReader, + int columnIndex) { // operation of getting the dimension chunk is not supported as its a // non leaf node // and in case of B+Tree data will be stored only in leaf node and @@ -205,11 +197,11 @@ public BTreeNonLeafNode() { * Below method will be used to get the measure chunk * * @param fileReader file reader to read the chunk from file - * @param blockIndexes block indexes to be read from file + * @param columnIndexRange block indexes to be read from file * @return measure column data chunk */ - @Override public MeasureRawColumnChunk[] getMeasureChunks(FileHolder fileReader, - int[][] blockIndexes) { + @Override public MeasureRawColumnChunk[] readMeasureChunks(FileReader fileReader, + int[][] columnIndexRange) { // operation of getting the measure chunk is not supported as its a non // leaf node // and in case of B+Tree data will be stored only in leaf node and @@ -222,11 +214,11 @@ public BTreeNonLeafNode() { * Below method will be used to read the measure chunk * * @param fileReader file read to read the file chunk - * @param blockIndex block index to be read from file + * @param columnIndex block index to be read from file * @return measure data chunk */ - @Override public MeasureRawColumnChunk getMeasureChunk(FileHolder fileReader, int blockIndex) { + @Override public MeasureRawColumnChunk readMeasureChunk(FileReader fileReader, int columnIndex) { // operation of getting the measure chunk is not supported as its a non // leaf node // and in case of B+Tree data will be stored only in leaf node and @@ -235,20 +227,6 @@ public BTreeNonLeafNode() { throw new UnsupportedOperationException("Unsupported operation"); } - /** - * @return the segmentProperties - */ - public void setDeleteDeltaDataCache(BlockletLevelDeleteDeltaDataCache deleteDeltaDataCache) { - - this.deleteDeltaDataCache = deleteDeltaDataCache; - } - /** - * @return the segmentProperties - */ - public BlockletLevelDeleteDeltaDataCache getDeleteDeltaDataCache() { - return deleteDeltaDataCache; - } - /** * number of pages in blocklet * @return diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockBTreeLeafNode.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockBTreeLeafNode.java index 25817f5d758..8af7eae1766 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockBTreeLeafNode.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockBTreeLeafNode.java @@ -38,7 +38,7 @@ public class BlockBTreeLeafNode extends AbstractBTreeLeafNode { * node * @param metadataIndex metadata index */ - public BlockBTreeLeafNode(BTreeBuilderInfo builderInfos, int metadataIndex, long nodeNumber) { + BlockBTreeLeafNode(BTreeBuilderInfo builderInfos, int metadataIndex, long nodeNumber) { DataFileFooter footer = builderInfos.getFooterList().get(metadataIndex); BlockletMinMaxIndex minMaxIndex = footer.getBlockletIndex().getMinMaxIndex(); maxKeyOfColumns = minMaxIndex.getMaxValues(); @@ -63,8 +63,8 @@ public TableBlockInfo getTableBlockInfo() { * Below method is suppose to return the Blocklet ID. * @return */ - @Override public String blockletId() { - return blockInfo.getTableBlockInfo().getDetailInfo().getBlockletId().toString(); + @Override public short blockletIndex() { + return blockInfo.getTableBlockInfo().getDetailInfo().getBlockletId(); } /** diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockletBTreeLeafNode.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockletBTreeLeafNode.java index 94221bae1ce..ddd7fcffdbb 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockletBTreeLeafNode.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockletBTreeLeafNode.java @@ -20,7 +20,7 @@ import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants; import org.apache.carbondata.core.datastore.BTreeBuilderInfo; -import org.apache.carbondata.core.datastore.FileHolder; +import org.apache.carbondata.core.datastore.FileReader; import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk; import org.apache.carbondata.core.datastore.chunk.reader.CarbonDataReaderFactory; @@ -69,7 +69,7 @@ public class BlockletBTreeLeafNode extends AbstractBTreeLeafNode { * this will be used during query execution when we can * give some leaf node of a btree to one executor some to other */ - public BlockletBTreeLeafNode(BTreeBuilderInfo builderInfos, int leafIndex, long nodeNumber) { + BlockletBTreeLeafNode(BTreeBuilderInfo builderInfos, int leafIndex, long nodeNumber) { // get a lead node min max BlockletMinMaxIndex minMaxIndex = builderInfos.getFooterList().get(0).getBlockletList().get(leafIndex).getBlockletIndex() @@ -124,23 +124,23 @@ public BlockletBTreeLeafNode(BTreeBuilderInfo builderInfos, int leafIndex, long } } - @Override public String blockletId() { - return "0"; + @Override public short blockletIndex() { + return 0; } /** * Below method will be used to get the dimension chunks * * @param fileReader file reader to read the chunks from file - * @param blockIndexes indexes of the blocks need to be read + * @param columnIndexRange indexes of the blocks need to be read * @return dimension data chunks */ - @Override public DimensionRawColumnChunk[] getDimensionChunks(FileHolder fileReader, - int[][] blockIndexes) throws IOException { + @Override public DimensionRawColumnChunk[] readDimensionChunks(FileReader fileReader, + int[][] columnIndexRange) throws IOException { if (fileReader.isReadPageByPage()) { - return dimensionChunksPageLevelReader.readRawDimensionChunks(fileReader, blockIndexes); + return dimensionChunksPageLevelReader.readRawDimensionChunks(fileReader, columnIndexRange); } else { - return dimensionChunksReader.readRawDimensionChunks(fileReader, blockIndexes); + return dimensionChunksReader.readRawDimensionChunks(fileReader, columnIndexRange); } } @@ -148,15 +148,15 @@ public BlockletBTreeLeafNode(BTreeBuilderInfo builderInfos, int leafIndex, long * Below method will be used to get the dimension chunk * * @param fileReader file reader to read the chunk from file - * @param blockIndex block index to be read + * @param columnIndex block index to be read * @return dimension data chunk */ - @Override public DimensionRawColumnChunk getDimensionChunk(FileHolder fileReader, int blockIndex) - throws IOException { + @Override public DimensionRawColumnChunk readDimensionChunk( + FileReader fileReader, int columnIndex) throws IOException { if (fileReader.isReadPageByPage()) { - return dimensionChunksPageLevelReader.readRawDimensionChunk(fileReader, blockIndex); + return dimensionChunksPageLevelReader.readRawDimensionChunk(fileReader, columnIndex); } else { - return dimensionChunksReader.readRawDimensionChunk(fileReader, blockIndex); + return dimensionChunksReader.readRawDimensionChunk(fileReader, columnIndex); } } @@ -164,15 +164,15 @@ public BlockletBTreeLeafNode(BTreeBuilderInfo builderInfos, int leafIndex, long * Below method will be used to get the measure chunk * * @param fileReader file reader to read the chunk from file - * @param blockIndexes block indexes to be read from file + * @param columnIndexRange block indexes to be read from file * @return measure column data chunk */ - @Override public MeasureRawColumnChunk[] getMeasureChunks(FileHolder fileReader, - int[][] blockIndexes) throws IOException { + @Override public MeasureRawColumnChunk[] readMeasureChunks(FileReader fileReader, + int[][] columnIndexRange) throws IOException { if (fileReader.isReadPageByPage()) { - return measureColumnChunkPageLevelReader.readRawMeasureChunks(fileReader, blockIndexes); + return measureColumnChunkPageLevelReader.readRawMeasureChunks(fileReader, columnIndexRange); } else { - return measureColumnChunkReader.readRawMeasureChunks(fileReader, blockIndexes); + return measureColumnChunkReader.readRawMeasureChunks(fileReader, columnIndexRange); } } @@ -180,15 +180,15 @@ public BlockletBTreeLeafNode(BTreeBuilderInfo builderInfos, int leafIndex, long * Below method will be used to read the measure chunk * * @param fileReader file read to read the file chunk - * @param blockIndex block index to be read from file + * @param columnIndex block index to be read from file * @return measure data chunk */ - @Override public MeasureRawColumnChunk getMeasureChunk(FileHolder fileReader, int blockIndex) + @Override public MeasureRawColumnChunk readMeasureChunk(FileReader fileReader, int columnIndex) throws IOException { if (fileReader.isReadPageByPage()) { - return measureColumnChunkPageLevelReader.readRawMeasureChunk(fileReader, blockIndex); + return measureColumnChunkPageLevelReader.readRawMeasureChunk(fileReader, columnIndex); } else { - return measureColumnChunkReader.readRawMeasureChunk(fileReader, blockIndex); + return measureColumnChunkReader.readRawMeasureChunk(fileReader, columnIndex); } } 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 6d96b3b47ab..597def07464 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 @@ -41,7 +41,13 @@ import org.apache.carbondata.core.util.CarbonUtil; import org.apache.carbondata.format.Encoding; -import static org.apache.carbondata.format.Encoding.*; +import static org.apache.carbondata.format.Encoding.ADAPTIVE_DELTA_FLOATING; +import static org.apache.carbondata.format.Encoding.ADAPTIVE_DELTA_INTEGRAL; +import static org.apache.carbondata.format.Encoding.ADAPTIVE_FLOATING; +import static org.apache.carbondata.format.Encoding.ADAPTIVE_INTEGRAL; +import static org.apache.carbondata.format.Encoding.BOOL_BYTE; +import static org.apache.carbondata.format.Encoding.DIRECT_COMPRESS; +import static org.apache.carbondata.format.Encoding.RLE_INTEGRAL; /** * Base class for encoding factory implementation. diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServer.java b/core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServer.java index c7411d63102..daba470ae77 100644 --- a/core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServer.java +++ b/core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServer.java @@ -27,7 +27,6 @@ import org.apache.carbondata.core.metadata.schema.table.CarbonTable; import org.apache.carbondata.core.util.CarbonProperties; - import io.netty.bootstrap.ServerBootstrap; import io.netty.channel.ChannelInitializer; import io.netty.channel.ChannelOption; diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailInfo.java b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailInfo.java index ce05fe2ef1a..2865d4ba571 100644 --- a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailInfo.java +++ b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailInfo.java @@ -213,10 +213,6 @@ public List getColumnSchemas() { return columnSchemas; } - public void setColumnSchemas(List columnSchemas) { - this.columnSchemas = columnSchemas; - } - public void setColumnSchemaBinary(byte[] columnSchemaBinary) { this.columnSchemaBinary = columnSchemaBinary; } diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNodeWrapper.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java similarity index 86% rename from core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNodeWrapper.java rename to core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java index 4e49ede0177..5cd59cb90bb 100644 --- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNodeWrapper.java +++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java @@ -19,11 +19,10 @@ import java.io.IOException; import java.util.List; -import org.apache.carbondata.core.cache.update.BlockletLevelDeleteDeltaDataCache; import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants; import org.apache.carbondata.core.constants.CarbonVersionConstants; import org.apache.carbondata.core.datastore.DataRefNode; -import org.apache.carbondata.core.datastore.FileHolder; +import org.apache.carbondata.core.datastore.FileReader; import org.apache.carbondata.core.datastore.block.TableBlockInfo; import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk; @@ -37,7 +36,7 @@ /** * wrapper for blocklet data map data */ -public class BlockletDataRefNodeWrapper implements DataRefNode { +public class BlockletDataRefNode implements DataRefNode { private List blockInfos; @@ -45,10 +44,7 @@ public class BlockletDataRefNodeWrapper implements DataRefNode { private int[] dimensionLens; - private BlockletLevelDeleteDeltaDataCache deleteDeltaDataCache; - - public BlockletDataRefNodeWrapper(List blockInfos, int index, - int[] dimensionLens) { + BlockletDataRefNode(List blockInfos, int index, int[] dimensionLens) { this.blockInfos = blockInfos; // Update row count and page count to blocklet info for (TableBlockInfo blockInfo : blockInfos) { @@ -84,21 +80,21 @@ public BlockletDataRefNodeWrapper(List blockInfos, int index, @Override public DataRefNode getNextDataRefNode() { if (index + 1 < blockInfos.size()) { - return new BlockletDataRefNodeWrapper(blockInfos, index + 1, dimensionLens); + return new BlockletDataRefNode(blockInfos, index + 1, dimensionLens); } return null; } - @Override public int nodeSize() { + @Override public int numRows() { return blockInfos.get(index).getDetailInfo().getRowCount(); } - @Override public long nodeNumber() { + @Override public long nodeIndex() { return index; } - @Override public String blockletId() { - return blockInfos.get(index).getDetailInfo().getBlockletId().toString(); + @Override public short blockletIndex() { + return blockInfos.get(index).getDetailInfo().getBlockletId(); } @Override @@ -124,34 +120,34 @@ public byte[][] getColumnsMinValue() { } @Override - public DimensionRawColumnChunk[] getDimensionChunks(FileHolder fileReader, int[][] blockIndexes) + public DimensionRawColumnChunk[] readDimensionChunks(FileReader fileReader, int[][] blockIndexes) throws IOException { DimensionColumnChunkReader dimensionChunksReader = getDimensionColumnChunkReader(fileReader); return dimensionChunksReader.readRawDimensionChunks(fileReader, blockIndexes); } @Override - public DimensionRawColumnChunk getDimensionChunk(FileHolder fileReader, int blockIndexes) + public DimensionRawColumnChunk readDimensionChunk(FileReader fileReader, int columnIndex) throws IOException { DimensionColumnChunkReader dimensionChunksReader = getDimensionColumnChunkReader(fileReader); - return dimensionChunksReader.readRawDimensionChunk(fileReader, blockIndexes); + return dimensionChunksReader.readRawDimensionChunk(fileReader, columnIndex); } @Override - public MeasureRawColumnChunk[] getMeasureChunks(FileHolder fileReader, int[][] blockIndexes) + public MeasureRawColumnChunk[] readMeasureChunks(FileReader fileReader, int[][] columnIndexRange) throws IOException { MeasureColumnChunkReader measureColumnChunkReader = getMeasureColumnChunkReader(fileReader); MeasureRawColumnChunk[] measureRawColumnChunks = - measureColumnChunkReader.readRawMeasureChunks(fileReader, blockIndexes); + measureColumnChunkReader.readRawMeasureChunks(fileReader, columnIndexRange); updateMeasureRawColumnChunkMinMaxValues(measureRawColumnChunks); return measureRawColumnChunks; } - @Override public MeasureRawColumnChunk getMeasureChunk(FileHolder fileReader, int blockIndex) + @Override public MeasureRawColumnChunk readMeasureChunk(FileReader fileReader, int columnIndex) throws IOException { MeasureColumnChunkReader measureColumnChunkReader = getMeasureColumnChunkReader(fileReader); MeasureRawColumnChunk measureRawColumnChunk = - measureColumnChunkReader.readRawMeasureChunk(fileReader, blockIndex); + measureColumnChunkReader.readRawMeasureChunk(fileReader, columnIndex); updateMeasureRawColumnChunkMinMaxValues(measureRawColumnChunk); return measureRawColumnChunk; } @@ -185,7 +181,7 @@ private void updateMeasureRawColumnChunkMinMaxValues( } } - private DimensionColumnChunkReader getDimensionColumnChunkReader(FileHolder fileReader) { + private DimensionColumnChunkReader getDimensionColumnChunkReader(FileReader fileReader) { ColumnarFormatVersion version = ColumnarFormatVersion.valueOf(blockInfos.get(index).getDetailInfo().getVersionNumber()); if (fileReader.isReadPageByPage()) { @@ -199,7 +195,7 @@ private DimensionColumnChunkReader getDimensionColumnChunkReader(FileHolder file } } - private MeasureColumnChunkReader getMeasureColumnChunkReader(FileHolder fileReader) { + private MeasureColumnChunkReader getMeasureColumnChunkReader(FileReader fileReader) { ColumnarFormatVersion version = ColumnarFormatVersion.valueOf(blockInfos.get(index).getDetailInfo().getVersionNumber()); if (fileReader.isReadPageByPage()) { @@ -213,15 +209,6 @@ private MeasureColumnChunkReader getMeasureColumnChunkReader(FileHolder fileRead } } - @Override - public void setDeleteDeltaDataCache(BlockletLevelDeleteDeltaDataCache deleteDeltaDataCache) { - this.deleteDeltaDataCache = deleteDeltaDataCache; - } - - @Override public BlockletLevelDeleteDeltaDataCache getDeleteDeltaDataCache() { - return deleteDeltaDataCache; - } - @Override public int numberOfPages() { return blockInfos.get(index).getDetailInfo().getPagesCount(); } diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/IndexWrapper.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/IndexWrapper.java index 17ad17f1f52..a30f64c50f1 100644 --- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/IndexWrapper.java +++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/IndexWrapper.java @@ -32,7 +32,7 @@ public class IndexWrapper extends AbstractIndex { public IndexWrapper(List blockInfos) { segmentProperties = new SegmentProperties(blockInfos.get(0).getDetailInfo().getColumnSchemas(), blockInfos.get(0).getDetailInfo().getDimLens()); - dataRefNode = new BlockletDataRefNodeWrapper(blockInfos, 0, + dataRefNode = new BlockletDataRefNode(blockInfos, 0, segmentProperties.getDimensionColumnsValueSize()); } diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java index 9364a7a72af..6803fc86af3 100644 --- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java +++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java @@ -19,7 +19,12 @@ import java.io.DataInputStream; import java.io.IOException; import java.nio.ByteBuffer; -import java.util.*; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; import org.apache.carbondata.common.logging.LogService; import org.apache.carbondata.common.logging.LogServiceFactory; diff --git a/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java b/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java index 242995b9fe5..53cbb1df9b2 100644 --- a/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java +++ b/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java @@ -17,11 +17,11 @@ package org.apache.carbondata.core.memory; +import javax.annotation.concurrent.GuardedBy; import java.lang.ref.WeakReference; import java.util.HashMap; import java.util.LinkedList; import java.util.Map; -import javax.annotation.concurrent.GuardedBy; import org.apache.carbondata.core.util.CarbonProperties; diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/SegmentInfo.java b/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/SegmentInfo.java index 0cb29184276..099fffd1d31 100644 --- a/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/SegmentInfo.java +++ b/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/SegmentInfo.java @@ -28,31 +28,12 @@ public class SegmentInfo implements Serializable { */ private static final long serialVersionUID = -1749874611112709431L; - /** - * number of column in the segment - */ - private int numberOfColumns; - /** * cardinality of each columns * column which is not participating in the multidimensional key cardinality will be -1; */ private int[] columnCardinality; - /** - * @return the numberOfColumns - */ - public int getNumberOfColumns() { - return numberOfColumns; - } - - /** - * @param numberOfColumns the numberOfColumns to set - */ - public void setNumberOfColumns(int numberOfColumns) { - this.numberOfColumns = numberOfColumns; - } - /** * @return the columnCardinality */ diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java index 6036569a03c..d17d8653fb8 100644 --- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java +++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java @@ -19,7 +19,13 @@ import java.io.IOException; import java.io.Serializable; -import java.util.*; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; import org.apache.carbondata.core.constants.CarbonCommonConstants; import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier; @@ -33,7 +39,10 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonImplicitDimension; import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure; import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema; +import org.apache.carbondata.core.scan.model.QueryModel; +import org.apache.carbondata.core.scan.model.QueryProjection; import org.apache.carbondata.core.util.CarbonUtil; +import org.apache.carbondata.core.util.DataTypeConverter; import org.apache.carbondata.core.util.DataTypeUtil; import org.apache.carbondata.core.util.path.CarbonTablePath; @@ -136,10 +145,7 @@ private CarbonTable() { /** * During creation of TableInfo from hivemetastore the DataMapSchemas and the columns * DataTypes are not converted to the appropriate child classes. - * * This method will cast the same to the appropriate classes - * - * @param tableInfo */ public static void updateTableInfo(TableInfo tableInfo) { List dataMapSchemas = new ArrayList<>(); @@ -153,8 +159,9 @@ public static void updateTableInfo(TableInfo tableInfo) { } tableInfo.setDataMapSchemaList(dataMapSchemas); for (ColumnSchema columnSchema : tableInfo.getFactTable().getListOfColumns()) { - columnSchema.setDataType(DataTypeUtil.valueOf(columnSchema.getDataType(), - columnSchema.getPrecision(), columnSchema.getScale())); + columnSchema.setDataType( + DataTypeUtil.valueOf( + columnSchema.getDataType(), columnSchema.getPrecision(), columnSchema.getScale())); } List childSchema = tableInfo.getDataMapSchemaList(); for (DataMapSchema dataMapSchema : childSchema) { @@ -168,10 +175,11 @@ public static void updateTableInfo(TableInfo tableInfo) { } } if (tableInfo.getFactTable().getBucketingInfo() != null) { - for (ColumnSchema columnSchema : tableInfo.getFactTable() - .getBucketingInfo().getListOfColumns()) { - columnSchema.setDataType(DataTypeUtil.valueOf(columnSchema.getDataType(), - columnSchema.getPrecision(), columnSchema.getScale())); + for (ColumnSchema columnSchema : + tableInfo.getFactTable().getBucketingInfo().getListOfColumns()) { + columnSchema.setDataType( + DataTypeUtil.valueOf( + columnSchema.getDataType(), columnSchema.getPrecision(), columnSchema.getScale())); } } if (tableInfo.getFactTable().getPartitionInfo() != null) { @@ -211,6 +219,7 @@ public static CarbonTable buildFromTableInfo(TableInfo tableInfo) { /** * fill columns as per user provided order + * * @param tableName */ private void fillCreateOrderColumn(String tableName) { @@ -229,7 +238,6 @@ private void fillCreateOrderColumn(String tableName) { this.createOrderColumn.put(tableName, columns); } - /** * Fill allDimensions and allMeasures for carbon table * @@ -254,8 +262,8 @@ private void fillDimensionsAndMeasuresForTables(TableSchema tableSchema) { if (columnSchema.isDimensionColumn()) { if (columnSchema.getNumberOfChild() > 0) { CarbonDimension complexDimension = - new CarbonDimension(columnSchema, dimensionOrdinal++, - columnSchema.getSchemaOrdinal(), -1, -1, ++complexTypeOrdinal); + new CarbonDimension(columnSchema, dimensionOrdinal++, columnSchema.getSchemaOrdinal(), + -1, -1, ++complexTypeOrdinal); complexDimension.initializeChildDimensionsList(columnSchema.getNumberOfChild()); allDimensions.add(complexDimension); dimensionOrdinal = @@ -268,9 +276,8 @@ private void fillDimensionsAndMeasuresForTables(TableSchema tableSchema) { this.numberOfSortColumns++; } if (!columnSchema.getEncodingList().contains(Encoding.DICTIONARY)) { - CarbonDimension dimension = - new CarbonDimension(columnSchema, dimensionOrdinal++, - columnSchema.getSchemaOrdinal(), -1, -1, -1); + CarbonDimension dimension = new CarbonDimension( + columnSchema, dimensionOrdinal++, columnSchema.getSchemaOrdinal(), -1, -1, -1); if (!columnSchema.isInvisible() && columnSchema.isSortColumn()) { this.numberOfNoDictSortColumns++; } @@ -278,25 +285,25 @@ private void fillDimensionsAndMeasuresForTables(TableSchema tableSchema) { primitiveDimensions.add(dimension); } else if (columnSchema.getEncodingList().contains(Encoding.DICTIONARY) && columnSchema.getColumnGroupId() == -1) { - CarbonDimension dimension = - new CarbonDimension(columnSchema, dimensionOrdinal++, - columnSchema.getSchemaOrdinal(), keyOrdinal++, -1, -1); + CarbonDimension dimension = new CarbonDimension( + columnSchema, dimensionOrdinal++, columnSchema.getSchemaOrdinal(), keyOrdinal++, + -1, -1); allDimensions.add(dimension); primitiveDimensions.add(dimension); } else { columnGroupOrdinal = previousColumnGroupId == columnSchema.getColumnGroupId() ? ++columnGroupOrdinal : 0; previousColumnGroupId = columnSchema.getColumnGroupId(); - CarbonDimension dimension = new CarbonDimension(columnSchema, dimensionOrdinal++, - columnSchema.getSchemaOrdinal(), keyOrdinal++, + CarbonDimension dimension = new CarbonDimension( + columnSchema, dimensionOrdinal++, columnSchema.getSchemaOrdinal(), keyOrdinal++, columnGroupOrdinal, -1); allDimensions.add(dimension); primitiveDimensions.add(dimension); } } } else { - allMeasures.add(new CarbonMeasure(columnSchema, measureOrdinal++, - columnSchema.getSchemaOrdinal())); + allMeasures.add( + new CarbonMeasure(columnSchema, measureOrdinal++, columnSchema.getSchemaOrdinal())); } } fillVisibleDimensions(tableSchema.getTableName()); @@ -347,8 +354,8 @@ private int readAllComplexTypeChildrens(int dimensionOrdinal, int childCount, if (columnSchema.isDimensionColumn()) { if (columnSchema.getNumberOfChild() > 0) { CarbonDimension complexDimension = - new CarbonDimension(columnSchema, dimensionOrdinal++, - columnSchema.getSchemaOrdinal(), -1, -1, -1); + new CarbonDimension(columnSchema, dimensionOrdinal++, columnSchema.getSchemaOrdinal(), + -1, -1, -1); complexDimension.initializeChildDimensionsList(columnSchema.getNumberOfChild()); parentDimension.getListOfChildDimensions().add(complexDimension); dimensionOrdinal = @@ -356,8 +363,8 @@ private int readAllComplexTypeChildrens(int dimensionOrdinal, int childCount, listOfColumns, complexDimension, primitiveDimensions); } else { CarbonDimension carbonDimension = - new CarbonDimension(columnSchema, dimensionOrdinal++, - columnSchema.getSchemaOrdinal(), -1, -1, -1); + new CarbonDimension(columnSchema, dimensionOrdinal++, columnSchema.getSchemaOrdinal(), + -1, -1, -1); parentDimension.getListOfChildDimensions().add(carbonDimension); primitiveDimensions.add(carbonDimension); } @@ -569,6 +576,7 @@ public CarbonColumn getColumnByName(String tableName, String columnName) { } return null; } + /** * gets all children dimension for complex type * @@ -771,18 +779,13 @@ public int getDimensionOrdinalMax() { return dimensionOrdinalMax; } - public void setDimensionOrdinalMax(int dimensionOrdinalMax) { - this.dimensionOrdinalMax = dimensionOrdinalMax; - } - - public boolean hasDataMapSchema() { return hasDataMapSchema; } public boolean isChildDataMap() { - return null != tableInfo.getParentRelationIdentifiers() - && !tableInfo.getParentRelationIdentifiers().isEmpty(); + return null != tableInfo.getParentRelationIdentifiers() && + !tableInfo.getParentRelationIdentifiers().isEmpty(); } /** @@ -806,4 +809,63 @@ public long size() throws IOException { } return dataSize + indexSize; } + + /** + * Create a new QueryModel with projection all columns in the table. + */ + public QueryModel createQueryModelWithProjectAllColumns(DataTypeConverter converter) { + QueryProjection projection = new QueryProjection(); + + List dimensions = getDimensionByTableName(getTableName()); + for (int i = 0; i < dimensions.size(); i++) { + projection.addDimension(dimensions.get(i), i); + } + List measures = getMeasureByTableName(getTableName()); + for (int i = 0; i < measures.size(); i++) { + projection.addMeasure(measures.get(i), i); + } + QueryModel model = QueryModel.newInstance(this); + model.setProjection(projection); + model.setConverter(converter); + return model; + } + + /** + * Create a new QueryModel with specified projection + */ + public QueryModel createQueryWithProjection(String[] projectionColumnNames, + DataTypeConverter converter) { + QueryProjection projection = createProjection(projectionColumnNames); + QueryModel queryModel = QueryModel.newInstance(this); + queryModel.setProjection(projection); + queryModel.setConverter(converter); + return queryModel; + } + + private QueryProjection createProjection(String[] projectionColumnNames) { + String factTableName = getTableName(); + QueryProjection projection = new QueryProjection(); + // fill dimensions + // If columns are null, set all dimensions and measures + int i = 0; + if (projectionColumnNames != null) { + for (String projectionColumnName : projectionColumnNames) { + CarbonDimension dimension = getDimensionByName(factTableName, projectionColumnName); + if (dimension != null) { + projection.addDimension(dimension, i); + i++; + } else { + CarbonMeasure measure = getMeasureByName(factTableName, projectionColumnName); + if (measure == null) { + throw new RuntimeException(projectionColumnName + + " column not found in the table " + factTableName); + } + projection.addMeasure(measure, i); + i++; + } + } + } + + return projection; + } } diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java index 20e539ba7ce..9a1dad11fed 100644 --- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java +++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java @@ -26,18 +26,6 @@ */ public class RelationIdentifier implements Serializable, Writable { - public void setDatabaseName(String databaseName) { - this.databaseName = databaseName; - } - - public void setTableName(String tableName) { - this.tableName = tableName; - } - - public void setTableId(String tableId) { - this.tableId = tableId; - } - private String databaseName; private String tableName; @@ -50,10 +38,6 @@ public RelationIdentifier(String databaseName, String tableName, String tableId) this.tableId = tableId; } - public RelationIdentifier() { - this(null, null, null); - } - public String getDatabaseName() { return databaseName; } diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java index 4deafd462e1..0d796c7388a 100644 --- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java +++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java @@ -24,7 +24,11 @@ import java.io.DataOutputStream; import java.io.IOException; import java.io.Serializable; -import java.util.*; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; import org.apache.carbondata.common.logging.LogService; import org.apache.carbondata.common.logging.LogServiceFactory; diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonColumn.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonColumn.java index 13e7d4938eb..c888418f58e 100644 --- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonColumn.java +++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonColumn.java @@ -45,7 +45,7 @@ public class CarbonColumn implements Serializable { /** * order in which user has created table */ - protected int schemaOrdinal; + private int schemaOrdinal; /** * Column identifier diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonDimension.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonDimension.java index b4c052f44b6..d15cde77acd 100644 --- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonDimension.java +++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonDimension.java @@ -122,18 +122,6 @@ public boolean isGlobalDictionaryEncoding() { return getEncoder().contains(Encoding.DICTIONARY); } - public int getNumDimensionsExpanded() { - if (listOfChildDimensions == null) { - // there is no child, return 1 column - return 1; - } - int columnCount = 1; - for (CarbonDimension dimension: listOfChildDimensions) { - columnCount += dimension.getNumDimensionsExpanded(); - } - return columnCount; - } - /** * @return is column participated in sorting or not */ diff --git a/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java b/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java index 18eae11b863..7289c989f69 100644 --- a/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java +++ b/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java @@ -128,8 +128,6 @@ public static boolean updateSegmentStatus(List updateDetai lockStatus = updateLock.lockWithRetries(); if (lockStatus) { - AbsoluteTableIdentifier absoluteTableIdentifier = table.getAbsoluteTableIdentifier(); - // read the existing file if present and update the same. SegmentUpdateDetails[] oldDetails = segmentUpdateStatusManager .getUpdateStatusDetails(); @@ -389,16 +387,6 @@ public static String getBlockName(String completeBlockName) { .substring(0, completeBlockName.lastIndexOf(CarbonCommonConstants.HYPHEN)); } - /** - * returns segment id from segment name - * - * @param segmentName - * @return - */ - public static String getSegmentId(String segmentName) { - return segmentName.split(CarbonCommonConstants.UNDERSCORE)[1]; - } - public static long getLatestTaskIdForSegment(String segmentId, CarbonTablePath tablePath) { String segmentDirPath = tablePath.getCarbonDataDirectoryPath(segmentId); @@ -523,7 +511,7 @@ public static void cleanUpDeltaFiles(CarbonTable table, boolean forceDelete) { // aborted scenario. invalidDeleteDeltaFiles = updateStatusManager - .getDeleteDeltaInvalidFilesList(segment.getLoadName(), block, false, + .getDeleteDeltaInvalidFilesList(block, false, allSegmentFiles, isAbortedFile); for (CarbonFile invalidFile : invalidDeleteDeltaFiles) { boolean doForceDelete = true; @@ -533,7 +521,7 @@ public static void cleanUpDeltaFiles(CarbonTable table, boolean forceDelete) { // case 1 if (CarbonUpdateUtil.isBlockInvalid(block.getSegmentStatus())) { completeListOfDeleteDeltaFiles = updateStatusManager - .getDeleteDeltaInvalidFilesList(segment.getLoadName(), block, true, + .getDeleteDeltaInvalidFilesList(block, true, allSegmentFiles, isInvalidFile); for (CarbonFile invalidFile : completeListOfDeleteDeltaFiles) { @@ -541,7 +529,7 @@ public static void cleanUpDeltaFiles(CarbonTable table, boolean forceDelete) { } CarbonFile[] blockRelatedFiles = updateStatusManager - .getAllBlockRelatedFiles(block.getBlockName(), allSegmentFiles, + .getAllBlockRelatedFiles(allSegmentFiles, block.getActualBlockName()); // now for each invalid index file need to check the query execution time out @@ -555,7 +543,7 @@ public static void cleanUpDeltaFiles(CarbonTable table, boolean forceDelete) { } else { invalidDeleteDeltaFiles = updateStatusManager - .getDeleteDeltaInvalidFilesList(segment.getLoadName(), block, false, + .getDeleteDeltaInvalidFilesList(block, false, allSegmentFiles, isInvalidFile); for (CarbonFile invalidFile : invalidDeleteDeltaFiles) { diff --git a/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockDetails.java b/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockDetails.java index 7affe125d8a..20db9374116 100644 --- a/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockDetails.java +++ b/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockDetails.java @@ -48,14 +48,6 @@ public DeleteDeltaBlockDetails(String blockName) { blockletDetailsMap = new TreeMap<>(); } - public String getBlockName() { - return blockName; - } - - public void setBlockName(String blockName) { - this.blockName = blockName; - } - @Override public boolean equals(Object obj) { if (this == obj) return true; if (obj == null || !(obj instanceof DeleteDeltaBlockDetails)) return false; diff --git a/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockletDetails.java b/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockletDetails.java index 22785fac6a3..6185134af1a 100644 --- a/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockletDetails.java +++ b/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockletDetails.java @@ -46,14 +46,6 @@ public boolean addDeletedRow(Integer row) { return deletedRows.add(row); } - public String getId() { - return id; - } - - public void setId(String id) { - this.id = id; - } - public Integer getPageId() { return pageId; } diff --git a/core/src/main/java/org/apache/carbondata/core/mutate/data/BlockletDeleteDeltaCacheLoader.java b/core/src/main/java/org/apache/carbondata/core/mutate/data/BlockletDeleteDeltaCacheLoader.java deleted file mode 100644 index 3c0dd4ffe0f..00000000000 --- a/core/src/main/java/org/apache/carbondata/core/mutate/data/BlockletDeleteDeltaCacheLoader.java +++ /dev/null @@ -1,87 +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.mutate.data; - -import java.util.Map; - -import org.apache.carbondata.common.logging.LogService; -import org.apache.carbondata.common.logging.LogServiceFactory; -import org.apache.carbondata.core.cache.update.BlockletLevelDeleteDeltaDataCache; -import org.apache.carbondata.core.datastore.DataRefNode; -import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier; -import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager; - -/** - * This class is responsible for loading delete delta file cache based on - * blocklet id of a particular block - */ -public class BlockletDeleteDeltaCacheLoader implements DeleteDeltaCacheLoaderIntf { - private String blockletID; - private DataRefNode blockletNode; - private AbsoluteTableIdentifier absoluteIdentifier; - private static final LogService LOGGER = - LogServiceFactory.getLogService(BlockletDeleteDeltaCacheLoader.class.getName()); - - public BlockletDeleteDeltaCacheLoader(String blockletID, DataRefNode blockletNode, - AbsoluteTableIdentifier absoluteIdentifier) { - this.blockletID = blockletID; - this.blockletNode = blockletNode; - this.absoluteIdentifier = absoluteIdentifier; - } - - /** - * This method will load the delete delta cache based on blocklet id of particular block with - * the help of SegmentUpdateStatusManager. - */ - public void loadDeleteDeltaFileDataToCache() { - SegmentUpdateStatusManager segmentUpdateStatusManager = - new SegmentUpdateStatusManager(absoluteIdentifier); - Map deleteDeltaFileData = null; - BlockletLevelDeleteDeltaDataCache deleteDeltaDataCache = null; - if (null == blockletNode.getDeleteDeltaDataCache()) { - try { - deleteDeltaFileData = - segmentUpdateStatusManager.getDeleteDeltaDataFromAllFiles(blockletID); - deleteDeltaDataCache = new BlockletLevelDeleteDeltaDataCache(deleteDeltaFileData, - segmentUpdateStatusManager.getTimestampForRefreshCache(blockletID, null)); - } catch (Exception e) { - if (LOGGER.isDebugEnabled()) { - LOGGER.debug("Unable to retrieve delete delta files"); - } - } - } else { - deleteDeltaDataCache = blockletNode.getDeleteDeltaDataCache(); - // if already cache is present then validate the cache using timestamp - String cacheTimeStamp = segmentUpdateStatusManager - .getTimestampForRefreshCache(blockletID, deleteDeltaDataCache.getCacheTimeStamp()); - if (null != cacheTimeStamp) { - try { - deleteDeltaFileData = - segmentUpdateStatusManager.getDeleteDeltaDataFromAllFiles(blockletID); - deleteDeltaDataCache = new BlockletLevelDeleteDeltaDataCache(deleteDeltaFileData, - segmentUpdateStatusManager.getTimestampForRefreshCache(blockletID, cacheTimeStamp)); - } catch (Exception e) { - if (LOGGER.isDebugEnabled()) { - LOGGER.debug("Unable to retrieve delete delta files"); - } - } - } - } - blockletNode.setDeleteDeltaDataCache(deleteDeltaDataCache); - } -} diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/ScannedResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/ScannedResultCollector.java index b14660c00a4..88b3065a298 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/collector/ScannedResultCollector.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/ScannedResultCollector.java @@ -18,7 +18,7 @@ import java.util.List; -import org.apache.carbondata.core.scan.result.AbstractScannedResult; +import org.apache.carbondata.core.scan.result.BlockletScannedResult; import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch; /** @@ -27,17 +27,13 @@ public interface ScannedResultCollector { /** - * Below method will be used to aggregate the scanned result - * - * @param scannedResult scanned result - * @return how many records was aggregated + * Return the aggregated scanned result in list of rows */ - List collectData(AbstractScannedResult scannedResult, int batchSize); + List collectResultInRow(BlockletScannedResult scannedResult, int batchSize); /** - * Collects data in columnar format. - * @param scannedResult - * @param columnarBatch + * Collects result and set it in the specified columnar batch */ - void collectVectorBatch(AbstractScannedResult scannedResult, CarbonColumnarBatch columnarBatch); + void collectResultInColumnarBatch(BlockletScannedResult scannedResult, + CarbonColumnarBatch columnarBatch); } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/AbstractScannedResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/AbstractScannedResultCollector.java index dc78ac684af..694271e6b7c 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/AbstractScannedResultCollector.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/AbstractScannedResultCollector.java @@ -19,8 +19,6 @@ import java.math.BigDecimal; import java.math.RoundingMode; -import org.apache.carbondata.common.logging.LogService; -import org.apache.carbondata.common.logging.LogServiceFactory; import org.apache.carbondata.core.datastore.page.ColumnPage; import org.apache.carbondata.core.metadata.datatype.DataType; import org.apache.carbondata.core.metadata.datatype.DataTypes; @@ -29,8 +27,8 @@ import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo; import org.apache.carbondata.core.scan.executor.infos.DimensionInfo; import org.apache.carbondata.core.scan.executor.infos.MeasureInfo; -import org.apache.carbondata.core.scan.model.QueryMeasure; -import org.apache.carbondata.core.scan.result.AbstractScannedResult; +import org.apache.carbondata.core.scan.model.ProjectionMeasure; +import org.apache.carbondata.core.scan.result.BlockletScannedResult; import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch; import org.apache.carbondata.core.util.DataTypeUtil; @@ -39,38 +37,35 @@ */ public abstract class AbstractScannedResultCollector implements ScannedResultCollector { - private static final LogService LOGGER = - LogServiceFactory.getLogService(AbstractScannedResultCollector.class.getName()); - /** * table block execution infos */ - protected BlockExecutionInfo tableBlockExecutionInfos; + BlockExecutionInfo executionInfo; /** * maintains the measure information like datatype, ordinal, measure existence */ - protected MeasureInfo measureInfo; + MeasureInfo measureInfo; /** * maintains the dimension information like datatype, ordinal, measure existence */ - protected DimensionInfo dimensionInfo; + DimensionInfo dimensionInfo; - public AbstractScannedResultCollector(BlockExecutionInfo blockExecutionInfos) { - this.tableBlockExecutionInfos = blockExecutionInfos; + AbstractScannedResultCollector(BlockExecutionInfo blockExecutionInfos) { + this.executionInfo = blockExecutionInfos; measureInfo = blockExecutionInfos.getMeasureInfo(); dimensionInfo = blockExecutionInfos.getDimensionInfo(); } protected void fillMeasureData(Object[] msrValues, int offset, - AbstractScannedResult scannedResult) { + BlockletScannedResult scannedResult) { int measureExistIndex = 0; for (short i = 0; i < measureInfo.getMeasureDataTypes().length; i++) { // if measure exists is block then pass measure column // data chunk to the collector if (measureInfo.getMeasureExists()[i]) { - QueryMeasure queryMeasure = tableBlockExecutionInfos.getQueryMeasures()[measureExistIndex]; + ProjectionMeasure queryMeasure = executionInfo.getProjectionMeasures()[measureExistIndex]; msrValues[i + offset] = getMeasureData( scannedResult.getMeasureChunk(measureInfo.getMeasureOrdinals()[measureExistIndex]), scannedResult.getCurrentRowId(), queryMeasure.getMeasure()); @@ -87,8 +82,7 @@ protected void fillMeasureData(Object[] msrValues, int offset, } } - protected Object getMeasureData(ColumnPage dataChunk, int index, - CarbonMeasure carbonMeasure) { + Object getMeasureData(ColumnPage dataChunk, int index, CarbonMeasure carbonMeasure) { if (!dataChunk.getNullBits().get(index)) { DataType dataType = carbonMeasure.getDataType(); if (dataType == DataTypes.BOOLEAN) { @@ -114,7 +108,8 @@ protected Object getMeasureData(ColumnPage dataChunk, int index, return null; } - @Override public void collectVectorBatch(AbstractScannedResult scannedResult, + @Override + public void collectResultInColumnarBatch(BlockletScannedResult scannedResult, CarbonColumnarBatch columnarBatch) { throw new UnsupportedOperationException("Works only for batch collectors"); } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollector.java index cf6e6d6abe3..43220346fe9 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollector.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollector.java @@ -29,9 +29,9 @@ import org.apache.carbondata.core.metadata.encoder.Encoding; import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo; import org.apache.carbondata.core.scan.filter.GenericQueryType; -import org.apache.carbondata.core.scan.model.QueryDimension; -import org.apache.carbondata.core.scan.model.QueryMeasure; -import org.apache.carbondata.core.scan.result.AbstractScannedResult; +import org.apache.carbondata.core.scan.model.ProjectionDimension; +import org.apache.carbondata.core.scan.model.ProjectionMeasure; +import org.apache.carbondata.core.scan.result.BlockletScannedResult; import org.apache.carbondata.core.util.CarbonUtil; import org.apache.carbondata.core.util.DataTypeUtil; @@ -42,49 +42,50 @@ */ public class DictionaryBasedResultCollector extends AbstractScannedResultCollector { - protected QueryDimension[] queryDimensions; + protected ProjectionDimension[] queryDimensions; - protected QueryMeasure[] queryMeasures; + protected ProjectionMeasure[] queryMeasures; - protected DirectDictionaryGenerator[] directDictionaryGenerators; + private DirectDictionaryGenerator[] directDictionaryGenerators; /** * query order */ protected int[] order; - protected int[] actualIndexInSurrogateKey; + private int[] actualIndexInSurrogateKey; - protected boolean[] dictionaryEncodingArray; + boolean[] dictionaryEncodingArray; - protected boolean[] directDictionaryEncodingArray; + boolean[] directDictionaryEncodingArray; - protected boolean[] implictColumnArray; + private boolean[] implictColumnArray; - protected boolean[] complexDataTypeArray; + private boolean[] complexDataTypeArray; - protected int dictionaryColumnIndex; - protected int noDictionaryColumnIndex; - protected int complexTypeColumnIndex; + int dictionaryColumnIndex; + int noDictionaryColumnIndex; + int complexTypeColumnIndex; - protected boolean isDimensionExists; + boolean isDimensionExists; - protected Map comlexDimensionInfoMap; + private Map comlexDimensionInfoMap; public DictionaryBasedResultCollector(BlockExecutionInfo blockExecutionInfos) { super(blockExecutionInfos); - queryDimensions = tableBlockExecutionInfos.getQueryDimensions(); - queryMeasures = tableBlockExecutionInfos.getQueryMeasures(); + queryDimensions = executionInfo.getProjectionDimensions(); + queryMeasures = executionInfo.getProjectionMeasures(); initDimensionAndMeasureIndexesForFillingData(); isDimensionExists = queryDimensions.length > 0; - this.comlexDimensionInfoMap = tableBlockExecutionInfos.getComlexDimensionInfoMap(); + this.comlexDimensionInfoMap = executionInfo.getComlexDimensionInfoMap(); } /** * This method will add a record both key and value to list object * it will keep track of how many record is processed, to handle limit scenario */ - @Override public List collectData(AbstractScannedResult scannedResult, int batchSize) { + @Override + public List collectResultInRow(BlockletScannedResult scannedResult, int batchSize) { // scan the record and add to list List listBasedResult = new ArrayList<>(batchSize); @@ -118,13 +119,13 @@ public DictionaryBasedResultCollector(BlockExecutionInfo blockExecutionInfos) { return listBasedResult; } - protected void fillDimensionData(AbstractScannedResult scannedResult, int[] surrogateResult, + void fillDimensionData(BlockletScannedResult scannedResult, int[] surrogateResult, byte[][] noDictionaryKeys, byte[][] complexTypeKeyArray, Map comlexDimensionInfoMap, Object[] row, int i) { if (!dictionaryEncodingArray[i]) { if (implictColumnArray[i]) { if (CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID - .equals(queryDimensions[i].getDimension().getColName())) { + .equals(queryDimensions[i].getColumnName())) { row[order[i]] = DataTypeUtil.getDataBasedOnDataType( scannedResult.getBlockletId() + CarbonCommonConstants.FILE_SEPARATOR + scannedResult .getCurrentPageCounter() + CarbonCommonConstants.FILE_SEPARATOR + scannedResult @@ -153,7 +154,7 @@ protected void fillDimensionData(AbstractScannedResult scannedResult, int[] surr } } - protected void fillMeasureData(AbstractScannedResult scannedResult, Object[] row) { + void fillMeasureData(BlockletScannedResult scannedResult, Object[] row) { if (measureInfo.getMeasureDataTypes().length > 0) { Object[] msrValues = new Object[measureInfo.getMeasureDataTypes().length]; fillMeasureData(msrValues, 0, scannedResult); @@ -163,7 +164,7 @@ protected void fillMeasureData(AbstractScannedResult scannedResult, Object[] row } } - protected void initDimensionAndMeasureIndexesForFillingData() { + void initDimensionAndMeasureIndexesForFillingData() { List dictionaryIndexes = new ArrayList(); for (int i = 0; i < queryDimensions.length; i++) { if (queryDimensions[i].getDimension().hasEncoding(Encoding.DICTIONARY) || queryDimensions[i] @@ -190,10 +191,10 @@ protected void initDimensionAndMeasureIndexesForFillingData() { complexDataTypeArray = CarbonUtil.getComplexDataTypeArray(queryDimensions); order = new int[queryDimensions.length + queryMeasures.length]; for (int i = 0; i < queryDimensions.length; i++) { - order[i] = queryDimensions[i].getQueryOrder(); + order[i] = queryDimensions[i].getOrdinal(); } for (int i = 0; i < queryMeasures.length; i++) { - order[i + queryDimensions.length] = queryMeasures[i].getQueryOrder(); + order[i + queryDimensions.length] = queryMeasures[i].getOrdinal(); } directDictionaryGenerators = new DirectDictionaryGenerator[queryDimensions.length]; for (int i = 0; i < queryDimensions.length; i++) { diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedVectorResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedVectorResultCollector.java index e9d6740736c..4947621bee4 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedVectorResultCollector.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedVectorResultCollector.java @@ -23,9 +23,9 @@ import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory; import org.apache.carbondata.core.metadata.encoder.Encoding; import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo; -import org.apache.carbondata.core.scan.model.QueryDimension; -import org.apache.carbondata.core.scan.model.QueryMeasure; -import org.apache.carbondata.core.scan.result.AbstractScannedResult; +import org.apache.carbondata.core.scan.model.ProjectionDimension; +import org.apache.carbondata.core.scan.model.ProjectionMeasure; +import org.apache.carbondata.core.scan.result.BlockletScannedResult; import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch; import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo; import org.apache.carbondata.core.scan.result.vector.MeasureDataVectorProcessor; @@ -35,35 +35,35 @@ */ public class DictionaryBasedVectorResultCollector extends AbstractScannedResultCollector { - protected QueryDimension[] queryDimensions; + protected ProjectionDimension[] queryDimensions; - protected QueryMeasure[] queryMeasures; + protected ProjectionMeasure[] queryMeasures; - protected ColumnVectorInfo[] dictionaryInfo; + private ColumnVectorInfo[] dictionaryInfo; - protected ColumnVectorInfo[] noDictionaryInfo; + private ColumnVectorInfo[] noDictionaryInfo; - protected ColumnVectorInfo[] complexInfo; + private ColumnVectorInfo[] complexInfo; - protected ColumnVectorInfo[] measureColumnInfo; + private ColumnVectorInfo[] measureColumnInfo; - protected ColumnVectorInfo[] allColumnInfo; + ColumnVectorInfo[] allColumnInfo; - protected ColumnVectorInfo[] implictColumnInfo; + private ColumnVectorInfo[] implictColumnInfo; public DictionaryBasedVectorResultCollector(BlockExecutionInfo blockExecutionInfos) { super(blockExecutionInfos); // initialize only if the current block is not a restructured block else the initialization // will be taken care by RestructureBasedVectorResultCollector if (!blockExecutionInfos.isRestructuredBlock()) { - queryDimensions = tableBlockExecutionInfos.getQueryDimensions(); - queryMeasures = tableBlockExecutionInfos.getQueryMeasures(); + queryDimensions = executionInfo.getProjectionDimensions(); + queryMeasures = executionInfo.getProjectionMeasures(); allColumnInfo = new ColumnVectorInfo[queryDimensions.length + queryMeasures.length]; prepareDimensionAndMeasureColumnVectors(); } } - protected void prepareDimensionAndMeasureColumnVectors() { + void prepareDimensionAndMeasureColumnVectors() { measureColumnInfo = new ColumnVectorInfo[queryMeasures.length]; List dictInfoList = new ArrayList<>(); List noDictInfoList = new ArrayList<>(); @@ -78,13 +78,13 @@ protected void prepareDimensionAndMeasureColumnVectors() { implictColumnList.add(columnVectorInfo); columnVectorInfo.dimension = queryDimensions[i]; columnVectorInfo.ordinal = queryDimensions[i].getDimension().getOrdinal(); - allColumnInfo[queryDimensions[i].getQueryOrder()] = columnVectorInfo; + allColumnInfo[queryDimensions[i].getOrdinal()] = columnVectorInfo; } else if (!queryDimensions[i].getDimension().hasEncoding(Encoding.DICTIONARY)) { ColumnVectorInfo columnVectorInfo = new ColumnVectorInfo(); noDictInfoList.add(columnVectorInfo); columnVectorInfo.dimension = queryDimensions[i]; columnVectorInfo.ordinal = queryDimensions[i].getDimension().getOrdinal(); - allColumnInfo[queryDimensions[i].getQueryOrder()] = columnVectorInfo; + allColumnInfo[queryDimensions[i].getOrdinal()] = columnVectorInfo; } else if (queryDimensions[i].getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) { ColumnVectorInfo columnVectorInfo = new ColumnVectorInfo(); dictInfoList.add(columnVectorInfo); @@ -92,21 +92,21 @@ protected void prepareDimensionAndMeasureColumnVectors() { columnVectorInfo.directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory .getDirectDictionaryGenerator(queryDimensions[i].getDimension().getDataType()); columnVectorInfo.ordinal = queryDimensions[i].getDimension().getOrdinal(); - allColumnInfo[queryDimensions[i].getQueryOrder()] = columnVectorInfo; + allColumnInfo[queryDimensions[i].getOrdinal()] = columnVectorInfo; } else if (queryDimensions[i].getDimension().isComplex()) { ColumnVectorInfo columnVectorInfo = new ColumnVectorInfo(); complexList.add(columnVectorInfo); columnVectorInfo.dimension = queryDimensions[i]; columnVectorInfo.ordinal = queryDimensions[i].getDimension().getOrdinal(); columnVectorInfo.genericQueryType = - tableBlockExecutionInfos.getComlexDimensionInfoMap().get(columnVectorInfo.ordinal); - allColumnInfo[queryDimensions[i].getQueryOrder()] = columnVectorInfo; + executionInfo.getComlexDimensionInfoMap().get(columnVectorInfo.ordinal); + allColumnInfo[queryDimensions[i].getOrdinal()] = columnVectorInfo; } else { ColumnVectorInfo columnVectorInfo = new ColumnVectorInfo(); dictInfoList.add(columnVectorInfo); columnVectorInfo.dimension = queryDimensions[i]; columnVectorInfo.ordinal = queryDimensions[i].getDimension().getOrdinal(); - allColumnInfo[queryDimensions[i].getQueryOrder()] = columnVectorInfo; + allColumnInfo[queryDimensions[i].getOrdinal()] = columnVectorInfo; } } for (int i = 0; i < queryMeasures.length; i++) { @@ -116,7 +116,7 @@ protected void prepareDimensionAndMeasureColumnVectors() { columnVectorInfo.ordinal = queryMeasures[i].getMeasure().getOrdinal(); columnVectorInfo.measure = queryMeasures[i]; this.measureColumnInfo[i] = columnVectorInfo; - allColumnInfo[queryMeasures[i].getQueryOrder()] = columnVectorInfo; + allColumnInfo[queryMeasures[i].getOrdinal()] = columnVectorInfo; } dictionaryInfo = dictInfoList.toArray(new ColumnVectorInfo[dictInfoList.size()]); noDictionaryInfo = noDictInfoList.toArray(new ColumnVectorInfo[noDictInfoList.size()]); @@ -126,11 +126,13 @@ protected void prepareDimensionAndMeasureColumnVectors() { Arrays.sort(complexInfo); } - @Override public List collectData(AbstractScannedResult scannedResult, int batchSize) { - throw new UnsupportedOperationException("collectData is not supported here"); + @Override + public List collectResultInRow(BlockletScannedResult scannedResult, int batchSize) { + throw new UnsupportedOperationException("collectResultInRow is not supported here"); } - @Override public void collectVectorBatch(AbstractScannedResult scannedResult, + @Override + public void collectResultInColumnarBatch(BlockletScannedResult scannedResult, CarbonColumnarBatch columnarBatch) { int numberOfPages = scannedResult.numberOfpages(); int filteredRows = 0; @@ -150,14 +152,15 @@ protected void prepareDimensionAndMeasureColumnVectors() { return; } fillColumnVectorDetails(columnarBatch, rowCounter, requiredRows); - filteredRows = scannedResult - .markFilteredRows(columnarBatch, rowCounter, requiredRows, columnarBatch.getRowCounter()); - scanAndFillResult(scannedResult, columnarBatch, rowCounter, availableRows, requiredRows); + filteredRows = scannedResult.markFilteredRows( + columnarBatch, rowCounter, requiredRows, columnarBatch.getRowCounter()); + fillResultToColumnarBatch( + scannedResult, columnarBatch, rowCounter, availableRows, requiredRows); columnarBatch.setActualSize(columnarBatch.getActualSize() + requiredRows - filteredRows); } } - protected void scanAndFillResult(AbstractScannedResult scannedResult, + void fillResultToColumnarBatch(BlockletScannedResult scannedResult, CarbonColumnarBatch columnarBatch, int rowCounter, int availableRows, int requiredRows) { scannedResult.fillColumnarDictionaryBatch(dictionaryInfo); scannedResult.fillColumnarNoDictionaryBatch(noDictionaryInfo); @@ -174,8 +177,8 @@ protected void scanAndFillResult(AbstractScannedResult scannedResult, columnarBatch.setRowCounter(columnarBatch.getRowCounter() + requiredRows); } - protected void fillColumnVectorDetails(CarbonColumnarBatch columnarBatch, int rowCounter, - int requiredRows) { + void fillColumnVectorDetails(CarbonColumnarBatch columnarBatch, int rowCounter, int requiredRows) + { for (int i = 0; i < allColumnInfo.length; i++) { allColumnInfo[i].size = requiredRows; allColumnInfo[i].offset = rowCounter; diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollector.java index a09676a2c08..07806758893 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollector.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollector.java @@ -16,14 +16,12 @@ */ package org.apache.carbondata.core.scan.collector.impl; -import java.nio.charset.Charset; import java.util.ArrayList; import java.util.List; -import org.apache.carbondata.core.constants.CarbonCommonConstants; import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo; -import org.apache.carbondata.core.scan.model.QueryMeasure; -import org.apache.carbondata.core.scan.result.AbstractScannedResult; +import org.apache.carbondata.core.scan.model.ProjectionMeasure; +import org.apache.carbondata.core.scan.result.BlockletScannedResult; import org.apache.carbondata.core.scan.wrappers.ByteArrayWrapper; /** @@ -31,15 +29,11 @@ */ public class RawBasedResultCollector extends AbstractScannedResultCollector { - protected ByteArrayWrapper wrapper; + byte[] dictionaryKeyArray; - protected byte[] dictionaryKeyArray; + byte[][] noDictionaryKeyArray; - protected byte[][] noDictionaryKeyArray; - - protected byte[][] complexTypeKeyArray; - - protected byte[] implicitColumnByteArray; + private byte[][] complexTypeKeyArray; public RawBasedResultCollector(BlockExecutionInfo blockExecutionInfos) { super(blockExecutionInfos); @@ -49,9 +43,10 @@ public RawBasedResultCollector(BlockExecutionInfo blockExecutionInfos) { * This method will add a record both key and value to list object * it will keep track of how many record is processed, to handle limit scenario */ - @Override public List collectData(AbstractScannedResult scannedResult, int batchSize) { + @Override + public List collectResultInRow(BlockletScannedResult scannedResult, int batchSize) { List listBasedResult = new ArrayList<>(batchSize); - QueryMeasure[] queryMeasures = tableBlockExecutionInfos.getQueryMeasures(); + ProjectionMeasure[] queryMeasures = executionInfo.getProjectionMeasures(); // scan the record and add to list int rowCounter = 0; while (scannedResult.hasNext() && rowCounter < batchSize) { @@ -65,24 +60,21 @@ public RawBasedResultCollector(BlockExecutionInfo blockExecutionInfos) { return listBasedResult; } - protected void prepareRow(AbstractScannedResult scannedResult, List listBasedResult, - QueryMeasure[] queryMeasures) { + void prepareRow(BlockletScannedResult scannedResult, List listBasedResult, + ProjectionMeasure[] queryMeasures) { Object[] row = new Object[1 + queryMeasures.length]; - wrapper = new ByteArrayWrapper(); + ByteArrayWrapper wrapper = new ByteArrayWrapper(); wrapper.setDictionaryKey(dictionaryKeyArray); wrapper.setNoDictionaryKeys(noDictionaryKeyArray); wrapper.setComplexTypesKeys(complexTypeKeyArray); - wrapper.setImplicitColumnByteArray(implicitColumnByteArray); row[0] = wrapper; fillMeasureData(row, 1, scannedResult); listBasedResult.add(row); } - protected void scanResultAndGetData(AbstractScannedResult scannedResult) { + void scanResultAndGetData(BlockletScannedResult scannedResult) { dictionaryKeyArray = scannedResult.getDictionaryKeyArray(); noDictionaryKeyArray = scannedResult.getNoDictionaryKeyArray(); complexTypeKeyArray = scannedResult.getComplexTypeKeyArray(); - implicitColumnByteArray = scannedResult.getBlockletId() - .getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)); } } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedDictionaryResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedDictionaryResultCollector.java index 296d9a6480e..8b42a4af711 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedDictionaryResultCollector.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedDictionaryResultCollector.java @@ -23,8 +23,8 @@ import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo; import org.apache.carbondata.core.scan.executor.util.RestructureUtil; import org.apache.carbondata.core.scan.filter.GenericQueryType; -import org.apache.carbondata.core.scan.model.QueryMeasure; -import org.apache.carbondata.core.scan.result.AbstractScannedResult; +import org.apache.carbondata.core.scan.model.ProjectionMeasure; +import org.apache.carbondata.core.scan.result.BlockletScannedResult; /** * class for handling restructure scenarios for filling result @@ -35,8 +35,8 @@ public class RestructureBasedDictionaryResultCollector extends DictionaryBasedRe public RestructureBasedDictionaryResultCollector(BlockExecutionInfo blockExecutionInfos) { super(blockExecutionInfos); - queryDimensions = tableBlockExecutionInfos.getActualQueryDimensions(); - queryMeasures = tableBlockExecutionInfos.getActualQueryMeasures(); + queryDimensions = executionInfo.getActualQueryDimensions(); + queryMeasures = executionInfo.getActualQueryMeasures(); measureDefaultValues = new Object[queryMeasures.length]; fillMeasureDefaultValues(); initDimensionAndMeasureIndexesForFillingData(); @@ -61,7 +61,8 @@ private void fillMeasureDefaultValues() { * This method will add a record both key and value to list object * it will keep track of how many record is processed, to handle limit scenario */ - @Override public List collectData(AbstractScannedResult scannedResult, int batchSize) { + @Override + public List collectResultInRow(BlockletScannedResult scannedResult, int batchSize) { // scan the record and add to list List listBasedResult = new ArrayList<>(batchSize); int rowCounter = 0; @@ -69,7 +70,7 @@ private void fillMeasureDefaultValues() { byte[][] noDictionaryKeys; byte[][] complexTypeKeyArray; Map comlexDimensionInfoMap = - tableBlockExecutionInfos.getComlexDimensionInfoMap(); + executionInfo.getComlexDimensionInfoMap(); while (scannedResult.hasNext() && rowCounter < batchSize) { Object[] row = new Object[queryDimensions.length + queryMeasures.length]; if (isDimensionExists) { @@ -107,13 +108,13 @@ private void fillMeasureDefaultValues() { } protected void fillMeasureData(Object[] msrValues, int offset, - AbstractScannedResult scannedResult) { + BlockletScannedResult scannedResult) { int measureExistIndex = 0; for (short i = 0; i < measureInfo.getMeasureDataTypes().length; i++) { // if measure exists is block then pass measure column // data chunk to the collector if (measureInfo.getMeasureExists()[i]) { - QueryMeasure queryMeasure = tableBlockExecutionInfos.getQueryMeasures()[measureExistIndex]; + ProjectionMeasure queryMeasure = executionInfo.getProjectionMeasures()[measureExistIndex]; msrValues[i + offset] = getMeasureData( scannedResult.getMeasureChunk(measureInfo.getMeasureOrdinals()[measureExistIndex]), scannedResult.getCurrentRowId(), queryMeasure.getMeasure()); diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedRawResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedRawResultCollector.java index b08a1d423b7..6544a750ab1 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedRawResultCollector.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedRawResultCollector.java @@ -30,9 +30,9 @@ import org.apache.carbondata.core.metadata.encoder.Encoding; import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension; import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo; -import org.apache.carbondata.core.scan.model.QueryDimension; -import org.apache.carbondata.core.scan.model.QueryMeasure; -import org.apache.carbondata.core.scan.result.AbstractScannedResult; +import org.apache.carbondata.core.scan.model.ProjectionDimension; +import org.apache.carbondata.core.scan.model.ProjectionMeasure; +import org.apache.carbondata.core.scan.result.BlockletScannedResult; import org.apache.carbondata.core.util.CarbonUtil; import org.apache.commons.lang3.ArrayUtils; @@ -70,15 +70,15 @@ public RestructureBasedRawResultCollector(BlockExecutionInfo blockExecutionInfos */ private void initRestructuredKeyGenerator() { SegmentProperties segmentProperties = - tableBlockExecutionInfos.getDataBlock().getSegmentProperties(); - QueryDimension[] queryDimensions = tableBlockExecutionInfos.getActualQueryDimensions(); + executionInfo.getDataBlock().getSegmentProperties(); + ProjectionDimension[] queryDimensions = executionInfo.getActualQueryDimensions(); List updatedColumnCardinality = new ArrayList<>(queryDimensions.length); List updatedDimensionPartitioner = new ArrayList<>(queryDimensions.length); - int[] dictionaryColumnBlockIndex = tableBlockExecutionInfos.getDictionaryColumnBlockIndex(); + int[] dictionaryColumnBlockIndex = executionInfo.getDictionaryColumnChunkIndex(); int dimCounterInCurrentBlock = 0; for (int i = 0; i < queryDimensions.length; i++) { if (queryDimensions[i].getDimension().hasEncoding(Encoding.DICTIONARY)) { - if (tableBlockExecutionInfos.getDimensionInfo().getDimensionExists()[i]) { + if (executionInfo.getDimensionInfo().getDimensionExists()[i]) { // get the dictionary key ordinal as column cardinality in segment properties // will only be for dictionary encoded columns CarbonDimension currentBlockDimension = segmentProperties.getDimensions() @@ -124,8 +124,8 @@ private void initRestructuredKeyGenerator() { */ private void initCurrentBlockKeyGenerator() { SegmentProperties segmentProperties = - tableBlockExecutionInfos.getDataBlock().getSegmentProperties(); - int[] dictionaryColumnBlockIndex = tableBlockExecutionInfos.getDictionaryColumnBlockIndex(); + executionInfo.getDataBlock().getSegmentProperties(); + int[] dictionaryColumnBlockIndex = executionInfo.getDictionaryColumnChunkIndex(); int[] updatedColumnCardinality = new int[dictionaryColumnBlockIndex.length]; int[] updatedDimensionPartitioner = new int[dictionaryColumnBlockIndex.length]; for (int i = 0; i < dictionaryColumnBlockIndex.length; i++) { @@ -149,9 +149,10 @@ private void initCurrentBlockKeyGenerator() { * This method will add a record both key and value to list object * it will keep track of how many record is processed, to handle limit scenario */ - @Override public List collectData(AbstractScannedResult scannedResult, int batchSize) { + @Override + public List collectResultInRow(BlockletScannedResult scannedResult, int batchSize) { List listBasedResult = new ArrayList<>(batchSize); - QueryMeasure[] queryMeasures = tableBlockExecutionInfos.getActualQueryMeasures(); + ProjectionMeasure[] queryMeasures = executionInfo.getActualQueryMeasures(); // scan the record and add to list int rowCounter = 0; while (scannedResult.hasNext() && rowCounter < batchSize) { @@ -179,7 +180,7 @@ private void initCurrentBlockKeyGenerator() { * @return */ private byte[] fillDictionaryKeyArrayWithLatestSchema(byte[] dictionaryKeyArray) { - QueryDimension[] actualQueryDimensions = tableBlockExecutionInfos.getActualQueryDimensions(); + ProjectionDimension[] actualQueryDimensions = executionInfo.getActualQueryDimensions(); int newKeyArrayLength = dimensionInfo.getNewDictionaryColumnCount(); long[] keyArray = null; if (null != updatedCurrentBlockKeyGenerator) { @@ -222,7 +223,7 @@ private byte[] fillDictionaryKeyArrayWithLatestSchema(byte[] dictionaryKeyArray) * @return */ private byte[][] fillNoDictionaryKeyArrayWithLatestSchema(byte[][] noDictionaryKeyArray) { - QueryDimension[] actualQueryDimensions = tableBlockExecutionInfos.getActualQueryDimensions(); + ProjectionDimension[] actualQueryDimensions = executionInfo.getActualQueryDimensions(); byte[][] noDictionaryKeyArrayWithNewlyAddedColumns = new byte[noDictionaryKeyArray.length + dimensionInfo.getNewNoDictionaryColumnCount()][]; int existingColumnValueIndex = 0; diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java index 0f1b9f96ad2..61a2992978c 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java @@ -26,7 +26,7 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure; import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo; import org.apache.carbondata.core.scan.executor.util.RestructureUtil; -import org.apache.carbondata.core.scan.result.AbstractScannedResult; +import org.apache.carbondata.core.scan.result.BlockletScannedResult; import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector; import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch; import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo; @@ -43,8 +43,8 @@ public class RestructureBasedVectorResultCollector extends DictionaryBasedVector public RestructureBasedVectorResultCollector(BlockExecutionInfo blockExecutionInfos) { super(blockExecutionInfos); - queryDimensions = tableBlockExecutionInfos.getActualQueryDimensions(); - queryMeasures = tableBlockExecutionInfos.getActualQueryMeasures(); + queryDimensions = executionInfo.getActualQueryDimensions(); + queryMeasures = executionInfo.getActualQueryMeasures(); measureDefaultValues = new Object[queryMeasures.length]; allColumnInfo = new ColumnVectorInfo[queryDimensions.length + queryMeasures.length]; createVectorForNewlyAddedDimensions(); @@ -66,7 +66,7 @@ private void createVectorForNewlyAddedDimensions() { columnVectorInfo.directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory .getDirectDictionaryGenerator(queryDimensions[i].getDimension().getDataType()); } - allColumnInfo[queryDimensions[i].getQueryOrder()] = columnVectorInfo; + allColumnInfo[queryDimensions[i].getOrdinal()] = columnVectorInfo; } } } @@ -79,7 +79,7 @@ private void createVectorForNewlyAddedMeasures() { if (!measureInfo.getMeasureExists()[i]) { // add a dummy column vector result collector object ColumnVectorInfo columnVectorInfo = new ColumnVectorInfo(); - allColumnInfo[queryMeasures[i].getQueryOrder()] = columnVectorInfo; + allColumnInfo[queryMeasures[i].getOrdinal()] = columnVectorInfo; columnVectorInfo.measure = queryMeasures[i]; measureDefaultValues[i] = getMeasureDefaultValue(queryMeasures[i].getMeasure()); } @@ -97,11 +97,13 @@ private Object getMeasureDefaultValue(CarbonMeasure carbonMeasure) { carbonMeasure.getDefaultValue()); } - @Override public List collectData(AbstractScannedResult scannedResult, int batchSize) { - throw new UnsupportedOperationException("collectData is not supported here"); + @Override + public List collectResultInRow(BlockletScannedResult scannedResult, int batchSize) { + throw new UnsupportedOperationException("collectResultInRow is not supported here"); } - @Override public void collectVectorBatch(AbstractScannedResult scannedResult, + @Override + public void collectResultInColumnarBatch(BlockletScannedResult scannedResult, CarbonColumnarBatch columnarBatch) { int numberOfPages = scannedResult.numberOfpages(); while (scannedResult.getCurrentPageCounter() < numberOfPages) { @@ -124,7 +126,8 @@ private Object getMeasureDefaultValue(CarbonMeasure carbonMeasure) { fillDataForNonExistingDimensions(); fillDataForNonExistingMeasures(); // fill existing dimensions and measures data - scanAndFillResult(scannedResult, columnarBatch, rowCounter, availableRows, requiredRows); + fillResultToColumnarBatch( + scannedResult, columnarBatch, rowCounter, availableRows, requiredRows); columnarBatch.setActualSize(columnarBatch.getActualSize() + requiredRows - filteredRows); } } @@ -133,11 +136,11 @@ private Object getMeasureDefaultValue(CarbonMeasure carbonMeasure) { * This method will fill the default values of non existing dimensions in the current block */ private void fillDataForNonExistingDimensions() { - for (int i = 0; i < tableBlockExecutionInfos.getActualQueryDimensions().length; i++) { + for (int i = 0; i < executionInfo.getActualQueryDimensions().length; i++) { if (!dimensionInfo.getDimensionExists()[i]) { - int queryOrder = tableBlockExecutionInfos.getActualQueryDimensions()[i].getQueryOrder(); + int queryOrder = executionInfo.getActualQueryDimensions()[i].getOrdinal(); CarbonDimension dimension = - tableBlockExecutionInfos.getActualQueryDimensions()[i].getDimension(); + executionInfo.getActualQueryDimensions()[i].getDimension(); if (dimension.hasEncoding(Encoding.DIRECT_DICTIONARY)) { // fill direct dictionary column data fillDirectDictionaryData(allColumnInfo[queryOrder].vector, allColumnInfo[queryOrder], @@ -215,10 +218,10 @@ private void fillNoDictionaryData(CarbonColumnVector vector, ColumnVectorInfo co * This method will fill the default values of non existing measures in the current block */ private void fillDataForNonExistingMeasures() { - for (int i = 0; i < tableBlockExecutionInfos.getActualQueryMeasures().length; i++) { + for (int i = 0; i < executionInfo.getActualQueryMeasures().length; i++) { if (!measureInfo.getMeasureExists()[i]) { - int queryOrder = tableBlockExecutionInfos.getActualQueryMeasures()[i].getQueryOrder(); - CarbonMeasure measure = tableBlockExecutionInfos.getActualQueryMeasures()[i].getMeasure(); + int queryOrder = executionInfo.getActualQueryMeasures()[i].getOrdinal(); + CarbonMeasure measure = executionInfo.getActualQueryMeasures()[i].getMeasure(); ColumnVectorInfo columnVectorInfo = allColumnInfo[queryOrder]; CarbonColumnVector vector = columnVectorInfo.vector; Object defaultValue = measureDefaultValues[i]; diff --git a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryType.java b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryType.java index 1fcccca5497..30dd1ddae97 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryType.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryType.java @@ -23,7 +23,7 @@ import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; import org.apache.carbondata.core.scan.filter.GenericQueryType; -import org.apache.carbondata.core.scan.processor.BlocksChunkHolder; +import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks; import org.apache.spark.sql.catalyst.util.GenericArrayData; import org.apache.spark.sql.types.ArrayType; @@ -86,7 +86,7 @@ public void parseBlocksAndReturnComplexColumnByteArray(DimensionRawColumnChunk[] return new ArrayType(null, true); } - @Override public void fillRequiredBlockData(BlocksChunkHolder blockChunkHolder) + @Override public void fillRequiredBlockData(RawBlockletColumnChunks blockChunkHolder) throws IOException { readBlockDataChunk(blockChunkHolder); children.fillRequiredBlockData(blockChunkHolder); diff --git a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ComplexQueryType.java b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ComplexQueryType.java index ee43a109087..98f0715f47a 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ComplexQueryType.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ComplexQueryType.java @@ -20,7 +20,7 @@ import java.io.IOException; import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; -import org.apache.carbondata.core.scan.processor.BlocksChunkHolder; +import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks; public class ComplexQueryType { protected String name; @@ -42,7 +42,7 @@ public ComplexQueryType(String name, String parentname, int blockIndex) { protected byte[] copyBlockDataChunk(DimensionRawColumnChunk[] rawColumnChunks, int rowNumber, int pageNumber) { byte[] data = - rawColumnChunks[blockIndex].convertToDimColDataChunk(pageNumber).getChunkData(rowNumber); + rawColumnChunks[blockIndex].decodeColumnPage(pageNumber).getChunkData(rowNumber); byte[] output = new byte[data.length]; System.arraycopy(data, 0, output, 0, output.length); return output; @@ -51,10 +51,10 @@ protected byte[] copyBlockDataChunk(DimensionRawColumnChunk[] rawColumnChunks, /* * This method will read the block data chunk from the respective block */ - protected void readBlockDataChunk(BlocksChunkHolder blockChunkHolder) throws IOException { - if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) { - blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex); + protected void readBlockDataChunk(RawBlockletColumnChunks blockChunkHolder) throws IOException { + if (null == blockChunkHolder.getDimensionRawColumnChunks()[blockIndex]) { + blockChunkHolder.getDimensionRawColumnChunks()[blockIndex] = blockChunkHolder.getDataBlock() + .readDimensionChunk(blockChunkHolder.getFileReader(), blockIndex); } } } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java index 978a652881e..b8aa9125838 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java @@ -27,7 +27,7 @@ import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory; import org.apache.carbondata.core.keygenerator.mdkey.Bits; import org.apache.carbondata.core.scan.filter.GenericQueryType; -import org.apache.carbondata.core.scan.processor.BlocksChunkHolder; +import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks; import org.apache.carbondata.core.util.DataTypeUtil; import org.apache.spark.sql.types.BooleanType$; @@ -113,7 +113,7 @@ public PrimitiveQueryType(String name, String parentname, int blockIndex, } } - @Override public void fillRequiredBlockData(BlocksChunkHolder blockChunkHolder) + @Override public void fillRequiredBlockData(RawBlockletColumnChunks blockChunkHolder) throws IOException { readBlockDataChunk(blockChunkHolder); } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/StructQueryType.java b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/StructQueryType.java index 23a9f812128..1d4f1413be4 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/StructQueryType.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/StructQueryType.java @@ -25,7 +25,7 @@ import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; import org.apache.carbondata.core.scan.filter.GenericQueryType; -import org.apache.carbondata.core.scan.processor.BlocksChunkHolder; +import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; import org.apache.spark.sql.types.DataType; @@ -106,7 +106,7 @@ public StructQueryType(String name, String parentname, int blockIndex) { return new StructType(fields); } - @Override public void fillRequiredBlockData(BlocksChunkHolder blockChunkHolder) + @Override public void fillRequiredBlockData(RawBlockletColumnChunks blockChunkHolder) throws IOException { readBlockDataChunk(blockChunkHolder); diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java index 69f5ceb20da..22d1df1af9e 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java @@ -43,10 +43,9 @@ import org.apache.carbondata.core.datastore.block.TableBlockInfo; import org.apache.carbondata.core.datastore.block.TableBlockUniqueIdentifier; import org.apache.carbondata.core.indexstore.BlockletDetailInfo; -import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataRefNodeWrapper; +import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataRefNode; import org.apache.carbondata.core.indexstore.blockletindex.IndexWrapper; import org.apache.carbondata.core.keygenerator.KeyGenException; -import org.apache.carbondata.core.keygenerator.KeyGenerator; import org.apache.carbondata.core.memory.UnsafeMemoryManager; import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier; import org.apache.carbondata.core.metadata.blocklet.BlockletInfo; @@ -64,8 +63,8 @@ import org.apache.carbondata.core.scan.filter.FilterUtil; import org.apache.carbondata.core.scan.filter.SingleTableProvider; import org.apache.carbondata.core.scan.filter.TableProvider; -import org.apache.carbondata.core.scan.model.QueryDimension; -import org.apache.carbondata.core.scan.model.QueryMeasure; +import org.apache.carbondata.core.scan.model.ProjectionDimension; +import org.apache.carbondata.core.scan.model.ProjectionMeasure; import org.apache.carbondata.core.scan.model.QueryModel; import org.apache.carbondata.core.stats.QueryStatistic; import org.apache.carbondata.core.stats.QueryStatisticsConstants; @@ -121,7 +120,6 @@ protected void initQuery(QueryModel queryModel) throws IOException { queryProperties.queryStatisticsRecorder = CarbonTimeStatisticsFactory.createExecutorRecorder(queryModel.getQueryId()); queryModel.setStatisticsRecorder(queryProperties.queryStatisticsRecorder); - QueryUtil.resolveQueryModel(queryModel); QueryStatistic queryStatistic = new QueryStatistic(); // sort the block info // so block will be loaded in sorted order this will be required for @@ -168,12 +166,12 @@ protected void initQuery(QueryModel queryModel) throws IOException { .addStatistics(QueryStatisticsConstants.LOAD_BLOCKS_EXECUTOR, System.currentTimeMillis()); queryProperties.queryStatisticsRecorder.recordStatistics(queryStatistic); // calculating the total number of aggeragted columns - int measureCount = queryModel.getQueryMeasures().size(); + int measureCount = queryModel.getProjectionMeasures().size(); int currentIndex = 0; DataType[] dataTypes = new DataType[measureCount]; - for (QueryMeasure carbonMeasure : queryModel.getQueryMeasures()) { + for (ProjectionMeasure carbonMeasure : queryModel.getProjectionMeasures()) { // adding the data type and aggregation type of all the measure this // can be used // to select the aggregator @@ -198,9 +196,11 @@ protected void initQuery(QueryModel queryModel) throws IOException { queryStatistic = new QueryStatistic(); // dictionary column unique column id to dictionary mapping // which will be used to get column actual data - queryProperties.columnToDictionayMapping = QueryUtil - .getDimensionDictionaryDetail(queryModel.getQueryDimension(), - queryProperties.complexFilterDimension, queryModel.getAbsoluteTableIdentifier(), + queryProperties.columnToDictionayMapping = + QueryUtil.getDimensionDictionaryDetail( + queryModel.getProjectionDimensions(), + queryProperties.complexFilterDimension, + queryModel.getAbsoluteTableIdentifier(), tableProvider); queryStatistic .addStatistics(QueryStatisticsConstants.LOAD_DICTIONARY, System.currentTimeMillis()); @@ -263,8 +263,8 @@ protected List getBlockExecutionInfos(QueryModel queryModel) // and query will be executed based on that infos for (int i = 0; i < queryProperties.dataBlocks.size(); i++) { AbstractIndex abstractIndex = queryProperties.dataBlocks.get(i); - BlockletDataRefNodeWrapper dataRefNode = - (BlockletDataRefNodeWrapper) abstractIndex.getDataRefNode(); + BlockletDataRefNode dataRefNode = + (BlockletDataRefNode) abstractIndex.getDataRefNode(); blockExecutionInfoList.add(getBlockExecutionInfoForBlock(queryModel, abstractIndex, dataRefNode.getBlockInfos().get(0).getBlockletInfos().getStartBlockletNumber(), dataRefNode.numberOfNodes(), dataRefNode.getBlockInfos().get(0).getFilePath(), @@ -289,47 +289,45 @@ protected List getBlockExecutionInfos(QueryModel queryModel) * @return block execution info * @throws QueryExecutionException any failure during block info creation */ - protected BlockExecutionInfo getBlockExecutionInfoForBlock(QueryModel queryModel, + private BlockExecutionInfo getBlockExecutionInfoForBlock(QueryModel queryModel, AbstractIndex blockIndex, int startBlockletIndex, int numberOfBlockletToScan, String filePath, String[] deleteDeltaFiles, String segmentId) throws QueryExecutionException { BlockExecutionInfo blockExecutionInfo = new BlockExecutionInfo(); SegmentProperties segmentProperties = blockIndex.getSegmentProperties(); List tableBlockDimensions = segmentProperties.getDimensions(); - KeyGenerator blockKeyGenerator = segmentProperties.getDimensionKeyGenerator(); // below is to get only those dimension in query which is present in the // table block - List currentBlockQueryDimensions = RestructureUtil + List projectDimensions = RestructureUtil .createDimensionInfoAndGetCurrentBlockQueryDimension(blockExecutionInfo, - queryModel.getQueryDimension(), tableBlockDimensions, - segmentProperties.getComplexDimensions(), queryModel.getQueryMeasures().size()); + queryModel.getProjectionDimensions(), tableBlockDimensions, + segmentProperties.getComplexDimensions(), queryModel.getProjectionMeasures().size()); blockExecutionInfo.setBlockId( CarbonUtil.getBlockId(queryModel.getAbsoluteTableIdentifier(), filePath, segmentId)); blockExecutionInfo.setDeleteDeltaFilePath(deleteDeltaFiles); blockExecutionInfo.setStartBlockletIndex(startBlockletIndex); blockExecutionInfo.setNumberOfBlockletToScan(numberOfBlockletToScan); - blockExecutionInfo.setQueryDimensions(currentBlockQueryDimensions - .toArray(new QueryDimension[currentBlockQueryDimensions.size()])); + blockExecutionInfo.setProjectionDimensions(projectDimensions + .toArray(new ProjectionDimension[projectDimensions.size()])); // get measures present in the current block - List currentBlockQueryMeasures = + List currentBlockQueryMeasures = getCurrentBlockQueryMeasures(blockExecutionInfo, queryModel, blockIndex); - blockExecutionInfo.setQueryMeasures( - currentBlockQueryMeasures.toArray(new QueryMeasure[currentBlockQueryMeasures.size()])); + blockExecutionInfo.setProjectionMeasures( + currentBlockQueryMeasures.toArray(new ProjectionMeasure[currentBlockQueryMeasures.size()])); blockExecutionInfo.setDataBlock(blockIndex); - blockExecutionInfo.setBlockKeyGenerator(blockKeyGenerator); // setting whether raw record query or not blockExecutionInfo.setRawRecordDetailQuery(queryModel.isForcedDetailRawQuery()); // total number dimension blockExecutionInfo - .setTotalNumberDimensionBlock(segmentProperties.getDimensionOrdinalToBlockMapping().size()); + .setTotalNumberDimensionToRead( + segmentProperties.getDimensionOrdinalToChunkMapping().size()); blockExecutionInfo.setPrefetchBlocklet(!queryModel.isReadPageByPage()); blockExecutionInfo - .setTotalNumberOfMeasureBlock(segmentProperties.getMeasuresOrdinalToBlockMapping().size()); - blockExecutionInfo.setAbsoluteTableIdentifier(queryModel.getAbsoluteTableIdentifier()); + .setTotalNumberOfMeasureToRead(segmentProperties.getMeasuresOrdinalToChunkMapping().size()); blockExecutionInfo.setComplexDimensionInfoMap(QueryUtil - .getComplexDimensionsMap(currentBlockQueryDimensions, - segmentProperties.getDimensionOrdinalToBlockMapping(), + .getComplexDimensionsMap(projectDimensions, + segmentProperties.getDimensionOrdinalToChunkMapping(), segmentProperties.getEachComplexDimColumnValueSize(), queryProperties.columnToDictionayMapping, queryProperties.complexFilterDimension)); IndexKey startIndexKey = null; @@ -363,45 +361,46 @@ protected BlockExecutionInfo getBlockExecutionInfoForBlock(QueryModel queryModel // create a list of filter dimensions present in the current block Set currentBlockFilterDimensions = getCurrentBlockFilterDimensions(queryProperties.complexFilterDimension, segmentProperties); - int[] dimensionsBlockIndexes = QueryUtil.getDimensionsBlockIndexes(currentBlockQueryDimensions, - segmentProperties.getDimensionOrdinalToBlockMapping(), expressionDimensions, - currentBlockFilterDimensions, allProjectionListDimensionIdexes); + int[] dimensionChunkIndexes = QueryUtil.getDimensionChunkIndexes( + projectDimensions, segmentProperties.getDimensionOrdinalToChunkMapping(), + expressionDimensions, currentBlockFilterDimensions, allProjectionListDimensionIdexes); int numberOfColumnToBeReadInOneIO = Integer.parseInt(CarbonProperties.getInstance() .getProperty(CarbonV3DataFormatConstants.NUMBER_OF_COLUMN_TO_READ_IN_IO, CarbonV3DataFormatConstants.NUMBER_OF_COLUMN_TO_READ_IN_IO_DEFAULTVALUE)); - if (dimensionsBlockIndexes.length > 0) { - numberOfElementToConsider = dimensionsBlockIndexes[dimensionsBlockIndexes.length - 1] + if (dimensionChunkIndexes.length > 0) { + numberOfElementToConsider = dimensionChunkIndexes[dimensionChunkIndexes.length - 1] == segmentProperties.getBlockTodimensionOrdinalMapping().size() - 1 ? - dimensionsBlockIndexes.length - 1 : - dimensionsBlockIndexes.length; - blockExecutionInfo.setAllSelectedDimensionBlocksIndexes(CarbonUtil - .getRangeIndex(dimensionsBlockIndexes, numberOfElementToConsider, + dimensionChunkIndexes.length - 1 : + dimensionChunkIndexes.length; + blockExecutionInfo.setAllSelectedDimensionColumnIndexRange( + CarbonUtil.getRangeIndex(dimensionChunkIndexes, numberOfElementToConsider, numberOfColumnToBeReadInOneIO)); } else { - blockExecutionInfo.setAllSelectedDimensionBlocksIndexes(new int[0][0]); + blockExecutionInfo.setAllSelectedDimensionColumnIndexRange(new int[0][0]); } // get the list of updated filter measures present in the current block - Set currentBlockFilterMeasures = + Set filterMeasures = getCurrentBlockFilterMeasures(queryProperties.filterMeasures, segmentProperties); // list of measures to be projected List allProjectionListMeasureIndexes = new ArrayList<>(); - int[] measureBlockIndexes = QueryUtil - .getMeasureBlockIndexes(currentBlockQueryMeasures, expressionMeasures, - segmentProperties.getMeasuresOrdinalToBlockMapping(), currentBlockFilterMeasures, - allProjectionListMeasureIndexes); - if (measureBlockIndexes.length > 0) { + int[] measureChunkIndexes = QueryUtil.getMeasureChunkIndexes( + currentBlockQueryMeasures, expressionMeasures, + segmentProperties.getMeasuresOrdinalToChunkMapping(), filterMeasures, + allProjectionListMeasureIndexes); + if (measureChunkIndexes.length > 0) { - numberOfElementToConsider = measureBlockIndexes[measureBlockIndexes.length - 1] + numberOfElementToConsider = measureChunkIndexes[measureChunkIndexes.length - 1] == segmentProperties.getMeasures().size() - 1 ? - measureBlockIndexes.length - 1 : - measureBlockIndexes.length; + measureChunkIndexes.length - 1 : + measureChunkIndexes.length; // setting all the measure chunk indexes to be read from file - blockExecutionInfo.setAllSelectedMeasureBlocksIndexes(CarbonUtil - .getRangeIndex(measureBlockIndexes, numberOfElementToConsider, + blockExecutionInfo.setAllSelectedMeasureIndexRange( + CarbonUtil.getRangeIndex( + measureChunkIndexes, numberOfElementToConsider, numberOfColumnToBeReadInOneIO)); } else { - blockExecutionInfo.setAllSelectedMeasureBlocksIndexes(new int[0][0]); + blockExecutionInfo.setAllSelectedMeasureIndexRange(new int[0][0]); } // setting the indexes of list of dimension in projection list blockExecutionInfo.setProjectionListDimensionIndexes(ArrayUtils.toPrimitive( @@ -413,27 +412,27 @@ protected BlockExecutionInfo getBlockExecutionInfoForBlock(QueryModel queryModel .toArray(new Integer[allProjectionListMeasureIndexes.size()]))); // setting the size of fixed key column (dictionary column) blockExecutionInfo - .setFixedLengthKeySize(getKeySize(currentBlockQueryDimensions, segmentProperties)); - Set dictionaryColumnBlockIndex = new HashSet(); - List noDictionaryColumnBlockIndex = new ArrayList(); + .setFixedLengthKeySize(getKeySize(projectDimensions, segmentProperties)); + Set dictionaryColumnChunkIndex = new HashSet(); + List noDictionaryColumnChunkIndex = new ArrayList(); // get the block index to be read from file for query dimension // for both dictionary columns and no dictionary columns - QueryUtil.fillQueryDimensionsBlockIndexes(currentBlockQueryDimensions, - segmentProperties.getDimensionOrdinalToBlockMapping(), dictionaryColumnBlockIndex, - noDictionaryColumnBlockIndex); - int[] queryDictionaryColumnBlockIndexes = ArrayUtils.toPrimitive( - dictionaryColumnBlockIndex.toArray(new Integer[dictionaryColumnBlockIndex.size()])); + QueryUtil.fillQueryDimensionChunkIndexes(projectDimensions, + segmentProperties.getDimensionOrdinalToChunkMapping(), dictionaryColumnChunkIndex, + noDictionaryColumnChunkIndex); + int[] queryDictionaryColumnChunkIndexes = ArrayUtils.toPrimitive( + dictionaryColumnChunkIndex.toArray(new Integer[dictionaryColumnChunkIndex.size()])); // need to sort the dictionary column as for all dimension // column key will be filled based on key order - Arrays.sort(queryDictionaryColumnBlockIndexes); - blockExecutionInfo.setDictionaryColumnBlockIndex(queryDictionaryColumnBlockIndexes); + Arrays.sort(queryDictionaryColumnChunkIndexes); + blockExecutionInfo.setDictionaryColumnChunkIndex(queryDictionaryColumnChunkIndexes); // setting the no dictionary column block indexes - blockExecutionInfo.setNoDictionaryBlockIndexes(ArrayUtils.toPrimitive( - noDictionaryColumnBlockIndex.toArray(new Integer[noDictionaryColumnBlockIndex.size()]))); + blockExecutionInfo.setNoDictionaryColumnChunkIndexes(ArrayUtils.toPrimitive( + noDictionaryColumnChunkIndex.toArray(new Integer[noDictionaryColumnChunkIndex.size()]))); // setting each column value size blockExecutionInfo.setEachColumnValueSize(segmentProperties.getEachDimColumnValueSize()); blockExecutionInfo.setComplexColumnParentBlockIndexes( - getComplexDimensionParentBlockIndexes(currentBlockQueryDimensions)); + getComplexDimensionParentBlockIndexes(projectDimensions)); blockExecutionInfo.setVectorBatchCollector(queryModel.isVectorReader()); try { // to set column group and its key structure info which will be used @@ -441,15 +440,15 @@ protected BlockExecutionInfo getBlockExecutionInfoForBlock(QueryModel queryModel // for getting the column group column data in case of final row // and in case of dimension aggregation blockExecutionInfo.setColumnGroupToKeyStructureInfo( - QueryUtil.getColumnGroupKeyStructureInfo(currentBlockQueryDimensions, segmentProperties)); + QueryUtil.getColumnGroupKeyStructureInfo(projectDimensions, segmentProperties)); } catch (KeyGenException e) { throw new QueryExecutionException(e); } // set actual query dimensions and measures. It may differ in case of restructure scenarios - blockExecutionInfo.setActualQueryDimensions(queryModel.getQueryDimension() - .toArray(new QueryDimension[queryModel.getQueryDimension().size()])); - blockExecutionInfo.setActualQueryMeasures(queryModel.getQueryMeasures() - .toArray(new QueryMeasure[queryModel.getQueryMeasures().size()])); + blockExecutionInfo.setActualQueryDimensions(queryModel.getProjectionDimensions() + .toArray(new ProjectionDimension[queryModel.getProjectionDimensions().size()])); + blockExecutionInfo.setActualQueryMeasures(queryModel.getProjectionMeasures() + .toArray(new ProjectionMeasure[queryModel.getProjectionMeasures().size()])); DataTypeUtil.setDataTypeConverter(queryModel.getConverter()); return blockExecutionInfo; } @@ -464,7 +463,7 @@ protected BlockExecutionInfo getBlockExecutionInfoForBlock(QueryModel queryModel * @param blockMetadataInfo block metadata info * @return key size */ - private int getKeySize(List queryDimension, + private int getKeySize(List queryDimension, SegmentProperties blockMetadataInfo) { // add the dimension block ordinal for each dictionary column // existing in the current block dimensions. Set is used because in case of column groups @@ -475,12 +474,11 @@ private int getKeySize(List queryDimension, while (counter < queryDimension.size()) { if (queryDimension.get(counter).getDimension().getNumberOfChild() > 0) { counter += queryDimension.get(counter).getDimension().getNumberOfChild(); - continue; } else if (!CarbonUtil.hasEncoding(queryDimension.get(counter).getDimension().getEncoder(), Encoding.DICTIONARY)) { counter++; } else { - fixedLengthDimensionOrdinal.add(blockMetadataInfo.getDimensionOrdinalToBlockMapping() + fixedLengthDimensionOrdinal.add(blockMetadataInfo.getDimensionOrdinalToChunkMapping() .get(queryDimension.get(counter).getDimension().getOrdinal())); counter++; } @@ -502,26 +500,26 @@ private int getKeySize(List queryDimension, /** * Below method will be used to get the measures present in the current block * - * @param blockExecutionInfo + * @param executionInfo * @param queryModel query model * @param tableBlock table block * @return */ - private List getCurrentBlockQueryMeasures(BlockExecutionInfo blockExecutionInfo, + private List getCurrentBlockQueryMeasures(BlockExecutionInfo executionInfo, QueryModel queryModel, AbstractIndex tableBlock) throws QueryExecutionException { // getting the measure info which will be used while filling up measure data - List updatedQueryMeasures = RestructureUtil - .createMeasureInfoAndGetCurrentBlockQueryMeasures(blockExecutionInfo, - queryModel.getQueryMeasures(), tableBlock.getSegmentProperties().getMeasures()); + List updatedQueryMeasures = RestructureUtil + .createMeasureInfoAndGetCurrentBlockQueryMeasures(executionInfo, + queryModel.getProjectionMeasures(), tableBlock.getSegmentProperties().getMeasures()); // setting the measure aggregator for all aggregation function selected // in query - blockExecutionInfo.getMeasureInfo().setMeasureDataTypes(queryProperties.measureDataTypes); + executionInfo.getMeasureInfo().setMeasureDataTypes(queryProperties.measureDataTypes); return updatedQueryMeasures; } - private int[] getComplexDimensionParentBlockIndexes(List queryDimensions) { + private int[] getComplexDimensionParentBlockIndexes(List queryDimensions) { List parentBlockIndexList = new ArrayList(); - for (QueryDimension queryDimension : queryDimensions) { + for (ProjectionDimension queryDimension : queryDimensions) { if (queryDimension.getDimension().getDataType().isComplexType()) { parentBlockIndexList.add(queryDimension.getDimension().getOrdinal()); } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/DetailQueryExecutor.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/DetailQueryExecutor.java index 931b1ab93f9..93d696b6fa3 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/DetailQueryExecutor.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/DetailQueryExecutor.java @@ -23,7 +23,7 @@ import org.apache.carbondata.core.scan.executor.exception.QueryExecutionException; import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo; import org.apache.carbondata.core.scan.model.QueryModel; -import org.apache.carbondata.core.scan.result.BatchResult; +import org.apache.carbondata.core.scan.result.RowBatch; import org.apache.carbondata.core.scan.result.iterator.DetailQueryResultIterator; /** @@ -31,10 +31,10 @@ * For executing the detail query it will pass all the block execution * info to detail query result iterator and iterator will be returned */ -public class DetailQueryExecutor extends AbstractQueryExecutor { +public class DetailQueryExecutor extends AbstractQueryExecutor { @Override - public CarbonIterator execute(QueryModel queryModel) + public CarbonIterator execute(QueryModel queryModel) throws QueryExecutionException, IOException { List blockExecutionInfoList = getBlockExecutionInfos(queryModel); this.queryIterator = new DetailQueryResultIterator( diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/infos/BlockExecutionInfo.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/infos/BlockExecutionInfo.java index 65f28381937..06adf8f4e14 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/executor/infos/BlockExecutionInfo.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/infos/BlockExecutionInfo.java @@ -21,13 +21,11 @@ import org.apache.carbondata.core.datastore.DataRefNode; import org.apache.carbondata.core.datastore.IndexKey; import org.apache.carbondata.core.datastore.block.AbstractIndex; -import org.apache.carbondata.core.keygenerator.KeyGenerator; -import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier; import org.apache.carbondata.core.mutate.DeleteDeltaVo; import org.apache.carbondata.core.scan.filter.GenericQueryType; import org.apache.carbondata.core.scan.filter.executer.FilterExecuter; -import org.apache.carbondata.core.scan.model.QueryDimension; -import org.apache.carbondata.core.scan.model.QueryMeasure; +import org.apache.carbondata.core.scan.model.ProjectionDimension; +import org.apache.carbondata.core.scan.model.ProjectionMeasure; /** * Below class will have all the properties which needed during query execution @@ -40,14 +38,6 @@ public class BlockExecutionInfo { */ private AbstractIndex blockIndex; - /** - * each segment key size can be different and in that case we need to update - * the fixed key with latest segment key generator. so this property will - * tell whether this is required or not if key size is same then it is not - * required - */ - private boolean isFixedKeyUpdateRequired; - /** * below to store all the information required for measures during query * execution @@ -73,22 +63,22 @@ public class BlockExecutionInfo { /** * total number of dimension in block */ - private int totalNumberDimensionBlock; + private int totalNumberDimensionToRead; /** * total number of measure in block */ - private int totalNumberOfMeasureBlock; + private int totalNumberOfMeasureToRead; /** * will be used to read the dimension block from file */ - private int[][] allSelectedDimensionBlocksIndexes; + private int[][] allSelectedDimensionColumnIndexRange; /** * will be used to read the measure block from file */ - private int[][] allSelectedMeasureBlocksIndexes; + private int[][] allSelectedMeasureIndexRange; /** * list of dimension present in the projection @@ -118,16 +108,11 @@ public class BlockExecutionInfo { /** * dictionary column block indexes based on query */ - private int[] dictionaryColumnBlockIndex; + private int[] dictionaryColumnChunkIndex; /** * no dictionary column block indexes in based on the query order */ - private int[] noDictionaryBlockIndexes; - - /** - * key generator used for generating the table block fixed length key - */ - private KeyGenerator blockKeyGenerator; + private int[] noDictionaryColumnChunkIndexes; /** * each column value size @@ -180,23 +165,23 @@ public AbstractIndex getDataBlock() { * list of dimension present in the current block. This will be * different in case of restructured block */ - private QueryDimension[] queryDimensions; + private ProjectionDimension[] projectionDimensions; /** * list of dimension selected for in query */ - private QueryDimension[] actualQueryDimensions; + private ProjectionDimension[] actualQueryDimensions; /** * list of dimension present in the current block. This will be * different in case of restructured block */ - private QueryMeasure[] queryMeasures; + private ProjectionMeasure[] projectionMeasures; /** * list of measure selected in query */ - private QueryMeasure[] actualQueryMeasures; + private ProjectionMeasure[] actualQueryMeasures; /** * variable to maintain dimension existence and default value info @@ -213,11 +198,6 @@ public AbstractIndex getDataBlock() { */ private boolean isRestructuredBlock; - /** - * absolute table identifier - */ - private AbsoluteTableIdentifier absoluteTableIdentifier; - /** * delete delta file path */ @@ -229,13 +209,6 @@ public AbstractIndex getDataBlock() { private boolean prefetchBlocklet = true; private Map deletedRecordsMap; - public AbsoluteTableIdentifier getAbsoluteTableIdentifier() { - return absoluteTableIdentifier; - } - - public void setAbsoluteTableIdentifier(AbsoluteTableIdentifier absoluteTableIdentifier) { - this.absoluteTableIdentifier = absoluteTableIdentifier; - } /** * @param blockIndex the tableBlock to set @@ -244,20 +217,6 @@ public void setDataBlock(AbstractIndex blockIndex) { this.blockIndex = blockIndex; } - /** - * @return the isFixedKeyUpdateRequired - */ - public boolean isFixedKeyUpdateRequired() { - return isFixedKeyUpdateRequired; - } - - /** - * @param isFixedKeyUpdateRequired the isFixedKeyUpdateRequired to set - */ - public void setFixedKeyUpdateRequired(boolean isFixedKeyUpdateRequired) { - this.isFixedKeyUpdateRequired = isFixedKeyUpdateRequired; - } - /** * @return the aggregatorInfos */ @@ -301,59 +260,60 @@ public void setEndKey(IndexKey endKey) { } /** - * @return the totalNumberDimensionBlock + * @return the totalNumberDimensionToRead */ - public int getTotalNumberDimensionBlock() { - return totalNumberDimensionBlock; + public int getTotalNumberDimensionToRead() { + return totalNumberDimensionToRead; } /** - * @param totalNumberDimensionBlock the totalNumberDimensionBlock to set + * @param totalNumberDimensionToRead the totalNumberDimensionToRead to set */ - public void setTotalNumberDimensionBlock(int totalNumberDimensionBlock) { - this.totalNumberDimensionBlock = totalNumberDimensionBlock; + public void setTotalNumberDimensionToRead(int totalNumberDimensionToRead) { + this.totalNumberDimensionToRead = totalNumberDimensionToRead; } /** - * @return the totalNumberOfMeasureBlock + * @return the totalNumberOfMeasureToRead */ - public int getTotalNumberOfMeasureBlock() { - return totalNumberOfMeasureBlock; + public int getTotalNumberOfMeasureToRead() { + return totalNumberOfMeasureToRead; } /** - * @param totalNumberOfMeasureBlock the totalNumberOfMeasureBlock to set + * @param totalNumberOfMeasureToRead the totalNumberOfMeasureToRead to set */ - public void setTotalNumberOfMeasureBlock(int totalNumberOfMeasureBlock) { - this.totalNumberOfMeasureBlock = totalNumberOfMeasureBlock; + public void setTotalNumberOfMeasureToRead(int totalNumberOfMeasureToRead) { + this.totalNumberOfMeasureToRead = totalNumberOfMeasureToRead; } /** - * @return the allSelectedDimensionBlocksIndexes + * @return the allSelectedDimensionColumnIndexRange */ - public int[][] getAllSelectedDimensionBlocksIndexes() { - return allSelectedDimensionBlocksIndexes; + public int[][] getAllSelectedDimensionColumnIndexRange() { + return allSelectedDimensionColumnIndexRange; } /** - * @param allSelectedDimensionBlocksIndexes the allSelectedDimensionBlocksIndexes to set + * @param allSelectedDimensionColumnIndexRange the allSelectedDimensionColumnIndexRange to set */ - public void setAllSelectedDimensionBlocksIndexes(int[][] allSelectedDimensionBlocksIndexes) { - this.allSelectedDimensionBlocksIndexes = allSelectedDimensionBlocksIndexes; + public void setAllSelectedDimensionColumnIndexRange(int[][] allSelectedDimensionColumnIndexRange) + { + this.allSelectedDimensionColumnIndexRange = allSelectedDimensionColumnIndexRange; } /** - * @return the allSelectedMeasureBlocksIndexes + * @return the allSelectedMeasureIndexRange */ - public int[][] getAllSelectedMeasureBlocksIndexes() { - return allSelectedMeasureBlocksIndexes; + public int[][] getAllSelectedMeasureIndexRange() { + return allSelectedMeasureIndexRange; } /** - * @param allSelectedMeasureBlocksIndexes the allSelectedMeasureBlocksIndexes to set + * @param allSelectedMeasureIndexRange the allSelectedMeasureIndexRange to set */ - public void setAllSelectedMeasureBlocksIndexes(int[][] allSelectedMeasureBlocksIndexes) { - this.allSelectedMeasureBlocksIndexes = allSelectedMeasureBlocksIndexes; + public void setAllSelectedMeasureIndexRange(int[][] allSelectedMeasureIndexRange) { + this.allSelectedMeasureIndexRange = allSelectedMeasureIndexRange; } /** @@ -412,20 +372,6 @@ public void setFilterExecuterTree(FilterExecuter filterExecuterTree) { this.filterExecuterTree = filterExecuterTree; } - /** - * @return the tableBlockKeyGenerator - */ - public KeyGenerator getBlockKeyGenerator() { - return blockKeyGenerator; - } - - /** - * @param tableBlockKeyGenerator the tableBlockKeyGenerator to set - */ - public void setBlockKeyGenerator(KeyGenerator tableBlockKeyGenerator) { - this.blockKeyGenerator = tableBlockKeyGenerator; - } - /** * @return the eachColumnValueSize */ @@ -441,31 +387,31 @@ public void setEachColumnValueSize(int[] eachColumnValueSize) { } /** - * @return the dictionaryColumnBlockIndex + * @return the dictionaryColumnChunkIndex */ - public int[] getDictionaryColumnBlockIndex() { - return dictionaryColumnBlockIndex; + public int[] getDictionaryColumnChunkIndex() { + return dictionaryColumnChunkIndex; } /** - * @param dictionaryColumnBlockIndex the dictionaryColumnBlockIndex to set + * @param dictionaryColumnChunkIndex the dictionaryColumnChunkIndex to set */ - public void setDictionaryColumnBlockIndex(int[] dictionaryColumnBlockIndex) { - this.dictionaryColumnBlockIndex = dictionaryColumnBlockIndex; + public void setDictionaryColumnChunkIndex(int[] dictionaryColumnChunkIndex) { + this.dictionaryColumnChunkIndex = dictionaryColumnChunkIndex; } /** - * @return the noDictionaryBlockIndexes + * @return the noDictionaryColumnChunkIndexes */ - public int[] getNoDictionaryBlockIndexes() { - return noDictionaryBlockIndexes; + public int[] getNoDictionaryColumnChunkIndexes() { + return noDictionaryColumnChunkIndexes; } /** - * @param noDictionaryBlockIndexes the noDictionaryBlockIndexes to set + * @param noDictionaryColumnChunkIndexes the noDictionaryColumnChunkIndexes to set */ - public void setNoDictionaryBlockIndexes(int[] noDictionaryBlockIndexes) { - this.noDictionaryBlockIndexes = noDictionaryBlockIndexes; + public void setNoDictionaryColumnChunkIndexes(int[] noDictionaryColumnChunkIndexes) { + this.noDictionaryColumnChunkIndexes = noDictionaryColumnChunkIndexes; } /** @@ -519,20 +465,20 @@ public void setComplexColumnParentBlockIndexes(int[] complexColumnParentBlockInd this.complexColumnParentBlockIndexes = complexColumnParentBlockIndexes; } - public QueryDimension[] getQueryDimensions() { - return queryDimensions; + public ProjectionDimension[] getProjectionDimensions() { + return projectionDimensions; } - public void setQueryDimensions(QueryDimension[] queryDimensions) { - this.queryDimensions = queryDimensions; + public void setProjectionDimensions(ProjectionDimension[] projectionDimensions) { + this.projectionDimensions = projectionDimensions; } - public QueryMeasure[] getQueryMeasures() { - return queryMeasures; + public ProjectionMeasure[] getProjectionMeasures() { + return projectionMeasures; } - public void setQueryMeasures(QueryMeasure[] queryMeasures) { - this.queryMeasures = queryMeasures; + public void setProjectionMeasures(ProjectionMeasure[] projectionMeasures) { + this.projectionMeasures = projectionMeasures; } /** @@ -579,7 +525,8 @@ public void setVectorBatchCollector(boolean vectorBatchCollector) { this.vectorBatchCollector = vectorBatchCollector; } - public String getBlockId() { + // Return file name and path, like Part0/Segment_0/part-0-0_batchno0-0-1517155583332.carbondata + public String getBlockIdString() { return blockId; } @@ -603,19 +550,19 @@ public void setDimensionInfo(DimensionInfo dimensionInfo) { this.dimensionInfo = dimensionInfo; } - public QueryDimension[] getActualQueryDimensions() { + public ProjectionDimension[] getActualQueryDimensions() { return actualQueryDimensions; } - public void setActualQueryDimensions(QueryDimension[] actualQueryDimensions) { + public void setActualQueryDimensions(ProjectionDimension[] actualQueryDimensions) { this.actualQueryDimensions = actualQueryDimensions; } - public QueryMeasure[] getActualQueryMeasures() { + public ProjectionMeasure[] getActualQueryMeasures() { return actualQueryMeasures; } - public void setActualQueryMeasures(QueryMeasure[] actualQueryMeasures) { + public void setActualQueryMeasures(ProjectionMeasure[] actualQueryMeasures) { this.actualQueryMeasures = actualQueryMeasures; } 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 8eb0a0d0bb1..00cdfe79e6d 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 @@ -60,9 +60,8 @@ import org.apache.carbondata.core.scan.filter.TableProvider; import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo; -import org.apache.carbondata.core.scan.model.QueryDimension; -import org.apache.carbondata.core.scan.model.QueryMeasure; -import org.apache.carbondata.core.scan.model.QueryModel; +import org.apache.carbondata.core.scan.model.ProjectionDimension; +import org.apache.carbondata.core.scan.model.ProjectionMeasure; import org.apache.carbondata.core.util.CarbonUtil; import org.apache.carbondata.core.util.path.CarbonStorePath; import org.apache.carbondata.core.util.path.CarbonTablePath; @@ -83,7 +82,7 @@ public class QueryUtil { * @param keyGenerator key generator * @return masked key */ - public static int[] getMaskedByteRange(List queryDimensions, + public static int[] getMaskedByteRange(List queryDimensions, KeyGenerator keyGenerator) { Set byteRangeSet = new TreeSet(); int[] byteRange = null; @@ -167,7 +166,7 @@ public static byte[] getMaxKeyBasedOnOrinal(List keyOrdinalList, KeyGen * @return max key for dimension * @throws KeyGenException if any problem while generating the key */ - public static byte[] getMaxKeyBasedOnDimensions(List queryDimensions, + public static byte[] getMaxKeyBasedOnDimensions(List queryDimensions, KeyGenerator generator) throws KeyGenException { long[] max = new long[generator.getDimCount()]; Arrays.fill(max, 0L); @@ -209,46 +208,46 @@ public static int[] getMaskedByte(int keySize, int[] maskedKeyRanges) { * on query dimension * * @param queryDimensions query dimension - * @param dimensionOrdinalToBlockMapping mapping of dimension block in file to query dimension + * @param dimensionOrdinalToChunkMapping mapping of dimension block in file to query dimension * @return block index of file */ - public static int[] getDimensionsBlockIndexes(List queryDimensions, - Map dimensionOrdinalToBlockMapping, + public static int[] getDimensionChunkIndexes(List queryDimensions, + Map dimensionOrdinalToChunkMapping, List customAggregationDimension, Set filterDimensions, Set allProjectionListDimensionIndexes) { // using set as in row group columns will point to same block - Set dimensionBlockIndex = new HashSet(); + Set dimensionChunkIndex = new HashSet(); Set filterDimensionOrdinal = getFilterDimensionOrdinal(filterDimensions); - int blockIndex = 0; + int chunkIndex = 0; for (int i = 0; i < queryDimensions.size(); i++) { if (queryDimensions.get(i).getDimension().hasEncoding(Encoding.IMPLICIT)) { continue; } Integer dimensionOrdinal = queryDimensions.get(i).getDimension().getOrdinal(); - allProjectionListDimensionIndexes.add(dimensionOrdinalToBlockMapping.get(dimensionOrdinal)); + allProjectionListDimensionIndexes.add(dimensionOrdinalToChunkMapping.get(dimensionOrdinal)); if (queryDimensions.get(i).getDimension().getNumberOfChild() > 0) { addChildrenBlockIndex(allProjectionListDimensionIndexes, queryDimensions.get(i).getDimension()); } if (!filterDimensionOrdinal.contains(dimensionOrdinal)) { - blockIndex = dimensionOrdinalToBlockMapping.get(dimensionOrdinal); - dimensionBlockIndex.add(blockIndex); + chunkIndex = dimensionOrdinalToChunkMapping.get(dimensionOrdinal); + dimensionChunkIndex.add(chunkIndex); if (queryDimensions.get(i).getDimension().getNumberOfChild() > 0) { - addChildrenBlockIndex(dimensionBlockIndex, queryDimensions.get(i).getDimension()); + addChildrenBlockIndex(dimensionChunkIndex, queryDimensions.get(i).getDimension()); } } } for (int i = 0; i < customAggregationDimension.size(); i++) { - blockIndex = - dimensionOrdinalToBlockMapping.get(customAggregationDimension.get(i).getOrdinal()); + chunkIndex = + dimensionOrdinalToChunkMapping.get(customAggregationDimension.get(i).getOrdinal()); // not adding the children dimension as dimension aggregation // is not push down in case of complex dimension - dimensionBlockIndex.add(blockIndex); + dimensionChunkIndex.add(chunkIndex); } int[] dimensionIndex = ArrayUtils - .toPrimitive(dimensionBlockIndex.toArray(new Integer[dimensionBlockIndex.size()])); + .toPrimitive(dimensionChunkIndex.toArray(new Integer[dimensionChunkIndex.size()])); Arrays.sort(dimensionIndex); return dimensionIndex; } @@ -278,7 +277,7 @@ private static void addChildrenBlockIndex(Set blockIndexes, CarbonDimen * @throws IOException */ public static Map getDimensionDictionaryDetail( - List queryDimensions, Set filterComplexDimensions, + List queryDimensions, Set filterComplexDimensions, AbsoluteTableIdentifier absoluteTableIdentifier, TableProvider tableProvider) throws IOException { // to store complex dimension and its child id unique column id list, this is required as @@ -433,23 +432,23 @@ public static AbsoluteTableIdentifier getTableIdentifierForColumn(CarbonDimensio * @param ordinalToBlockIndexMapping measure ordinal to block mapping * @return block indexes */ - public static int[] getMeasureBlockIndexes(List queryMeasures, + public static int[] getMeasureChunkIndexes(List queryMeasures, List expressionMeasure, Map ordinalToBlockIndexMapping, Set filterMeasures, List allProjectionListMeasureIdexes) { - Set measureBlockIndex = new HashSet(); + Set measureChunkIndex = new HashSet(); Set filterMeasureOrdinal = getFilterMeasureOrdinal(filterMeasures); for (int i = 0; i < queryMeasures.size(); i++) { Integer measureOrdinal = queryMeasures.get(i).getMeasure().getOrdinal(); allProjectionListMeasureIdexes.add(measureOrdinal); if (!filterMeasureOrdinal.contains(measureOrdinal)) { - measureBlockIndex.add(ordinalToBlockIndexMapping.get(measureOrdinal)); + measureChunkIndex.add(ordinalToBlockIndexMapping.get(measureOrdinal)); } } for (int i = 0; i < expressionMeasure.size(); i++) { - measureBlockIndex.add(ordinalToBlockIndexMapping.get(expressionMeasure.get(i).getOrdinal())); + measureChunkIndex.add(ordinalToBlockIndexMapping.get(expressionMeasure.get(i).getOrdinal())); } int[] measureIndexes = - ArrayUtils.toPrimitive(measureBlockIndex.toArray(new Integer[measureBlockIndex.size()])); + ArrayUtils.toPrimitive(measureChunkIndex.toArray(new Integer[measureChunkIndex.size()])); Arrays.sort(measureIndexes); return measureIndexes; } @@ -462,8 +461,8 @@ public static int[] getMeasureBlockIndexes(List queryMeasures, * @param queryDimensions query dimension * @return sort dimension indexes */ - public static byte[] getSortDimensionIndexes(List sortedDimensions, - List queryDimensions) { + public static byte[] getSortDimensionIndexes(List sortedDimensions, + List queryDimensions) { byte[] sortedDims = new byte[queryDimensions.size()]; int indexOf = 0; for (int i = 0; i < sortedDims.length; i++) { @@ -485,7 +484,7 @@ public static byte[] getSortDimensionIndexes(List sortedDimensio * @throws KeyGenException if problem while key generation */ public static Map getColumnGroupKeyStructureInfo( - List queryDimensions, SegmentProperties segmentProperties) + List queryDimensions, SegmentProperties segmentProperties) throws KeyGenException { Map rowGroupToItsRSInfo = new HashMap(); // get column group id and its ordinal mapping of column group @@ -521,7 +520,7 @@ public static Map getColumnGroupKeyStructureInfo( restructureInfos.setMdkeyQueryDimensionOrdinal(ArrayUtils .toPrimitive(mdKeyOrdinalForQuery.toArray(new Integer[mdKeyOrdinalForQuery.size()]))); rowGroupToItsRSInfo - .put(segmentProperties.getDimensionOrdinalToBlockMapping().get(ordinal.get(0)), + .put(segmentProperties.getDimensionOrdinalToChunkMapping().get(ordinal.get(0)), restructureInfos); } return rowGroupToItsRSInfo; @@ -552,16 +551,16 @@ public static boolean searchInArray(int[] data, int key) { * stored in bit level */ private static Map> getColumnGroupAndItsOrdinalMapping( - List origdimensions) { + List origDimensions) { - List dimensions = new ArrayList(origdimensions.size()); - dimensions.addAll(origdimensions); + List dimensions = new ArrayList<>(origDimensions.size()); + dimensions.addAll(origDimensions); /* * sort based on column group id */ - Collections.sort(dimensions, new Comparator() { + Collections.sort(dimensions, new Comparator() { - @Override public int compare(QueryDimension o1, QueryDimension o2) { + @Override public int compare(ProjectionDimension o1, ProjectionDimension o2) { return Integer .compare(o1.getDimension().columnGroupId(), o2.getDimension().columnGroupId()); } @@ -636,70 +635,26 @@ public static byte[] getMaskedKey(byte[] data, byte[] maxKey, int[] maskByteRang * dictionary column. This is done for specific purpose so that in one * iteration we will be able to fill both type dimension block indexes * - * @param queryDimensions dimension present in the query - * @param columnOrdinalToBlockIndexMapping column ordinal to block index mapping - * @param dictionaryDimensionBlockIndex list to store dictionary column block indexes - * @param noDictionaryDimensionBlockIndex list to store no dictionary block indexes + * @param projectDimensions dimension present in the query + * @param columnOrdinalToChunkIndexMapping column ordinal to block index mapping + * @param dictionaryDimensionChunkIndex list to store dictionary column block indexes + * @param noDictionaryDimensionChunkIndex list to store no dictionary block indexes */ - public static void fillQueryDimensionsBlockIndexes(List queryDimensions, - Map columnOrdinalToBlockIndexMapping, - Set dictionaryDimensionBlockIndex, List noDictionaryDimensionBlockIndex) { - for (QueryDimension queryDimension : queryDimensions) { + public static void fillQueryDimensionChunkIndexes( + List projectDimensions, + Map columnOrdinalToChunkIndexMapping, + Set dictionaryDimensionChunkIndex, + List noDictionaryDimensionChunkIndex) { + for (ProjectionDimension queryDimension : projectDimensions) { if (CarbonUtil.hasEncoding(queryDimension.getDimension().getEncoder(), Encoding.DICTIONARY) && queryDimension.getDimension().getNumberOfChild() == 0) { - dictionaryDimensionBlockIndex - .add(columnOrdinalToBlockIndexMapping.get(queryDimension.getDimension().getOrdinal())); + dictionaryDimensionChunkIndex + .add(columnOrdinalToChunkIndexMapping.get(queryDimension.getDimension().getOrdinal())); } else if ( !CarbonUtil.hasEncoding(queryDimension.getDimension().getEncoder(), Encoding.IMPLICIT) && queryDimension.getDimension().getNumberOfChild() == 0) { - noDictionaryDimensionBlockIndex - .add(columnOrdinalToBlockIndexMapping.get(queryDimension.getDimension().getOrdinal())); - } - } - } - - /** - * Below method will be used to resolve the query model - * resolve will be setting the actual dimension and measure object - * as from driver only column name will be passes to avoid the heavy object - * serialization - * - * @param queryModel query model - */ - public static void resolveQueryModel(QueryModel queryModel) { - CarbonMetadata.getInstance().addCarbonTable(queryModel.getTable()); - // TODO need to load the table from table identifier - CarbonTable carbonTable = queryModel.getTable(); - String tableName = - queryModel.getAbsoluteTableIdentifier().getCarbonTableIdentifier().getTableName(); - // resolve query dimension - for (QueryDimension queryDimension : queryModel.getQueryDimension()) { - queryDimension - .setDimension(carbonTable.getDimensionByName(tableName, queryDimension.getColumnName())); - } - // resolve query measure - for (QueryMeasure queryMeasure : queryModel.getQueryMeasures()) { - // in case of count start column name will be count * so - // first need to check any measure is present or not and as if measure - // if measure is present and if first measure is not a default - // measure than add measure otherwise - // than add first dimension as a measure - //as currently if measure is not present then - //we are adding default measure so first condition will - //never come false but if in future we can remove so not removing first if check - if (queryMeasure.getColumnName().equals("count(*)")) { - if (carbonTable.getMeasureByTableName(tableName).size() > 0 && !carbonTable - .getMeasureByTableName(tableName).get(0).getColName() - .equals(CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE)) { - queryMeasure.setMeasure(carbonTable.getMeasureByTableName(tableName).get(0)); - } else { - CarbonMeasure dummyMeasure = new CarbonMeasure( - carbonTable.getDimensionByTableName(tableName).get(0).getColumnSchema(), 0); - queryMeasure.setMeasure(dummyMeasure); - } - } else { - queryMeasure - .setMeasure(carbonTable.getMeasureByName(tableName, queryMeasure.getColumnName())); + noDictionaryDimensionChunkIndex + .add(columnOrdinalToChunkIndexMapping.get(queryDimension.getDimension().getOrdinal())); } } } @@ -759,11 +714,11 @@ public static int getColumnGroupId(SegmentProperties segmentProperties, int ordi * @return complex dimension and query type */ public static Map getComplexDimensionsMap( - List queryDimensions, Map dimensionToBlockIndexMap, + List queryDimensions, Map dimensionToBlockIndexMap, int[] eachComplexColumnValueSize, Map columnIdToDictionaryMap, Set filterDimensions) { Map complexTypeMap = new HashMap(); - for (QueryDimension dimension : queryDimensions) { + for (ProjectionDimension dimension : queryDimensions) { CarbonDimension actualDimension = dimension.getDimension(); if (actualDimension.getNumberOfChild() == 0) { continue; diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java index a0299860f92..2712cbc9680 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java @@ -34,8 +34,8 @@ import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo; import org.apache.carbondata.core.scan.executor.infos.DimensionInfo; import org.apache.carbondata.core.scan.executor.infos.MeasureInfo; -import org.apache.carbondata.core.scan.model.QueryDimension; -import org.apache.carbondata.core.scan.model.QueryMeasure; +import org.apache.carbondata.core.scan.model.ProjectionDimension; +import org.apache.carbondata.core.scan.model.ProjectionMeasure; import org.apache.carbondata.core.util.ByteUtil; import org.apache.carbondata.core.util.CarbonUtil; import org.apache.carbondata.core.util.DataTypeUtil; @@ -61,12 +61,12 @@ public class RestructureUtil { * @param tableComplexDimension * @return list of query dimension which is present in the table block */ - public static List createDimensionInfoAndGetCurrentBlockQueryDimension( - BlockExecutionInfo blockExecutionInfo, List queryDimensions, + public static List createDimensionInfoAndGetCurrentBlockQueryDimension( + BlockExecutionInfo blockExecutionInfo, List queryDimensions, List tableBlockDimensions, List tableComplexDimension, int measureCount) { - List presentDimension = - new ArrayList(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); + List presentDimension = + new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); boolean[] isDimensionExists = new boolean[queryDimensions.size()]; Object[] defaultValues = new Object[queryDimensions.size()]; // create dimension information instance @@ -76,7 +76,7 @@ public static List createDimensionInfoAndGetCurrentBlockQueryDim int newNoDictionaryColumnCount = 0; // selecting only those dimension which is present in the query int dimIndex = 0; - for (QueryDimension queryDimension : queryDimensions) { + for (ProjectionDimension queryDimension : queryDimensions) { if (queryDimension.getDimension().hasEncoding(Encoding.IMPLICIT)) { presentDimension.add(queryDimension); isDimensionExists[dimIndex] = true; @@ -85,15 +85,14 @@ public static List createDimensionInfoAndGetCurrentBlockQueryDim } else { for (CarbonDimension tableDimension : tableBlockDimensions) { if (tableDimension.getColumnId().equals(queryDimension.getDimension().getColumnId())) { - QueryDimension currentBlockDimension = new QueryDimension(tableDimension.getColName()); + ProjectionDimension currentBlockDimension = new ProjectionDimension(tableDimension); tableDimension.getColumnSchema() .setPrecision(queryDimension.getDimension().getColumnSchema().getPrecision()); tableDimension.getColumnSchema() .setScale(queryDimension.getDimension().getColumnSchema().getScale()); tableDimension.getColumnSchema() .setDefaultValue(queryDimension.getDimension().getDefaultValue()); - currentBlockDimension.setDimension(tableDimension); - currentBlockDimension.setQueryOrder(queryDimension.getQueryOrder()); + currentBlockDimension.setOrdinal(queryDimension.getOrdinal()); presentDimension.add(currentBlockDimension); isDimensionExists[dimIndex] = true; dimensionInfo.dataType[currentBlockDimension.getQueryOrder()] = @@ -108,11 +107,10 @@ public static List createDimensionInfoAndGetCurrentBlockQueryDim } for (CarbonDimension tableDimension : tableComplexDimension) { if (tableDimension.getColumnId().equals(queryDimension.getDimension().getColumnId())) { - QueryDimension currentBlockDimension = new QueryDimension(tableDimension.getColName()); + ProjectionDimension currentBlockDimension = new ProjectionDimension(tableDimension); // TODO: for complex dimension set scale and precision by traversing // the child dimensions - currentBlockDimension.setDimension(tableDimension); - currentBlockDimension.setQueryOrder(queryDimension.getQueryOrder()); + currentBlockDimension.setOrdinal(queryDimension.getOrdinal()); presentDimension.add(currentBlockDimension); isDimensionExists[dimIndex] = true; dimensionInfo.dataType[currentBlockDimension.getQueryOrder()] = @@ -249,39 +247,6 @@ private static boolean isDefaultValueNull(byte[] defaultValue) { return null == defaultValue; } - /** - * Below method is to add dimension children for complex type dimension as - * internally we are creating dimension column for each each complex - * dimension so when complex query dimension request will come in the query, - * we need to add its children as it is hidden from the user For example if - * complex dimension is of Array of String[2] so we are storing 3 dimension - * and when user will query for complex type i.e. array type we need to add - * its children and then we will read respective block and create a tuple - * based on all three dimension - * - * @param queryDimensions current query dimensions - * @param tableBlockDimensions dimensions which is present in the table block - * @return updated dimension(after adding complex type children) - */ - public static List addChildrenForComplexTypeDimension( - List queryDimensions, List tableBlockDimensions) { - List updatedQueryDimension = new ArrayList(); - int numberOfChildren = 0; - for (CarbonDimension queryDimension : queryDimensions) { - // if number of child is zero, then it is not a complex dimension - // so directly add it query dimension - if (queryDimension.getNumberOfChild() == 0) { - updatedQueryDimension.add(queryDimension); - } - // if number of child is more than 1 then add all its children - numberOfChildren = queryDimension.getOrdinal() + queryDimension.getNumberOfChild(); - for (int j = queryDimension.getOrdinal(); j < numberOfChildren; j++) { - updatedQueryDimension.add(tableBlockDimensions.get(j)); - } - } - return updatedQueryDimension; - } - /** * Method for computing measure default value based on the data type * @@ -375,30 +340,29 @@ public static Object getMeasureDefaultValueByType(ColumnSchema columnSchema, * @param currentBlockMeasures current block measures * @return measures present in the block */ - public static List createMeasureInfoAndGetCurrentBlockQueryMeasures( - BlockExecutionInfo blockExecutionInfo, List queryMeasures, + public static List createMeasureInfoAndGetCurrentBlockQueryMeasures( + BlockExecutionInfo blockExecutionInfo, List queryMeasures, List currentBlockMeasures) { MeasureInfo measureInfo = new MeasureInfo(); - List presentMeasure = new ArrayList<>(queryMeasures.size()); + List presentMeasure = new ArrayList<>(queryMeasures.size()); int numberOfMeasureInQuery = queryMeasures.size(); List measureOrdinalList = new ArrayList<>(numberOfMeasureInQuery); Object[] defaultValues = new Object[numberOfMeasureInQuery]; boolean[] measureExistsInCurrentBlock = new boolean[numberOfMeasureInQuery]; int index = 0; - for (QueryMeasure queryMeasure : queryMeasures) { + for (ProjectionMeasure queryMeasure : queryMeasures) { // if query measure exists in current dimension measures // then setting measure exists is true // otherwise adding a default value of a measure for (CarbonMeasure carbonMeasure : currentBlockMeasures) { if (carbonMeasure.getColumnId().equals(queryMeasure.getMeasure().getColumnId())) { - QueryMeasure currentBlockMeasure = new QueryMeasure(carbonMeasure.getColName()); + ProjectionMeasure currentBlockMeasure = new ProjectionMeasure(carbonMeasure); carbonMeasure.getColumnSchema().setDataType(queryMeasure.getMeasure().getDataType()); carbonMeasure.getColumnSchema().setPrecision(queryMeasure.getMeasure().getPrecision()); carbonMeasure.getColumnSchema().setScale(queryMeasure.getMeasure().getScale()); carbonMeasure.getColumnSchema() .setDefaultValue(queryMeasure.getMeasure().getDefaultValue()); - currentBlockMeasure.setMeasure(carbonMeasure); - currentBlockMeasure.setQueryOrder(queryMeasure.getQueryOrder()); + currentBlockMeasure.setOrdinal(queryMeasure.getOrdinal()); presentMeasure.add(currentBlockMeasure); measureOrdinalList.add(carbonMeasure.getOrdinal()); measureExistsInCurrentBlock[index] = true; diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/ColumnExpression.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/ColumnExpression.java index 981efb51ded..5cd2d345624 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/expression/ColumnExpression.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/ColumnExpression.java @@ -70,10 +70,6 @@ public String getColumnName() { return columnName; } - public void setColumnName(String columnName) { - this.columnName = columnName; - } - public boolean isDimension() { return isDimension; } @@ -102,10 +98,6 @@ public DataType getDataType() { return dataType; } - public void setDataType(DataType dataType) { - this.dataType = dataType; - } - @Override public ExpressionResult evaluate(RowIntf value) { return new ExpressionResult(dataType, (null == value ? null : value.getVal(colIndex))); } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/FilterModificationNode.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/FilterModificationNode.java index dad3975bf2b..8b060e1f13a 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/expression/FilterModificationNode.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/FilterModificationNode.java @@ -17,62 +17,29 @@ package org.apache.carbondata.core.scan.expression; -import org.apache.carbondata.core.metadata.datatype.DataType; import org.apache.carbondata.core.scan.filter.intf.ExpressionType; public class FilterModificationNode { private Expression currentExp; private Expression parentExp; private ExpressionType expType; - private ExpressionResult literalValue; - private String columnName; - public FilterModificationNode(Expression currentNode, Expression parentNode, - ExpressionType expType, DataType dataType, Object literalVal, String colName) { + FilterModificationNode(Expression currentNode, Expression parentNode, ExpressionType expType) { this.currentExp = currentNode; this.parentExp = parentNode; this.expType = expType; - this.columnName = colName; - this.literalValue = new ExpressionResult(dataType, literalVal); } public Expression getCurrentExp() { return currentExp; } - public void setCurrentExp(Expression currentExp) { - this.currentExp = currentExp; - } - public Expression getParentExp() { return parentExp; } - public void setParentExp(Expression parentExp) { - this.parentExp = parentExp; - } - public ExpressionType getExpType() { return expType; } - public void setExpType(ExpressionType expType) { - this.expType = expType; - } - - public ExpressionResult getLiteralValue() { - return literalValue; - } - - public void setLiteralValue(ExpressionResult literalValue) { - this.literalValue = literalValue; - } - - public String getColumnName() { - return columnName; - } - - public void setColumnName(String columnName) { - this.columnName = columnName; - } } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/RangeExpressionEvaluator.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/RangeExpressionEvaluator.java index e7cc1c3c6e1..585bf60878e 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/expression/RangeExpressionEvaluator.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/RangeExpressionEvaluator.java @@ -48,7 +48,6 @@ public class RangeExpressionEvaluator { LogServiceFactory.getLogService(RangeExpressionEvaluator.class.getName()); private Expression expr; private Expression srcNode; - private Expression srcParentNode; private Expression tarNode; private Expression tarParentNode; @@ -64,15 +63,15 @@ public void setExpr(Expression expr) { this.expr = expr; } - public Expression getSrcNode() { + private Expression getSrcNode() { return srcNode; } - public void setTarNode(Expression expr) { + private void setTarNode(Expression expr) { this.tarNode = expr; } - public void setTarParentNode(Expression expr) { + private void setTarParentNode(Expression expr) { this.tarParentNode = expr; } @@ -237,12 +236,10 @@ && eligibleForRangeExpConv(currentNode))) { private void addFilterExpressionMap(Map> filterExpressionMap, Expression currentNode, Expression parentNode) { String colName = getColumnName(currentNode); - DataType dataType = getLiteralDataType(currentNode); - Object literalVal = getLiteralValue(currentNode); ExpressionType expType = getExpressionType(currentNode); FilterModificationNode filterExpression = - new FilterModificationNode(currentNode, parentNode, expType, dataType, literalVal, colName); + new FilterModificationNode(currentNode, parentNode, expType); if (null == filterExpressionMap.get(colName)) { filterExpressionMap.put(colName, new ArrayList()); diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java index 3e23aa340e3..b882b511da7 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java @@ -105,8 +105,7 @@ public FilterResolverIntf getFilterResolver(Expression expressionTree, * */ public List getFilterredBlocks(DataRefNode btreeNode, - FilterResolverIntf filterResolver, AbstractIndex tableSegment, - AbsoluteTableIdentifier tableIdentifier) { + FilterResolverIntf filterResolver, AbstractIndex tableSegment) { // Need to get the current dimension tables List listOfDataBlocksToScan = new ArrayList(); // getting the start and end index key based on filter for hitting the @@ -401,7 +400,7 @@ private FilterResolverIntf getFilterResolverBasedOnExpressionType( case FALSE: return new FalseConditionalResolverImpl(expression, false, false, tableIdentifier); case TRUE: - return new TrueConditionalResolverImpl(expression, false, false, tableIdentifier); + return new TrueConditionalResolverImpl(expression, false, false); case EQUALS: currentCondExpression = (BinaryConditionalExpression) expression; // check for implicit column in the expression @@ -410,7 +409,6 @@ private FilterResolverIntf getFilterResolverBasedOnExpressionType( currentCondExpression.getColumnList().get(0).getCarbonColumn(); if (carbonColumn.hasEncoding(Encoding.IMPLICIT)) { return new ConditionalFilterResolverImpl(expression, isExpressionResolve, true, - tableIdentifier, currentCondExpression.getColumnList().get(0).getCarbonColumn().isMeasure()); } } @@ -436,7 +434,6 @@ private FilterResolverIntf getFilterResolverBasedOnExpressionType( tableIdentifier); } return new ConditionalFilterResolverImpl(expression, isExpressionResolve, true, - tableIdentifier, currentCondExpression.getColumnList().get(0).getCarbonColumn().isMeasure()); } // getting new dim index. @@ -462,14 +459,12 @@ private FilterResolverIntf getFilterResolverBasedOnExpressionType( } } return new ConditionalFilterResolverImpl(expression, isExpressionResolve, true, - tableIdentifier, currentCondExpression.getColumnList().get(0).getCarbonColumn().isMeasure()); } break; case RANGE: - return new ConditionalFilterResolverImpl(expression, isExpressionResolve, true, - tableIdentifier, false); + return new ConditionalFilterResolverImpl(expression, isExpressionResolve, true, false); case NOT_EQUALS: currentCondExpression = (BinaryConditionalExpression) expression; column = currentCondExpression.getColumnList().get(0).getCarbonColumn(); @@ -492,8 +487,7 @@ private FilterResolverIntf getFilterResolverBasedOnExpressionType( return new RowLevelRangeFilterResolverImpl(expression, isExpressionResolve, false, tableIdentifier); } - return new ConditionalFilterResolverImpl(expression, isExpressionResolve, false, - tableIdentifier, true); + return new ConditionalFilterResolverImpl(expression, isExpressionResolve, false, true); } if (!currentCondExpression.getColumnList().get(0).getCarbonColumn() @@ -516,11 +510,9 @@ private FilterResolverIntf getFilterResolverBasedOnExpressionType( tableIdentifier); } - return new ConditionalFilterResolverImpl(expression, isExpressionResolve, false, - tableIdentifier, false); + return new ConditionalFilterResolverImpl(expression, isExpressionResolve, false, false); } - return new ConditionalFilterResolverImpl(expression, isExpressionResolve, false, - tableIdentifier, false); + return new ConditionalFilterResolverImpl(expression, isExpressionResolve, false, false); } break; @@ -534,7 +526,7 @@ private FilterResolverIntf getFilterResolverBasedOnExpressionType( .hasEncoding(Encoding.DICTIONARY) && !condExpression.getColumnList().get(0) .getCarbonColumn().hasEncoding(Encoding.DIRECT_DICTIONARY)) || (condExpression.getColumnList().get(0).getCarbonColumn().isMeasure())) { - return new ConditionalFilterResolverImpl(expression, true, true, tableIdentifier, + return new ConditionalFilterResolverImpl(expression, true, true, condExpression.getColumnList().get(0).getCarbonColumn().isMeasure()); } } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterProcessor.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterProcessor.java index dc392baa038..dbb6eb7f8d1 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterProcessor.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterProcessor.java @@ -53,7 +53,7 @@ FilterResolverIntf getFilterResolver(Expression expressionTree, * @return list of DataRefNode. */ List getFilterredBlocks(DataRefNode dataRefNode, FilterResolverIntf filterResolver, - AbstractIndex segmentIndexBuilder, AbsoluteTableIdentifier tableIdentifier); + AbstractIndex segmentIndexBuilder); /** * This API will get the map of required partitions. 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 8dcac308c59..0b72e773030 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 @@ -49,7 +49,7 @@ import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants; import org.apache.carbondata.core.datastore.IndexKey; import org.apache.carbondata.core.datastore.block.SegmentProperties; -import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk; +import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage; import org.apache.carbondata.core.keygenerator.KeyGenException; import org.apache.carbondata.core.keygenerator.KeyGenerator; import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier; @@ -172,10 +172,8 @@ private static FilterExecuter createFilterExecuterTree( segmentProperties); case RANGE: return new RangeValueFilterExecuterImpl( - ((ConditionalFilterResolverImpl) filterExpressionResolverTree) - .getDimColResolvedFilterInfo(), - null, filterExpressionResolverTree.getFilterExpression(), - ((ConditionalFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier(), + filterExpressionResolverTree.getDimColResolvedFilterInfo(), + filterExpressionResolverTree.getFilterExpression(), ((ConditionalFilterResolverImpl) filterExpressionResolverTree) .getFilterRangeValues(segmentProperties), segmentProperties); case TRUE: @@ -551,10 +549,9 @@ public static ColumnFilterInfo getFilterValues(AbsoluteTableIdentifier tableIden sortFilterModelMembers(columnExpression, evaluateResultList); getDictionaryValue(evaluateResultList, forwardDictionary, surrogates); filterInfo = - getFilterValues(columnExpression, forwardDictionary, isIncludeFilter, null, surrogates); + getFilterValues(forwardDictionary, isIncludeFilter, surrogates); if (filterInfo.isOptimized()) { - return getDimColumnFilterInfoAfterApplyingCBO(columnExpression, - forwardDictionary, filterInfo); + return getDimColumnFilterInfoAfterApplyingCBO(forwardDictionary, filterInfo); } } finally { CarbonUtil.clearDictionaryCache(forwardDictionary); @@ -566,22 +563,20 @@ public static ColumnFilterInfo getFilterValues(AbsoluteTableIdentifier tableIden * Method will prepare the dimfilterinfo instance by resolving the filter * expression value to its respective surrogates. * - * @param columnExpression * @param forwardDictionary * @param isIncludeFilter * @param filterInfo * @param surrogates * @return */ - private static ColumnFilterInfo getFilterValues(ColumnExpression columnExpression, - Dictionary forwardDictionary, boolean isIncludeFilter, ColumnFilterInfo filterInfo, - List surrogates) throws QueryExecutionException { + private static ColumnFilterInfo getFilterValues(Dictionary forwardDictionary, + boolean isIncludeFilter, List surrogates) { // Default value has to be added if (surrogates.isEmpty()) { surrogates.add(0); } boolean isExcludeFilterNeedsToApply = false; - if (null == filterInfo && isIncludeFilter) { + if (isIncludeFilter) { isExcludeFilterNeedsToApply = isExcludeFilterNeedsToApply(forwardDictionary, surrogates.size()); } @@ -593,17 +588,10 @@ private static ColumnFilterInfo getFilterValues(ColumnExpression columnExpressio columnFilterInfo.setOptimized(true); } columnFilterInfo.setIncludeFilter(isIncludeFilter); - if (null != filterInfo) { - filterInfo.setIncludeFilter(isIncludeFilter); - filterInfo.setOptimized(true); - filterInfo.setExcludeFilterList(surrogates); - return filterInfo; + if (!isIncludeFilter) { + columnFilterInfo.setExcludeFilterList(surrogates); } else { - if (!isIncludeFilter) { - columnFilterInfo.setExcludeFilterList(surrogates); - } else { - columnFilterInfo.setFilterList(surrogates); - } + columnFilterInfo.setFilterList(surrogates); } } return columnFilterInfo; @@ -619,8 +607,7 @@ private static boolean isExcludeFilterNeedsToApply(Dictionary forwardDictionary, } private static ColumnFilterInfo getDimColumnFilterInfoAfterApplyingCBO( - ColumnExpression columnExpression, Dictionary forwardDictionary, - ColumnFilterInfo filterInfo) throws FilterUnsupportedException, QueryExecutionException { + Dictionary forwardDictionary, ColumnFilterInfo filterInfo) throws FilterUnsupportedException { List excludeMemberSurrogates = prepareExcludeFilterMembers(forwardDictionary, filterInfo.getFilterList()); filterInfo.setExcludeFilterList(excludeMemberSurrogates); @@ -730,14 +717,12 @@ public static ColumnFilterInfo getFilterListForAllValues(AbsoluteTableIdentifier prepareIncludeFilterMembers(expression, columnExpression, isIncludeFilter, forwardDictionary, surrogates); ColumnFilterInfo filterInfo = - getFilterValues(columnExpression, forwardDictionary, isIncludeFilter, null, surrogates); + getFilterValues(forwardDictionary, isIncludeFilter, surrogates); if (filterInfo.isOptimized()) { - return getDimColumnFilterInfoAfterApplyingCBO(columnExpression, forwardDictionary, + return getDimColumnFilterInfoAfterApplyingCBO(forwardDictionary, filterInfo); } return filterInfo; - } catch (QueryExecutionException e) { - throw new FilterUnsupportedException(e.getMessage()); } finally { CarbonUtil.clearDictionaryCache(forwardDictionary); } @@ -761,21 +746,16 @@ private static void sortFilterModelMembers(final ColumnExpression columnExpressi * expression value to its respective surrogates in the scenario of restructure. * * @param expression - * @param columnExpression * @param defaultValues * @param defaultSurrogate * @return * @throws FilterUnsupportedException */ - public static ColumnFilterInfo getFilterListForRS(Expression expression, - ColumnExpression columnExpression, String defaultValues, int defaultSurrogate) - throws FilterUnsupportedException { + public static ColumnFilterInfo getFilterListForRS(Expression expression, String defaultValues, + int defaultSurrogate) throws FilterUnsupportedException { List filterValuesList = new ArrayList(20); ColumnFilterInfo columnFilterInfo = null; - // List filterValuesList = new ArrayList(20); List evaluateResultListFinal = new ArrayList(20); - // KeyGenerator keyGenerator = - // KeyGeneratorFactory.getKeyGenerator(new int[] { defaultSurrogate }); try { List evaluateResultList = expression.evaluate(null).getList(); for (ExpressionResult result : evaluateResultList) { @@ -821,8 +801,6 @@ public static ColumnFilterInfo getFilterListForAllMembersRS(Expression expressio List evaluateResultListFinal = new ArrayList(20); ColumnFilterInfo columnFilterInfo = null; - // KeyGenerator keyGenerator = - // KeyGeneratorFactory.getKeyGenerator(new int[] { defaultSurrogate }); try { RowIntf row = new RowImpl(); if (defaultValues.equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL)) { @@ -1775,14 +1753,14 @@ public static BitSetGroup createBitSetGroupWithDefaultValue(int pageCount, int t * This method will compare the selected data against null values and * flip the bitSet if any null value is found * - * @param dimensionColumnDataChunk + * @param dimensionColumnPage * @param bitSet */ - public static void removeNullValues(DimensionColumnDataChunk dimensionColumnDataChunk, + public static void removeNullValues(DimensionColumnPage dimensionColumnPage, BitSet bitSet, byte[] defaultValue) { if (!bitSet.isEmpty()) { for (int i = bitSet.nextSetBit(0); i >= 0; i = bitSet.nextSetBit(i + 1)) { - if (dimensionColumnDataChunk.compareTo(i, defaultValue) == 0) { + if (dimensionColumnPage.compareTo(i, defaultValue) == 0) { bitSet.flip(i); } } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/GenericQueryType.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/GenericQueryType.java index 3742e7ed80d..214bd9dedd8 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/GenericQueryType.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/GenericQueryType.java @@ -22,7 +22,7 @@ import java.nio.ByteBuffer; import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; -import org.apache.carbondata.core.scan.processor.BlocksChunkHolder; +import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks; import org.apache.spark.sql.types.DataType; @@ -45,7 +45,7 @@ void parseBlocksAndReturnComplexColumnByteArray(DimensionRawColumnChunk[] rawCol DataType getSchemaType(); - void fillRequiredBlockData(BlocksChunkHolder blockChunkHolder) throws IOException; + void fillRequiredBlockData(RawBlockletColumnChunks blockChunkHolder) throws IOException; Object getDataBasedOnDataTypeFromSurrogates(ByteBuffer surrogateData); } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/AndFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/AndFilterExecuterImpl.java index f882162fbd8..e30945147c2 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/AndFilterExecuterImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/AndFilterExecuterImpl.java @@ -21,7 +21,7 @@ import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException; import org.apache.carbondata.core.scan.filter.intf.RowIntf; -import org.apache.carbondata.core.scan.processor.BlocksChunkHolder; +import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks; import org.apache.carbondata.core.util.BitSetGroup; public class AndFilterExecuterImpl implements FilterExecuter, ImplicitColumnFilterExecutor { @@ -35,18 +35,18 @@ public AndFilterExecuterImpl(FilterExecuter leftExecuter, FilterExecuter rightEx } @Override - public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine) - throws FilterUnsupportedException, IOException { - BitSetGroup leftFilters = leftExecuter.applyFilter(blockChunkHolder, useBitsetPipeLine); + public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks, + boolean useBitsetPipeLine) throws FilterUnsupportedException, IOException { + BitSetGroup leftFilters = leftExecuter.applyFilter(rawBlockletColumnChunks, useBitsetPipeLine); if (leftFilters.isEmpty()) { return leftFilters; } - BitSetGroup rightFilter = rightExecuter.applyFilter(blockChunkHolder, useBitsetPipeLine); + BitSetGroup rightFilter = rightExecuter.applyFilter(rawBlockletColumnChunks, useBitsetPipeLine); if (rightFilter.isEmpty()) { return rightFilter; } leftFilters.and(rightFilter); - blockChunkHolder.setBitSetGroup(leftFilters); + rawBlockletColumnChunks.setBitSetGroup(leftFilters); return leftFilters; } @@ -69,9 +69,10 @@ public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBi return leftFilters; } - @Override public void readBlocks(BlocksChunkHolder blocksChunkHolder) throws IOException { - leftExecuter.readBlocks(blocksChunkHolder); - rightExecuter.readBlocks(blocksChunkHolder); + @Override + public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException { + leftExecuter.readColumnChunks(rawBlockletColumnChunks); + rightExecuter.readColumnChunks(rawBlockletColumnChunks); } @Override @@ -93,8 +94,7 @@ public BitSet isFilterValuesPresentInBlockOrBlocklet(byte[][] maxValue, byte[][] rightFilter = ((ImplicitColumnFilterExecutor) rightExecuter) .isFilterValuesPresentInBlockOrBlocklet(maxValue, minValue, uniqueBlockPath); } else { - rightFilter = rightExecuter - .isScanRequired(maxValue, minValue); + rightFilter = rightExecuter.isScanRequired(maxValue, minValue); } if (rightFilter.isEmpty()) { return rightFilter; diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeColGroupFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeColGroupFilterExecuterImpl.java index 9391ebd10e7..44f7c079360 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeColGroupFilterExecuterImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeColGroupFilterExecuterImpl.java @@ -16,20 +16,10 @@ */ package org.apache.carbondata.core.scan.filter.executer; -import java.util.ArrayList; import java.util.BitSet; -import java.util.List; -import org.apache.carbondata.common.logging.LogService; -import org.apache.carbondata.common.logging.LogServiceFactory; import org.apache.carbondata.core.datastore.block.SegmentProperties; -import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk; -import org.apache.carbondata.core.keygenerator.KeyGenException; -import org.apache.carbondata.core.keygenerator.KeyGenerator; -import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo; -import org.apache.carbondata.core.scan.executor.util.QueryUtil; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo; -import org.apache.carbondata.core.util.ByteUtil; /** * It checks if filter is required on given block and if required, it does @@ -37,12 +27,6 @@ */ public class ExcludeColGroupFilterExecuterImpl extends ExcludeFilterExecuterImpl { - /** - * LOGGER - */ - private static final LogService LOGGER = - LogServiceFactory.getLogService(ExcludeColGroupFilterExecuterImpl.class.getName()); - /** * @param dimColResolvedFilterInfo * @param segmentProperties @@ -52,54 +36,6 @@ public ExcludeColGroupFilterExecuterImpl(DimColumnResolvedFilterInfo dimColResol super(dimColResolvedFilterInfo, null, segmentProperties, false); } - /** - * It fills BitSet with row index which matches filter key - */ - protected BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk, - int numerOfRows) { - BitSet bitSet = new BitSet(numerOfRows); - bitSet.flip(0, numerOfRows); - try { - KeyStructureInfo keyStructureInfo = getKeyStructureInfo(); - byte[][] filterValues = dimColumnExecuterInfo.getExcludeFilterKeys(); - for (int i = 0; i < filterValues.length; i++) { - byte[] filterVal = filterValues[i]; - for (int rowId = 0; rowId < numerOfRows; rowId++) { - byte[] colData = new byte[keyStructureInfo.getMaskByteRanges().length]; - dimensionColumnDataChunk.fillChunkData(colData, 0, rowId, keyStructureInfo); - if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterVal, colData) == 0) { - bitSet.flip(rowId); - } - } - } - - } catch (Exception e) { - LOGGER.error(e); - } - - return bitSet; - } - - /** - * It is required for extracting column data from columngroup chunk - * - * @return - * @throws KeyGenException - */ - private KeyStructureInfo getKeyStructureInfo() throws KeyGenException { - int colGrpId = getColumnGroupId(dimColEvaluatorInfo.getColumnIndex()); - KeyGenerator keyGenerator = segmentProperties.getColumnGroupAndItsKeygenartor().get(colGrpId); - List mdKeyOrdinal = new ArrayList(); - mdKeyOrdinal.add(getMdkeyOrdinal(dimColEvaluatorInfo.getColumnIndex(), colGrpId)); - int[] maskByteRanges = QueryUtil.getMaskedByteRangeBasedOrdinal(mdKeyOrdinal, keyGenerator); - byte[] maxKey = QueryUtil.getMaxKeyBasedOnOrinal(mdKeyOrdinal, keyGenerator); - KeyStructureInfo restructureInfos = new KeyStructureInfo(); - restructureInfos.setKeyGenerator(keyGenerator); - restructureInfos.setMaskByteRanges(maskByteRanges); - restructureInfos.setMaxKey(maxKey); - return restructureInfos; - } - /** * Check if scan is required on given block based on min and max value */ @@ -109,25 +45,4 @@ public BitSet isScanRequired(byte[][] blkMaxVal, byte[][] blkMinVal) { return bitSet; } - private int getMdkeyOrdinal(int ordinal, int colGrpId) { - return segmentProperties.getColumnGroupMdKeyOrdinal(colGrpId, ordinal); - } - - private int getColumnGroupId(int ordinal) { - int[][] columnGroups = segmentProperties.getColumnGroups(); - int colGrpId = -1; - for (int i = 0; i < columnGroups.length; i++) { - if (columnGroups[i].length > 1) { - colGrpId++; - if (QueryUtil.searchInArray(columnGroups[i], ordinal)) { - break; - } - } - } - return colGrpId; - } - - public KeyGenerator getKeyGenerator(int colGrpId) { - return segmentProperties.getColumnGroupAndItsKeygenartor().get(colGrpId); - } } \ No newline at end of file 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 fad37fc0179..fe983a5745e 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 @@ -20,7 +20,7 @@ import java.util.BitSet; import org.apache.carbondata.core.datastore.block.SegmentProperties; -import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk; +import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage; import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk; import org.apache.carbondata.core.datastore.page.ColumnPage; @@ -30,7 +30,7 @@ import org.apache.carbondata.core.scan.filter.intf.RowIntf; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo; -import org.apache.carbondata.core.scan.processor.BlocksChunkHolder; +import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks; import org.apache.carbondata.core.util.BitSetGroup; import org.apache.carbondata.core.util.ByteUtil; import org.apache.carbondata.core.util.CarbonUtil; @@ -40,13 +40,13 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter { - protected DimColumnResolvedFilterInfo dimColEvaluatorInfo; - protected DimColumnExecuterFilterInfo dimColumnExecuterInfo; - protected MeasureColumnResolvedFilterInfo msrColumnEvaluatorInfo; - protected MeasureColumnExecuterFilterInfo msrColumnExecutorInfo; + private DimColumnResolvedFilterInfo dimColEvaluatorInfo; + private DimColumnExecuterFilterInfo dimColumnExecuterInfo; + private MeasureColumnResolvedFilterInfo msrColumnEvaluatorInfo; + private MeasureColumnExecuterFilterInfo msrColumnExecutorInfo; protected SegmentProperties segmentProperties; - protected boolean isDimensionPresentInCurrentBlock = false; - protected boolean isMeasurePresentInCurrentBlock = false; + private boolean isDimensionPresentInCurrentBlock = false; + private boolean isMeasurePresentInCurrentBlock = false; private SerializableComparator comparator; /** * is dimension column data is natural sorted @@ -82,48 +82,50 @@ public ExcludeFilterExecuterImpl(DimColumnResolvedFilterInfo dimColEvaluatorInfo } @Override - public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine) - throws IOException { + public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks, + boolean useBitsetPipeLine) throws IOException { if (isDimensionPresentInCurrentBlock) { - int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping() + int chunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping() .get(dimColEvaluatorInfo.getColumnIndex()); - if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) { - blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex); + if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) { + rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] = + rawBlockletColumnChunks.getDataBlock().readDimensionChunk( + rawBlockletColumnChunks.getFileReader(), chunkIndex); } DimensionRawColumnChunk dimensionRawColumnChunk = - blockChunkHolder.getDimensionRawDataChunk()[blockIndex]; - DimensionColumnDataChunk[] dimensionColumnDataChunks = - dimensionRawColumnChunk.convertToDimColDataChunks(); + rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]; + DimensionColumnPage[] dimensionColumnPages = + dimensionRawColumnChunk.decodeAllColumnPages(); BitSetGroup bitSetGroup = new BitSetGroup(dimensionRawColumnChunk.getPagesCount()); - for (int i = 0; i < dimensionColumnDataChunks.length; i++) { - BitSet bitSet = getFilteredIndexes(dimensionColumnDataChunks[i], + for (int i = 0; i < dimensionColumnPages.length; i++) { + BitSet bitSet = getFilteredIndexes(dimensionColumnPages[i], dimensionRawColumnChunk.getRowCount()[i], useBitsetPipeLine, - blockChunkHolder.getBitSetGroup(), i); + rawBlockletColumnChunks.getBitSetGroup(), i); bitSetGroup.setBitSet(bitSet, i); } return bitSetGroup; } else if (isMeasurePresentInCurrentBlock) { - int blockIndex = segmentProperties.getMeasuresOrdinalToBlockMapping() + int chunkIndex = segmentProperties.getMeasuresOrdinalToChunkMapping() .get(msrColumnEvaluatorInfo.getColumnIndex()); - if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) { - blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex); + if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) { + rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] = + rawBlockletColumnChunks.getDataBlock().readMeasureChunk( + rawBlockletColumnChunks.getFileReader(), chunkIndex); } MeasureRawColumnChunk measureRawColumnChunk = - blockChunkHolder.getMeasureRawDataChunk()[blockIndex]; + rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]; ColumnPage[] ColumnPages = - measureRawColumnChunk.convertToColumnPage(); + measureRawColumnChunk.decodeAllColumnPages(); BitSetGroup bitSetGroup = new BitSetGroup(measureRawColumnChunk.getPagesCount()); DataType msrType = getMeasureDataType(msrColumnEvaluatorInfo); for (int i = 0; i < ColumnPages.length; i++) { BitSet bitSet = getFilteredIndexesForMeasure( - measureRawColumnChunk.convertToColumnPage(i), + measureRawColumnChunk.decodeColumnPage(i), measureRawColumnChunk.getRowCount()[i], useBitsetPipeLine, - blockChunkHolder.getBitSetGroup(), + rawBlockletColumnChunks.getBitSetGroup(), i, msrType); bitSetGroup.setBitSet(bitSet, i); @@ -133,7 +135,8 @@ public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBi return null; } - @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax) { + @Override + public boolean applyFilter(RowIntf value, int dimOrdinalMax) { if (isDimensionPresentInCurrentBlock) { byte[][] filterValues = dimColumnExecuterInfo.getExcludeFilterKeys(); byte[] col = (byte[])value.getVal(dimColEvaluatorInfo.getDimension().getOrdinal()); @@ -275,43 +278,41 @@ private BitSet getFilteredIndexesForMsrUsingPrvBitSet(ColumnPage measureColumnPa /** * Below method will be used to apply filter on dimension column - * @param dimensionColumnDataChunk + * @param dimensionColumnPage * @param numberOfRows * @param useBitsetPipeLine * @param prvBitSetGroup * @param pageNumber * @return filtered indexes bitset */ - private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk, + private BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage, int numberOfRows, boolean useBitsetPipeLine, BitSetGroup prvBitSetGroup, int pageNumber) { // check whether applying filtered based on previous bitset will be optimal if (CarbonUtil.usePreviousFilterBitsetGroup(useBitsetPipeLine, prvBitSetGroup, pageNumber, dimColumnExecuterInfo.getExcludeFilterKeys().length)) { - return getFilteredIndexesUisngPrvBitset(dimensionColumnDataChunk, prvBitSetGroup, pageNumber, - numberOfRows); + return getFilteredIndexesUisngPrvBitset(dimensionColumnPage, prvBitSetGroup, pageNumber); } else { - return getFilteredIndexes(dimensionColumnDataChunk, numberOfRows); + return getFilteredIndexes(dimensionColumnPage, numberOfRows); } } - private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk, + private BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage, int numberOfRows) { - if (dimensionColumnDataChunk.isExplicitSorted()) { - return setFilterdIndexToBitSetWithColumnIndex(dimensionColumnDataChunk, numberOfRows); + if (dimensionColumnPage.isExplicitSorted()) { + return setFilterdIndexToBitSetWithColumnIndex(dimensionColumnPage, numberOfRows); } - return setFilterdIndexToBitSet(dimensionColumnDataChunk, numberOfRows); + return setFilterdIndexToBitSet(dimensionColumnPage, numberOfRows); } /** * Below method will be used to apply filter based on previous filtered bitset - * @param dimensionColumnDataChunk + * @param dimensionColumnPage * @param prvBitSetGroup * @param pageNumber - * @param numberOfRows * @return filtered indexes bitset */ - private BitSet getFilteredIndexesUisngPrvBitset(DimensionColumnDataChunk dimensionColumnDataChunk, - BitSetGroup prvBitSetGroup, int pageNumber, int numberOfRows) { + private BitSet getFilteredIndexesUisngPrvBitset(DimensionColumnPage dimensionColumnPage, + BitSetGroup prvBitSetGroup, int pageNumber) { BitSet prvPageBitSet = prvBitSetGroup.getBitSet(pageNumber); BitSet bitSet = new BitSet(); bitSet.or(prvPageBitSet); @@ -319,11 +320,11 @@ private BitSet getFilteredIndexesUisngPrvBitset(DimensionColumnDataChunk dimensi int compareResult = 0; // if dimension data was natural sorted then get the index from previous bitset // and use the same in next column data, otherwise use the inverted index reverse - if (!dimensionColumnDataChunk.isExplicitSorted()) { + if (!dimensionColumnPage.isExplicitSorted()) { for (int index = prvPageBitSet.nextSetBit(0); index >= 0; index = prvPageBitSet.nextSetBit(index + 1)) { compareResult = CarbonUtil - .isFilterPresent(filterKeys, dimensionColumnDataChunk, 0, filterKeys.length - 1, index); + .isFilterPresent(filterKeys, dimensionColumnPage, 0, filterKeys.length - 1, index); if (compareResult != 0) { bitSet.set(index); } else { @@ -336,8 +337,8 @@ private BitSet getFilteredIndexesUisngPrvBitset(DimensionColumnDataChunk dimensi for (int index = prvPageBitSet.nextSetBit(0); index >= 0; index = prvPageBitSet.nextSetBit(index + 1)) { compareResult = CarbonUtil - .isFilterPresent(filterKeys, dimensionColumnDataChunk, 0, filterKeys.length - 1, - dimensionColumnDataChunk.getInvertedReverseIndex(index)); + .isFilterPresent(filterKeys, dimensionColumnPage, 0, filterKeys.length - 1, + dimensionColumnPage.getInvertedReverseIndex(index)); if (compareResult != 0) { bitSet.set(index); } else { @@ -351,7 +352,7 @@ private BitSet getFilteredIndexesUisngPrvBitset(DimensionColumnDataChunk dimensi } private BitSet setFilterdIndexToBitSetWithColumnIndex( - DimensionColumnDataChunk dimensionColumnDataChunk, int numerOfRows) { + DimensionColumnPage dimensionColumnPage, int numerOfRows) { BitSet bitSet = new BitSet(numerOfRows); bitSet.flip(0, numerOfRows); int startIndex = 0; @@ -361,10 +362,10 @@ private BitSet setFilterdIndexToBitSetWithColumnIndex( break; } int[] rangeIndex = CarbonUtil - .getRangeIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1, + .getRangeIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1, filterValues[i]); for (int j = rangeIndex[0]; j <= rangeIndex[1]; j++) { - bitSet.flip(dimensionColumnDataChunk.getInvertedIndex(j)); + bitSet.flip(dimensionColumnPage.getInvertedIndex(j)); } if (rangeIndex[1] >= 0) { startIndex = rangeIndex[1] + 1; @@ -373,7 +374,7 @@ private BitSet setFilterdIndexToBitSetWithColumnIndex( return bitSet; } - private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk, + private BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage, int numerOfRows) { BitSet bitSet = new BitSet(numerOfRows); bitSet.flip(0, numerOfRows); @@ -390,7 +391,7 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD break; } int[] rangeIndex = CarbonUtil - .getRangeIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1, + .getRangeIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1, filterValues[i]); for (int j = rangeIndex[0]; j <= rangeIndex[1]; j++) { bitSet.flip(j); @@ -403,14 +404,14 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD if (filterValues.length > 1) { for (int i = 0; i < numerOfRows; i++) { int index = CarbonUtil.binarySearch(filterValues, 0, filterValues.length - 1, - dimensionColumnDataChunk.getChunkData(i)); + dimensionColumnPage.getChunkData(i)); if (index >= 0) { bitSet.flip(i); } } } else { for (int j = 0; j < numerOfRows; j++) { - if (dimensionColumnDataChunk.compareTo(j, filterValues[0]) == 0) { + if (dimensionColumnPage.compareTo(j, filterValues[0]) == 0) { bitSet.flip(j); } } @@ -419,26 +420,30 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD return bitSet; } - @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) { + @Override + public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) { BitSet bitSet = new BitSet(1); bitSet.flip(0, 1); return bitSet; } - @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException { + @Override + public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException { if (isDimensionPresentInCurrentBlock) { - int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping() + int chunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping() .get(dimColEvaluatorInfo.getColumnIndex()); - if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) { - blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex); + if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) { + rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] = + rawBlockletColumnChunks.getDataBlock().readDimensionChunk( + rawBlockletColumnChunks.getFileReader(), chunkIndex); } } else if (isMeasurePresentInCurrentBlock) { - int blockIndex = segmentProperties.getMeasuresOrdinalToBlockMapping() + int chunkIndex = segmentProperties.getMeasuresOrdinalToChunkMapping() .get(msrColumnEvaluatorInfo.getColumnIndex()); - if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) { - blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex); + if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) { + rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] = + rawBlockletColumnChunks.getDataBlock().readMeasureChunk( + rawBlockletColumnChunks.getFileReader(), chunkIndex); } } } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FilterExecuter.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FilterExecuter.java index 53d30681e33..a64341e7b74 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FilterExecuter.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FilterExecuter.java @@ -21,7 +21,7 @@ import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException; import org.apache.carbondata.core.scan.filter.intf.RowIntf; -import org.apache.carbondata.core.scan.processor.BlocksChunkHolder; +import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks; import org.apache.carbondata.core.util.BitSetGroup; public interface FilterExecuter { @@ -32,8 +32,8 @@ public interface FilterExecuter { * @return * @throws FilterUnsupportedException */ - BitSetGroup applyFilter(BlocksChunkHolder blocksChunkHolder, boolean useBitsetPipeLine) - throws FilterUnsupportedException, IOException; + BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks, + boolean useBitsetPipeLine) throws FilterUnsupportedException, IOException; /** * apply range filter on a row @@ -54,7 +54,7 @@ boolean applyFilter(RowIntf value, int dimOrdinalMax) /** * It just reads necessary block for filter executor, it does not uncompress the data. - * @param blockChunkHolder + * @param rawBlockletColumnChunks */ - void readBlocks(BlocksChunkHolder blockChunkHolder)throws IOException; + void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks)throws IOException; } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ImplicitIncludeFilterExecutorImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ImplicitIncludeFilterExecutorImpl.java index 9c4c7ba8b05..b683fd6d6bd 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ImplicitIncludeFilterExecutorImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ImplicitIncludeFilterExecutorImpl.java @@ -23,7 +23,7 @@ import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException; import org.apache.carbondata.core.scan.filter.intf.RowIntf; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo; -import org.apache.carbondata.core.scan.processor.BlocksChunkHolder; +import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks; import org.apache.carbondata.core.util.BitSetGroup; import org.apache.carbondata.core.util.path.CarbonTablePath; @@ -41,26 +41,31 @@ public ImplicitIncludeFilterExecutorImpl(DimColumnResolvedFilterInfo dimColumnEv } @Override - public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeline) - throws FilterUnsupportedException { - BitSetGroup bitSetGroup = new BitSetGroup(blockChunkHolder.getDataBlock().numberOfPages()); - for (int i = 0; i < blockChunkHolder.getDataBlock().numberOfPages(); i++) { + public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks, + boolean useBitsetPipeline) { + BitSetGroup bitSetGroup = new BitSetGroup( + rawBlockletColumnChunks.getDataBlock().numberOfPages()); + for (int i = 0; i < rawBlockletColumnChunks.getDataBlock().numberOfPages(); i++) { bitSetGroup.setBitSet( - setBitSetForCompleteDimensionData(blockChunkHolder.getDataBlock().getPageRowCount(i)), i); + setBitSetForCompleteDimensionData( + rawBlockletColumnChunks.getDataBlock().getPageRowCount(i)), i); } return bitSetGroup; } - @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax) + @Override + public boolean applyFilter(RowIntf value, int dimOrdinalMax) throws FilterUnsupportedException, IOException { return false; } - @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) { + @Override + public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) { return null; } - @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException { + @Override + public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException { } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeColGroupFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeColGroupFilterExecuterImpl.java index 1cbc0bcfb20..e4da26fe9ce 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeColGroupFilterExecuterImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeColGroupFilterExecuterImpl.java @@ -24,14 +24,14 @@ import org.apache.carbondata.common.logging.LogService; import org.apache.carbondata.common.logging.LogServiceFactory; import org.apache.carbondata.core.datastore.block.SegmentProperties; -import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk; +import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage; import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; import org.apache.carbondata.core.keygenerator.KeyGenException; import org.apache.carbondata.core.keygenerator.KeyGenerator; import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo; import org.apache.carbondata.core.scan.executor.util.QueryUtil; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo; -import org.apache.carbondata.core.scan.processor.BlocksChunkHolder; +import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks; import org.apache.carbondata.core.util.BitSetGroup; import org.apache.carbondata.core.util.ByteUtil; @@ -59,7 +59,7 @@ public IncludeColGroupFilterExecuterImpl(DimColumnResolvedFilterInfo dimColResol /** * It fills BitSet with row index which matches filter key */ - protected BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk, + protected BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage, int numerOfRows) { BitSet bitSet = new BitSet(numerOfRows); @@ -70,7 +70,7 @@ protected BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnData byte[] filterVal = filterValues[i]; for (int rowId = 0; rowId < numerOfRows; rowId++) { byte[] colData = new byte[keyStructureInfo.getMaskByteRanges().length]; - dimensionColumnDataChunk.fillChunkData(colData, 0, rowId, keyStructureInfo); + dimensionColumnPage.fillRawData(rowId, 0, colData, keyStructureInfo); if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterVal, colData) == 0) { bitSet.set(rowId); } @@ -85,20 +85,21 @@ protected BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnData } @Override - public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine) - throws IOException { - int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping() + public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks, + boolean useBitsetPipeLine) throws IOException { + int chunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping() .get(dimColumnEvaluatorInfo.getColumnIndex()); - if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) { - blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex); + if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) { + rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] = + rawBlockletColumnChunks.getDataBlock().readDimensionChunk( + rawBlockletColumnChunks.getFileReader(), chunkIndex); } DimensionRawColumnChunk dimensionRawColumnChunk = - blockChunkHolder.getDimensionRawDataChunk()[blockIndex]; + rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]; BitSetGroup bitSetGroup = new BitSetGroup(dimensionRawColumnChunk.getPagesCount()); for (int i = 0; i < dimensionRawColumnChunk.getPagesCount(); i++) { if (dimensionRawColumnChunk.getMaxValues() != null) { - BitSet bitSet = getFilteredIndexes(dimensionRawColumnChunk.convertToDimColDataChunk(i), + BitSet bitSet = getFilteredIndexes(dimensionRawColumnChunk.decodeColumnPage(i), dimensionRawColumnChunk.getRowCount()[i]); bitSetGroup.setBitSet(bitSet, i); } @@ -133,10 +134,10 @@ public BitSet isScanRequired(byte[][] blkMaxVal, byte[][] blkMinVal) { BitSet bitSet = new BitSet(1); byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys(); int columnIndex = dimColumnEvaluatorInfo.getColumnIndex(); - int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping().get(columnIndex); + int chunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping().get(columnIndex); int[] cols = getAllColumns(columnIndex); - byte[] maxValue = getMinMaxData(cols, blkMaxVal[blockIndex], columnIndex); - byte[] minValue = getMinMaxData(cols, blkMinVal[blockIndex], columnIndex); + byte[] maxValue = getMinMaxData(cols, blkMaxVal[chunkIndex], columnIndex); + byte[] minValue = getMinMaxData(cols, blkMinVal[chunkIndex], columnIndex); boolean isScanRequired = false; for (int k = 0; k < filterValues.length; k++) { // filter value should be in range of max and min value i.e 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 fe1421c9e4c..05328f31146 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 @@ -20,7 +20,7 @@ import java.util.BitSet; import org.apache.carbondata.core.datastore.block.SegmentProperties; -import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk; +import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage; import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk; import org.apache.carbondata.core.datastore.page.ColumnPage; @@ -30,7 +30,7 @@ import org.apache.carbondata.core.scan.filter.intf.RowIntf; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo; -import org.apache.carbondata.core.scan.processor.BlocksChunkHolder; +import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks; import org.apache.carbondata.core.util.BitSetGroup; import org.apache.carbondata.core.util.ByteUtil; import org.apache.carbondata.core.util.CarbonUtil; @@ -41,12 +41,12 @@ public class IncludeFilterExecuterImpl implements FilterExecuter { protected DimColumnResolvedFilterInfo dimColumnEvaluatorInfo; - protected DimColumnExecuterFilterInfo dimColumnExecuterInfo; - protected MeasureColumnResolvedFilterInfo msrColumnEvaluatorInfo; - protected MeasureColumnExecuterFilterInfo msrColumnExecutorInfo; + DimColumnExecuterFilterInfo dimColumnExecuterInfo; + private MeasureColumnResolvedFilterInfo msrColumnEvaluatorInfo; + private MeasureColumnExecuterFilterInfo msrColumnExecutorInfo; protected SegmentProperties segmentProperties; - protected boolean isDimensionPresentInCurrentBlock = false; - protected boolean isMeasurePresentInCurrentBlock = false; + private boolean isDimensionPresentInCurrentBlock = false; + private boolean isMeasurePresentInCurrentBlock = false; protected SerializableComparator comparator; /** * is dimension column data is natural sorted @@ -86,44 +86,46 @@ public IncludeFilterExecuterImpl(DimColumnResolvedFilterInfo dimColumnEvaluatorI } @Override - public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine) - throws IOException { - if (isDimensionPresentInCurrentBlock == true) { - int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping() + public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks, + boolean useBitsetPipeLine) throws IOException { + if (isDimensionPresentInCurrentBlock) { + int chunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping() .get(dimColumnEvaluatorInfo.getColumnIndex()); - if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) { - blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex); + if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) { + rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] = + rawBlockletColumnChunks.getDataBlock().readDimensionChunk( + rawBlockletColumnChunks.getFileReader(), chunkIndex); } DimensionRawColumnChunk dimensionRawColumnChunk = - blockChunkHolder.getDimensionRawDataChunk()[blockIndex]; + rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]; BitSetGroup bitSetGroup = new BitSetGroup(dimensionRawColumnChunk.getPagesCount()); for (int i = 0; i < dimensionRawColumnChunk.getPagesCount(); i++) { if (dimensionRawColumnChunk.getMaxValues() != null) { if (isScanRequired(dimensionRawColumnChunk.getMaxValues()[i], dimensionRawColumnChunk.getMinValues()[i], dimColumnExecuterInfo.getFilterKeys())) { - BitSet bitSet = getFilteredIndexes(dimensionRawColumnChunk.convertToDimColDataChunk(i), + BitSet bitSet = getFilteredIndexes(dimensionRawColumnChunk.decodeColumnPage(i), dimensionRawColumnChunk.getRowCount()[i], useBitsetPipeLine, - blockChunkHolder.getBitSetGroup(), i); + rawBlockletColumnChunks.getBitSetGroup(), i); bitSetGroup.setBitSet(bitSet, i); } } else { - BitSet bitSet = getFilteredIndexes(dimensionRawColumnChunk.convertToDimColDataChunk(i), + BitSet bitSet = getFilteredIndexes(dimensionRawColumnChunk.decodeColumnPage(i), dimensionRawColumnChunk.getRowCount()[i], useBitsetPipeLine, - blockChunkHolder.getBitSetGroup(), i); + rawBlockletColumnChunks.getBitSetGroup(), i); bitSetGroup.setBitSet(bitSet, i); } } return bitSetGroup; } else if (isMeasurePresentInCurrentBlock) { - int blockIndex = segmentProperties.getMeasuresOrdinalToBlockMapping() + int chunkIndex = segmentProperties.getMeasuresOrdinalToChunkMapping() .get(msrColumnEvaluatorInfo.getColumnIndex()); - if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) { - blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex); + if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) { + rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] = + rawBlockletColumnChunks.getDataBlock().readMeasureChunk( + rawBlockletColumnChunks.getFileReader(), chunkIndex); } MeasureRawColumnChunk measureRawColumnChunk = - blockChunkHolder.getMeasureRawDataChunk()[blockIndex]; + rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]; BitSetGroup bitSetGroup = new BitSetGroup(measureRawColumnChunk.getPagesCount()); DataType msrType = getMeasureDataType(msrColumnEvaluatorInfo); for (int i = 0; i < measureRawColumnChunk.getPagesCount(); i++) { @@ -132,16 +134,16 @@ public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBi measureRawColumnChunk.getMinValues()[i], msrColumnExecutorInfo.getFilterKeys(), msrColumnEvaluatorInfo.getType())) { BitSet bitSet = - getFilteredIndexesForMeasure(measureRawColumnChunk.convertToColumnPage(i), + getFilteredIndexesForMeasure(measureRawColumnChunk.decodeColumnPage(i), measureRawColumnChunk.getRowCount()[i], useBitsetPipeLine, - blockChunkHolder.getBitSetGroup(), i, msrType); + rawBlockletColumnChunks.getBitSetGroup(), i, msrType); bitSetGroup.setBitSet(bitSet, i); } } else { BitSet bitSet = - getFilteredIndexesForMeasure(measureRawColumnChunk.convertToColumnPage(i), + getFilteredIndexesForMeasure(measureRawColumnChunk.decodeColumnPage(i), measureRawColumnChunk.getRowCount()[i], useBitsetPipeLine, - blockChunkHolder.getBitSetGroup(), i, msrType); + rawBlockletColumnChunks.getBitSetGroup(), i, msrType); bitSetGroup.setBitSet(bitSet, i); } } @@ -293,43 +295,43 @@ private BitSet getFilteredIndexesForMsrUsingPrvBitSet(ColumnPage measureColumnPa /** * Below method will be used to apply filter on dimension column - * @param dimensionColumnDataChunk + * @param dimensionColumnPage * @param numberOfRows * @param useBitsetPipeLine * @param prvBitSetGroup * @param pageNumber * @return filtered indexes bitset */ - private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk, + private BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage, int numberOfRows, boolean useBitsetPipeLine, BitSetGroup prvBitSetGroup, int pageNumber) { // check whether previous indexes can be optimal to apply filter on dimension column if (CarbonUtil.usePreviousFilterBitsetGroup(useBitsetPipeLine, prvBitSetGroup, pageNumber, dimColumnExecuterInfo.getFilterKeys().length)) { - return getFilteredIndexesUisngPrvBitset(dimensionColumnDataChunk, prvBitSetGroup, pageNumber, + return getFilteredIndexesUisngPrvBitset(dimensionColumnPage, prvBitSetGroup, pageNumber, numberOfRows); } else { - return getFilteredIndexes(dimensionColumnDataChunk, numberOfRows); + return getFilteredIndexes(dimensionColumnPage, numberOfRows); } } - private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk, + private BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage, int numberOfRows) { - if (dimensionColumnDataChunk.isExplicitSorted()) { - return setFilterdIndexToBitSetWithColumnIndex(dimensionColumnDataChunk, numberOfRows); + if (dimensionColumnPage.isExplicitSorted()) { + return setFilterdIndexToBitSetWithColumnIndex(dimensionColumnPage, numberOfRows); } - return setFilterdIndexToBitSet(dimensionColumnDataChunk, numberOfRows); + return setFilterdIndexToBitSet(dimensionColumnPage, numberOfRows); } /** * Below method will be used to apply filter on dimension * column based on previous filtered indexes - * @param dimensionColumnDataChunk + * @param dimensionColumnPage * @param prvBitSetGroup * @param pageNumber * @param numberOfRows * @return filtered bitset */ - private BitSet getFilteredIndexesUisngPrvBitset(DimensionColumnDataChunk dimensionColumnDataChunk, + private BitSet getFilteredIndexesUisngPrvBitset(DimensionColumnPage dimensionColumnPage, BitSetGroup prvBitSetGroup, int pageNumber, int numberOfRows) { BitSet prvPageBitSet = prvBitSetGroup.getBitSet(pageNumber); BitSet bitSet = new BitSet(numberOfRows); @@ -337,11 +339,11 @@ private BitSet getFilteredIndexesUisngPrvBitset(DimensionColumnDataChunk dimensi int compareResult = 0; // if dimension data was natural sorted then get the index from previous bitset // and use the same in next column data, otherwise use the inverted index reverse - if (!dimensionColumnDataChunk.isExplicitSorted()) { + if (!dimensionColumnPage.isExplicitSorted()) { for (int index = prvPageBitSet.nextSetBit(0); index >= 0; index = prvPageBitSet.nextSetBit(index + 1)) { compareResult = CarbonUtil - .isFilterPresent(filterKeys, dimensionColumnDataChunk, 0, filterKeys.length - 1, index); + .isFilterPresent(filterKeys, dimensionColumnPage, 0, filterKeys.length - 1, index); if (compareResult == 0) { bitSet.set(index); } @@ -350,8 +352,8 @@ private BitSet getFilteredIndexesUisngPrvBitset(DimensionColumnDataChunk dimensi for (int index = prvPageBitSet.nextSetBit(0); index >= 0; index = prvPageBitSet.nextSetBit(index + 1)) { compareResult = CarbonUtil - .isFilterPresent(filterKeys, dimensionColumnDataChunk, 0, filterKeys.length - 1, - dimensionColumnDataChunk.getInvertedReverseIndex(index)); + .isFilterPresent(filterKeys, dimensionColumnPage, 0, filterKeys.length - 1, + dimensionColumnPage.getInvertedReverseIndex(index)); if (compareResult == 0) { bitSet.set(index); } @@ -360,7 +362,7 @@ private BitSet getFilteredIndexesUisngPrvBitset(DimensionColumnDataChunk dimensi return bitSet; } private BitSet setFilterdIndexToBitSetWithColumnIndex( - DimensionColumnDataChunk dimensionColumnDataChunk, int numerOfRows) { + DimensionColumnPage dimensionColumnPage, int numerOfRows) { BitSet bitSet = new BitSet(numerOfRows); int startIndex = 0; byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys(); @@ -369,10 +371,10 @@ private BitSet setFilterdIndexToBitSetWithColumnIndex( break; } int[] rangeIndex = CarbonUtil - .getRangeIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1, + .getRangeIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1, filterValues[i]); for (int j = rangeIndex[0]; j <= rangeIndex[1]; j++) { - bitSet.set(dimensionColumnDataChunk.getInvertedIndex(j)); + bitSet.set(dimensionColumnPage.getInvertedIndex(j)); } if (rangeIndex[1] >= 0) { startIndex = rangeIndex[1] + 1; @@ -381,7 +383,7 @@ private BitSet setFilterdIndexToBitSetWithColumnIndex( return bitSet; } - private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk, + private BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage, int numerOfRows) { BitSet bitSet = new BitSet(numerOfRows); byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys(); @@ -394,7 +396,7 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD break; } int[] rangeIndex = CarbonUtil - .getRangeIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1, + .getRangeIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1, filterValues[i]); for (int j = rangeIndex[0]; j <= rangeIndex[1]; j++) { bitSet.set(j); @@ -407,14 +409,14 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD if (filterValues.length > 1) { for (int i = 0; i < numerOfRows; i++) { int index = CarbonUtil.binarySearch(filterValues, 0, filterValues.length - 1, - dimensionColumnDataChunk.getChunkData(i)); + dimensionColumnPage.getChunkData(i)); if (index >= 0) { bitSet.set(i); } } } else { for (int j = 0; j < numerOfRows; j++) { - if (dimensionColumnDataChunk.compareTo(j, filterValues[0]) == 0) { + if (dimensionColumnPage.compareTo(j, filterValues[0]) == 0) { bitSet.set(j); } } @@ -423,26 +425,25 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD return bitSet; } + @Override public BitSet isScanRequired(byte[][] blkMaxVal, byte[][] blkMinVal) { BitSet bitSet = new BitSet(1); byte[][] filterValues = null; int columnIndex = 0; - int blockIndex = 0; + int chunkIndex = 0; boolean isScanRequired = false; if (isDimensionPresentInCurrentBlock) { filterValues = dimColumnExecuterInfo.getFilterKeys(); columnIndex = dimColumnEvaluatorInfo.getColumnIndex(); - blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping().get(columnIndex); - isScanRequired = - isScanRequired(blkMaxVal[blockIndex], blkMinVal[blockIndex], filterValues); - + chunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping().get(columnIndex); + isScanRequired = isScanRequired(blkMaxVal[chunkIndex], blkMinVal[chunkIndex], filterValues); } else if (isMeasurePresentInCurrentBlock) { columnIndex = msrColumnEvaluatorInfo.getColumnIndex(); - blockIndex = - segmentProperties.getMeasuresOrdinalToBlockMapping().get(columnIndex) + segmentProperties - .getLastDimensionColOrdinal(); - isScanRequired = isScanRequired(blkMaxVal[blockIndex], blkMinVal[blockIndex], + chunkIndex = + segmentProperties.getMeasuresOrdinalToChunkMapping().get(columnIndex) + + segmentProperties.getLastDimensionColOrdinal(); + isScanRequired = isScanRequired(blkMaxVal[chunkIndex], blkMinVal[chunkIndex], msrColumnExecutorInfo.getFilterKeys(), msrColumnEvaluatorInfo.getType()); } @@ -492,20 +493,23 @@ private boolean isScanRequired(byte[] maxValue, byte[] minValue, Object[] filter return false; } - @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException { - if (isDimensionPresentInCurrentBlock == true) { - int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping() + @Override + public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException { + if (isDimensionPresentInCurrentBlock) { + int chunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping() .get(dimColumnEvaluatorInfo.getColumnIndex()); - if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) { - blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex); + if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) { + rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] = + rawBlockletColumnChunks.getDataBlock().readDimensionChunk( + rawBlockletColumnChunks.getFileReader(), chunkIndex); } - } else if (isMeasurePresentInCurrentBlock == true) { - int blockIndex = segmentProperties.getMeasuresOrdinalToBlockMapping() + } else if (isMeasurePresentInCurrentBlock) { + int chunkIndex = segmentProperties.getMeasuresOrdinalToChunkMapping() .get(msrColumnEvaluatorInfo.getColumnIndex()); - if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) { - blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex); + if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) { + rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] = + rawBlockletColumnChunks.getDataBlock().readMeasureChunk( + rawBlockletColumnChunks.getFileReader(), chunkIndex); } } } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/OrFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/OrFilterExecuterImpl.java index 87273bb6d4f..9f0afb5be00 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/OrFilterExecuterImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/OrFilterExecuterImpl.java @@ -21,7 +21,7 @@ import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException; import org.apache.carbondata.core.scan.filter.intf.RowIntf; -import org.apache.carbondata.core.scan.processor.BlocksChunkHolder; +import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks; import org.apache.carbondata.core.util.BitSetGroup; public class OrFilterExecuterImpl implements FilterExecuter { @@ -35,30 +35,33 @@ public OrFilterExecuterImpl(FilterExecuter leftExecuter, FilterExecuter rightExe } @Override - public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine) - throws FilterUnsupportedException, IOException { - BitSetGroup leftFilters = leftExecuter.applyFilter(blockChunkHolder, false); - BitSetGroup rightFilters = rightExecuter.applyFilter(blockChunkHolder, false); + public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks, + boolean useBitsetPipeLine) throws FilterUnsupportedException, IOException { + BitSetGroup leftFilters = leftExecuter.applyFilter(rawBlockletColumnChunks, false); + BitSetGroup rightFilters = rightExecuter.applyFilter(rawBlockletColumnChunks, false); leftFilters.or(rightFilters); - blockChunkHolder.setBitSetGroup(leftFilters); + rawBlockletColumnChunks.setBitSetGroup(leftFilters); return leftFilters; } - @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax) + @Override + public boolean applyFilter(RowIntf value, int dimOrdinalMax) throws FilterUnsupportedException, IOException { return leftExecuter.applyFilter(value, dimOrdinalMax) || rightExecuter.applyFilter(value, dimOrdinalMax); } - @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) { + @Override + public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) { BitSet leftFilters = leftExecuter.isScanRequired(blockMaxValue, blockMinValue); BitSet rightFilters = rightExecuter.isScanRequired(blockMaxValue, blockMinValue); leftFilters.or(rightFilters); return leftFilters; } - @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException { - leftExecuter.readBlocks(blockChunkHolder); - rightExecuter.readBlocks(blockChunkHolder); + @Override + public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException { + leftExecuter.readColumnChunks(rawBlockletColumnChunks); + rightExecuter.readColumnChunks(rawBlockletColumnChunks); } } 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 797fe9ddd09..34555e1bc65 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 @@ -22,11 +22,10 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants; import org.apache.carbondata.core.datastore.block.SegmentProperties; -import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk; +import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage; import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator; import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory; -import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier; 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.CarbonDimension; @@ -39,8 +38,7 @@ import org.apache.carbondata.core.scan.filter.FilterUtil; import org.apache.carbondata.core.scan.filter.intf.RowIntf; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo; -import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo; -import org.apache.carbondata.core.scan.processor.BlocksChunkHolder; +import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks; import org.apache.carbondata.core.util.BitSetGroup; import org.apache.carbondata.core.util.ByteUtil; import org.apache.carbondata.core.util.CarbonUtil; @@ -60,25 +58,23 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl { /** * it has index at which given dimension is stored in file */ - protected int dimensionBlocksIndex; + private int dimensionChunkIndex; /** * flag to check whether the filter dimension is present in current block list of dimensions. * Applicable for restructure scenarios */ - protected boolean isDimensionPresentInCurrentBlock; - boolean lessThanExp; - boolean lessThanEqualExp; - boolean greaterThanExp; - boolean greaterThanEqualExp; - boolean startBlockMinIsDefaultStart; - boolean endBlockMaxisDefaultEnd; - boolean isRangeFullyCoverBlock; + private boolean isDimensionPresentInCurrentBlock; + private boolean lessThanExp; + private boolean lessThanEqualExp; + private boolean greaterThanExp; + private boolean greaterThanEqualExp; + private boolean startBlockMinIsDefaultStart; + private boolean endBlockMaxisDefaultEnd; + private boolean isRangeFullyCoverBlock; public RangeValueFilterExecuterImpl(DimColumnResolvedFilterInfo dimColEvaluatorInfo, - MeasureColumnResolvedFilterInfo msrColEvaluatorInfo, Expression exp, - AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues, - SegmentProperties segmentProperties) { + Expression exp, byte[][] filterRangeValues, SegmentProperties segmentProperties) { this.dimColEvaluatorInfo = dimColEvaluatorInfo; this.exp = exp; @@ -91,7 +87,7 @@ public RangeValueFilterExecuterImpl(DimColumnResolvedFilterInfo dimColEvaluatorI startBlockMinIsDefaultStart = false; endBlockMaxisDefaultEnd = false; isRangeFullyCoverBlock = false; - initDimensionBlockIndexes(); + initDimensionChunkIndexes(); ifDefaultValueMatchesFilter(); } @@ -100,13 +96,13 @@ public RangeValueFilterExecuterImpl(DimColumnResolvedFilterInfo dimColEvaluatorI * This method will initialize the dimension info for the current block to be * used for filtering the data */ - private void initDimensionBlockIndexes() { + private void initDimensionChunkIndexes() { // find the dimension in the current block dimensions list CarbonDimension dimensionFromCurrentBlock = segmentProperties.getDimensionFromCurrentBlock(dimColEvaluatorInfo.getDimension()); if (null != dimensionFromCurrentBlock) { dimColEvaluatorInfo.setColumnIndex(dimensionFromCurrentBlock.getOrdinal()); - this.dimensionBlocksIndex = segmentProperties.getDimensionOrdinalToBlockMapping() + this.dimensionChunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping() .get(dimensionFromCurrentBlock.getOrdinal()); isDimensionPresentInCurrentBlock = true; } @@ -136,14 +132,14 @@ private void ifDefaultValueMatchesFilter() { /** * Method to apply the filter. - * @param blockChunkHolder + * @param rawBlockletColumnChunks * @return * @throws FilterUnsupportedException * @throws IOException */ - public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine) - throws FilterUnsupportedException, IOException { - return applyNoAndDirectFilter(blockChunkHolder); + public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks, + boolean useBitsetPipeLine) throws FilterUnsupportedException, IOException { + return applyNoAndDirectFilter(rawBlockletColumnChunks); } /** @@ -272,32 +268,32 @@ public boolean isScanRequired(byte[] blockMinValue, byte[] blockMaxValue, byte[] // Filter Min <-----------------------------------------------> Filter Max if (isDimensionPresentInCurrentBlock) { - if (((lessThanExp == true) && ( + if (((lessThanExp) && ( ByteUtil.UnsafeComparer.INSTANCE.compareTo(blockMinValue, filterValues[1]) >= 0)) || ( - (lessThanEqualExp == true) && ( + (lessThanEqualExp) && ( ByteUtil.UnsafeComparer.INSTANCE.compareTo(blockMinValue, filterValues[1]) > 0)) || ( - (greaterThanExp == true) && ( + (greaterThanExp) && ( ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[0], blockMaxValue) >= 0)) || ( - (greaterThanEqualExp == true) && ( + (greaterThanEqualExp) && ( ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[0], blockMaxValue) > 0))) { // completely out of block boundary isScanRequired = false; } else { - if (((greaterThanExp == true) && ( + if (((greaterThanExp) && ( ByteUtil.UnsafeComparer.INSTANCE.compareTo(blockMinValue, filterValues[0]) > 0)) || ( - (greaterThanEqualExp == true) && ( + (greaterThanEqualExp) && ( ByteUtil.UnsafeComparer.INSTANCE.compareTo(blockMinValue, filterValues[0]) >= 0))) { startBlockMinIsDefaultStart = true; } - if (((lessThanExp == true) && ( + if (((lessThanExp) && ( ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[1], blockMaxValue) > 0)) || ( - (lessThanEqualExp == true) && ( + (lessThanEqualExp) && ( ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[1], blockMaxValue) >= 0))) { endBlockMaxisDefaultEnd = true; } - if (startBlockMinIsDefaultStart == true && endBlockMaxisDefaultEnd == true) { + if (startBlockMinIsDefaultStart && endBlockMaxisDefaultEnd) { isRangeFullyCoverBlock = true; } } @@ -313,7 +309,8 @@ public boolean isScanRequired(byte[] blockMinValue, byte[] blockMaxValue, byte[] * @param blockMinValue * @return */ - @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) { + @Override + public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) { BitSet bitSet = new BitSet(1); byte[][] filterValues = this.filterRangesValues; int columnIndex = this.dimColEvaluatorInfo.getColumnIndex(); @@ -329,50 +326,49 @@ public boolean isScanRequired(byte[] blockMinValue, byte[] blockMaxValue, byte[] * Method to apply the Range Filter. * @param blockChunkHolder * @return - * @throws FilterUnsupportedException * @throws IOException */ - public BitSetGroup applyNoAndDirectFilter(BlocksChunkHolder blockChunkHolder) - throws FilterUnsupportedException, IOException { + private BitSetGroup applyNoAndDirectFilter(RawBlockletColumnChunks blockChunkHolder) + throws IOException { // In case of Alter Table Add and Delete Columns the isDimensionPresentInCurrentBlock can be // false, in that scenario the default values of the column should be shown. // select all rows if dimension does not exists in the current block if (!isDimensionPresentInCurrentBlock) { - int numberOfRows = blockChunkHolder.getDataBlock().nodeSize(); - return FilterUtil - .createBitSetGroupWithDefaultValue(blockChunkHolder.getDataBlock().numberOfPages(), - numberOfRows, true); + int numberOfRows = blockChunkHolder.getDataBlock().numRows(); + return FilterUtil.createBitSetGroupWithDefaultValue( + blockChunkHolder.getDataBlock().numberOfPages(), numberOfRows, true); } - int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping() + int chunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping() .get(dimColEvaluatorInfo.getColumnIndex()); - if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) { - blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex); + if (null == blockChunkHolder.getDimensionRawColumnChunks()[chunkIndex]) { + blockChunkHolder.getDimensionRawColumnChunks()[chunkIndex] = + blockChunkHolder.getDataBlock().readDimensionChunk( + blockChunkHolder.getFileReader(), chunkIndex); } DimensionRawColumnChunk rawColumnChunk = - blockChunkHolder.getDimensionRawDataChunk()[blockIndex]; + blockChunkHolder.getDimensionRawColumnChunks()[chunkIndex]; BitSetGroup bitSetGroup = new BitSetGroup(rawColumnChunk.getPagesCount()); for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) { if (rawColumnChunk.getMaxValues() != null) { if (isScanRequired(rawColumnChunk.getMinValues()[i], rawColumnChunk.getMaxValues()[i], this.filterRangesValues)) { - if (isRangeFullyCoverBlock == true) { + if (isRangeFullyCoverBlock) { // Set all the bits in this case as filter Min Max values cover the whole block. BitSet bitSet = new BitSet(rawColumnChunk.getRowCount()[i]); bitSet.flip(0, rawColumnChunk.getRowCount()[i]); bitSetGroup.setBitSet(bitSet, i); } else { - BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i), + BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i), rawColumnChunk.getRowCount()[i]); bitSetGroup.setBitSet(bitSet, i); } } } else { - BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i), + BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i), rawColumnChunk.getRowCount()[i]); bitSetGroup.setBitSet(bitSet, i); } @@ -380,12 +376,12 @@ public BitSetGroup applyNoAndDirectFilter(BlocksChunkHolder blockChunkHolder) return bitSetGroup; } - private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk, + private BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage, int numerOfRows) { - if (dimensionColumnDataChunk.isExplicitSorted()) { - return setFilterdIndexToBitSetWithColumnIndex(dimensionColumnDataChunk, numerOfRows); + if (dimensionColumnPage.isExplicitSorted()) { + return setFilterdIndexToBitSetWithColumnIndex(dimensionColumnPage, numerOfRows); } - return setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows); + return setFilterdIndexToBitSet(dimensionColumnPage, numerOfRows); } /** @@ -394,12 +390,12 @@ private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataCh * column is not supported by default so column index mapping will be present for * accesing the members from the block. * - * @param dimensionColumnDataChunk + * @param dimensionColumnPage * @param numerOfRows * @return BitSet. */ private BitSet setFilterdIndexToBitSetWithColumnIndex( - DimensionColumnDataChunk dimensionColumnDataChunk, int numerOfRows) { + DimensionColumnPage dimensionColumnPage, int numerOfRows) { BitSet bitSet = new BitSet(numerOfRows); int start = 0; int startIndex = 0; @@ -410,13 +406,13 @@ private BitSet setFilterdIndexToBitSetWithColumnIndex( // For Range expression we expect two values. The First is the Min Value and Second is the // Max value. // Get the Min Value - if (startBlockMinIsDefaultStart == false) { + if (!startBlockMinIsDefaultStart) { start = CarbonUtil - .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1, + .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1, filterValues[0], greaterThanExp); - if (greaterThanExp == true && start >= 0) { + if (greaterThanExp && start >= 0) { start = CarbonUtil - .nextGreaterValueToTarget(start, dimensionColumnDataChunk, filterValues[0], + .nextGreaterValueToTarget(start, dimensionColumnPage, filterValues[0], numerOfRows); } @@ -428,8 +424,8 @@ private BitSet setFilterdIndexToBitSetWithColumnIndex( // Method will compare the tentative index value after binary search, this tentative // index needs to be compared by the filter member if its >= filter then from that // index the bitset will be considered for filtering process. - if ((ByteUtil.compare(filterValues[0], dimensionColumnDataChunk - .getChunkData(dimensionColumnDataChunk.getInvertedIndex(start)))) > 0) { + if ((ByteUtil.compare(filterValues[0], dimensionColumnPage + .getChunkData(dimensionColumnPage.getInvertedIndex(start)))) > 0) { start = start + 1; } } @@ -440,14 +436,14 @@ private BitSet setFilterdIndexToBitSetWithColumnIndex( } // Get the Max value - if (endBlockMaxisDefaultEnd == false) { + if (!endBlockMaxisDefaultEnd) { start = CarbonUtil - .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1, + .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1, filterValues[1], lessThanEqualExp); - if (lessThanExp == true && start >= 0) { + if (lessThanExp && start >= 0) { start = - CarbonUtil.nextLesserValueToTarget(start, dimensionColumnDataChunk, filterValues[1]); + CarbonUtil.nextLesserValueToTarget(start, dimensionColumnPage, filterValues[1]); } if (start < 0) { @@ -458,7 +454,7 @@ private BitSet setFilterdIndexToBitSetWithColumnIndex( // In case the start is less than 0, then positive value of start is pointing to the next // value of the searched key. So move to the previous one. if ((ByteUtil.compare(filterValues[1], - dimensionColumnDataChunk.getChunkData(dimensionColumnDataChunk.getInvertedIndex(start))) + dimensionColumnPage.getChunkData(dimensionColumnPage.getInvertedIndex(start))) < 0)) { start = start - 1; } @@ -469,18 +465,18 @@ private BitSet setFilterdIndexToBitSetWithColumnIndex( } for (int j = startMin; j <= endMax; j++) { - bitSet.set(dimensionColumnDataChunk.getInvertedIndex(j)); + bitSet.set(dimensionColumnPage.getInvertedIndex(j)); } // Binary Search cannot be done on '@NU#LL$!", so need to check and compare for null on // matching row. - if (dimensionColumnDataChunk.isNoDicitionaryColumn()) { - updateForNoDictionaryColumn(startMin, endMax, dimensionColumnDataChunk, bitSet); + if (dimensionColumnPage.isNoDicitionaryColumn()) { + updateForNoDictionaryColumn(startMin, endMax, dimensionColumnPage, bitSet); } return bitSet; } - private void updateForNoDictionaryColumn(int start, int end, DimensionColumnDataChunk dataChunk, + private void updateForNoDictionaryColumn(int start, int end, DimensionColumnPage dataChunk, BitSet bitset) { for (int j = start; j <= end; j++) { if (dataChunk.compareTo(j, CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY) == 0 @@ -496,31 +492,31 @@ private void updateForNoDictionaryColumn(int start, int end, DimensionColumnData * be called if the column is sorted default so column index * mapping will be present for accesaing the members from the block. * - * @param dimensionColumnDataChunk + * @param dimensionColumnPage * @param numerOfRows * @return BitSet. */ - private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk, + private BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage, int numerOfRows) { BitSet bitSet = new BitSet(numerOfRows); - // if (dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) { + // if (dimensionColumnPage instanceof FixedLengthDimensionColumnPage) { byte[][] filterValues = this.filterRangesValues; - if (dimensionColumnDataChunk.isExplicitSorted()) { + if (dimensionColumnPage.isExplicitSorted()) { int start = 0; int startMin = 0; int endMax = 0; int startIndex = 0; // For Range expression we expect two values. The First is the Min Value and Second is the // Max value. - if (startBlockMinIsDefaultStart == false) { + if (!startBlockMinIsDefaultStart) { start = CarbonUtil - .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1, + .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1, filterValues[0], greaterThanExp); - if (greaterThanExp == true && start >= 0) { + if (greaterThanExp && start >= 0) { start = CarbonUtil - .nextGreaterValueToTarget(start, dimensionColumnDataChunk, filterValues[0], + .nextGreaterValueToTarget(start, dimensionColumnPage, filterValues[0], numerOfRows); } @@ -532,7 +528,7 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD // Method will compare the tentative index value after binary search, this tentative // index needs to be compared by the filter member if its >= filter then from that // index the bitset will be considered for filtering process. - if ((ByteUtil.compare(filterValues[0], dimensionColumnDataChunk.getChunkData(start))) + if ((ByteUtil.compare(filterValues[0], dimensionColumnPage.getChunkData(start))) > 0) { start = start + 1; } @@ -542,14 +538,14 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD startMin = startIndex; } - if (endBlockMaxisDefaultEnd == false) { + if (!endBlockMaxisDefaultEnd) { start = CarbonUtil - .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1, + .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1, filterValues[1], lessThanEqualExp); - if (lessThanExp == true && start >= 0) { + if (lessThanExp && start >= 0) { start = - CarbonUtil.nextLesserValueToTarget(start, dimensionColumnDataChunk, filterValues[1]); + CarbonUtil.nextLesserValueToTarget(start, dimensionColumnPage, filterValues[1]); } if (start < 0) { @@ -559,7 +555,7 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD } // In case the start is less than 0, then positive value of start is pointing to the next // value of the searched key. So move to the previous one. - if ((ByteUtil.compare(filterValues[1], dimensionColumnDataChunk.getChunkData(start)) + if ((ByteUtil.compare(filterValues[1], dimensionColumnPage.getChunkData(start)) < 0)) { start = start - 1; } @@ -575,8 +571,8 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD // Binary Search cannot be done on '@NU#LL$!", so need to check and compare for null on // matching row. - if (dimensionColumnDataChunk.isNoDicitionaryColumn()) { - updateForNoDictionaryColumn(startMin, endMax, dimensionColumnDataChunk, bitSet); + if (dimensionColumnPage.isNoDicitionaryColumn()) { + updateForNoDictionaryColumn(startMin, endMax, dimensionColumnPage, bitSet); } } else { byte[] defaultValue = null; @@ -585,7 +581,7 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD .getDirectDictionaryGenerator(dimColEvaluatorInfo.getDimension().getDataType()); int key = directDictionaryGenerator.generateDirectSurrogateKey(null); CarbonDimension currentBlockDimension = - segmentProperties.getDimensions().get(dimensionBlocksIndex); + segmentProperties.getDimensions().get(dimensionChunkIndex); if (currentBlockDimension.isSortColumn()) { defaultValue = FilterUtil.getMaskKey(key, currentBlockDimension, this.segmentProperties.getSortColumnsGenerator()); @@ -601,55 +597,37 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD } // evaluate result for lower range value first and then perform and operation in the // upper range value in order to compute the final result - bitSet = evaluateGreaterThanFilterForUnsortedColumn(dimensionColumnDataChunk, filterValues[0], + bitSet = evaluateGreaterThanFilterForUnsortedColumn(dimensionColumnPage, filterValues[0], numerOfRows); BitSet upperRangeBitSet = - evaluateLessThanFilterForUnsortedColumn(dimensionColumnDataChunk, filterValues[1], + evaluateLessThanFilterForUnsortedColumn(dimensionColumnPage, filterValues[1], numerOfRows); bitSet.and(upperRangeBitSet); - FilterUtil.removeNullValues(dimensionColumnDataChunk, bitSet, defaultValue); + FilterUtil.removeNullValues(dimensionColumnPage, bitSet, defaultValue); } return bitSet; } - /** - * This method will compare the selected data against null values and - * flip the bitSet if any null value is found - * - * @param dimensionColumnDataChunk - * @param bitSet - */ - private void removeNullValues(DimensionColumnDataChunk dimensionColumnDataChunk, BitSet bitSet) { - if (!bitSet.isEmpty()) { - for (int i = bitSet.nextSetBit(0); i >= 0; i = bitSet.nextSetBit(i + 1)) { - if (dimensionColumnDataChunk.compareTo(i, CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY) - == 0) { - bitSet.flip(i); - } - } - } - } - /** * This method will evaluate the result for filter column based on the lower range value * - * @param dimensionColumnDataChunk + * @param dimensionColumnPage * @param filterValue * @param numberOfRows * @return */ private BitSet evaluateGreaterThanFilterForUnsortedColumn( - DimensionColumnDataChunk dimensionColumnDataChunk, byte[] filterValue, int numberOfRows) { + DimensionColumnPage dimensionColumnPage, byte[] filterValue, int numberOfRows) { BitSet bitSet = new BitSet(numberOfRows); if (greaterThanExp) { for (int i = 0; i < numberOfRows; i++) { - if ((ByteUtil.compare(dimensionColumnDataChunk.getChunkData(i), filterValue) > 0)) { + if ((ByteUtil.compare(dimensionColumnPage.getChunkData(i), filterValue) > 0)) { bitSet.set(i); } } } else if (greaterThanEqualExp) { for (int i = 0; i < numberOfRows; i++) { - if ((ByteUtil.compare(dimensionColumnDataChunk.getChunkData(i), filterValue) >= 0)) { + if ((ByteUtil.compare(dimensionColumnPage.getChunkData(i), filterValue) >= 0)) { bitSet.set(i); } } @@ -660,23 +638,23 @@ private BitSet evaluateGreaterThanFilterForUnsortedColumn( /** * This method will evaluate the result for filter column based on the upper range value * - * @param dimensionColumnDataChunk + * @param dimensionColumnPage * @param filterValue * @param numberOfRows * @return */ private BitSet evaluateLessThanFilterForUnsortedColumn( - DimensionColumnDataChunk dimensionColumnDataChunk, byte[] filterValue, int numberOfRows) { + DimensionColumnPage dimensionColumnPage, byte[] filterValue, int numberOfRows) { BitSet bitSet = new BitSet(numberOfRows); if (lessThanExp) { for (int i = 0; i < numberOfRows; i++) { - if ((ByteUtil.compare(dimensionColumnDataChunk.getChunkData(i), filterValue) < 0)) { + if ((ByteUtil.compare(dimensionColumnPage.getChunkData(i), filterValue) < 0)) { bitSet.set(i); } } } else if (lessThanEqualExp) { for (int i = 0; i < numberOfRows; i++) { - if ((ByteUtil.compare(dimensionColumnDataChunk.getChunkData(i), filterValue) <= 0)) { + if ((ByteUtil.compare(dimensionColumnPage.getChunkData(i), filterValue) <= 0)) { bitSet.set(i); } } @@ -686,16 +664,18 @@ private BitSet evaluateLessThanFilterForUnsortedColumn( /** * Method to read the blocks. - * @param blockChunkHolder + * @param rawBlockletColumnChunks * @throws IOException */ - @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException { - if (isDimensionPresentInCurrentBlock == true) { - int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping() + @Override + public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException { + if (isDimensionPresentInCurrentBlock) { + int chunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping() .get(dimColEvaluatorInfo.getColumnIndex()); - if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) { - blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex); + if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) { + rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] = + rawBlockletColumnChunks.getDataBlock().readDimensionChunk( + rawBlockletColumnChunks.getFileReader(), chunkIndex); } } } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureExcludeFilterExecutorImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureExcludeFilterExecutorImpl.java index 5707eb44b7d..d7bec7e002b 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureExcludeFilterExecutorImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureExcludeFilterExecutorImpl.java @@ -24,7 +24,7 @@ import org.apache.carbondata.core.scan.filter.intf.RowIntf; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo; -import org.apache.carbondata.core.scan.processor.BlocksChunkHolder; +import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks; import org.apache.carbondata.core.util.BitSetGroup; public class RestructureExcludeFilterExecutorImpl extends RestructureEvaluatorImpl { @@ -34,7 +34,7 @@ public class RestructureExcludeFilterExecutorImpl extends RestructureEvaluatorIm * flag to check whether filter values contain the default value applied on the dimension column * which does not exist in the current block */ - protected boolean isDefaultValuePresentInFilterValues; + private boolean isDefaultValuePresentInFilterValues; public RestructureExcludeFilterExecutorImpl(DimColumnResolvedFilterInfo dimColEvaluatorInfo, MeasureColumnResolvedFilterInfo measureColumnResolvedFilterInfo, boolean isMeasure) { @@ -48,26 +48,29 @@ public RestructureExcludeFilterExecutorImpl(DimColumnResolvedFilterInfo dimColEv } @Override - public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine) - throws IOException { - int numberOfRows = blockChunkHolder.getDataBlock().nodeSize(); + public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks, + boolean useBitsetPipeLine) throws IOException { + int numberOfRows = rawBlockletColumnChunks.getDataBlock().numRows(); return FilterUtil - .createBitSetGroupWithDefaultValue(blockChunkHolder.getDataBlock().numberOfPages(), + .createBitSetGroupWithDefaultValue(rawBlockletColumnChunks.getDataBlock().numberOfPages(), numberOfRows, !isDefaultValuePresentInFilterValues); } - @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax) + @Override + public boolean applyFilter(RowIntf value, int dimOrdinalMax) throws FilterUnsupportedException { throw new FilterUnsupportedException("Unsupported RestructureExcludeFilterExecutorImpl on row"); } - @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) { + @Override + public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) { BitSet bitSet = new BitSet(1); bitSet.flip(0, 1); return bitSet; } - @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException { + @Override + public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) { } } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureIncludeFilterExecutorImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureIncludeFilterExecutorImpl.java index 8bcc53f2e29..c874fc5c73b 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureIncludeFilterExecutorImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureIncludeFilterExecutorImpl.java @@ -24,7 +24,7 @@ import org.apache.carbondata.core.scan.filter.intf.RowIntf; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo; -import org.apache.carbondata.core.scan.processor.BlocksChunkHolder; +import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks; import org.apache.carbondata.core.util.BitSetGroup; public class RestructureIncludeFilterExecutorImpl extends RestructureEvaluatorImpl { @@ -33,7 +33,7 @@ public class RestructureIncludeFilterExecutorImpl extends RestructureEvaluatorIm * flag to check whether filter values contain the default value applied on the dimension column * which does not exist in the current block */ - protected boolean isDefaultValuePresentInFilterValues; + private boolean isDefaultValuePresentInFilterValues; public RestructureIncludeFilterExecutorImpl(DimColumnResolvedFilterInfo dimColumnEvaluatorInfo, MeasureColumnResolvedFilterInfo measureColumnResolvedFilterInfo, boolean isMeasure) { @@ -47,15 +47,16 @@ public RestructureIncludeFilterExecutorImpl(DimColumnResolvedFilterInfo dimColum } @Override - public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine) - throws IOException { - int numberOfRows = blockChunkHolder.getDataBlock().nodeSize(); - return FilterUtil - .createBitSetGroupWithDefaultValue(blockChunkHolder.getDataBlock().numberOfPages(), - numberOfRows, isDefaultValuePresentInFilterValues); + public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks, + boolean useBitsetPipeLine) throws IOException { + int numberOfRows = rawBlockletColumnChunks.getDataBlock().numRows(); + return FilterUtil.createBitSetGroupWithDefaultValue( + rawBlockletColumnChunks.getDataBlock().numberOfPages(), + numberOfRows, isDefaultValuePresentInFilterValues); } - @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax) + @Override + public boolean applyFilter(RowIntf value, int dimOrdinalMax) throws FilterUnsupportedException { throw new FilterUnsupportedException("Unsupported RestructureIncludeFilterExecutorImpl on row"); } @@ -66,7 +67,8 @@ public BitSet isScanRequired(byte[][] blkMaxVal, byte[][] blkMinVal) { return bitSet; } - @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException { + @Override + public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) { } 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 de97e82c6c2..540607d274d 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java @@ -34,8 +34,8 @@ import org.apache.carbondata.core.cache.dictionary.Dictionary; import org.apache.carbondata.core.constants.CarbonCommonConstants; import org.apache.carbondata.core.datastore.block.SegmentProperties; -import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk; -import org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionDataChunk; +import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage; +import org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionColumnPage; import org.apache.carbondata.core.datastore.page.ColumnPage; import org.apache.carbondata.core.keygenerator.KeyGenException; import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator; @@ -58,7 +58,7 @@ import org.apache.carbondata.core.scan.filter.intf.RowIntf; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo; -import org.apache.carbondata.core.scan.processor.BlocksChunkHolder; +import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks; import org.apache.carbondata.core.util.BitSetGroup; import org.apache.carbondata.core.util.ByteUtil; import org.apache.carbondata.core.util.CarbonUtil; @@ -68,20 +68,20 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter { private static final LogService LOGGER = LogServiceFactory.getLogService(RowLevelFilterExecuterImpl.class.getName()); - protected List dimColEvaluatorInfoList; - protected List msrColEvalutorInfoList; + List dimColEvaluatorInfoList; + List msrColEvalutorInfoList; protected Expression exp; protected AbsoluteTableIdentifier tableIdentifier; protected SegmentProperties segmentProperties; /** * it has index at which given dimension is stored in file */ - protected int[] dimensionBlocksIndex; + int[] dimensionChunkIndex; /** * it has index at which given measure is stored in file */ - protected int[] measureBlocksIndex; + int[] measureChunkIndex; private Map complexDimensionInfoMap; @@ -89,18 +89,18 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter { * flag to check whether the filter dimension is present in current block list of dimensions. * Applicable for restructure scenarios */ - protected boolean[] isDimensionPresentInCurrentBlock; + boolean[] isDimensionPresentInCurrentBlock; /** * flag to check whether the filter measure is present in current block list of measures. * Applicable for restructure scenarios */ - protected boolean[] isMeasurePresentInCurrentBlock; + boolean[] isMeasurePresentInCurrentBlock; /** * is dimension column data is natural sorted */ - protected boolean isNaturalSorted; + boolean isNaturalSorted; /** * date direct dictionary generator @@ -124,10 +124,10 @@ public RowLevelFilterExecuterImpl(List dimColEvalua } if (this.dimColEvaluatorInfoList.size() > 0) { this.isDimensionPresentInCurrentBlock = new boolean[dimColEvaluatorInfoList.size()]; - this.dimensionBlocksIndex = new int[dimColEvaluatorInfoList.size()]; + this.dimensionChunkIndex = new int[dimColEvaluatorInfoList.size()]; } else { this.isDimensionPresentInCurrentBlock = new boolean[]{false}; - this.dimensionBlocksIndex = new int[]{0}; + this.dimensionChunkIndex = new int[]{0}; } if (null == msrColEvalutorInfoList) { this.msrColEvalutorInfoList = new ArrayList(20); @@ -136,10 +136,10 @@ public RowLevelFilterExecuterImpl(List dimColEvalua } if (this.msrColEvalutorInfoList.size() > 0) { this.isMeasurePresentInCurrentBlock = new boolean[msrColEvalutorInfoList.size()]; - this.measureBlocksIndex = new int[msrColEvalutorInfoList.size()]; + this.measureChunkIndex = new int[msrColEvalutorInfoList.size()]; } else { this.isMeasurePresentInCurrentBlock = new boolean[]{false}; - this.measureBlocksIndex = new int[] {0}; + this.measureChunkIndex = new int[] {0}; } this.exp = exp; this.tableIdentifier = tableIdentifier; @@ -148,22 +148,22 @@ public RowLevelFilterExecuterImpl(List dimColEvalua DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(DataTypes.DATE); this.timestampDictionaryGenerator = DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(DataTypes.TIMESTAMP); - initDimensionBlockIndexes(); - initMeasureBlockIndexes(); + initDimensionChunkIndexes(); + initMeasureChunkIndexes(); } /** * This method will initialize the dimension info for the current block to be * used for filtering the data */ - private void initDimensionBlockIndexes() { + private void initDimensionChunkIndexes() { for (int i = 0; i < dimColEvaluatorInfoList.size(); i++) { // find the dimension in the current block dimensions list CarbonDimension dimensionFromCurrentBlock = segmentProperties .getDimensionFromCurrentBlock(dimColEvaluatorInfoList.get(i).getDimension()); if (null != dimensionFromCurrentBlock) { dimColEvaluatorInfoList.get(i).setColumnIndex(dimensionFromCurrentBlock.getOrdinal()); - this.dimensionBlocksIndex[i] = segmentProperties.getDimensionOrdinalToBlockMapping() + this.dimensionChunkIndex[i] = segmentProperties.getDimensionOrdinalToChunkMapping() .get(dimensionFromCurrentBlock.getOrdinal()); isDimensionPresentInCurrentBlock[i] = true; } @@ -174,14 +174,14 @@ private void initDimensionBlockIndexes() { * This method will initialize the measure info for the current block to be * used for filtering the data */ - private void initMeasureBlockIndexes() { + private void initMeasureChunkIndexes() { for (int i = 0; i < msrColEvalutorInfoList.size(); i++) { // find the measure in the current block measures list CarbonMeasure measureFromCurrentBlock = segmentProperties.getMeasureFromCurrentBlock( msrColEvalutorInfoList.get(i).getCarbonColumn().getColumnId()); if (null != measureFromCurrentBlock) { msrColEvalutorInfoList.get(i).setColumnIndex(measureFromCurrentBlock.getOrdinal()); - this.measureBlocksIndex[i] = segmentProperties.getMeasuresOrdinalToBlockMapping() + this.measureChunkIndex[i] = segmentProperties.getMeasuresOrdinalToChunkMapping() .get(measureFromCurrentBlock.getOrdinal()); isMeasurePresentInCurrentBlock[i] = true; } @@ -189,9 +189,9 @@ private void initMeasureBlockIndexes() { } @Override - public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine) - throws FilterUnsupportedException, IOException { - readBlocks(blockChunkHolder); + public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks, + boolean useBitsetPipeLine) throws FilterUnsupportedException, IOException { + readColumnChunks(rawBlockletColumnChunks); // CHECKSTYLE:ON int[] numberOfRows = null; @@ -199,31 +199,31 @@ public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBi if (dimColEvaluatorInfoList.size() > 0) { if (isDimensionPresentInCurrentBlock[0]) { - pageNumbers = - blockChunkHolder.getDimensionRawDataChunk()[dimensionBlocksIndex[0]].getPagesCount(); - numberOfRows = - blockChunkHolder.getDimensionRawDataChunk()[dimensionBlocksIndex[0]].getRowCount(); + pageNumbers = rawBlockletColumnChunks.getDimensionRawColumnChunks()[dimensionChunkIndex[0]] + .getPagesCount(); + numberOfRows = rawBlockletColumnChunks.getDimensionRawColumnChunks()[dimensionChunkIndex[0]] + .getRowCount(); } else { // specific for restructure case where default values need to be filled - pageNumbers = blockChunkHolder.getDataBlock().numberOfPages(); + pageNumbers = rawBlockletColumnChunks.getDataBlock().numberOfPages(); numberOfRows = new int[pageNumbers]; for (int i = 0; i < pageNumbers; i++) { - numberOfRows[i] = blockChunkHolder.getDataBlock().getPageRowCount(i); + numberOfRows[i] = rawBlockletColumnChunks.getDataBlock().getPageRowCount(i); } } } if (msrColEvalutorInfoList.size() > 0) { if (isMeasurePresentInCurrentBlock[0]) { - pageNumbers = - blockChunkHolder.getMeasureRawDataChunk()[measureBlocksIndex[0]].getPagesCount(); - numberOfRows = - blockChunkHolder.getMeasureRawDataChunk()[measureBlocksIndex[0]].getRowCount(); + pageNumbers = rawBlockletColumnChunks.getMeasureRawColumnChunks()[measureChunkIndex[0]] + .getPagesCount(); + numberOfRows = rawBlockletColumnChunks.getMeasureRawColumnChunks()[measureChunkIndex[0]] + .getRowCount(); } else { // specific for restructure case where default values need to be filled - pageNumbers = blockChunkHolder.getDataBlock().numberOfPages(); + pageNumbers = rawBlockletColumnChunks.getDataBlock().numberOfPages(); numberOfRows = new int[pageNumbers]; for (int i = 0; i < pageNumbers; i++) { - numberOfRows[i] = blockChunkHolder.getDataBlock().getPageRowCount(i); + numberOfRows[i] = rawBlockletColumnChunks.getDataBlock().getPageRowCount(i); } } } @@ -234,10 +234,12 @@ public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBi BitSet prvBitset = null; // if bitset pipe line is enabled then use rowid from previous bitset // otherwise use older flow - if (!useBitsetPipeLine || null == blockChunkHolder.getBitSetGroup() || null == bitSetGroup - .getBitSet(i) || blockChunkHolder.getBitSetGroup().getBitSet(i).isEmpty()) { + if (!useBitsetPipeLine || + null == rawBlockletColumnChunks.getBitSetGroup() || + null == bitSetGroup.getBitSet(i) || + rawBlockletColumnChunks.getBitSetGroup().getBitSet(i).isEmpty()) { for (int index = 0; index < numberOfRows[i]; index++) { - createRow(blockChunkHolder, row, i, index); + createRow(rawBlockletColumnChunks, row, i, index); Boolean rslt = false; try { rslt = exp.evaluate(row).getBoolean(); @@ -253,10 +255,10 @@ public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBi } } } else { - prvBitset = blockChunkHolder.getBitSetGroup().getBitSet(i); + prvBitset = rawBlockletColumnChunks.getBitSetGroup().getBitSet(i); for (int index = prvBitset.nextSetBit(0); index >= 0; index = prvBitset.nextSetBit(index + 1)) { - createRow(blockChunkHolder, row, i, index); + createRow(rawBlockletColumnChunks, row, i, index); Boolean rslt = false; try { rslt = exp.evaluate(row).getBoolean(); @@ -371,8 +373,8 @@ private RowIntf convertRow(RowIntf value, int dimOrdinalMax) throws IOException * @param index * @throws IOException */ - private void createRow(BlocksChunkHolder blockChunkHolder, RowIntf row, int pageIndex, int index) - throws IOException { + private void createRow(RawBlockletColumnChunks blockChunkHolder, RowIntf row, int pageIndex, + int index) throws IOException { Object[] record = new Object[dimColEvaluatorInfoList.size() + msrColEvalutorInfoList.size()]; String memberString; for (int i = 0; i < dimColEvaluatorInfoList.size(); i++) { @@ -389,14 +391,14 @@ private void createRow(BlocksChunkHolder blockChunkHolder, RowIntf row, int page record[dimColumnEvaluatorInfo.getRowIndex()] = dimColumnEvaluatorInfo.getDimension().getDefaultValue(); } - DimensionColumnDataChunk columnDataChunk = - blockChunkHolder.getDimensionRawDataChunk()[dimensionBlocksIndex[i]] - .convertToDimColDataChunk(pageIndex); + DimensionColumnPage columnDataChunk = + blockChunkHolder.getDimensionRawColumnChunks()[dimensionChunkIndex[i]] + .decodeColumnPage(pageIndex); if (!dimColumnEvaluatorInfo.getDimension().hasEncoding(Encoding.DICTIONARY) - && columnDataChunk instanceof VariableLengthDimensionDataChunk) { + && columnDataChunk instanceof VariableLengthDimensionColumnPage) { - VariableLengthDimensionDataChunk dimensionColumnDataChunk = - (VariableLengthDimensionDataChunk) columnDataChunk; + VariableLengthDimensionColumnPage dimensionColumnDataChunk = + (VariableLengthDimensionColumnPage) columnDataChunk; byte[] memberBytes = dimensionColumnDataChunk.getChunkData(index); if (null != memberBytes) { if (Arrays.equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY, memberBytes)) { @@ -409,8 +411,8 @@ private void createRow(BlocksChunkHolder blockChunkHolder, RowIntf row, int page dimColumnEvaluatorInfo.getDimension().getDataType()); } } else { - int dictionaryValue = readSurrogatesFromColumnBlock(blockChunkHolder, index, pageIndex, - dimColumnEvaluatorInfo, dimensionBlocksIndex[i]); + int dictionaryValue = readSurrogatesFromColumnChunk(blockChunkHolder, index, pageIndex, + dimColumnEvaluatorInfo, dimensionChunkIndex[i]); if (dimColumnEvaluatorInfo.getDimension().hasEncoding(Encoding.DICTIONARY) && !dimColumnEvaluatorInfo.getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) { memberString = @@ -428,11 +430,11 @@ private void createRow(BlocksChunkHolder blockChunkHolder, RowIntf row, int page } } else { try { - GenericQueryType complexType = complexDimensionInfoMap.get(dimensionBlocksIndex[i]); + GenericQueryType complexType = complexDimensionInfoMap.get(dimensionChunkIndex[i]); ByteArrayOutputStream byteStream = new ByteArrayOutputStream(); DataOutputStream dataOutputStream = new DataOutputStream(byteStream); complexType.parseBlocksAndReturnComplexColumnByteArray( - blockChunkHolder.getDimensionRawDataChunk(), index, pageIndex, dataOutputStream); + blockChunkHolder.getDimensionRawColumnChunks(), index, pageIndex, dataOutputStream); record[dimColumnEvaluatorInfo.getRowIndex()] = complexType .getDataBasedOnDataTypeFromSurrogates(ByteBuffer.wrap(byteStream.toByteArray())); byteStream.close(); @@ -471,8 +473,8 @@ private void createRow(BlocksChunkHolder blockChunkHolder, RowIntf row, int page Object msrValue; ColumnPage columnPage = - blockChunkHolder.getMeasureRawDataChunk()[measureBlocksIndex[0]] - .convertToColumnPage(pageIndex); + blockChunkHolder.getMeasureRawColumnChunks()[measureChunkIndex[0]] + .decodeColumnPage(pageIndex); if (msrType == DataTypes.BOOLEAN) { msrValue = columnPage.getBoolean(index); } else if (msrType == DataTypes.SHORT) { @@ -574,10 +576,10 @@ private String getFilterActualValueFromDictionaryValue( * @param dimColumnEvaluatorInfo * @return */ - private int readSurrogatesFromColumnBlock(BlocksChunkHolder blockChunkHolder, int index, int page, - DimColumnResolvedFilterInfo dimColumnEvaluatorInfo, int blockIndex) { - DimensionColumnDataChunk dataChunk = - blockChunkHolder.getDimensionRawDataChunk()[blockIndex].convertToDimColDataChunk(page); + private int readSurrogatesFromColumnChunk(RawBlockletColumnChunks blockChunkHolder, int index, + int page, DimColumnResolvedFilterInfo dimColumnEvaluatorInfo, int chunkIndex) { + DimensionColumnPage dataChunk = + blockChunkHolder.getDimensionRawColumnChunks()[chunkIndex].decodeColumnPage(page); if (dimColumnEvaluatorInfo.getDimension().isColumnar()) { byte[] rawData = dataChunk.getChunkData(index); ByteBuffer byteBuffer = ByteBuffer.allocate(CarbonCommonConstants.INT_SIZE_IN_BYTE); @@ -593,7 +595,7 @@ private int readSurrogatesFromColumnBlock(BlocksChunkHolder blockChunkHolder, in * @param dimColumnEvaluatorInfo * @return read surrogate of given row of given column group dimension */ - private int readSurrogatesFromColumnGroupBlock(DimensionColumnDataChunk chunk, int index, + private int readSurrogatesFromColumnGroupBlock(DimensionColumnPage chunk, int index, DimColumnResolvedFilterInfo dimColumnEvaluatorInfo) { try { KeyStructureInfo keyStructureInfo = @@ -601,9 +603,9 @@ private int readSurrogatesFromColumnGroupBlock(DimensionColumnDataChunk chunk, i byte[] colData = chunk.getChunkData(index); long[] result = keyStructureInfo.getKeyGenerator().getKeyArray(colData); int colGroupId = - QueryUtil.getColumnGroupId(segmentProperties, dimensionBlocksIndex[0]); + QueryUtil.getColumnGroupId(segmentProperties, dimensionChunkIndex[0]); return (int) result[segmentProperties - .getColumnGroupMdKeyOrdinal(colGroupId, dimensionBlocksIndex[0])]; + .getColumnGroupMdKeyOrdinal(colGroupId, dimensionChunkIndex[0])]; } catch (KeyGenException e) { LOGGER.error(e); } @@ -611,33 +613,36 @@ private int readSurrogatesFromColumnGroupBlock(DimensionColumnDataChunk chunk, i } - @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) { + @Override + public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) { BitSet bitSet = new BitSet(1); bitSet.set(0); return bitSet; } - @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException { + @Override + public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException { for (int i = 0; i < dimColEvaluatorInfoList.size(); i++) { DimColumnResolvedFilterInfo dimColumnEvaluatorInfo = dimColEvaluatorInfoList.get(i); if (!dimColumnEvaluatorInfo.getDimension().getDataType().isComplexType()) { - if (null == blockChunkHolder.getDimensionRawDataChunk()[dimensionBlocksIndex[i]]) { - blockChunkHolder.getDimensionRawDataChunk()[dimensionBlocksIndex[i]] = - blockChunkHolder.getDataBlock() - .getDimensionChunk(blockChunkHolder.getFileReader(), dimensionBlocksIndex[i]); + if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[dimensionChunkIndex[i]]) + { + rawBlockletColumnChunks.getDimensionRawColumnChunks()[dimensionChunkIndex[i]] = + rawBlockletColumnChunks.getDataBlock().readDimensionChunk( + rawBlockletColumnChunks.getFileReader(), dimensionChunkIndex[i]); } } else { - GenericQueryType complexType = complexDimensionInfoMap.get(dimensionBlocksIndex[i]); - complexType.fillRequiredBlockData(blockChunkHolder); + GenericQueryType complexType = complexDimensionInfoMap.get(dimensionChunkIndex[i]); + complexType.fillRequiredBlockData(rawBlockletColumnChunks); } } if (null != msrColEvalutorInfoList) { for (MeasureColumnResolvedFilterInfo msrColumnEvalutorInfo : msrColEvalutorInfoList) { - if (null == blockChunkHolder.getMeasureRawDataChunk()[measureBlocksIndex[0]]) { - blockChunkHolder.getMeasureRawDataChunk()[measureBlocksIndex[0]] = - blockChunkHolder.getDataBlock() - .getMeasureChunk(blockChunkHolder.getFileReader(), measureBlocksIndex[0]); + if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[measureChunkIndex[0]]) { + rawBlockletColumnChunks.getMeasureRawColumnChunks()[measureChunkIndex[0]] = + rawBlockletColumnChunks.getDataBlock() + .readMeasureChunk(rawBlockletColumnChunks.getFileReader(), measureChunkIndex[0]); } } } 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 39812111d23..1f63a813bba 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 @@ -22,7 +22,7 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants; import org.apache.carbondata.core.datastore.block.SegmentProperties; -import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk; +import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage; import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk; import org.apache.carbondata.core.datastore.page.ColumnPage; @@ -32,12 +32,11 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension; import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure; import org.apache.carbondata.core.scan.expression.Expression; -import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException; import org.apache.carbondata.core.scan.filter.FilterUtil; import org.apache.carbondata.core.scan.filter.intf.RowIntf; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo; -import org.apache.carbondata.core.scan.processor.BlocksChunkHolder; +import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks; import org.apache.carbondata.core.util.BitSetGroup; import org.apache.carbondata.core.util.ByteUtil; import org.apache.carbondata.core.util.CarbonUtil; @@ -56,12 +55,11 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute */ private boolean isDefaultValuePresentInFilter; private int lastDimensionColOrdinal = 0; - public RowLevelRangeGrtThanFiterExecuterImpl( - List dimColEvaluatorInfoList, + + RowLevelRangeGrtThanFiterExecuterImpl(List dimColEvaluatorInfoList, List msrColEvalutorInfoList, Expression exp, AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues, - Object[] msrFilterRangeValues, - SegmentProperties segmentProperties) { + Object[] msrFilterRangeValues, SegmentProperties segmentProperties) { super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier, segmentProperties, null); this.filterRangeValues = filterRangeValues; @@ -111,17 +109,18 @@ private void ifDefaultValueMatchesFilter() { } } - @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) { + @Override + public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) { BitSet bitSet = new BitSet(1); boolean isScanRequired = false; byte[] maxValue = null; if (isMeasurePresentInCurrentBlock[0] || isDimensionPresentInCurrentBlock[0]) { if (isMeasurePresentInCurrentBlock[0]) { - maxValue = blockMaxValue[measureBlocksIndex[0] + lastDimensionColOrdinal]; + maxValue = blockMaxValue[measureChunkIndex[0] + lastDimensionColOrdinal]; isScanRequired = isScanRequired(maxValue, msrFilterRangeValues, msrColEvalutorInfoList.get(0).getType()); } else { - maxValue = blockMaxValue[dimensionBlocksIndex[0]]; + maxValue = blockMaxValue[dimensionChunkIndex[0]]; isScanRequired = isScanRequired(maxValue, filterRangeValues); } } else { @@ -168,24 +167,25 @@ private boolean isScanRequired(byte[] maxValue, Object[] filterValue, } @Override - public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine) - throws FilterUnsupportedException, IOException { + public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks, + boolean useBitsetPipeLine) throws IOException { // select all rows if dimension does not exists in the current block if (!isDimensionPresentInCurrentBlock[0] && !isMeasurePresentInCurrentBlock[0]) { - int numberOfRows = blockChunkHolder.getDataBlock().nodeSize(); + int numberOfRows = rawBlockletColumnChunks.getDataBlock().numRows(); return FilterUtil - .createBitSetGroupWithDefaultValue(blockChunkHolder.getDataBlock().numberOfPages(), + .createBitSetGroupWithDefaultValue(rawBlockletColumnChunks.getDataBlock().numberOfPages(), numberOfRows, true); } if (isDimensionPresentInCurrentBlock[0]) { - int blockIndex = - segmentProperties.getDimensionOrdinalToBlockMapping().get(dimensionBlocksIndex[0]); - if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) { - blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex); + int chunkIndex = + segmentProperties.getDimensionOrdinalToChunkMapping().get(dimensionChunkIndex[0]); + if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) { + rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] = + rawBlockletColumnChunks.getDataBlock().readDimensionChunk( + rawBlockletColumnChunks.getFileReader(), chunkIndex); } DimensionRawColumnChunk rawColumnChunk = - blockChunkHolder.getDimensionRawDataChunk()[blockIndex]; + rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]; BitSetGroup bitSetGroup = new BitSetGroup(rawColumnChunk.getPagesCount()); for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) { if (rawColumnChunk.getMaxValues() != null) { @@ -197,26 +197,28 @@ public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBi bitSet.flip(0, rawColumnChunk.getRowCount()[i]); bitSetGroup.setBitSet(bitSet, i); } else { - BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i), + BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i), rawColumnChunk.getRowCount()[i]); bitSetGroup.setBitSet(bitSet, i); } } } else { - BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i), + BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i), rawColumnChunk.getRowCount()[i]); bitSetGroup.setBitSet(bitSet, i); } } return bitSetGroup; } else if (isMeasurePresentInCurrentBlock[0]) { - int blockIndex = - segmentProperties.getMeasuresOrdinalToBlockMapping().get(measureBlocksIndex[0]); - if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) { - blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex); + int chunkIndex = + segmentProperties.getMeasuresOrdinalToChunkMapping().get(measureChunkIndex[0]); + if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) { + rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] = + rawBlockletColumnChunks.getDataBlock().readMeasureChunk( + rawBlockletColumnChunks.getFileReader(), chunkIndex); } - MeasureRawColumnChunk rawColumnChunk = blockChunkHolder.getMeasureRawDataChunk()[blockIndex]; + MeasureRawColumnChunk rawColumnChunk = + rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]; BitSetGroup bitSetGroup = new BitSetGroup(rawColumnChunk.getPagesCount()); for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) { if (rawColumnChunk.getMaxValues() != null) { @@ -226,7 +228,7 @@ public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBi .getMeasureObjectFromDataType(rawColumnChunk.getMinValues()[i], msrColEvalutorInfoList.get(0).getType())); ColumnPage columnPage = - rawColumnChunk.convertToColumnPage(i); + rawColumnChunk.decodeColumnPage(i); if (compare < 0 && columnPage.getNullBits().isEmpty()) { BitSet bitSet = new BitSet(rawColumnChunk.getRowCount()[i]); bitSet.flip(0, rawColumnChunk.getRowCount()[i]); @@ -239,7 +241,7 @@ public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBi } } else { BitSet bitSet = - getFilteredIndexesForMeasures(rawColumnChunk.convertToColumnPage(i), + getFilteredIndexesForMeasures(rawColumnChunk.decodeColumnPage(i), rawColumnChunk.getRowCount()[i]); bitSetGroup.setBitSet(bitSet, i); } @@ -296,16 +298,16 @@ private BitSet getFilteredIndexesForMeasures(ColumnPage columnPage, return bitSet; } - private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk, + private BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage, int numerOfRows) { BitSet bitSet = null; - if (dimensionColumnDataChunk.isExplicitSorted()) { - bitSet = setFilterdIndexToBitSetWithColumnIndex(dimensionColumnDataChunk, numerOfRows); + if (dimensionColumnPage.isExplicitSorted()) { + bitSet = setFilterdIndexToBitSetWithColumnIndex(dimensionColumnPage, numerOfRows); } else { - bitSet = setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows); + bitSet = setFilterdIndexToBitSet(dimensionColumnPage, numerOfRows); } - if (dimensionColumnDataChunk.isNoDicitionaryColumn()) { - FilterUtil.removeNullValues(dimensionColumnDataChunk, bitSet, + if (dimensionColumnPage.isNoDicitionaryColumn()) { + FilterUtil.removeNullValues(dimensionColumnPage, bitSet, CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY); } return bitSet; @@ -317,12 +319,12 @@ private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataCh * column is not supported by default so column index mapping will be present for * accesing the members from the block. * - * @param dimensionColumnDataChunk + * @param dimensionColumnPage * @param numerOfRows * @return BitSet. */ private BitSet setFilterdIndexToBitSetWithColumnIndex( - DimensionColumnDataChunk dimensionColumnDataChunk, int numerOfRows) { + DimensionColumnPage dimensionColumnPage, int numerOfRows) { BitSet bitSet = new BitSet(numerOfRows); int start = 0; int last = 0; @@ -330,11 +332,11 @@ private BitSet setFilterdIndexToBitSetWithColumnIndex( byte[][] filterValues = this.filterRangeValues; for (int i = 0; i < filterValues.length; i++) { start = CarbonUtil - .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1, + .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1, filterValues[i], true); if (start >= 0) { start = CarbonUtil - .nextGreaterValueToTarget(start, dimensionColumnDataChunk, filterValues[i], + .nextGreaterValueToTarget(start, dimensionColumnPage, filterValues[i], numerOfRows); } // Logic will handle the case where the range filter member is not present in block @@ -350,7 +352,7 @@ private BitSet setFilterdIndexToBitSetWithColumnIndex( // index needs to be compared by the filter member if its > filter then from that // index the bitset will be considered for filtering process. if (ByteUtil.compare(filterValues[i], - dimensionColumnDataChunk.getChunkData(dimensionColumnDataChunk.getInvertedIndex(start))) + dimensionColumnPage.getChunkData(dimensionColumnPage.getInvertedIndex(start))) > 0) { start = start + 1; } @@ -358,7 +360,7 @@ private BitSet setFilterdIndexToBitSetWithColumnIndex( last = start; for (int j = start; j < numerOfRows; j++) { - bitSet.set(dimensionColumnDataChunk.getInvertedIndex(j)); + bitSet.set(dimensionColumnPage.getInvertedIndex(j)); last++; } startIndex = last; @@ -376,11 +378,11 @@ private BitSet setFilterdIndexToBitSetWithColumnIndex( * be called if the column is sorted default so column index * mapping will be present for accesing the members from the block. * - * @param dimensionColumnDataChunk + * @param dimensionColumnPage * @param numerOfRows * @return BitSet. */ - private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk, + private BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage, int numerOfRows) { BitSet bitSet = new BitSet(numerOfRows); byte[][] filterValues = this.filterRangeValues; @@ -391,11 +393,11 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD int startIndex = 0; for (int k = 0; k < filterValues.length; k++) { start = CarbonUtil - .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, + .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1, filterValues[k], true); if (start >= 0) { start = CarbonUtil - .nextGreaterValueToTarget(start, dimensionColumnDataChunk, filterValues[k], + .nextGreaterValueToTarget(start, dimensionColumnPage, filterValues[k], numerOfRows); } if (start < 0) { @@ -406,7 +408,7 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD // Method will compare the tentative index value after binary search, this tentative // index needs to be compared by the filter member if its > filter then from that // index the bitset will be considered for filtering process. - if (ByteUtil.compare(filterValues[k], dimensionColumnDataChunk.getChunkData(start)) > 0) { + if (ByteUtil.compare(filterValues[k], dimensionColumnPage.getChunkData(start)) > 0) { start = start + 1; } } @@ -423,7 +425,7 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD } else { for (int k = 0; k < filterValues.length; k++) { for (int i = 0; i < numerOfRows; i++) { - if (ByteUtil.compare(dimensionColumnDataChunk.getChunkData(i), filterValues[k]) > 0) { + if (ByteUtil.compare(dimensionColumnPage.getChunkData(i), filterValues[k]) > 0) { bitSet.set(i); } } @@ -432,21 +434,24 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD return bitSet; } - @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException { + @Override + public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException { if (isDimensionPresentInCurrentBlock[0]) { if (!dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY)) { - super.readBlocks(blockChunkHolder); + super.readColumnChunks(rawBlockletColumnChunks); } - int blockIndex = dimensionBlocksIndex[0]; - if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) { - blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex); + int chunkIndex = dimensionChunkIndex[0]; + if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) { + rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] = + rawBlockletColumnChunks.getDataBlock().readDimensionChunk( + rawBlockletColumnChunks.getFileReader(), chunkIndex); } } else if (isMeasurePresentInCurrentBlock[0]) { - int blockIndex = measureBlocksIndex[0]; - if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) { - blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex); + int chunkIndex = measureChunkIndex[0]; + if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) { + rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] = + rawBlockletColumnChunks.getDataBlock().readMeasureChunk( + rawBlockletColumnChunks.getFileReader(), chunkIndex); } } } 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 f2ddcb6a68e..9140a1156f8 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 @@ -22,7 +22,7 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants; import org.apache.carbondata.core.datastore.block.SegmentProperties; -import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk; +import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage; import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk; import org.apache.carbondata.core.datastore.page.ColumnPage; @@ -32,12 +32,11 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension; import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure; import org.apache.carbondata.core.scan.expression.Expression; -import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException; import org.apache.carbondata.core.scan.filter.FilterUtil; import org.apache.carbondata.core.scan.filter.intf.RowIntf; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo; -import org.apache.carbondata.core.scan.processor.BlocksChunkHolder; +import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks; import org.apache.carbondata.core.util.BitSetGroup; import org.apache.carbondata.core.util.ByteUtil; import org.apache.carbondata.core.util.CarbonUtil; @@ -47,7 +46,7 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilterExecuterImpl { - protected byte[][] filterRangeValues; + private byte[][] filterRangeValues; private Object[] msrFilterRangeValues; private SerializableComparator comparator; /** @@ -56,7 +55,7 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte private boolean isDefaultValuePresentInFilter; private int lastDimensionColOrdinal = 0; - public RowLevelRangeGrtrThanEquaToFilterExecuterImpl( + RowLevelRangeGrtrThanEquaToFilterExecuterImpl( List dimColEvaluatorInfoList, List msrColEvalutorInfoList, Expression exp, AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues, @@ -116,11 +115,11 @@ private void ifDefaultValueMatchesFilter() { byte[] maxValue = null; if (isMeasurePresentInCurrentBlock[0] || isDimensionPresentInCurrentBlock[0]) { if (isMeasurePresentInCurrentBlock[0]) { - maxValue = blockMaxValue[measureBlocksIndex[0] + lastDimensionColOrdinal]; + maxValue = blockMaxValue[measureChunkIndex[0] + lastDimensionColOrdinal]; isScanRequired = isScanRequired(maxValue, msrFilterRangeValues, msrColEvalutorInfoList.get(0).getType()); } else { - maxValue = blockMaxValue[dimensionBlocksIndex[0]]; + maxValue = blockMaxValue[dimensionChunkIndex[0]]; isScanRequired = isScanRequired(maxValue, filterRangeValues); } } else { @@ -166,25 +165,26 @@ private boolean isScanRequired(byte[] maxValue, Object[] filterValue, } @Override - public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine) - throws FilterUnsupportedException, IOException { + public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks, + boolean useBitsetPipeLine) throws IOException { // select all rows if dimension does not exists in the current block if (!isDimensionPresentInCurrentBlock[0] && !isMeasurePresentInCurrentBlock[0]) { - int numberOfRows = blockChunkHolder.getDataBlock().nodeSize(); + int numberOfRows = rawBlockletColumnChunks.getDataBlock().numRows(); return FilterUtil - .createBitSetGroupWithDefaultValue(blockChunkHolder.getDataBlock().numberOfPages(), + .createBitSetGroupWithDefaultValue(rawBlockletColumnChunks.getDataBlock().numberOfPages(), numberOfRows, true); } if (isDimensionPresentInCurrentBlock[0]) { - int blockIndex = - segmentProperties.getDimensionOrdinalToBlockMapping().get(dimensionBlocksIndex[0]); - if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) { - blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex); + int chunkIndex = + segmentProperties.getDimensionOrdinalToChunkMapping().get(dimensionChunkIndex[0]); + if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) { + rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] = + rawBlockletColumnChunks.getDataBlock().readDimensionChunk( + rawBlockletColumnChunks.getFileReader(), chunkIndex); } DimensionRawColumnChunk rawColumnChunk = - blockChunkHolder.getDimensionRawDataChunk()[blockIndex]; + rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]; BitSetGroup bitSetGroup = new BitSetGroup(rawColumnChunk.getPagesCount()); for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) { if (rawColumnChunk.getMaxValues() != null) { @@ -196,26 +196,28 @@ public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBi bitSet.flip(0, rawColumnChunk.getRowCount()[i]); bitSetGroup.setBitSet(bitSet, i); } else { - BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i), + BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i), rawColumnChunk.getRowCount()[i]); bitSetGroup.setBitSet(bitSet, i); } } } else { - BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i), + BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i), rawColumnChunk.getRowCount()[i]); bitSetGroup.setBitSet(bitSet, i); } } return bitSetGroup; - } else if (isMeasurePresentInCurrentBlock[0]) { - int blockIndex = - segmentProperties.getMeasuresOrdinalToBlockMapping().get(measureBlocksIndex[0]); - if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) { - blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex); + } else { + int chunkIndex = + segmentProperties.getMeasuresOrdinalToChunkMapping().get(measureChunkIndex[0]); + if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) { + rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] = + rawBlockletColumnChunks.getDataBlock().readMeasureChunk( + rawBlockletColumnChunks.getFileReader(), chunkIndex); } - MeasureRawColumnChunk rawColumnChunk = blockChunkHolder.getMeasureRawDataChunk()[blockIndex]; + MeasureRawColumnChunk rawColumnChunk = + rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]; BitSetGroup bitSetGroup = new BitSetGroup(rawColumnChunk.getPagesCount()); for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) { if (rawColumnChunk.getMaxValues() != null) { @@ -225,28 +227,27 @@ public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBi .getMeasureObjectFromDataType(rawColumnChunk.getMinValues()[i], msrColEvalutorInfoList.get(0).getType())); ColumnPage columnPage = - rawColumnChunk.convertToColumnPage(i); + rawColumnChunk.decodeColumnPage(i); if (compare <= 0 && columnPage.getNullBits().isEmpty()) { BitSet bitSet = new BitSet(rawColumnChunk.getRowCount()[i]); bitSet.flip(0, rawColumnChunk.getRowCount()[i]); bitSetGroup.setBitSet(bitSet, i); } else { BitSet bitSet = - getFilteredIndexesForMeasures(rawColumnChunk.convertToColumnPage(i), + getFilteredIndexesForMeasures(rawColumnChunk.decodeColumnPage(i), rawColumnChunk.getRowCount()[i]); bitSetGroup.setBitSet(bitSet, i); } } } else { BitSet bitSet = - getFilteredIndexesForMeasures(rawColumnChunk.convertToColumnPage(i), + getFilteredIndexesForMeasures(rawColumnChunk.decodeColumnPage(i), rawColumnChunk.getRowCount()[i]); bitSetGroup.setBitSet(bitSet, i); } } return bitSetGroup; } - return null; } @Override @@ -297,16 +298,16 @@ private BitSet getFilteredIndexesForMeasures(ColumnPage columnPage, } - private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk, + private BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage, int numerOfRows) { BitSet bitSet = null; - if (dimensionColumnDataChunk.isExplicitSorted()) { - bitSet = setFilterdIndexToBitSetWithColumnIndex(dimensionColumnDataChunk, numerOfRows); + if (dimensionColumnPage.isExplicitSorted()) { + bitSet = setFilterdIndexToBitSetWithColumnIndex(dimensionColumnPage, numerOfRows); } else { - bitSet = setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows); + bitSet = setFilterdIndexToBitSet(dimensionColumnPage, numerOfRows); } - if (dimensionColumnDataChunk.isNoDicitionaryColumn()) { - FilterUtil.removeNullValues(dimensionColumnDataChunk, bitSet, + if (dimensionColumnPage.isNoDicitionaryColumn()) { + FilterUtil.removeNullValues(dimensionColumnPage, bitSet, CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY); } return bitSet; @@ -318,12 +319,12 @@ private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataCh * column is not supported by default so column index mapping will be present for * accesing the members from the block. * - * @param dimensionColumnDataChunk + * @param dimensionColumnPage * @param numerOfRows * @return BitSet. */ private BitSet setFilterdIndexToBitSetWithColumnIndex( - DimensionColumnDataChunk dimensionColumnDataChunk, int numerOfRows) { + DimensionColumnPage dimensionColumnPage, int numerOfRows) { BitSet bitSet = new BitSet(numerOfRows); int start = 0; int last = 0; @@ -331,7 +332,7 @@ private BitSet setFilterdIndexToBitSetWithColumnIndex( byte[][] filterValues = this.filterRangeValues; for (int i = 0; i < filterValues.length; i++) { start = CarbonUtil - .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1, + .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1, filterValues[i], false); if (start < 0) { start = -(start + 1); @@ -342,14 +343,14 @@ private BitSet setFilterdIndexToBitSetWithColumnIndex( // index needs to be compared by the filter member if its >= filter then from that // index the bitset will be considered for filtering process. if (ByteUtil.compare(filterValues[i], - dimensionColumnDataChunk.getChunkData(dimensionColumnDataChunk.getInvertedIndex(start))) + dimensionColumnPage.getChunkData(dimensionColumnPage.getInvertedIndex(start))) > 0) { start = start + 1; } } last = start; for (int j = start; j < numerOfRows; j++) { - bitSet.set(dimensionColumnDataChunk.getInvertedIndex(j)); + bitSet.set(dimensionColumnPage.getInvertedIndex(j)); last++; } startIndex = last; @@ -366,11 +367,11 @@ private BitSet setFilterdIndexToBitSetWithColumnIndex( * be called if the column is sorted default so column index * mapping will be present for accesing the members from the block. * - * @param dimensionColumnDataChunk + * @param dimensionColumnPage * @param numerOfRows * @return BitSet. */ - private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk, + private BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage, int numerOfRows) { BitSet bitSet = new BitSet(numerOfRows); byte[][] filterValues = this.filterRangeValues; @@ -381,7 +382,7 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD int startIndex = 0; for (int k = 0; k < filterValues.length; k++) { start = CarbonUtil - .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, + .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1, filterValues[k], false); if (start < 0) { start = -(start + 1); @@ -391,7 +392,7 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD // Method will compare the tentative index value after binary search, this tentative // index needs to be compared by the filter member if its >= filter then from that // index the bitset will be considered for filtering process. - if (ByteUtil.compare(filterValues[k], dimensionColumnDataChunk.getChunkData(start)) > 0) { + if (ByteUtil.compare(filterValues[k], dimensionColumnPage.getChunkData(start)) > 0) { start = start + 1; } } @@ -409,7 +410,7 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD } else { for (int k = 0; k < filterValues.length; k++) { for (int i = 0; i < numerOfRows; i++) { - if (ByteUtil.compare(dimensionColumnDataChunk.getChunkData(i), filterValues[k]) >= 0) { + if (ByteUtil.compare(dimensionColumnPage.getChunkData(i), filterValues[k]) >= 0) { bitSet.set(i); } } @@ -418,21 +419,24 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD return bitSet; } - @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException { + @Override + public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException { if (isDimensionPresentInCurrentBlock[0]) { if (!dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY)) { - super.readBlocks(blockChunkHolder); + super.readColumnChunks(rawBlockletColumnChunks); } - int blockIndex = dimensionBlocksIndex[0]; - if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) { - blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex); + int chunkIndex = dimensionChunkIndex[0]; + if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) { + rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] = + rawBlockletColumnChunks.getDataBlock().readDimensionChunk( + rawBlockletColumnChunks.getFileReader(), chunkIndex); } } else if (isMeasurePresentInCurrentBlock[0]) { - int blockIndex = measureBlocksIndex[0]; - if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) { - blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex); + int chunkIndex = measureChunkIndex[0]; + if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) { + rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] = + rawBlockletColumnChunks.getDataBlock().readMeasureChunk( + rawBlockletColumnChunks.getFileReader(), chunkIndex); } } } 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 a44bc1ada2c..120671fadef 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 @@ -22,7 +22,7 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants; import org.apache.carbondata.core.datastore.block.SegmentProperties; -import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk; +import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage; import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk; import org.apache.carbondata.core.datastore.page.ColumnPage; @@ -35,12 +35,11 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension; import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure; import org.apache.carbondata.core.scan.expression.Expression; -import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException; import org.apache.carbondata.core.scan.filter.FilterUtil; import org.apache.carbondata.core.scan.filter.intf.RowIntf; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo; -import org.apache.carbondata.core.scan.processor.BlocksChunkHolder; +import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks; import org.apache.carbondata.core.util.BitSetGroup; import org.apache.carbondata.core.util.ByteUtil; import org.apache.carbondata.core.util.CarbonUtil; @@ -118,11 +117,11 @@ private void ifDefaultValueMatchesFilter() { boolean isScanRequired = false; if (isMeasurePresentInCurrentBlock[0] || isDimensionPresentInCurrentBlock[0]) { if (isMeasurePresentInCurrentBlock[0]) { - minValue = blockMinValue[measureBlocksIndex[0] + lastDimensionColOrdinal]; + minValue = blockMinValue[measureChunkIndex[0] + lastDimensionColOrdinal]; isScanRequired = isScanRequired(minValue, msrFilterRangeValues, msrColEvalutorInfoList.get(0).getType()); } else { - minValue = blockMinValue[dimensionBlocksIndex[0]]; + minValue = blockMinValue[dimensionChunkIndex[0]]; isScanRequired = isScanRequired(minValue, filterRangeValues); } } else { @@ -168,60 +167,63 @@ private boolean isScanRequired(byte[] minValue, Object[] filterValue, } @Override - public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine) - throws FilterUnsupportedException, IOException { + public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks, + boolean useBitsetPipeLine) throws IOException { // select all rows if dimension does not exists in the current block if (!isDimensionPresentInCurrentBlock[0] && !isMeasurePresentInCurrentBlock[0]) { - int numberOfRows = blockChunkHolder.getDataBlock().nodeSize(); + int numberOfRows = rawBlockletColumnChunks.getDataBlock().numRows(); return FilterUtil - .createBitSetGroupWithDefaultValue(blockChunkHolder.getDataBlock().numberOfPages(), + .createBitSetGroupWithDefaultValue(rawBlockletColumnChunks.getDataBlock().numberOfPages(), numberOfRows, true); } if (isDimensionPresentInCurrentBlock[0]) { - int blockIndex = - segmentProperties.getDimensionOrdinalToBlockMapping().get(dimensionBlocksIndex[0]); - if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) { - blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex); + int chunkIndex = + segmentProperties.getDimensionOrdinalToChunkMapping().get(dimensionChunkIndex[0]); + if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) { + rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] = + rawBlockletColumnChunks.getDataBlock().readDimensionChunk( + rawBlockletColumnChunks.getFileReader(), chunkIndex); } DimensionRawColumnChunk rawColumnChunk = - blockChunkHolder.getDimensionRawDataChunk()[blockIndex]; + rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]; BitSetGroup bitSetGroup = new BitSetGroup(rawColumnChunk.getPagesCount()); for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) { if (rawColumnChunk.getMinValues() != null) { if (isScanRequired(rawColumnChunk.getMinValues()[i], this.filterRangeValues)) { - BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i), + BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i), rawColumnChunk.getRowCount()[i]); bitSetGroup.setBitSet(bitSet, i); } } else { - BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i), + BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i), rawColumnChunk.getRowCount()[i]); bitSetGroup.setBitSet(bitSet, i); } } return bitSetGroup; } else if (isMeasurePresentInCurrentBlock[0]) { - int blockIndex = - segmentProperties.getMeasuresOrdinalToBlockMapping().get(measureBlocksIndex[0]); - if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) { - blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex); + int chunkIndex = + segmentProperties.getMeasuresOrdinalToChunkMapping().get(measureChunkIndex[0]); + if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) { + rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] = + rawBlockletColumnChunks.getDataBlock().readMeasureChunk( + rawBlockletColumnChunks.getFileReader(), chunkIndex); } - MeasureRawColumnChunk rawColumnChunk = blockChunkHolder.getMeasureRawDataChunk()[blockIndex]; + MeasureRawColumnChunk rawColumnChunk = + rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]; BitSetGroup bitSetGroup = new BitSetGroup(rawColumnChunk.getPagesCount()); for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) { if (rawColumnChunk.getMinValues() != null) { if (isScanRequired(rawColumnChunk.getMinValues()[i], this.msrFilterRangeValues, msrColEvalutorInfoList.get(0).getType())) { BitSet bitSet = - getFilteredIndexesForMeasures(rawColumnChunk.convertToColumnPage(i), + getFilteredIndexesForMeasures(rawColumnChunk.decodeColumnPage(i), rawColumnChunk.getRowCount()[i]); bitSetGroup.setBitSet(bitSet, i); } } else { BitSet bitSet = - getFilteredIndexesForMeasures(rawColumnChunk.convertToColumnPage(i), + getFilteredIndexesForMeasures(rawColumnChunk.decodeColumnPage(i), rawColumnChunk.getRowCount()[i]); bitSetGroup.setBitSet(bitSet, i); } @@ -278,7 +280,7 @@ private BitSet getFilteredIndexesForMeasures(ColumnPage columnPage, return bitSet; } - private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk, + private BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage, int numerOfRows) { byte[] defaultValue = null; if (dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) { @@ -287,7 +289,7 @@ private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataCh dimColEvaluatorInfoList.get(0).getDimension().getDataType()); int key = directDictionaryGenerator.generateDirectSurrogateKey(null); CarbonDimension currentBlockDimension = - segmentProperties.getDimensions().get(dimensionBlocksIndex[0]); + segmentProperties.getDimensions().get(dimensionChunkIndex[0]); if (currentBlockDimension.isSortColumn()) { defaultValue = FilterUtil.getMaskKey(key, currentBlockDimension, this.segmentProperties.getSortColumnsGenerator()); @@ -298,14 +300,14 @@ private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataCh defaultValue = CarbonCommonConstants.EMPTY_BYTE_ARRAY; } BitSet bitSet = null; - if (dimensionColumnDataChunk.isExplicitSorted()) { - bitSet = setFilterdIndexToBitSetWithColumnIndex(dimensionColumnDataChunk, numerOfRows, + if (dimensionColumnPage.isExplicitSorted()) { + bitSet = setFilterdIndexToBitSetWithColumnIndex(dimensionColumnPage, numerOfRows, defaultValue); } else { - bitSet = setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows, defaultValue); + bitSet = setFilterdIndexToBitSet(dimensionColumnPage, numerOfRows, defaultValue); } - if (dimensionColumnDataChunk.isNoDicitionaryColumn()) { - FilterUtil.removeNullValues(dimensionColumnDataChunk, bitSet, + if (dimensionColumnPage.isNoDicitionaryColumn()) { + FilterUtil.removeNullValues(dimensionColumnPage, bitSet, CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY); } return bitSet; @@ -317,12 +319,12 @@ private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataCh * column is not supported by default so column index mapping will be present for * accesing the members from the block. * - * @param dimensionColumnDataChunk + * @param dimensionColumnPage * @param numerOfRows * @return BitSet. */ private BitSet setFilterdIndexToBitSetWithColumnIndex( - DimensionColumnDataChunk dimensionColumnDataChunk, int numerOfRows, + DimensionColumnPage dimensionColumnPage, int numerOfRows, byte[] defaultValue) { BitSet bitSet = new BitSet(numerOfRows); int start = 0; @@ -333,7 +335,7 @@ private BitSet setFilterdIndexToBitSetWithColumnIndex( //find the number of default values to skip the null value in case of direct dictionary if (null != defaultValue) { start = CarbonUtil - .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1, + .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1, defaultValue, true); if (start < 0) { skip = -(start + 1); @@ -350,7 +352,7 @@ private BitSet setFilterdIndexToBitSetWithColumnIndex( } for (int i = 0; i < filterValues.length; i++) { start = CarbonUtil - .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1, + .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1, filterValues[i], true); if (start < 0) { start = -(start + 1); @@ -361,14 +363,14 @@ private BitSet setFilterdIndexToBitSetWithColumnIndex( // will be pointing to the next consecutive position. So compare it again and point to the // previous value returned from getFirstIndexUsingBinarySearch. if (ByteUtil.compare(filterValues[i], - dimensionColumnDataChunk.getChunkData(dimensionColumnDataChunk.getInvertedIndex(start))) + dimensionColumnPage.getChunkData(dimensionColumnPage.getInvertedIndex(start))) < 0) { start = start - 1; } } last = start; for (int j = start; j >= skip; j--) { - bitSet.set(dimensionColumnDataChunk.getInvertedIndex(j)); + bitSet.set(dimensionColumnPage.getInvertedIndex(j)); last--; } startIndex = last; @@ -385,12 +387,12 @@ private BitSet setFilterdIndexToBitSetWithColumnIndex( * be called if the column is sorted default so column index * mapping will be present for accesing the members from the block. * - * @param dimensionColumnDataChunk + * @param dimensionColumnPage * @param numerOfRows * @param defaultValue * @return BitSet. */ - private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk, + private BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage, int numerOfRows, byte[] defaultValue) { BitSet bitSet = new BitSet(numerOfRows); byte[][] filterValues = this.filterRangeValues; @@ -403,7 +405,7 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD //find the number of default values to skip the null value in case of direct dictionary if (null != defaultValue) { start = CarbonUtil - .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, + .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1, defaultValue, true); if (start < 0) { skip = -(start + 1); @@ -420,7 +422,7 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD } for (int k = 0; k < filterValues.length; k++) { start = CarbonUtil - .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, + .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1, filterValues[k], true); if (start < 0) { start = -(start + 1); @@ -430,7 +432,7 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD // When negative value of start is returned from getFirstIndexUsingBinarySearch the Start // will be pointing to the next consecutive position. So compare it again and point to the // previous value returned from getFirstIndexUsingBinarySearch. - if (ByteUtil.compare(filterValues[k], dimensionColumnDataChunk.getChunkData(start)) < 0) { + if (ByteUtil.compare(filterValues[k], dimensionColumnPage.getChunkData(start)) < 0) { start = start - 1; } } @@ -447,7 +449,7 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD } else { for (int k = 0; k < filterValues.length; k++) { for (int i = 0; i < numerOfRows; i++) { - if (ByteUtil.compare(dimensionColumnDataChunk.getChunkData(i), filterValues[k]) <= 0) { + if (ByteUtil.compare(dimensionColumnPage.getChunkData(i), filterValues[k]) <= 0) { bitSet.set(i); } } @@ -456,21 +458,24 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD return bitSet; } - @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException { + @Override public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) + throws IOException { if (isDimensionPresentInCurrentBlock[0]) { if (!dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY)) { - super.readBlocks(blockChunkHolder); + super.readColumnChunks(rawBlockletColumnChunks); } - int blockIndex = dimensionBlocksIndex[0]; - if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) { - blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex); + int chunkIndex = dimensionChunkIndex[0]; + if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) { + rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] = + rawBlockletColumnChunks.getDataBlock().readDimensionChunk( + rawBlockletColumnChunks.getFileReader(), chunkIndex); } } else if (isMeasurePresentInCurrentBlock[0]) { - int blockIndex = measureBlocksIndex[0]; - if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) { - blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex); + int chunkIndex = measureChunkIndex[0]; + if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) { + rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] = + rawBlockletColumnChunks.getDataBlock().readMeasureChunk( + rawBlockletColumnChunks.getFileReader(), chunkIndex); } } } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java index 447ab46fb1a..547ecaa3e3e 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java @@ -22,7 +22,7 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants; import org.apache.carbondata.core.datastore.block.SegmentProperties; -import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk; +import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage; import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk; import org.apache.carbondata.core.datastore.page.ColumnPage; @@ -35,12 +35,11 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension; import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure; import org.apache.carbondata.core.scan.expression.Expression; -import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException; import org.apache.carbondata.core.scan.filter.FilterUtil; import org.apache.carbondata.core.scan.filter.intf.RowIntf; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo; -import org.apache.carbondata.core.scan.processor.BlocksChunkHolder; +import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks; import org.apache.carbondata.core.util.BitSetGroup; import org.apache.carbondata.core.util.ByteUtil; import org.apache.carbondata.core.util.CarbonUtil; @@ -73,7 +72,7 @@ public RowLevelRangeLessThanFiterExecuterImpl( comparator = Comparator.getComparatorByDataTypeForMeasure(measure.getDataType()); } ifDefaultValueMatchesFilter(); - if (isDimensionPresentInCurrentBlock[0] == true) { + if (isDimensionPresentInCurrentBlock[0]) { isNaturalSorted = dimColEvaluatorInfoList.get(0).getDimension().isUseInvertedIndex() && dimColEvaluatorInfoList.get(0).getDimension().isSortColumn(); } @@ -120,11 +119,11 @@ private void ifDefaultValueMatchesFilter() { boolean isScanRequired = false; if (isMeasurePresentInCurrentBlock[0] || isDimensionPresentInCurrentBlock[0]) { if (isMeasurePresentInCurrentBlock[0]) { - minValue = blockMinValue[measureBlocksIndex[0] + lastDimensionColOrdinal]; + minValue = blockMinValue[measureChunkIndex[0] + lastDimensionColOrdinal]; isScanRequired = isScanRequired(minValue, msrFilterRangeValues, msrColEvalutorInfoList.get(0).getType()); } else { - minValue = blockMinValue[dimensionBlocksIndex[0]]; + minValue = blockMinValue[dimensionChunkIndex[0]]; isScanRequired = isScanRequired(minValue, filterRangeValues); } } else { @@ -170,67 +169,69 @@ private boolean isScanRequired(byte[] minValue, Object[] filterValue, } @Override - public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine) - throws FilterUnsupportedException, IOException { + public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks, + boolean useBitsetPipeLine) throws IOException { // select all rows if dimension does not exists in the current block if (!isDimensionPresentInCurrentBlock[0] && !isMeasurePresentInCurrentBlock[0]) { - int numberOfRows = blockChunkHolder.getDataBlock().nodeSize(); + int numberOfRows = rawBlockletColumnChunks.getDataBlock().numRows(); return FilterUtil - .createBitSetGroupWithDefaultValue(blockChunkHolder.getDataBlock().numberOfPages(), + .createBitSetGroupWithDefaultValue(rawBlockletColumnChunks.getDataBlock().numberOfPages(), numberOfRows, true); } if (isDimensionPresentInCurrentBlock[0]) { - int blockIndex = - segmentProperties.getDimensionOrdinalToBlockMapping().get(dimensionBlocksIndex[0]); - if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) { - blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex); + int chunkIndex = + segmentProperties.getDimensionOrdinalToChunkMapping().get(dimensionChunkIndex[0]); + if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) { + rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] = + rawBlockletColumnChunks.getDataBlock().readDimensionChunk( + rawBlockletColumnChunks.getFileReader(), chunkIndex); } DimensionRawColumnChunk rawColumnChunk = - blockChunkHolder.getDimensionRawDataChunk()[blockIndex]; + rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]; BitSetGroup bitSetGroup = new BitSetGroup(rawColumnChunk.getPagesCount()); for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) { if (rawColumnChunk.getMinValues() != null) { if (isScanRequired(rawColumnChunk.getMinValues()[i], this.filterRangeValues)) { - BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i), + BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i), rawColumnChunk.getRowCount()[i]); bitSetGroup.setBitSet(bitSet, i); } } else { - BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i), + BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i), rawColumnChunk.getRowCount()[i]); bitSetGroup.setBitSet(bitSet, i); } } return bitSetGroup; - } else if (isMeasurePresentInCurrentBlock[0]) { - int blockIndex = - segmentProperties.getMeasuresOrdinalToBlockMapping().get(measureBlocksIndex[0]); - if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) { - blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex); + } else { + int chunkIndex = + segmentProperties.getMeasuresOrdinalToChunkMapping().get(measureChunkIndex[0]); + if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) { + rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] = + rawBlockletColumnChunks.getDataBlock().readMeasureChunk( + rawBlockletColumnChunks.getFileReader(), chunkIndex); } - MeasureRawColumnChunk rawColumnChunk = blockChunkHolder.getMeasureRawDataChunk()[blockIndex]; + MeasureRawColumnChunk rawColumnChunk = + rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]; BitSetGroup bitSetGroup = new BitSetGroup(rawColumnChunk.getPagesCount()); for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) { if (rawColumnChunk.getMinValues() != null) { if (isScanRequired(rawColumnChunk.getMinValues()[i], this.msrFilterRangeValues, msrColEvalutorInfoList.get(0).getType())) { BitSet bitSet = - getFilteredIndexesForMeasures(rawColumnChunk.convertToColumnPage(i), + getFilteredIndexesForMeasures(rawColumnChunk.decodeColumnPage(i), rawColumnChunk.getRowCount()[i]); bitSetGroup.setBitSet(bitSet, i); } } else { BitSet bitSet = - getFilteredIndexesForMeasures(rawColumnChunk.convertToColumnPage(i), + getFilteredIndexesForMeasures(rawColumnChunk.decodeColumnPage(i), rawColumnChunk.getRowCount()[i]); bitSetGroup.setBitSet(bitSet, i); } } return bitSetGroup; } - return null; } @Override @@ -280,7 +281,7 @@ private BitSet getFilteredIndexesForMeasures(ColumnPage columnPage, return bitSet; } - private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk, + private BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage, int numerOfRows) { byte[] defaultValue = null; if (dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) { @@ -289,7 +290,7 @@ private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataCh dimColEvaluatorInfoList.get(0).getDimension().getDataType()); int key = directDictionaryGenerator.generateDirectSurrogateKey(null); CarbonDimension currentBlockDimension = - segmentProperties.getDimensions().get(dimensionBlocksIndex[0]); + segmentProperties.getDimensions().get(dimensionChunkIndex[0]); if (currentBlockDimension.isSortColumn()) { defaultValue = FilterUtil.getMaskKey(key, currentBlockDimension, this.segmentProperties.getSortColumnsGenerator()); @@ -300,14 +301,14 @@ private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataCh defaultValue = CarbonCommonConstants.EMPTY_BYTE_ARRAY; } BitSet bitSet = null; - if (dimensionColumnDataChunk.isExplicitSorted()) { - bitSet = setFilterdIndexToBitSetWithColumnIndex(dimensionColumnDataChunk, numerOfRows, + if (dimensionColumnPage.isExplicitSorted()) { + bitSet = setFilterdIndexToBitSetWithColumnIndex(dimensionColumnPage, numerOfRows, defaultValue); } else { - bitSet = setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows, defaultValue); + bitSet = setFilterdIndexToBitSet(dimensionColumnPage, numerOfRows, defaultValue); } - if (dimensionColumnDataChunk.isNoDicitionaryColumn()) { - FilterUtil.removeNullValues(dimensionColumnDataChunk, bitSet, + if (dimensionColumnPage.isNoDicitionaryColumn()) { + FilterUtil.removeNullValues(dimensionColumnPage, bitSet, CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY); } return bitSet; @@ -319,12 +320,12 @@ private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataCh * column is not supported by default so column index mapping will be present for * accesing the members from the block. * - * @param dimensionColumnDataChunk + * @param dimensionColumnPage * @param numerOfRows * @return BitSet. */ private BitSet setFilterdIndexToBitSetWithColumnIndex( - DimensionColumnDataChunk dimensionColumnDataChunk, int numerOfRows, + DimensionColumnPage dimensionColumnPage, int numerOfRows, byte[] defaultValue) { BitSet bitSet = new BitSet(numerOfRows); int start = 0; @@ -336,7 +337,7 @@ private BitSet setFilterdIndexToBitSetWithColumnIndex( //find the number of default values to skip the null value in case of direct dictionary if (null != defaultValue) { start = CarbonUtil - .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1, + .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1, defaultValue, true); if (start < 0) { skip = -(start + 1); @@ -354,7 +355,7 @@ private BitSet setFilterdIndexToBitSetWithColumnIndex( for (int i = 0; i < filterValues.length; i++) { start = CarbonUtil - .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1, + .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1, filterValues[i], false); if (start >= 0) { // Logic will handle the case where the range filter member is not present in block @@ -362,7 +363,7 @@ private BitSet setFilterdIndexToBitSetWithColumnIndex( // set inorder to apply filters. this is Lesser than filter so the range will be taken // from the prev element which is Lesser than filter member. start = - CarbonUtil.nextLesserValueToTarget(start, dimensionColumnDataChunk, filterValues[i]); + CarbonUtil.nextLesserValueToTarget(start, dimensionColumnPage, filterValues[i]); } if (start < 0) { start = -(start + 1); @@ -373,14 +374,14 @@ private BitSet setFilterdIndexToBitSetWithColumnIndex( // will be pointing to the next consecutive position. So compare it again and point to the // previous value returned from getFirstIndexUsingBinarySearch. if (ByteUtil.compare(filterValues[i], - dimensionColumnDataChunk.getChunkData(dimensionColumnDataChunk.getInvertedIndex(start))) + dimensionColumnPage.getChunkData(dimensionColumnPage.getInvertedIndex(start))) < 0) { start = start - 1; } } last = start; for (int j = start; j >= skip; j--) { - bitSet.set(dimensionColumnDataChunk.getInvertedIndex(j)); + bitSet.set(dimensionColumnPage.getInvertedIndex(j)); last--; } startIndex = last; @@ -397,11 +398,11 @@ private BitSet setFilterdIndexToBitSetWithColumnIndex( * be called if the column is sorted default so column index * mapping will be present for accesing the members from the block. * - * @param dimensionColumnDataChunk + * @param dimensionColumnPage * @param numerOfRows * @return BitSet. */ - private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk, + private BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage, int numerOfRows, byte[] defaultValue) { BitSet bitSet = new BitSet(numerOfRows); byte[][] filterValues = this.filterRangeValues; @@ -414,7 +415,7 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD //find the number of default values to skip the null value in case of direct dictionary if (null != defaultValue) { start = CarbonUtil - .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, + .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1, defaultValue, true); if (start < 0) { skip = -(start + 1); @@ -431,11 +432,11 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD } for (int k = 0; k < filterValues.length; k++) { start = CarbonUtil - .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, + .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1, filterValues[k], false); if (start >= 0) { start = - CarbonUtil.nextLesserValueToTarget(start, dimensionColumnDataChunk, filterValues[k]); + CarbonUtil.nextLesserValueToTarget(start, dimensionColumnPage, filterValues[k]); } if (start < 0) { start = -(start + 1); @@ -446,7 +447,7 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD // When negative value of start is returned from getFirstIndexUsingBinarySearch the Start // will be pointing to the next consecutive position. So compare it again and point to the // previous value returned from getFirstIndexUsingBinarySearch. - if (ByteUtil.compare(filterValues[k], dimensionColumnDataChunk.getChunkData(start)) < 0) { + if (ByteUtil.compare(filterValues[k], dimensionColumnPage.getChunkData(start)) < 0) { start = start - 1; } } @@ -463,7 +464,7 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD } else { for (int k = 0; k < filterValues.length; k++) { for (int i = 0; i < numerOfRows; i++) { - if (ByteUtil.compare(dimensionColumnDataChunk.getChunkData(i), filterValues[k]) < 0) { + if (ByteUtil.compare(dimensionColumnPage.getChunkData(i), filterValues[k]) < 0) { bitSet.set(i); } } @@ -472,21 +473,24 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD return bitSet; } - @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException { + @Override + public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException { if (isDimensionPresentInCurrentBlock[0]) { if (!dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY)) { - super.readBlocks(blockChunkHolder); + super.readColumnChunks(rawBlockletColumnChunks); } - int blockIndex = dimensionBlocksIndex[0]; - if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) { - blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex); + int chunkIndex = dimensionChunkIndex[0]; + if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) { + rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] = + rawBlockletColumnChunks.getDataBlock().readDimensionChunk( + rawBlockletColumnChunks.getFileReader(), chunkIndex); } } else if (isMeasurePresentInCurrentBlock[0]) { - int blockIndex = measureBlocksIndex[0]; - if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) { - blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock() - .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex); + int chunkIndex = measureChunkIndex[0]; + if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) { + rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] = + rawBlockletColumnChunks.getDataBlock().readMeasureChunk( + rawBlockletColumnChunks.getFileReader(), chunkIndex); } } } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/TrueFilterExecutor.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/TrueFilterExecutor.java index 4b3738a97b9..2cbb234750d 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/TrueFilterExecutor.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/TrueFilterExecutor.java @@ -22,7 +22,7 @@ import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException; import org.apache.carbondata.core.scan.filter.intf.RowIntf; -import org.apache.carbondata.core.scan.processor.BlocksChunkHolder; +import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks; import org.apache.carbondata.core.util.BitSetGroup; public class TrueFilterExecutor implements FilterExecuter { @@ -33,19 +33,20 @@ public class TrueFilterExecutor implements FilterExecuter { * @return * @throws FilterUnsupportedException */ - public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine) - throws FilterUnsupportedException, IOException { - int numberOfPages = blockChunkHolder.getDataBlock().numberOfPages(); + public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks, + boolean useBitsetPipeLine) throws FilterUnsupportedException, IOException { + int numberOfPages = rawBlockletColumnChunks.getDataBlock().numberOfPages(); BitSetGroup group = new BitSetGroup(numberOfPages); for (int i = 0; i < numberOfPages; i++) { BitSet set = new BitSet(); - set.flip(0, blockChunkHolder.getDataBlock().getPageRowCount(i)); + set.flip(0, rawBlockletColumnChunks.getDataBlock().getPageRowCount(i)); group.setBitSet(set, i); } return group; } - @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax) { + @Override + public boolean applyFilter(RowIntf value, int dimOrdinalMax) { return true; } @@ -66,9 +67,9 @@ public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) { /** * It just reads necessary block for filter executor, it does not uncompress the data. * - * @param blockChunkHolder + * @param rawBlockletColumnChunks */ - public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException { + public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) { // do nothing } } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ValueBasedFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ValueBasedFilterExecuterImpl.java index 516447f4187..bc3b39c9933 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ValueBasedFilterExecuterImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ValueBasedFilterExecuterImpl.java @@ -21,27 +21,29 @@ import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException; import org.apache.carbondata.core.scan.filter.intf.RowIntf; -import org.apache.carbondata.core.scan.processor.BlocksChunkHolder; +import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks; import org.apache.carbondata.core.util.BitSetGroup; public class ValueBasedFilterExecuterImpl implements FilterExecuter { @Override - public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine) - throws FilterUnsupportedException, IOException { - + public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks, + boolean useBitsetPipeLine) throws FilterUnsupportedException, IOException { return new BitSetGroup(0); } - @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax) + @Override + public boolean applyFilter(RowIntf value, int dimOrdinalMax) throws FilterUnsupportedException, IOException { throw new FilterUnsupportedException("Unsupported ValueBasedFilterExecuterImpl on row"); } - @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) { + @Override + public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) { return new BitSet(1); } - @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException { + @Override + public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException { } } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/intf/FilterOptimizerBasic.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/intf/FilterOptimizerBasic.java deleted file mode 100644 index b3ed34e877a..00000000000 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/intf/FilterOptimizerBasic.java +++ /dev/null @@ -1,26 +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.scan.filter.intf; - -import org.apache.carbondata.core.scan.expression.Expression; - -public class FilterOptimizerBasic implements FilterOptimizer { - @Override public Expression optimizeFilter() { - return null; - } -} diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/optimizer/RangeFilterOptmizer.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/optimizer/RangeFilterOptmizer.java index e5cb0ad8a2d..bbd33ecf311 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/optimizer/RangeFilterOptmizer.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/optimizer/RangeFilterOptmizer.java @@ -19,14 +19,13 @@ import org.apache.carbondata.core.scan.expression.Expression; import org.apache.carbondata.core.scan.expression.RangeExpressionEvaluator; -import org.apache.carbondata.core.scan.filter.intf.FilterOptimizer; import org.apache.carbondata.core.scan.filter.intf.FilterOptimizerImpl; public class RangeFilterOptmizer extends FilterOptimizerImpl { RangeExpressionEvaluator rangeExpEvaluator; - public RangeFilterOptmizer(FilterOptimizer filterOptimizer, Expression filterExpression) { + public RangeFilterOptmizer(Expression filterExpression) { super(filterExpression); this.rangeExpEvaluator = new RangeExpressionEvaluator(filterExpression); diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java index bb66d185ab7..996965db5bf 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java @@ -46,17 +46,13 @@ public class ConditionalFilterResolverImpl implements FilterResolverIntf { protected boolean isIncludeFilter; private DimColumnResolvedFilterInfo dimColResolvedFilterInfo; private MeasureColumnResolvedFilterInfo msrColResolvedFilterInfo; - private AbsoluteTableIdentifier tableIdentifier; - private boolean isMeasure; public ConditionalFilterResolverImpl(Expression exp, boolean isExpressionResolve, - boolean isIncludeFilter, AbsoluteTableIdentifier tableIdentifier, boolean isMeasure) { + boolean isIncludeFilter, boolean isMeasure) { this.exp = exp; this.isExpressionResolve = isExpressionResolve; this.isIncludeFilter = isIncludeFilter; - this.tableIdentifier = tableIdentifier; - this.isMeasure = isMeasure; - if (isMeasure == false) { + if (!isMeasure) { this.dimColResolvedFilterInfo = new DimColumnResolvedFilterInfo(); } else { this.msrColResolvedFilterInfo = new MeasureColumnResolvedFilterInfo(); @@ -230,10 +226,6 @@ public MeasureColumnResolvedFilterInfo getMsrColResolvedFilterInfo() { return msrColResolvedFilterInfo; } - public AbsoluteTableIdentifier getTableIdentifier() { - return tableIdentifier; - } - /** * method will calculates the start key based on the filter surrogates */ @@ -245,12 +237,6 @@ public void getStartKey(SegmentProperties segmentProperties, long[] startKey, FilterUtil.getStartKeyForNoDictionaryDimension(dimColResolvedFilterInfo, segmentProperties, setOfStartKeyByteArray); } -// else { -// FilterUtil.getStartKey(dimColResolvedFilterInfo.getDimensionResolvedFilterInstance(), -// segmentProperties, startKey, startKeyList); -// FilterUtil.getStartKeyForNoDictionaryDimension(dimColResolvedFilterInfo, segmentProperties, -// setOfStartKeyByteArray); -// } } /** diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelFilterResolverImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelFilterResolverImpl.java index 33bafb171d9..c846c02fdea 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelFilterResolverImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelFilterResolverImpl.java @@ -40,7 +40,7 @@ public class RowLevelFilterResolverImpl extends ConditionalFilterResolverImpl { public RowLevelFilterResolverImpl(Expression exp, boolean isExpressionResolve, boolean isIncludeFilter, AbsoluteTableIdentifier tableIdentifier) { - super(exp, isExpressionResolve, isIncludeFilter, tableIdentifier, false); + super(exp, isExpressionResolve, isIncludeFilter, false); dimColEvaluatorInfoList = new ArrayList(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); msrColEvalutorInfoList = new ArrayList( 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 0e7666cb381..e85e9d18617 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 @@ -61,7 +61,7 @@ public class RowLevelRangeFilterResolverImpl extends ConditionalFilterResolverIm public RowLevelRangeFilterResolverImpl(Expression exp, boolean isExpressionResolve, boolean isIncludeFilter, AbsoluteTableIdentifier tableIdentifier) { - super(exp, isExpressionResolve, isIncludeFilter, tableIdentifier, false); + super(exp, isExpressionResolve, isIncludeFilter, false); dimColEvaluatorInfoList = new ArrayList(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); msrColEvalutorInfoList = new ArrayList( diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java index f8586edc17b..4165424eba4 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java @@ -77,10 +77,6 @@ public void addMeasureResolvedFilterInstance(CarbonMeasure measures, } } - public Map> getMeasureResolvedFilterInstance() { - return measureResolvedFilter; - } - public ColumnFilterInfo getFilterValues() { return resolvedFilterValueObj; } @@ -109,10 +105,6 @@ public void setType(org.apache.carbondata.core.metadata.datatype.DataType dataTy this.type = dataType; } - public boolean isMeasureExistsInCurrentSlice() { - return true; - } - public CarbonColumn getCarbonColumn() { return carbonColumn; } @@ -125,10 +117,6 @@ public CarbonMeasure getMeasure() { return carbonMeasure; } - public boolean isMeasureExistsInCurrentSilce() { - return isMeasureExistsInCurrentSilce; - } - public void setMeasureExistsInCurrentSilce(boolean measureExistsInCurrentSilce) { isMeasureExistsInCurrentSilce = measureExistsInCurrentSilce; } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/TrueConditionalResolverImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/TrueConditionalResolverImpl.java index d403ef4884c..104f69e58e3 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/TrueConditionalResolverImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/TrueConditionalResolverImpl.java @@ -28,9 +28,9 @@ public class TrueConditionalResolverImpl extends ConditionalFilterResolverImpl { public TrueConditionalResolverImpl(Expression exp, boolean isExpressionResolve, - boolean isIncludeFilter, AbsoluteTableIdentifier tableIdentifier) { + boolean isIncludeFilter) { - super(exp, isExpressionResolve, isIncludeFilter, tableIdentifier, false); + super(exp, isExpressionResolve, isIncludeFilter, false); } @Override public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier, @@ -47,16 +47,4 @@ public TrueConditionalResolverImpl(Expression exp, boolean isExpressionResolve, return FilterExecuterType.TRUE; } - /** - * Method will the read filter expression corresponding to the resolver. - * This method is required in row level executer inorder to evaluate the filter - * expression against spark, as mentioned above row level is a special type - * filter resolver. - * - * @return Expression - */ - public Expression getFilterExpresion() { - return exp; - } - } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/visitor/RangeDictionaryColumnVisitor.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/visitor/RangeDictionaryColumnVisitor.java index c90816a16d1..b1e5a7d3761 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/visitor/RangeDictionaryColumnVisitor.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/visitor/RangeDictionaryColumnVisitor.java @@ -19,7 +19,6 @@ import java.io.IOException; import java.util.Collections; -import java.util.List; import org.apache.carbondata.core.constants.CarbonCommonConstants; import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException; @@ -47,7 +46,6 @@ public void populateFilterResolvedInfo(ColumnResolvedFilterInfo visitableObj, if (visitableObj instanceof DimColumnResolvedFilterInfo) { DimColumnResolvedFilterInfo resolveDimension = (DimColumnResolvedFilterInfo) visitableObj; ColumnFilterInfo resolvedFilterObject = null; - List evaluateResultListFinal; resolvedFilterObject = FilterUtil .getFilterListForAllValues(metadata.getTableIdentifier(), metadata.getExpression(), metadata.getColumnExpression(), metadata.isIncludeFilter(), diff --git a/core/src/main/java/org/apache/carbondata/core/scan/model/CarbonQueryPlan.java b/core/src/main/java/org/apache/carbondata/core/scan/model/CarbonQueryPlan.java deleted file mode 100644 index 5a5b0425ad8..00000000000 --- a/core/src/main/java/org/apache/carbondata/core/scan/model/CarbonQueryPlan.java +++ /dev/null @@ -1,151 +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.scan.model; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; - -import org.apache.carbondata.core.constants.CarbonCommonConstants; -import org.apache.carbondata.core.scan.expression.Expression; - -/** - * This class contains all the logical information about the query like dimensions,measures, - * sort order, topN etc.. - */ -public class CarbonQueryPlan implements Serializable { - /** - * - */ - private static final long serialVersionUID = -9036044826928017164L; - - /** - * Database name - */ - private String databaseName; - - /** - * Table name - */ - private String tableName; - - /** - * List of dimensions. - * Ex : select employee_name,department_name,sum(salary) from employee, then employee_name - * and department_name are dimensions - * If there is no dimensions asked in query then it would be remained as empty. - */ - private List dimensions = - new ArrayList(CarbonCommonConstants.CONSTANT_SIZE_TEN); - - /** - * List of measures. - * Ex : select employee_name,department_name,sum(salary) from employee, then sum(salary) - * would be measure. - * If there is no dimensions asked in query then it would be remained as empty. - */ - private List measures = - new ArrayList(CarbonCommonConstants.CONSTANT_SIZE_TEN); - - /** - * expression - */ - private Expression expression; - - /** - * queryId - */ - private String queryId; - - /** - * If it is raw detail query, no need to aggregate in backend. And it returns with dictionary data - * with out decoding. - */ - private boolean rawDetailQuery; - - /** - * Constructor created with database name and table name. - * - * @param databaseName - * @param tableName - */ - public CarbonQueryPlan(String databaseName, String tableName) { - this.tableName = tableName; - this.databaseName = databaseName; - } - - /** - * @return the dimensions - */ - public List getDimensions() { - return dimensions; - } - - public void addDimension(QueryDimension dimension) { - this.dimensions.add(dimension); - } - - /** - * @return the measures - */ - public List getMeasures() { - return measures; - } - - public void addMeasure(QueryMeasure measure) { - this.measures.add(measure); - } - - public Expression getFilterExpression() { - return expression; - } - - public void setFilterExpression(Expression expression) { - this.expression = expression; - } - - /** - * @return the databaseName - */ - public String getDatabaseName() { - return databaseName; - } - - /** - * @return the tableName - */ - public String getTableName() { - return tableName; - } - - public String getQueryId() { - return queryId; - } - - public void setQueryId(String queryId) { - this.queryId = queryId; - } - - public boolean isRawDetailQuery() { - return rawDetailQuery; - } - - public void setRawDetailQuery(boolean rawDetailQuery) { - this.rawDetailQuery = rawDetailQuery; - } -} diff --git a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryColumn.java b/core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionColumn.java similarity index 69% rename from core/src/main/java/org/apache/carbondata/core/scan/model/QueryColumn.java rename to core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionColumn.java index 1336b877533..0768221e94d 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryColumn.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionColumn.java @@ -16,17 +16,10 @@ */ package org.apache.carbondata.core.scan.model; -import java.io.Serializable; - /** - * query column which will have information about column + * Contains information for a column for projection */ -public class QueryColumn implements Serializable { - - /** - * serialVersionUID - */ - private static final long serialVersionUID = -4222306600480181084L; +public class ProjectionColumn { /** * name of the column @@ -36,9 +29,9 @@ public class QueryColumn implements Serializable { /** * query order in which result of the query will be send */ - private int queryOrder; + private int projectionOrdinal; - public QueryColumn(String columnName) { + ProjectionColumn(String columnName) { this.columnName = columnName; } @@ -50,17 +43,17 @@ public String getColumnName() { } /** - * @return the queryOrder + * @return the projectionOrdinal */ - public int getQueryOrder() { - return queryOrder; + public int getOrdinal() { + return projectionOrdinal; } /** - * @param queryOrder the queryOrder to set + * @param projectionOrdinal the projectionOrdinal to set */ - public void setQueryOrder(int queryOrder) { - this.queryOrder = queryOrder; + public void setOrdinal(int projectionOrdinal) { + this.projectionOrdinal = projectionOrdinal; } } \ No newline at end of file diff --git a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryDimension.java b/core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionDimension.java similarity index 69% rename from core/src/main/java/org/apache/carbondata/core/scan/model/QueryDimension.java rename to core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionDimension.java index e8f07a72f9c..806de857b42 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryDimension.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionDimension.java @@ -17,27 +17,22 @@ package org.apache.carbondata.core.scan.model; -import java.io.Serializable; - import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension; /** - * query plan dimension which will holds the information about the query plan dimension + * dimension column which holds the information about the column * this is done to avoid heavy object serialization */ -public class QueryDimension extends QueryColumn implements Serializable { +public class ProjectionDimension extends ProjectionColumn { - /** - * serialVersionUID - */ - private static final long serialVersionUID = -8492704093776645651L; /** * actual dimension column */ - private transient CarbonDimension dimension; + private CarbonDimension dimension; - public QueryDimension(String columnName) { - super(columnName); + public ProjectionDimension(CarbonDimension dimension) { + super(dimension.getColName()); + this.dimension = dimension; } /** @@ -47,10 +42,4 @@ public CarbonDimension getDimension() { return dimension; } - /** - * @param dimension the dimension to set - */ - public void setDimension(CarbonDimension dimension) { - this.dimension = dimension; - } } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryMeasure.java b/core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionMeasure.java similarity index 75% rename from core/src/main/java/org/apache/carbondata/core/scan/model/QueryMeasure.java rename to core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionMeasure.java index 73a8ad587e1..93d9c82dc9a 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryMeasure.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionMeasure.java @@ -17,8 +17,6 @@ package org.apache.carbondata.core.scan.model; -import java.io.Serializable; - import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure; /** @@ -26,20 +24,16 @@ * about measure present in the query, this is done to avoid the serialization * of the heavy object */ -public class QueryMeasure extends QueryColumn implements Serializable { - - /** - * serialVersionUID - */ - private static final long serialVersionUID = 1035512411375495414L; +public class ProjectionMeasure extends ProjectionColumn { /** * actual carbon measure object */ - private transient CarbonMeasure measure; + private CarbonMeasure measure; - public QueryMeasure(String columnName) { - super(columnName); + public ProjectionMeasure(CarbonMeasure measure) { + super(measure.getColName()); + this.measure = measure; } /** @@ -49,11 +43,4 @@ public CarbonMeasure getMeasure() { return measure; } - /** - * @param measure the measure to set - */ - public void setMeasure(CarbonMeasure measure) { - this.measure = measure; - } - } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModel.java b/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModel.java index 655ed5e98b7..4e3665f49ab 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModel.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModel.java @@ -17,7 +17,6 @@ package org.apache.carbondata.core.scan.model; -import java.io.Serializable; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -46,25 +45,17 @@ * about the query, This will be sent from driver to executor ' * This will be refereed to executing the query. */ -public class QueryModel implements Serializable { +public class QueryModel { - /** - * serialization version - */ - private static final long serialVersionUID = -4674677234007089052L; /** * this will hold the information about the dictionary dimension * which to */ - public transient Map columnToDictionaryMapping; - /** - * list of dimension selected for in query - */ - private List queryDimension; + private transient Map columnToDictionaryMapping; /** - * list of measure selected in query + * list of projection columns in query */ - private List queryMeasures; + private QueryProjection projection; /** * query id */ @@ -78,10 +69,6 @@ public class QueryModel implements Serializable { * table block information in which query will be executed */ private List tableBlockInfos; - /** - * absolute table identifier - */ - private AbsoluteTableIdentifier absoluteTableIdentifier; /** * To handle most of the computation in query engines like spark and hive, carbon should give * raw detailed records to it. @@ -117,43 +104,15 @@ public class QueryModel implements Serializable { */ private boolean readPageByPage; - public QueryModel() { + private QueryModel(CarbonTable carbonTable) { tableBlockInfos = new ArrayList(); - queryDimension = new ArrayList(); - queryMeasures = new ArrayList(); invalidSegmentIds = new ArrayList<>(); + this.table = carbonTable; + this.queryId = String.valueOf(System.nanoTime()); } - public static QueryModel createModel(AbsoluteTableIdentifier absoluteTableIdentifier, - CarbonQueryPlan queryPlan, CarbonTable carbonTable, DataTypeConverter converter) { - QueryModel queryModel = new QueryModel(); - queryModel.setAbsoluteTableIdentifier(absoluteTableIdentifier); - - fillQueryModel(queryPlan, carbonTable, queryModel); - - queryModel.setForcedDetailRawQuery(queryPlan.isRawDetailQuery()); - queryModel.setQueryId(queryPlan.getQueryId()); - queryModel.setConverter(converter); - return queryModel; - } - - private static void fillQueryModel(CarbonQueryPlan queryPlan, CarbonTable carbonTable, - QueryModel queryModel) { - queryModel.setAbsoluteTableIdentifier(carbonTable.getAbsoluteTableIdentifier()); - queryModel.setQueryDimension(queryPlan.getDimensions()); - queryModel.setQueryMeasures(queryPlan.getMeasures()); - if (null != queryPlan.getFilterExpression()) { - boolean[] isFilterDimensions = new boolean[carbonTable.getDimensionOrdinalMax()]; - boolean[] isFilterMeasures = - new boolean[carbonTable.getNumberOfMeasures(carbonTable.getTableName())]; - processFilterExpression(carbonTable, queryPlan.getFilterExpression(), isFilterDimensions, - isFilterMeasures); - queryModel.setIsFilterDimensions(isFilterDimensions); - queryModel.setIsFilterMeasures(isFilterMeasures); - } - //TODO need to remove this code, and executor will load the table - // from file metadata - queryModel.setTable(carbonTable); + public static QueryModel newInstance(CarbonTable carbonTable) { + return new QueryModel(carbonTable); } public static void processFilterExpression(CarbonTable carbonTable, Expression filterExpression, @@ -239,42 +198,26 @@ private static void setDimAndMsrColumnNode(CarbonTable carbonTable, ColumnExpres */ public CarbonColumn[] getProjectionColumns() { CarbonColumn[] carbonColumns = - new CarbonColumn[getQueryDimension().size() + getQueryMeasures().size()]; - for (QueryDimension dimension : getQueryDimension()) { - carbonColumns[dimension.getQueryOrder()] = dimension.getDimension(); + new CarbonColumn[getProjectionDimensions().size() + getProjectionMeasures().size()]; + for (ProjectionDimension dimension : getProjectionDimensions()) { + carbonColumns[dimension.getOrdinal()] = dimension.getDimension(); } - for (QueryMeasure msr : getQueryMeasures()) { - carbonColumns[msr.getQueryOrder()] = msr.getMeasure(); + for (ProjectionMeasure msr : getProjectionMeasures()) { + carbonColumns[msr.getOrdinal()] = msr.getMeasure(); } return carbonColumns; } - /** - * @return the queryDimension - */ - public List getQueryDimension() { - return queryDimension; + public void setProjection(QueryProjection projection) { + this.projection = projection; } - /** - * @param queryDimension the queryDimension to set - */ - public void setQueryDimension(List queryDimension) { - this.queryDimension = queryDimension; + public List getProjectionDimensions() { + return projection.getDimensions(); } - /** - * @return the queryMeasures - */ - public List getQueryMeasures() { - return queryMeasures; - } - - /** - * @param queryMeasures the queryMeasures to set - */ - public void setQueryMeasures(List queryMeasures) { - this.queryMeasures = queryMeasures; + public List getProjectionMeasures() { + return projection.getMeasures(); } /** @@ -284,13 +227,6 @@ public String getQueryId() { return queryId; } - /** - * @param queryId the queryId to set - */ - public void setQueryId(String queryId) { - this.queryId = queryId; - } - /** * @return the tableBlockInfos */ @@ -320,14 +256,7 @@ public void setFilterExpressionResolverTree(FilterResolverIntf filterExpressionR * @return the absoluteTableIdentifier */ public AbsoluteTableIdentifier getAbsoluteTableIdentifier() { - return absoluteTableIdentifier; - } - - /** - * @param absoluteTableIdentifier the absoluteTableIdentifier to set - */ - public void setAbsoluteTableIdentifier(AbsoluteTableIdentifier absoluteTableIdentifier) { - this.absoluteTableIdentifier = absoluteTableIdentifier; + return table.getAbsoluteTableIdentifier(); } /** @@ -337,13 +266,6 @@ public CarbonTable getTable() { return table; } - /** - * @param table the table to set - */ - public void setTable(CarbonTable table) { - this.table = table; - } - public boolean isForcedDetailRawQuery() { return forcedDetailRawQuery; } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryProjection.java b/core/src/main/java/org/apache/carbondata/core/scan/model/QueryProjection.java new file mode 100644 index 00000000000..d4496e06499 --- /dev/null +++ b/core/src/main/java/org/apache/carbondata/core/scan/model/QueryProjection.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.carbondata.core.scan.model; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.carbondata.core.constants.CarbonCommonConstants; +import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension; +import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure; + +/** + * Contains projection columns in the query + */ +public class QueryProjection { + + /** + * List of dimensions. + * Ex : select employee_name,department_name,sum(salary) from employee, then employee_name + * and department_name are dimensions + * If there is no dimensions asked in query then it would be remained as empty. + */ + private List dimensions = + new ArrayList(CarbonCommonConstants.CONSTANT_SIZE_TEN); + + /** + * List of measures. + * Ex : select employee_name,department_name,sum(salary) from employee, then sum(salary) + * would be measure. + * If there is no dimensions asked in query then it would be remained as empty. + */ + private List measures = + new ArrayList(CarbonCommonConstants.CONSTANT_SIZE_TEN); + + /** + * Constructor created with database name and table name. + * + */ + public QueryProjection() { + } + + /** + * @return the dimensions + */ + public List getDimensions() { + return dimensions; + } + + public void addDimension(CarbonDimension dimension, int queryOrdinal) { + ProjectionDimension queryDimension = new ProjectionDimension(dimension); + queryDimension.setOrdinal(queryOrdinal); + this.dimensions.add(queryDimension); + } + + /** + * @return the measures + */ + public List getMeasures() { + return measures; + } + + public void addMeasure(CarbonMeasure measure, int queryOrdinal) { + ProjectionMeasure queryMeasure = new ProjectionMeasure(measure); + queryMeasure.setOrdinal(queryOrdinal); + this.measures.add(queryMeasure); + } + +} diff --git a/core/src/main/java/org/apache/carbondata/core/scan/processor/AbstractDataBlockIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/processor/AbstractDataBlockIterator.java deleted file mode 100644 index eb5e3f9689a..00000000000 --- a/core/src/main/java/org/apache/carbondata/core/scan/processor/AbstractDataBlockIterator.java +++ /dev/null @@ -1,251 +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.scan.processor; - -import java.io.IOException; -import java.util.List; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.carbondata.common.CarbonIterator; -import org.apache.carbondata.common.logging.LogService; -import org.apache.carbondata.common.logging.LogServiceFactory; -import org.apache.carbondata.core.datastore.DataRefNode; -import org.apache.carbondata.core.datastore.FileHolder; -import org.apache.carbondata.core.scan.collector.ResultCollectorFactory; -import org.apache.carbondata.core.scan.collector.ScannedResultCollector; -import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo; -import org.apache.carbondata.core.scan.result.AbstractScannedResult; -import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch; -import org.apache.carbondata.core.scan.scanner.BlockletScanner; -import org.apache.carbondata.core.scan.scanner.impl.FilterScanner; -import org.apache.carbondata.core.scan.scanner.impl.NonFilterScanner; -import org.apache.carbondata.core.stats.QueryStatisticsModel; -import org.apache.carbondata.core.util.TaskMetricsMap; - -/** - * This abstract class provides a skeletal implementation of the - * Block iterator. - */ -public abstract class AbstractDataBlockIterator extends CarbonIterator> { - - private static final LogService LOGGER = - LogServiceFactory.getLogService(AbstractDataBlockIterator.class.getName()); - - /** - * iterator which will be used to iterate over data blocks - */ - protected CarbonIterator dataBlockIterator; - - /** - * result collector which will be used to aggregate the scanned result - */ - protected ScannedResultCollector scannerResultAggregator; - - /** - * processor which will be used to process the block processing can be - * filter processing or non filter processing - */ - protected BlockletScanner blockletScanner; - - /** - * batch size of result - */ - protected int batchSize; - - protected ExecutorService executorService; - - private Future future; - - private Future futureIo; - - protected AbstractScannedResult scannedResult; - - private BlockExecutionInfo blockExecutionInfo; - - private FileHolder fileReader; - - private AtomicBoolean nextBlock; - - private AtomicBoolean nextRead; - - public AbstractDataBlockIterator(BlockExecutionInfo blockExecutionInfo, FileHolder fileReader, - int batchSize, QueryStatisticsModel queryStatisticsModel, ExecutorService executorService) { - this.blockExecutionInfo = blockExecutionInfo; - this.fileReader = fileReader; - dataBlockIterator = new BlockletIterator(blockExecutionInfo.getFirstDataBlock(), - blockExecutionInfo.getNumberOfBlockToScan()); - if (blockExecutionInfo.getFilterExecuterTree() != null) { - blockletScanner = new FilterScanner(blockExecutionInfo, queryStatisticsModel); - } else { - blockletScanner = new NonFilterScanner(blockExecutionInfo, queryStatisticsModel); - } - this.scannerResultAggregator = - ResultCollectorFactory.getScannedResultCollector(blockExecutionInfo); - this.batchSize = batchSize; - this.executorService = executorService; - this.nextBlock = new AtomicBoolean(false); - this.nextRead = new AtomicBoolean(false); - } - - public boolean hasNext() { - if (scannedResult != null && scannedResult.hasNext()) { - return true; - } else { - if (null != scannedResult) { - scannedResult.freeMemory(); - } - return dataBlockIterator.hasNext() || nextBlock.get() || nextRead.get(); - } - } - - protected boolean updateScanner() { - try { - if (scannedResult != null && scannedResult.hasNext()) { - return true; - } else { - scannedResult = getNextScannedResult(); - while (scannedResult != null) { - if (scannedResult.hasNext()) { - return true; - } - scannedResult = getNextScannedResult(); - } - nextBlock.set(false); - nextRead.set(false); - return false; - } - } catch (Exception ex) { - throw new RuntimeException(ex); - } - } - - private AbstractScannedResult getNextScannedResult() throws Exception { - AbstractScannedResult result = null; - if (blockExecutionInfo.isPrefetchBlocklet()) { - if (dataBlockIterator.hasNext() || nextBlock.get() || nextRead.get()) { - if (future == null) { - future = execute(); - } - result = future.get(); - nextBlock.set(false); - if (dataBlockIterator.hasNext() || nextRead.get()) { - nextBlock.set(true); - future = execute(); - } - } - } else { - if (dataBlockIterator.hasNext()) { - BlocksChunkHolder blocksChunkHolder = getBlocksChunkHolder(); - if (blocksChunkHolder != null) { - result = blockletScanner.scanBlocklet(blocksChunkHolder); - } - } - } - return result; - } - - private BlocksChunkHolder getBlocksChunkHolder() throws IOException { - BlocksChunkHolder blocksChunkHolder = getBlocksChunkHolderInternal(); - while (blocksChunkHolder == null && dataBlockIterator.hasNext()) { - blocksChunkHolder = getBlocksChunkHolderInternal(); - } - return blocksChunkHolder; - } - - private BlocksChunkHolder getBlocksChunkHolderInternal() throws IOException { - BlocksChunkHolder blocksChunkHolder = - new BlocksChunkHolder(blockExecutionInfo.getTotalNumberDimensionBlock(), - blockExecutionInfo.getTotalNumberOfMeasureBlock(), fileReader); - blocksChunkHolder.setDataBlock(dataBlockIterator.next()); - if (blocksChunkHolder.getDataBlock().getColumnsMaxValue() == null) { - return blocksChunkHolder; - } - if (blockletScanner.isScanRequired(blocksChunkHolder)) { - return blocksChunkHolder; - } - return null; - } - - private Future execute() { - return executorService.submit(new Callable() { - @Override public AbstractScannedResult call() throws Exception { - if (futureIo == null) { - futureIo = executeRead(); - } - BlocksChunkHolder blocksChunkHolder = futureIo.get(); - futureIo = null; - nextRead.set(false); - if (blocksChunkHolder != null) { - if (dataBlockIterator.hasNext()) { - nextRead.set(true); - futureIo = executeRead(); - } - return blockletScanner.scanBlocklet(blocksChunkHolder); - } - return null; - } - }); - } - - private Future executeRead() { - return executorService.submit(new Callable() { - @Override public BlocksChunkHolder call() throws Exception { - try { - TaskMetricsMap.getInstance().registerThreadCallback(); - if (dataBlockIterator.hasNext()) { - BlocksChunkHolder blocksChunkHolder = getBlocksChunkHolder(); - if (blocksChunkHolder != null) { - blockletScanner.readBlocklet(blocksChunkHolder); - return blocksChunkHolder; - } - } - return null; - } finally { - // update read bytes metrics for this thread - TaskMetricsMap.getInstance().updateReadBytes(Thread.currentThread().getId()); - } - } - }); - } - - public abstract void processNextBatch(CarbonColumnarBatch columnarBatch); - - /** - * Close the resources - */ - public void close() { - // free the current scanned result - if (null != scannedResult && !scannedResult.hasNext()) { - scannedResult.freeMemory(); - } - // free any pre-fetched memory if present - if (null != future) { - try { - AbstractScannedResult abstractScannedResult = future.get(); - if (abstractScannedResult != null) { - abstractScannedResult.freeMemory(); - } - } catch (InterruptedException | ExecutionException e) { - throw new RuntimeException(e); - } - } - } -} \ No newline at end of file diff --git a/core/src/main/java/org/apache/carbondata/core/scan/processor/BlockletIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/processor/BlockletIterator.java index 3ea4e1d1217..f0d81ffe94b 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/processor/BlockletIterator.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/processor/BlockletIterator.java @@ -23,7 +23,7 @@ /** * Below class will be used to iterate over data block */ -public class BlockletIterator extends CarbonIterator { +class BlockletIterator extends CarbonIterator { /** * data store block */ @@ -47,17 +47,18 @@ public class BlockletIterator extends CarbonIterator { * Constructor * * @param datablock first data block - * @param totalNumberOfBlocksToScan total number of blocks to be scanned + * @param totalNumberOfBlockletToScan total number of blocklets to be scanned */ - public BlockletIterator(DataRefNode datablock, long totalNumberOfBlocksToScan) { + BlockletIterator(DataRefNode datablock, long totalNumberOfBlockletToScan) { this.datablock = datablock; - this.totalNumberOfBlocksToScan = totalNumberOfBlocksToScan; + this.totalNumberOfBlocksToScan = totalNumberOfBlockletToScan; } /** * is all the blocks assigned to this iterator has been processed */ - @Override public boolean hasNext() { + @Override + public boolean hasNext() { return hasNext; } @@ -67,7 +68,8 @@ public BlockletIterator(DataRefNode datablock, long totalNumberOfBlocksToScan) { * @return next data block * */ - @Override public DataRefNode next() { + @Override + public DataRefNode next() { // get the current blocks DataRefNode datablockTemp = datablock; // store the next data block diff --git a/core/src/main/java/org/apache/carbondata/core/scan/processor/BlocksChunkHolder.java b/core/src/main/java/org/apache/carbondata/core/scan/processor/BlocksChunkHolder.java deleted file mode 100644 index 60090d04af9..00000000000 --- a/core/src/main/java/org/apache/carbondata/core/scan/processor/BlocksChunkHolder.java +++ /dev/null @@ -1,141 +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.scan.processor; - -import org.apache.carbondata.core.datastore.DataRefNode; -import org.apache.carbondata.core.datastore.FileHolder; -import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; -import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk; -import org.apache.carbondata.core.util.BitSetGroup; - -/** - * Block chunk holder which will hold the dimension and - * measure chunk - */ -public class BlocksChunkHolder { - - /** - * dimension column data chunk - */ - private DimensionRawColumnChunk[] dimensionRawDataChunk; - - /** - * measure column data chunk - */ - private MeasureRawColumnChunk[] measureRawDataChunk; - - /** - * file reader which will use to read the block from file - */ - private FileHolder fileReader; - - /** - * data block - */ - private DataRefNode dataBlock; - - private BitSetGroup bitSetGroup; - - public BlocksChunkHolder(int numberOfDimensionBlock, int numberOfMeasureBlock) { - dimensionRawDataChunk = new DimensionRawColumnChunk[numberOfDimensionBlock]; - measureRawDataChunk = new MeasureRawColumnChunk[numberOfMeasureBlock]; - } - - public BlocksChunkHolder(int numberOfDimensionBlock, int numberOfMeasureBlock, - FileHolder fileReader) { - dimensionRawDataChunk = new DimensionRawColumnChunk[numberOfDimensionBlock]; - measureRawDataChunk = new MeasureRawColumnChunk[numberOfMeasureBlock]; - this.fileReader = fileReader; - } - - /** - * @return the dimensionRawDataChunk - */ - public DimensionRawColumnChunk[] getDimensionRawDataChunk() { - return dimensionRawDataChunk; - } - - /** - * @param dimensionRawDataChunk the dimensionRawDataChunk to set - */ - public void setDimensionRawDataChunk(DimensionRawColumnChunk[] dimensionRawDataChunk) { - this.dimensionRawDataChunk = dimensionRawDataChunk; - } - - /** - * @return the measureRawDataChunk - */ - public MeasureRawColumnChunk[] getMeasureRawDataChunk() { - return measureRawDataChunk; - } - - /** - * @param measureRawDataChunk the measureRawDataChunk to set - */ - public void setMeasureRawDataChunk(MeasureRawColumnChunk[] measureRawDataChunk) { - this.measureRawDataChunk = measureRawDataChunk; - } - - /** - * @return the fileReader - */ - public FileHolder getFileReader() { - return fileReader; - } - - /** - * @param fileReader the fileReader to set - */ - public void setFileReader(FileHolder fileReader) { - this.fileReader = fileReader; - } - - /** - * @return the dataBlock - */ - public DataRefNode getDataBlock() { - return dataBlock; - } - - /** - * @param dataBlock the dataBlock to set - */ - public void setDataBlock(DataRefNode dataBlock) { - this.dataBlock = dataBlock; - } - - /*** - * To reset the measure chunk and dimension chunk - * array - */ - public void reset() { - for (int i = 0; i < measureRawDataChunk.length; i++) { - this.measureRawDataChunk[i] = null; - } - for (int i = 0; i < dimensionRawDataChunk.length; i++) { - this.dimensionRawDataChunk[i] = null; - } - } - - public BitSetGroup getBitSetGroup() { - return bitSetGroup; - } - - public void setBitSetGroup(BitSetGroup bitSetGroup) { - this.bitSetGroup = bitSetGroup; - } -} diff --git a/core/src/main/java/org/apache/carbondata/core/scan/processor/DataBlockIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/processor/DataBlockIterator.java new file mode 100644 index 00000000000..fde4e552bcf --- /dev/null +++ b/core/src/main/java/org/apache/carbondata/core/scan/processor/DataBlockIterator.java @@ -0,0 +1,269 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.carbondata.core.scan.processor; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.carbondata.common.CarbonIterator; +import org.apache.carbondata.core.datastore.DataRefNode; +import org.apache.carbondata.core.datastore.FileReader; +import org.apache.carbondata.core.scan.collector.ResultCollectorFactory; +import org.apache.carbondata.core.scan.collector.ScannedResultCollector; +import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo; +import org.apache.carbondata.core.scan.result.BlockletScannedResult; +import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch; +import org.apache.carbondata.core.scan.scanner.BlockletScanner; +import org.apache.carbondata.core.scan.scanner.impl.BlockletFilterScanner; +import org.apache.carbondata.core.scan.scanner.impl.BlockletFullScanner; +import org.apache.carbondata.core.stats.QueryStatisticsModel; +import org.apache.carbondata.core.util.TaskMetricsMap; + +/** + * This abstract class provides a skeletal implementation of the + * Block iterator. + */ +public class DataBlockIterator extends CarbonIterator> { + + /** + * iterator which will be used to iterate over blocklets + */ + private BlockletIterator blockletIterator; + + /** + * result collector which will be used to aggregate the scanned result + */ + private ScannedResultCollector scannerResultAggregator; + + /** + * processor which will be used to process the block processing can be + * filter processing or non filter processing + */ + private BlockletScanner blockletScanner; + + /** + * batch size of result + */ + private int batchSize; + + private ExecutorService executorService; + + private Future future; + + private Future futureIo; + + private BlockletScannedResult scannedResult; + + private BlockExecutionInfo blockExecutionInfo; + + private FileReader fileReader; + + private AtomicBoolean nextBlock; + + private AtomicBoolean nextRead; + + public DataBlockIterator(BlockExecutionInfo blockExecutionInfo, FileReader fileReader, + int batchSize, QueryStatisticsModel queryStatisticsModel, ExecutorService executorService) { + this.blockExecutionInfo = blockExecutionInfo; + this.fileReader = fileReader; + blockletIterator = new BlockletIterator(blockExecutionInfo.getFirstDataBlock(), + blockExecutionInfo.getNumberOfBlockToScan()); + if (blockExecutionInfo.getFilterExecuterTree() != null) { + blockletScanner = new BlockletFilterScanner(blockExecutionInfo, queryStatisticsModel); + } else { + blockletScanner = new BlockletFullScanner(blockExecutionInfo, queryStatisticsModel); + } + this.scannerResultAggregator = + ResultCollectorFactory.getScannedResultCollector(blockExecutionInfo); + this.batchSize = batchSize; + this.executorService = executorService; + this.nextBlock = new AtomicBoolean(false); + this.nextRead = new AtomicBoolean(false); + } + + @Override + public List next() { + List collectedResult = null; + if (updateScanner()) { + collectedResult = this.scannerResultAggregator.collectResultInRow(scannedResult, batchSize); + while (collectedResult.size() < batchSize && updateScanner()) { + List data = this.scannerResultAggregator + .collectResultInRow(scannedResult, batchSize - collectedResult.size()); + collectedResult.addAll(data); + } + } else { + collectedResult = new ArrayList<>(); + } + return collectedResult; + } + + @Override + public boolean hasNext() { + if (scannedResult != null && scannedResult.hasNext()) { + return true; + } else { + if (null != scannedResult) { + scannedResult.freeMemory(); + } + return blockletIterator.hasNext() || nextBlock.get() || nextRead.get(); + } + } + + /** + * Return true if scan result if non-empty + */ + private boolean updateScanner() { + try { + if (scannedResult != null && scannedResult.hasNext()) { + return true; + } else { + scannedResult = processNextBlocklet(); + while (scannedResult != null) { + if (scannedResult.hasNext()) { + return true; + } + scannedResult = processNextBlocklet(); + } + nextBlock.set(false); + nextRead.set(false); + return false; + } + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + + private BlockletScannedResult processNextBlocklet() throws Exception { + BlockletScannedResult result = null; + if (blockExecutionInfo.isPrefetchBlocklet()) { + if (blockletIterator.hasNext() || nextBlock.get() || nextRead.get()) { + if (future == null) { + future = scanNextBlockletAsync(); + } + result = future.get(); + nextBlock.set(false); + if (blockletIterator.hasNext() || nextRead.get()) { + nextBlock.set(true); + future = scanNextBlockletAsync(); + } + } + } else { + if (blockletIterator.hasNext()) { + RawBlockletColumnChunks rawChunks = readNextBlockletColumnChunks(); + if (rawChunks != null) { + result = blockletScanner.scanBlocklet(rawChunks); + } + } + } + return result; + } + + private RawBlockletColumnChunks readNextBlockletColumnChunks() throws IOException { + RawBlockletColumnChunks rawBlockletColumnChunks = getNextBlockletColumnChunks(); + if (rawBlockletColumnChunks != null) { + blockletScanner.readBlocklet(rawBlockletColumnChunks); + return rawBlockletColumnChunks; + } + return null; + } + + private RawBlockletColumnChunks getNextBlockletColumnChunks() { + RawBlockletColumnChunks rawBlockletColumnChunks = null; + do { + DataRefNode dataBlock = blockletIterator.next(); + if (dataBlock.getColumnsMaxValue() == null || blockletScanner.isScanRequired(dataBlock)) { + rawBlockletColumnChunks = RawBlockletColumnChunks.newInstance( + blockExecutionInfo.getTotalNumberDimensionToRead(), + blockExecutionInfo.getTotalNumberOfMeasureToRead(), fileReader, dataBlock); + } + } while (rawBlockletColumnChunks == null && blockletIterator.hasNext()); + return rawBlockletColumnChunks; + } + + private Future scanNextBlockletAsync() { + return executorService.submit(new Callable() { + @Override public BlockletScannedResult call() throws Exception { + if (futureIo == null) { + futureIo = readNextBlockletAsync(); + } + RawBlockletColumnChunks rawBlockletColumnChunks = futureIo.get(); + futureIo = null; + nextRead.set(false); + if (rawBlockletColumnChunks != null) { + if (blockletIterator.hasNext()) { + nextRead.set(true); + futureIo = readNextBlockletAsync(); + } + return blockletScanner.scanBlocklet(rawBlockletColumnChunks); + } + return null; + } + }); + } + + private Future readNextBlockletAsync() { + return executorService.submit(new Callable() { + @Override public RawBlockletColumnChunks call() throws Exception { + try { + TaskMetricsMap.getInstance().registerThreadCallback(); + if (blockletIterator.hasNext()) { + return readNextBlockletColumnChunks(); + } else { + return null; + } + } finally { + // update read bytes metrics for this thread + TaskMetricsMap.getInstance().updateReadBytes(Thread.currentThread().getId()); + } + } + }); + } + + public void processNextBatch(CarbonColumnarBatch columnarBatch) { + if (updateScanner()) { + this.scannerResultAggregator.collectResultInColumnarBatch(scannedResult, columnarBatch); + } + } + + + /** + * Close the resources + */ + public void close() { + // free the current scanned result + if (null != scannedResult && !scannedResult.hasNext()) { + scannedResult.freeMemory(); + } + // free any pre-fetched memory if present + if (null != future) { + try { + BlockletScannedResult blockletScannedResult = future.get(); + if (blockletScannedResult != null) { + blockletScannedResult.freeMemory(); + } + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } + } + } +} \ No newline at end of file diff --git a/core/src/main/java/org/apache/carbondata/core/scan/processor/RawBlockletColumnChunks.java b/core/src/main/java/org/apache/carbondata/core/scan/processor/RawBlockletColumnChunks.java new file mode 100644 index 00000000000..6b7e8806966 --- /dev/null +++ b/core/src/main/java/org/apache/carbondata/core/scan/processor/RawBlockletColumnChunks.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.carbondata.core.scan.processor; + +import org.apache.carbondata.core.datastore.DataRefNode; +import org.apache.carbondata.core.datastore.FileReader; +import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; +import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk; +import org.apache.carbondata.core.util.BitSetGroup; + +/** + * Contains dimension and measure raw column chunks of one blocklet + */ +public class RawBlockletColumnChunks { + + /** + * dimension column data chunk + */ + private DimensionRawColumnChunk[] dimensionRawColumnChunks; + + /** + * measure column data chunk + */ + private MeasureRawColumnChunk[] measureRawColumnChunks; + + /** + * file reader which will use to read the block from file + */ + private FileReader fileReader; + + /** + * data block + */ + private DataRefNode dataBlock; + + private BitSetGroup bitSetGroup; + + private RawBlockletColumnChunks() { } + + public static RawBlockletColumnChunks newInstance(int numberOfDimensionChunk, + int numberOfMeasureChunk, FileReader fileReader, DataRefNode dataBlock) { + RawBlockletColumnChunks instance = new RawBlockletColumnChunks(); + instance.dimensionRawColumnChunks = new DimensionRawColumnChunk[numberOfDimensionChunk]; + instance.measureRawColumnChunks = new MeasureRawColumnChunk[numberOfMeasureChunk]; + instance.fileReader = fileReader; + instance.dataBlock = dataBlock; + return instance; + } + + /** + * @return the dimensionRawColumnChunks + */ + public DimensionRawColumnChunk[] getDimensionRawColumnChunks() { + return dimensionRawColumnChunks; + } + + /** + * @param dimensionRawColumnChunks the dimensionRawColumnChunks to set + */ + public void setDimensionRawColumnChunks(DimensionRawColumnChunk[] dimensionRawColumnChunks) { + this.dimensionRawColumnChunks = dimensionRawColumnChunks; + } + + /** + * @return the measureRawColumnChunks + */ + public MeasureRawColumnChunk[] getMeasureRawColumnChunks() { + return measureRawColumnChunks; + } + + /** + * @param measureRawColumnChunks the measureRawColumnChunks to set + */ + public void setMeasureRawColumnChunks(MeasureRawColumnChunk[] measureRawColumnChunks) { + this.measureRawColumnChunks = measureRawColumnChunks; + } + + /** + * @return the fileReader + */ + public FileReader getFileReader() { + return fileReader; + } + + /** + * @return the dataBlock + */ + public DataRefNode getDataBlock() { + return dataBlock; + } + + public BitSetGroup getBitSetGroup() { + return bitSetGroup; + } + + public void setBitSetGroup(BitSetGroup bitSetGroup) { + this.bitSetGroup = bitSetGroup; + } +} diff --git a/core/src/main/java/org/apache/carbondata/core/scan/processor/impl/DataBlockIteratorImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/processor/impl/DataBlockIteratorImpl.java deleted file mode 100644 index 1c97725c821..00000000000 --- a/core/src/main/java/org/apache/carbondata/core/scan/processor/impl/DataBlockIteratorImpl.java +++ /dev/null @@ -1,69 +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.scan.processor.impl; - -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.ExecutorService; - -import org.apache.carbondata.core.datastore.FileHolder; -import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo; -import org.apache.carbondata.core.scan.processor.AbstractDataBlockIterator; -import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch; -import org.apache.carbondata.core.stats.QueryStatisticsModel; - -/** - * Below class will be used to process the block for detail query - */ -public class DataBlockIteratorImpl extends AbstractDataBlockIterator { - /** - * DataBlockIteratorImpl Constructor - * - * @param blockExecutionInfo execution information - */ - public DataBlockIteratorImpl(BlockExecutionInfo blockExecutionInfo, FileHolder fileReader, - int batchSize, QueryStatisticsModel queryStatisticsModel, ExecutorService executorService) { - super(blockExecutionInfo, fileReader, batchSize, queryStatisticsModel, executorService); - } - - /** - * It scans the block and returns the result with @batchSize - * - * @return Result of @batchSize - */ - public List next() { - List collectedResult = null; - if (updateScanner()) { - collectedResult = this.scannerResultAggregator.collectData(scannedResult, batchSize); - while (collectedResult.size() < batchSize && updateScanner()) { - List data = this.scannerResultAggregator - .collectData(scannedResult, batchSize - collectedResult.size()); - collectedResult.addAll(data); - } - } else { - collectedResult = new ArrayList<>(); - } - return collectedResult; - } - - public void processNextBatch(CarbonColumnarBatch columnarBatch) { - if (updateScanner()) { - this.scannerResultAggregator.collectVectorBatch(scannedResult, columnarBatch); - } - } - -} diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/AbstractScannedResult.java b/core/src/main/java/org/apache/carbondata/core/scan/result/BlockletScannedResult.java similarity index 67% rename from core/src/main/java/org/apache/carbondata/core/scan/result/AbstractScannedResult.java rename to core/src/main/java/org/apache/carbondata/core/scan/result/BlockletScannedResult.java index b089fad8c4b..29404b4f967 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/result/AbstractScannedResult.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/result/BlockletScannedResult.java @@ -26,7 +26,7 @@ import org.apache.carbondata.common.logging.LogService; import org.apache.carbondata.common.logging.LogServiceFactory; import org.apache.carbondata.core.constants.CarbonCommonConstants; -import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk; +import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage; import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk; import org.apache.carbondata.core.datastore.page.ColumnPage; @@ -45,10 +45,10 @@ /** * Scanned result class which will store and provide the result on request */ -public abstract class AbstractScannedResult { +public abstract class BlockletScannedResult { private static final LogService LOGGER = - LogServiceFactory.getLogService(AbstractScannedResult.class.getName()); + LogServiceFactory.getLogService(BlockletScannedResult.class.getName()); /** * current row number */ @@ -56,22 +56,18 @@ public abstract class AbstractScannedResult { protected int pageCounter; /** - * row mapping indexes + * matched rowId for each page */ - protected int[][] rowMapping; + protected int[][] pageFilteredRowId; /** * key size of the fixed length column */ private int fixedLengthKeySize; /** - * total number of rows per page + * total number of filtered rows for each page */ - private int[] numberOfRows; + private int[] pageFilteredRowCount; - /** - * Total number of rows. - */ - private int totalNumberOfRows; /** * to keep track of number of rows process */ @@ -79,7 +75,7 @@ public abstract class AbstractScannedResult { /** * dimension column data chunk */ - protected DimensionColumnDataChunk[][] dimensionDataChunks; + protected DimensionColumnPage[][] dimensionColumnPages; /** * Raw dimension chunks; @@ -93,16 +89,16 @@ public abstract class AbstractScannedResult { /** * measure column data chunk */ - protected ColumnPage[][] measureDataChunks; + protected ColumnPage[][] measureColumnPages; /** * dictionary column block index in file */ - protected int[] dictionaryColumnBlockIndexes; + protected int[] dictionaryColumnChunkIndexes; /** - * no dictionary column block index in file + * no dictionary column chunk index in file */ - protected int[] noDictionaryColumnBlockIndexes; + protected int[] noDictionaryColumnChunkIndexes; /** * column group to is key structure info @@ -126,8 +122,6 @@ public abstract class AbstractScannedResult { */ private String blockletId; - private long rowId; - /** * parent block indexes */ @@ -148,14 +142,14 @@ public abstract class AbstractScannedResult { */ private String blockletNumber; - public AbstractScannedResult(BlockExecutionInfo blockExecutionInfo) { + public BlockletScannedResult(BlockExecutionInfo blockExecutionInfo) { this.fixedLengthKeySize = blockExecutionInfo.getFixedLengthKeySize(); - this.noDictionaryColumnBlockIndexes = blockExecutionInfo.getNoDictionaryBlockIndexes(); - this.dictionaryColumnBlockIndexes = blockExecutionInfo.getDictionaryColumnBlockIndex(); + this.noDictionaryColumnChunkIndexes = blockExecutionInfo.getNoDictionaryColumnChunkIndexes(); + this.dictionaryColumnChunkIndexes = blockExecutionInfo.getDictionaryColumnChunkIndex(); this.columnGroupKeyStructureInfo = blockExecutionInfo.getColumnGroupToKeyStructureInfo(); this.complexParentIndexToQueryMap = blockExecutionInfo.getComlexDimensionInfoMap(); this.complexParentBlockIndexes = blockExecutionInfo.getComplexColumnParentBlockIndexes(); - this.totalDimensionsSize = blockExecutionInfo.getQueryDimensions().length; + this.totalDimensionsSize = blockExecutionInfo.getProjectionDimensions().length; this.deletedRecordMap = blockExecutionInfo.getDeletedRecordsMap(); } @@ -163,19 +157,19 @@ public AbstractScannedResult(BlockExecutionInfo blockExecutionInfo) { * Below method will be used to set the dimension chunks * which will be used to create a row * - * @param dataChunks dimension chunks used in query + * @param columnPages dimension chunks used in query */ - public void setDimensionChunks(DimensionColumnDataChunk[][] dataChunks) { - this.dimensionDataChunks = dataChunks; + public void setDimensionColumnPages(DimensionColumnPage[][] columnPages) { + this.dimensionColumnPages = columnPages; } /** * Below method will be used to set the measure column chunks * - * @param measureDataChunks measure data chunks + * @param columnPages measure data chunks */ - public void setMeasureChunks(ColumnPage[][] measureDataChunks) { - this.measureDataChunks = measureDataChunks; + public void setMeasureColumnPages(ColumnPage[][] columnPages) { + this.measureColumnPages = columnPages; } public void setDimRawColumnChunks(DimensionRawColumnChunk[] dimRawColumnChunks) { @@ -193,7 +187,7 @@ public void setMsrRawColumnChunks(MeasureRawColumnChunk[] msrRawColumnChunks) { * @return measure column chunk */ public ColumnPage getMeasureChunk(int ordinal) { - return measureDataChunks[ordinal][pageCounter]; + return measureColumnPages[ordinal][pageCounter]; } /** @@ -206,10 +200,10 @@ public ColumnPage getMeasureChunk(int ordinal) { protected byte[] getDictionaryKeyArray(int rowId) { byte[] completeKey = new byte[fixedLengthKeySize]; int offset = 0; - for (int i = 0; i < this.dictionaryColumnBlockIndexes.length; i++) { - offset += dimensionDataChunks[dictionaryColumnBlockIndexes[i]][pageCounter] - .fillChunkData(completeKey, offset, rowId, - columnGroupKeyStructureInfo.get(dictionaryColumnBlockIndexes[i])); + for (int i = 0; i < this.dictionaryColumnChunkIndexes.length; i++) { + offset += dimensionColumnPages[dictionaryColumnChunkIndexes[i]][pageCounter].fillRawData( + rowId, offset, completeKey, + columnGroupKeyStructureInfo.get(dictionaryColumnChunkIndexes[i])); } rowCounter++; return completeKey; @@ -225,10 +219,10 @@ protected byte[] getDictionaryKeyArray(int rowId) { protected int[] getDictionaryKeyIntegerArray(int rowId) { int[] completeKey = new int[totalDimensionsSize]; int column = 0; - for (int i = 0; i < this.dictionaryColumnBlockIndexes.length; i++) { - column = dimensionDataChunks[dictionaryColumnBlockIndexes[i]][pageCounter] - .fillConvertedChunkData(rowId, column, completeKey, - columnGroupKeyStructureInfo.get(dictionaryColumnBlockIndexes[i])); + for (int i = 0; i < this.dictionaryColumnChunkIndexes.length; i++) { + column = dimensionColumnPages[dictionaryColumnChunkIndexes[i]][pageCounter] + .fillSurrogateKey(rowId, column, completeKey, + columnGroupKeyStructureInfo.get(dictionaryColumnChunkIndexes[i])); } rowCounter++; return completeKey; @@ -239,10 +233,10 @@ protected int[] getDictionaryKeyIntegerArray(int rowId) { */ public void fillColumnarDictionaryBatch(ColumnVectorInfo[] vectorInfo) { int column = 0; - for (int i = 0; i < this.dictionaryColumnBlockIndexes.length; i++) { - column = dimensionDataChunks[dictionaryColumnBlockIndexes[i]][pageCounter] - .fillConvertedChunkData(vectorInfo, column, - columnGroupKeyStructureInfo.get(dictionaryColumnBlockIndexes[i])); + for (int i = 0; i < this.dictionaryColumnChunkIndexes.length; i++) { + column = dimensionColumnPages[dictionaryColumnChunkIndexes[i]][pageCounter] + .fillVector(vectorInfo, column, + columnGroupKeyStructureInfo.get(dictionaryColumnChunkIndexes[i])); } } @@ -251,10 +245,10 @@ public void fillColumnarDictionaryBatch(ColumnVectorInfo[] vectorInfo) { */ public void fillColumnarNoDictionaryBatch(ColumnVectorInfo[] vectorInfo) { int column = 0; - for (int i = 0; i < this.noDictionaryColumnBlockIndexes.length; i++) { - column = dimensionDataChunks[noDictionaryColumnBlockIndexes[i]][pageCounter] - .fillConvertedChunkData(vectorInfo, column, - columnGroupKeyStructureInfo.get(noDictionaryColumnBlockIndexes[i])); + for (int i = 0; i < this.noDictionaryColumnChunkIndexes.length; i++) { + column = dimensionColumnPages[noDictionaryColumnChunkIndexes[i]][pageCounter] + .fillVector(vectorInfo, column, + columnGroupKeyStructureInfo.get(noDictionaryColumnChunkIndexes[i])); } } @@ -264,7 +258,7 @@ public void fillColumnarNoDictionaryBatch(ColumnVectorInfo[] vectorInfo) { public void fillColumnarMeasureBatch(ColumnVectorInfo[] vectorInfo, int[] measuresOrdinal) { for (int i = 0; i < measuresOrdinal.length; i++) { vectorInfo[i].measureVectorFiller - .fillMeasureVector(measureDataChunks[measuresOrdinal[i]][pageCounter], vectorInfo[i]); + .fillMeasureVector(measureColumnPages[measuresOrdinal[i]][pageCounter], vectorInfo[i]); } } @@ -278,9 +272,10 @@ public void fillColumnarComplexBatch(ColumnVectorInfo[] vectorInfos) { ByteArrayOutputStream byteStream = new ByteArrayOutputStream(); DataOutputStream dataOutput = new DataOutputStream(byteStream); try { - vectorInfos[i].genericQueryType - .parseBlocksAndReturnComplexColumnByteArray(dimRawColumnChunks, - rowMapping == null ? j : rowMapping[pageCounter][j], pageCounter, dataOutput); + vectorInfos[i].genericQueryType.parseBlocksAndReturnComplexColumnByteArray( + dimRawColumnChunks, + pageFilteredRowId == null ? j : pageFilteredRowId[pageCounter][j], pageCounter, + dataOutput); Object data = vectorInfos[i].genericQueryType .getDataBasedOnDataTypeFromSurrogates(ByteBuffer.wrap(byteStream.toByteArray())); vector.putObject(vectorOffset++, data); @@ -310,9 +305,9 @@ public void fillColumnarImplicitBatch(ColumnVectorInfo[] vectorInfo) { if (CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID .equals(columnVectorInfo.dimension.getColumnName())) { data = data + CarbonCommonConstants.FILE_SEPARATOR + pageCounter - + CarbonCommonConstants.FILE_SEPARATOR + (rowMapping == null ? + + CarbonCommonConstants.FILE_SEPARATOR + (pageFilteredRowId == null ? j : - rowMapping[pageCounter][j]); + pageFilteredRowId[pageCounter][j]); } vector.putBytes(vectorOffset++, data.getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET))); @@ -346,19 +341,19 @@ public void incrementPageCounter() { */ public void fillDataChunks() { freeDataChunkMemory(); - if (pageCounter >= numberOfRows.length) { + if (pageCounter >= pageFilteredRowCount.length) { return; } - for (int i = 0; i < dimensionDataChunks.length; i++) { - if (dimensionDataChunks[i][pageCounter] == null && dimRawColumnChunks[i] != null) { - dimensionDataChunks[i][pageCounter] = + for (int i = 0; i < dimensionColumnPages.length; i++) { + if (dimensionColumnPages[i][pageCounter] == null && dimRawColumnChunks[i] != null) { + dimensionColumnPages[i][pageCounter] = dimRawColumnChunks[i].convertToDimColDataChunkWithOutCache(pageCounter); } } - for (int i = 0; i < measureDataChunks.length; i++) { - if (measureDataChunks[i][pageCounter] == null && msrRawColumnChunks[i] != null) { - measureDataChunks[i][pageCounter] = + for (int i = 0; i < measureColumnPages.length; i++) { + if (measureColumnPages[i][pageCounter] == null && msrRawColumnChunks[i] != null) { + measureColumnPages[i][pageCounter] = msrRawColumnChunks[i].convertToColumnPageWithOutCache(pageCounter); } } @@ -366,22 +361,22 @@ public void fillDataChunks() { // free the memory for the last page chunk private void freeDataChunkMemory() { - for (int i = 0; i < dimensionDataChunks.length; i++) { - if (pageCounter > 0 && dimensionDataChunks[i][pageCounter - 1] != null) { - dimensionDataChunks[i][pageCounter - 1].freeMemory(); - dimensionDataChunks[i][pageCounter - 1] = null; + for (int i = 0; i < dimensionColumnPages.length; i++) { + if (pageCounter > 0 && dimensionColumnPages[i][pageCounter - 1] != null) { + dimensionColumnPages[i][pageCounter - 1].freeMemory(); + dimensionColumnPages[i][pageCounter - 1] = null; } } - for (int i = 0; i < measureDataChunks.length; i++) { - if (pageCounter > 0 && measureDataChunks[i][pageCounter - 1] != null) { - measureDataChunks[i][pageCounter - 1].freeMemory(); - measureDataChunks[i][pageCounter - 1] = null; + for (int i = 0; i < measureColumnPages.length; i++) { + if (pageCounter > 0 && measureColumnPages[i][pageCounter - 1] != null) { + measureColumnPages[i][pageCounter - 1].freeMemory(); + measureColumnPages[i][pageCounter - 1] = null; } } } public int numberOfpages() { - return numberOfRows.length; + return pageFilteredRowCount.length; } /** @@ -390,7 +385,7 @@ public int numberOfpages() { * @return */ public int getCurrentPageRowCount() { - return numberOfRows[pageCounter]; + return pageFilteredRowCount[pageCounter]; } public int getCurrentPageCounter() { @@ -404,18 +399,6 @@ public void setRowCounter(int rowCounter) { this.rowCounter = rowCounter; } - /** - * Below method will be used to get the dimension data based on dimension - * ordinal and index - * - * @param dimOrdinal dimension ordinal present in the query - * @param rowId row index - * @return dimension data based on row id - */ - protected byte[] getDimensionData(int dimOrdinal, int rowId) { - return dimensionDataChunks[dimOrdinal][pageCounter].getChunkData(rowId); - } - /** * Below method will be used to get the dimension key array * for all the no dictionary dimension present in the query @@ -424,29 +407,11 @@ protected byte[] getDimensionData(int dimOrdinal, int rowId) { * @return no dictionary keys for all no dictionary dimension */ protected byte[][] getNoDictionaryKeyArray(int rowId) { - byte[][] noDictionaryColumnsKeys = new byte[noDictionaryColumnBlockIndexes.length][]; + byte[][] noDictionaryColumnsKeys = new byte[noDictionaryColumnChunkIndexes.length][]; int position = 0; - for (int i = 0; i < this.noDictionaryColumnBlockIndexes.length; i++) { + for (int i = 0; i < this.noDictionaryColumnChunkIndexes.length; i++) { noDictionaryColumnsKeys[position++] = - dimensionDataChunks[noDictionaryColumnBlockIndexes[i]][pageCounter].getChunkData(rowId); - } - return noDictionaryColumnsKeys; - } - - /** - * Below method will be used to get the dimension key array - * for all the no dictionary dimension present in the query - * - * @param rowId row number - * @return no dictionary keys for all no dictionary dimension - */ - protected String[] getNoDictionaryKeyStringArray(int rowId) { - String[] noDictionaryColumnsKeys = new String[noDictionaryColumnBlockIndexes.length]; - int position = 0; - for (int i = 0; i < this.noDictionaryColumnBlockIndexes.length; i++) { - noDictionaryColumnsKeys[position++] = new String( - dimensionDataChunks[noDictionaryColumnBlockIndexes[i]][pageCounter].getChunkData(rowId), - Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)); + dimensionColumnPages[noDictionaryColumnChunkIndexes[i]][pageCounter].getChunkData(rowId); } return noDictionaryColumnsKeys; } @@ -459,7 +424,8 @@ public String getBlockletId() { } /** - * @param blockletId + * Set blocklet id, which looks like + * "Part0/Segment_0/part-0-0_batchno0-0-1517155583332.carbondata/0" */ public void setBlockletId(String blockletId) { this.blockletId = CarbonTablePath.getShortBlockId(blockletId); @@ -471,20 +437,6 @@ public void setBlockletId(String blockletId) { } } - /** - * @return blockletId - */ - public long getRowId() { - return rowId; - } - - /** - * @param rowId - */ - public void setRowId(long rowId) { - this.rowId = rowId; - } - /** * Below method will be used to get the complex type keys array based * on row id for all the complex type dimension selected in query @@ -514,22 +466,16 @@ protected byte[][] getComplexTypeKeyArray(int rowId) { return complexTypeData; } - /** - * @return return the total number of row after scanning - */ - public int numberOfOutputRows() { - return this.totalNumberOfRows; - } - /** * to check whether any more row is present in the result * * @return */ public boolean hasNext() { - if (pageCounter < numberOfRows.length && rowCounter < this.numberOfRows[pageCounter]) { + if (pageCounter + < pageFilteredRowCount.length && rowCounter < this.pageFilteredRowCount[pageCounter]) { return true; - } else if (pageCounter < numberOfRows.length) { + } else if (pageCounter < pageFilteredRowCount.length) { pageCounter++; fillDataChunks(); rowCounter = 0; @@ -547,24 +493,24 @@ public boolean hasNext() { */ public void freeMemory() { // first free the dimension chunks - if (null != dimensionDataChunks) { - for (int i = 0; i < dimensionDataChunks.length; i++) { - if (null != dimensionDataChunks[i]) { - for (int j = 0; j < dimensionDataChunks[i].length; j++) { - if (null != dimensionDataChunks[i][j]) { - dimensionDataChunks[i][j].freeMemory(); + if (null != dimensionColumnPages) { + for (int i = 0; i < dimensionColumnPages.length; i++) { + if (null != dimensionColumnPages[i]) { + for (int j = 0; j < dimensionColumnPages[i].length; j++) { + if (null != dimensionColumnPages[i][j]) { + dimensionColumnPages[i][j].freeMemory(); } } } } } // free the measure data chunks - if (null != measureDataChunks) { - for (int i = 0; i < measureDataChunks.length; i++) { - if (null != measureDataChunks[i]) { - for (int j = 0; j < measureDataChunks[i].length; j++) { - if (null != measureDataChunks[i][j]) { - measureDataChunks[i][j].freeMemory(); + if (null != measureColumnPages) { + for (int i = 0; i < measureColumnPages.length; i++) { + if (null != measureColumnPages[i]) { + for (int j = 0; j < measureColumnPages[i].length; j++) { + if (null != measureColumnPages[i][j]) { + measureColumnPages[i][j].freeMemory(); } } } @@ -581,36 +527,18 @@ public void freeMemory() { } /** - * As this class will be a flyweight object so - * for one block all the blocklet scanning will use same result object - * in that case we need to reset the counter to zero so - * for new result it will give the result from zero - */ - public void reset() { - rowCounter = 0; - currentRow = -1; - pageCounter = 0; - } - - /** - * @param numberOfRows set total of number rows valid after scanning + * @param pageFilteredRowCount set total of number rows valid after scanning */ - public void setNumberOfRows(int[] numberOfRows) { - this.numberOfRows = numberOfRows; - - for (int count : numberOfRows) { - totalNumberOfRows += count; - } + public void setPageFilteredRowCount(int[] pageFilteredRowCount) { + this.pageFilteredRowCount = pageFilteredRowCount; } /** * After applying filter it will return the bit set with the valid row indexes * so below method will be used to set the row indexes - * - * @param indexes */ - public void setIndexes(int[][] indexes) { - this.rowMapping = indexes; + public void setPageFilteredRowId(int[][] pageFilteredRowId) { + this.pageFilteredRowId = pageFilteredRowId; } public int getRowCounter() { @@ -651,14 +579,6 @@ public int getRowCounter() { */ public abstract byte[][] getNoDictionaryKeyArray(); - /** - * Below method will be used to get the no dictionary key - * array in string array format for all the no dictionary dimension selected in query - * - * @return no dictionary key array for all the no dictionary dimension - */ - public abstract String[] getNoDictionaryKeyStringArray(); - /** * Mark the filtered rows in columnar batch. These rows will not be added to vector batches later. * @param columnarBatch @@ -672,7 +592,7 @@ public int markFilteredRows(CarbonColumnarBatch columnarBatch, int startRow, int if (currentDeleteDeltaVo != null) { int len = startRow + size; for (int i = startRow; i < len; i++) { - int rowId = rowMapping != null ? rowMapping[pageCounter][i] : i; + int rowId = pageFilteredRowId != null ? pageFilteredRowId[pageCounter][i] : i; if (currentDeleteDeltaVo.containsRow(rowId)) { columnarBatch.markFiltered(vectorOffset); rowsFiltered++; diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/BatchResult.java b/core/src/main/java/org/apache/carbondata/core/scan/result/RowBatch.java similarity index 96% rename from core/src/main/java/org/apache/carbondata/core/scan/result/BatchResult.java rename to core/src/main/java/org/apache/carbondata/core/scan/result/RowBatch.java index 56ca2ac5201..c129161519b 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/result/BatchResult.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/result/RowBatch.java @@ -26,7 +26,7 @@ /** * Below class holds the query result */ -public class BatchResult extends CarbonIterator { +public class RowBatch extends CarbonIterator { /** * list of keys @@ -38,7 +38,7 @@ public class BatchResult extends CarbonIterator { */ protected int counter; - public BatchResult() { + public RowBatch() { this.rows = new ArrayList<>(); } 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 81203109a66..bcc5634fd4d 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 @@ -17,7 +17,7 @@ package org.apache.carbondata.core.scan.result.impl; import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo; -import org.apache.carbondata.core.scan.result.AbstractScannedResult; +import org.apache.carbondata.core.scan.result.BlockletScannedResult; import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo; /** @@ -25,7 +25,7 @@ * In case of filter query data will be send * based on filtered row index */ -public class FilterQueryScannedResult extends AbstractScannedResult { +public class FilterQueryScannedResult extends BlockletScannedResult { public FilterQueryScannedResult(BlockExecutionInfo tableBlockExecutionInfos) { super(tableBlockExecutionInfos); @@ -37,7 +37,7 @@ public FilterQueryScannedResult(BlockExecutionInfo tableBlockExecutionInfos) { */ @Override public byte[] getDictionaryKeyArray() { ++currentRow; - return getDictionaryKeyArray(rowMapping[pageCounter][currentRow]); + return getDictionaryKeyArray(pageFilteredRowId[pageCounter][currentRow]); } /** @@ -46,7 +46,7 @@ public FilterQueryScannedResult(BlockExecutionInfo tableBlockExecutionInfos) { */ @Override public int[] getDictionaryKeyIntegerArray() { ++currentRow; - return getDictionaryKeyIntegerArray(rowMapping[pageCounter][currentRow]); + return getDictionaryKeyIntegerArray(pageFilteredRowId[pageCounter][currentRow]); } /** @@ -55,7 +55,7 @@ public FilterQueryScannedResult(BlockExecutionInfo tableBlockExecutionInfos) { * @return complex type key array */ @Override public byte[][] getComplexTypeKeyArray() { - return getComplexTypeKeyArray(rowMapping[pageCounter][currentRow]); + return getComplexTypeKeyArray(pageFilteredRowId[pageCounter][currentRow]); } /** @@ -65,17 +65,7 @@ public FilterQueryScannedResult(BlockExecutionInfo tableBlockExecutionInfos) { * @return no dictionary key array for all the no dictionary dimension */ @Override public byte[][] getNoDictionaryKeyArray() { - return getNoDictionaryKeyArray(rowMapping[pageCounter][currentRow]); - } - - /** - * Below method will be used to get the no dictionary key - * string array for all the no dictionary dimension selected in query - * - * @return no dictionary key array for all the no dictionary dimension - */ - @Override public String[] getNoDictionaryKeyStringArray() { - return getNoDictionaryKeyStringArray(rowMapping[pageCounter][currentRow]); + return getNoDictionaryKeyArray(pageFilteredRowId[pageCounter][currentRow]); } /** @@ -84,7 +74,7 @@ public FilterQueryScannedResult(BlockExecutionInfo tableBlockExecutionInfos) { * @return valid row id */ @Override public int getCurrentRowId() { - return rowMapping[pageCounter][currentRow]; + return pageFilteredRowId[pageCounter][currentRow]; } /** @@ -92,10 +82,12 @@ public FilterQueryScannedResult(BlockExecutionInfo tableBlockExecutionInfos) { */ public void fillColumnarDictionaryBatch(ColumnVectorInfo[] vectorInfo) { int column = 0; - for (int i = 0; i < this.dictionaryColumnBlockIndexes.length; i++) { - column = dimensionDataChunks[dictionaryColumnBlockIndexes[i]][pageCounter] - .fillConvertedChunkData(rowMapping[pageCounter], vectorInfo, column, - columnGroupKeyStructureInfo.get(dictionaryColumnBlockIndexes[i])); + for (int chunkIndex : this.dictionaryColumnChunkIndexes) { + column = dimensionColumnPages[chunkIndex][pageCounter].fillVector( + pageFilteredRowId[pageCounter], + vectorInfo, + column, + columnGroupKeyStructureInfo.get(chunkIndex)); } } @@ -104,10 +96,12 @@ public void fillColumnarDictionaryBatch(ColumnVectorInfo[] vectorInfo) { */ public void fillColumnarNoDictionaryBatch(ColumnVectorInfo[] vectorInfo) { int column = 0; - for (int i = 0; i < this.noDictionaryColumnBlockIndexes.length; i++) { - column = dimensionDataChunks[noDictionaryColumnBlockIndexes[i]][pageCounter] - .fillConvertedChunkData(rowMapping[pageCounter], vectorInfo, column, - columnGroupKeyStructureInfo.get(noDictionaryColumnBlockIndexes[i])); + for (int chunkIndex : this.noDictionaryColumnChunkIndexes) { + column = dimensionColumnPages[chunkIndex][pageCounter].fillVector( + pageFilteredRowId[pageCounter], + vectorInfo, + column, + columnGroupKeyStructureInfo.get(chunkIndex)); } } @@ -116,8 +110,10 @@ public void fillColumnarNoDictionaryBatch(ColumnVectorInfo[] vectorInfo) { */ public void fillColumnarMeasureBatch(ColumnVectorInfo[] vectorInfo, int[] measuresOrdinal) { for (int i = 0; i < measuresOrdinal.length; i++) { - vectorInfo[i].measureVectorFiller.fillMeasureVectorForFilter(rowMapping[pageCounter], - measureDataChunks[measuresOrdinal[i]][pageCounter], vectorInfo[i]); + vectorInfo[i].measureVectorFiller.fillMeasureVector( + pageFilteredRowId[pageCounter], + measureColumnPages[measuresOrdinal[i]][pageCounter], + vectorInfo[i]); } } } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/impl/NonFilterQueryScannedResult.java b/core/src/main/java/org/apache/carbondata/core/scan/result/impl/NonFilterQueryScannedResult.java index 3978f9eef79..06687c25eec 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/result/impl/NonFilterQueryScannedResult.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/result/impl/NonFilterQueryScannedResult.java @@ -17,14 +17,14 @@ package org.apache.carbondata.core.scan.result.impl; import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo; -import org.apache.carbondata.core.scan.result.AbstractScannedResult; +import org.apache.carbondata.core.scan.result.BlockletScannedResult; /** * Result provide class for non filter query * In case of no filter query we need to return * complete data */ -public class NonFilterQueryScannedResult extends AbstractScannedResult { +public class NonFilterQueryScannedResult extends BlockletScannedResult { public NonFilterQueryScannedResult(BlockExecutionInfo blockExecutionInfo) { super(blockExecutionInfo); @@ -67,16 +67,6 @@ public NonFilterQueryScannedResult(BlockExecutionInfo blockExecutionInfo) { return getNoDictionaryKeyArray(currentRow); } - /** - * Below method will be used to get the no dictionary key - * string array for all the no dictionary dimension selected in query - * - * @return no dictionary key array for all the no dictionary dimension - */ - @Override public String[] getNoDictionaryKeyStringArray() { - return getNoDictionaryKeyStringArray(currentRow); - } - /** * will return the current valid row id * diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/AbstractDetailQueryResultIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/AbstractDetailQueryResultIterator.java index 6172b407151..4e628fedcd8 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/AbstractDetailQueryResultIterator.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/AbstractDetailQueryResultIterator.java @@ -28,18 +28,17 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants; import org.apache.carbondata.core.datastore.DataRefNode; import org.apache.carbondata.core.datastore.DataRefNodeFinder; -import org.apache.carbondata.core.datastore.FileHolder; +import org.apache.carbondata.core.datastore.FileReader; import org.apache.carbondata.core.datastore.block.AbstractIndex; import org.apache.carbondata.core.datastore.impl.FileFactory; import org.apache.carbondata.core.datastore.impl.btree.BTreeDataRefNodeFinder; -import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataRefNodeWrapper; +import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataRefNode; import org.apache.carbondata.core.mutate.DeleteDeltaVo; import org.apache.carbondata.core.reader.CarbonDeleteFilesDataReader; import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo; import org.apache.carbondata.core.scan.executor.infos.DeleteDeltaInfo; import org.apache.carbondata.core.scan.model.QueryModel; -import org.apache.carbondata.core.scan.processor.AbstractDataBlockIterator; -import org.apache.carbondata.core.scan.processor.impl.DataBlockIteratorImpl; +import org.apache.carbondata.core.scan.processor.DataBlockIterator; import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch; import org.apache.carbondata.core.stats.QueryStatistic; import org.apache.carbondata.core.stats.QueryStatisticsConstants; @@ -63,23 +62,23 @@ public abstract class AbstractDetailQueryResultIterator extends CarbonIterato private static final Map deleteDeltaToLockObjectMap = new ConcurrentHashMap<>(); - protected ExecutorService execService; + private ExecutorService execService; /** * execution info of the block */ - protected List blockExecutionInfos; + private List blockExecutionInfos; /** * file reader which will be used to execute the query */ - protected FileHolder fileReader; + protected FileReader fileReader; - protected AbstractDataBlockIterator dataBlockIterator; + DataBlockIterator dataBlockIterator; /** * QueryStatisticsRecorder */ - protected QueryStatisticsRecorder recorder; + private QueryStatisticsRecorder recorder; /** * number of cores which can be used */ @@ -89,7 +88,7 @@ public abstract class AbstractDetailQueryResultIterator extends CarbonIterato */ private QueryStatisticsModel queryStatisticsModel; - public AbstractDetailQueryResultIterator(List infos, QueryModel queryModel, + AbstractDetailQueryResultIterator(List infos, QueryModel queryModel, ExecutorService execService) { String batchSizeString = CarbonProperties.getInstance().getProperty(CarbonCommonConstants.DETAIL_QUERY_BATCH_SIZE); @@ -107,7 +106,6 @@ public AbstractDetailQueryResultIterator(List infos, QueryMo this.blockExecutionInfos = infos; this.fileReader = FileFactory.getFileHolder( FileFactory.getFileType(queryModel.getAbsoluteTableIdentifier().getTablePath())); - this.fileReader.setQueryId(queryModel.getQueryId()); this.fileReader.setReadPageByPage(queryModel.isReadPageByPage()); this.execService = execService; intialiseInfos(); @@ -130,22 +128,21 @@ private void intialiseInfos() { blockInfo.setDeletedRecordsMap(deletedRowsMap); } DataRefNode dataRefNode = blockInfo.getDataBlock().getDataRefNode(); - if (dataRefNode instanceof BlockletDataRefNodeWrapper) { - BlockletDataRefNodeWrapper wrapper = (BlockletDataRefNodeWrapper) dataRefNode; - blockInfo.setFirstDataBlock(wrapper); - blockInfo.setNumberOfBlockToScan(wrapper.numberOfNodes()); - + if (dataRefNode instanceof BlockletDataRefNode) { + BlockletDataRefNode node = (BlockletDataRefNode) dataRefNode; + blockInfo.setFirstDataBlock(node); + blockInfo.setNumberOfBlockToScan(node.numberOfNodes()); } else { DataRefNode startDataBlock = finder.findFirstDataBlock(dataRefNode, blockInfo.getStartKey()); - while (startDataBlock.nodeNumber() < blockInfo.getStartBlockletIndex()) { + while (startDataBlock.nodeIndex() < blockInfo.getStartBlockletIndex()) { startDataBlock = startDataBlock.getNextDataRefNode(); } long numberOfBlockToScan = blockInfo.getNumberOfBlockletToScan(); //if number of block is less than 0 then take end block. if (numberOfBlockToScan <= 0) { DataRefNode endDataBlock = finder.findLastDataBlock(dataRefNode, blockInfo.getEndKey()); - numberOfBlockToScan = endDataBlock.nodeNumber() - startDataBlock.nodeNumber() + 1; + numberOfBlockToScan = endDataBlock.nodeIndex() - startDataBlock.nodeIndex() + 1; } blockInfo.setFirstDataBlock(startDataBlock); blockInfo.setNumberOfBlockToScan(numberOfBlockToScan); @@ -230,7 +227,8 @@ private void setDeltedDeltaBoToDataBlock(DeleteDeltaInfo deleteDeltaInfo, } } - @Override public boolean hasNext() { + @Override + public boolean hasNext() { if ((dataBlockIterator != null && dataBlockIterator.hasNext())) { return true; } else if (blockExecutionInfos.size() > 0) { @@ -240,7 +238,7 @@ private void setDeltedDeltaBoToDataBlock(DeleteDeltaInfo deleteDeltaInfo, } } - protected void updateDataBlockIterator() { + void updateDataBlockIterator() { if (dataBlockIterator == null || !dataBlockIterator.hasNext()) { dataBlockIterator = getDataBlockIterator(); while (dataBlockIterator != null && !dataBlockIterator.hasNext()) { @@ -249,17 +247,17 @@ protected void updateDataBlockIterator() { } } - private DataBlockIteratorImpl getDataBlockIterator() { + private DataBlockIterator getDataBlockIterator() { if (blockExecutionInfos.size() > 0) { BlockExecutionInfo executionInfo = blockExecutionInfos.get(0); blockExecutionInfos.remove(executionInfo); - return new DataBlockIteratorImpl(executionInfo, fileReader, batchSize, queryStatisticsModel, + return new DataBlockIterator(executionInfo, fileReader, batchSize, queryStatisticsModel, execService); } return null; } - protected void initQueryStatiticsModel() { + private void initQueryStatiticsModel() { this.queryStatisticsModel = new QueryStatisticsModel(); this.queryStatisticsModel.setRecorder(recorder); QueryStatistic queryStatisticTotalBlocklet = new QueryStatistic(); diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/ChunkRowIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/ChunkRowIterator.java index 1efac302132..12357894c5a 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/ChunkRowIterator.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/ChunkRowIterator.java @@ -18,7 +18,7 @@ package org.apache.carbondata.core.scan.result.iterator; import org.apache.carbondata.common.CarbonIterator; -import org.apache.carbondata.core.scan.result.BatchResult; +import org.apache.carbondata.core.scan.result.RowBatch; /** * Iterator over row result @@ -28,14 +28,14 @@ public class ChunkRowIterator extends CarbonIterator { /** * iterator over chunk result */ - private CarbonIterator iterator; + private CarbonIterator iterator; /** * currect chunk */ - private BatchResult currentchunk; + private RowBatch currentchunk; - public ChunkRowIterator(CarbonIterator iterator) { + public ChunkRowIterator(CarbonIterator iterator) { this.iterator = iterator; if (iterator.hasNext()) { currentchunk = iterator.next(); diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/DetailQueryResultIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/DetailQueryResultIterator.java index 747f5a99208..c073c783957 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/DetailQueryResultIterator.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/DetailQueryResultIterator.java @@ -21,14 +21,14 @@ import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo; import org.apache.carbondata.core.scan.model.QueryModel; -import org.apache.carbondata.core.scan.result.BatchResult; +import org.apache.carbondata.core.scan.result.RowBatch; /** * In case of detail query we cannot keep all the records in memory so for * executing that query are returning a iterator over block and every time next * call will come it will execute the block and return the result */ -public class DetailQueryResultIterator extends AbstractDetailQueryResultIterator { +public class DetailQueryResultIterator extends AbstractDetailQueryResultIterator { private final Object lock = new Object(); @@ -37,18 +37,18 @@ public DetailQueryResultIterator(List infos, QueryModel quer super(infos, queryModel, execService); } - @Override public BatchResult next() { + @Override public RowBatch next() { return getBatchResult(); } - private BatchResult getBatchResult() { - BatchResult batchResult = new BatchResult(); + private RowBatch getBatchResult() { + RowBatch rowBatch = new RowBatch(); synchronized (lock) { updateDataBlockIterator(); if (dataBlockIterator != null) { - batchResult.setRows(dataBlockIterator.next()); + rowBatch.setRows(dataBlockIterator.next()); } } - return batchResult; + return rowBatch; } } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java index 553f85e19a5..773fbd78b04 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java @@ -17,20 +17,15 @@ package org.apache.carbondata.core.scan.result.iterator; import org.apache.carbondata.common.CarbonIterator; -import org.apache.carbondata.common.logging.LogService; -import org.apache.carbondata.common.logging.LogServiceFactory; -import org.apache.carbondata.core.scan.result.BatchResult; +import org.apache.carbondata.core.scan.result.RowBatch; public class PartitionSpliterRawResultIterator extends CarbonIterator { - private CarbonIterator iterator; - private BatchResult batch; + private CarbonIterator iterator; + private RowBatch batch; private int counter; - private static final LogService LOGGER = - LogServiceFactory.getLogService(PartitionSpliterRawResultIterator.class.getName()); - - public PartitionSpliterRawResultIterator(CarbonIterator iterator) { + public PartitionSpliterRawResultIterator(CarbonIterator iterator) { this.iterator = iterator; } @@ -65,7 +60,7 @@ public PartitionSpliterRawResultIterator(CarbonIterator iterator) { * @param batch * @return */ - private boolean checkBatchEnd(BatchResult batch) { + private boolean checkBatchEnd(RowBatch batch) { return !(counter < batch.getSize()); } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java index 70d095820c6..1dd159573e9 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java @@ -21,7 +21,7 @@ import org.apache.carbondata.common.logging.LogServiceFactory; import org.apache.carbondata.core.datastore.block.SegmentProperties; import org.apache.carbondata.core.keygenerator.KeyGenException; -import org.apache.carbondata.core.scan.result.BatchResult; +import org.apache.carbondata.core.scan.result.RowBatch; import org.apache.carbondata.core.scan.wrappers.ByteArrayWrapper; /** @@ -37,7 +37,7 @@ public class RawResultIterator extends CarbonIterator { /** * Iterator of the Batch raw result. */ - private CarbonIterator detailRawQueryResultIterator; + private CarbonIterator detailRawQueryResultIterator; /** * Counter to maintain the row counter. @@ -55,9 +55,9 @@ public class RawResultIterator extends CarbonIterator { /** * batch of the result. */ - private BatchResult batch; + private RowBatch batch; - public RawResultIterator(CarbonIterator detailRawQueryResultIterator, + public RawResultIterator(CarbonIterator detailRawQueryResultIterator, SegmentProperties sourceSegProperties, SegmentProperties destinationSegProperties) { this.detailRawQueryResultIterator = detailRawQueryResultIterator; this.sourceSegProperties = sourceSegProperties; @@ -155,7 +155,7 @@ private Object[] convertRow(Object[] rawRow) throws KeyGenException { * @param batch * @return */ - private boolean checkIfBatchIsProcessedCompletely(BatchResult batch) { + private boolean checkIfBatchIsProcessedCompletely(RowBatch batch) { if (counter < batch.getSize()) { return false; } else { diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/VectorDetailQueryResultIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/VectorDetailQueryResultIterator.java index cc9710e4320..c7cb00d3a58 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/VectorDetailQueryResultIterator.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/VectorDetailQueryResultIterator.java @@ -35,10 +35,12 @@ public VectorDetailQueryResultIterator(List infos, QueryMode super(infos, queryModel, execService); } - @Override public Object next() { + @Override + public Object next() { throw new UnsupportedOperationException("call processNextBatch instead"); } + @Override public void processNextBatch(CarbonColumnarBatch columnarBatch) { synchronized (lock) { updateDataBlockIterator(); diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/CarbonColumnarBatch.java b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/CarbonColumnarBatch.java index cfc2f16fc61..973ce0f72d7 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/CarbonColumnarBatch.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/CarbonColumnarBatch.java @@ -87,7 +87,4 @@ public void markFiltered(int rowId) { } } - public int getRowsFilteredCount() { - return rowsFiltered; - } } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/ColumnVectorInfo.java b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/ColumnVectorInfo.java index a5f81b99ac8..59117ddc981 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/ColumnVectorInfo.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/ColumnVectorInfo.java @@ -18,16 +18,16 @@ import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator; import org.apache.carbondata.core.scan.filter.GenericQueryType; -import org.apache.carbondata.core.scan.model.QueryDimension; -import org.apache.carbondata.core.scan.model.QueryMeasure; +import org.apache.carbondata.core.scan.model.ProjectionDimension; +import org.apache.carbondata.core.scan.model.ProjectionMeasure; public class ColumnVectorInfo implements Comparable { public int offset; public int size; public CarbonColumnVector vector; public int vectorOffset; - public QueryDimension dimension; - public QueryMeasure measure; + public ProjectionDimension dimension; + public ProjectionMeasure measure; public int ordinal; public DirectDictionaryGenerator directDictionaryGenerator; public MeasureDataVectorProcessor.MeasureVectorFiller measureVectorFiller; diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java index db4c9821861..8902dfb5aab 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java @@ -29,7 +29,7 @@ public interface MeasureVectorFiller { void fillMeasureVector(ColumnPage dataChunk, ColumnVectorInfo info); - void fillMeasureVectorForFilter(int[] rowMapping, ColumnPage dataChunk, + void fillMeasureVector(int[] filteredRowId, ColumnPage dataChunk, ColumnVectorInfo info); } @@ -60,7 +60,7 @@ public void fillMeasureVector(ColumnPage dataChunk, ColumnVectorInfo info) { } @Override - public void fillMeasureVectorForFilter(int[] rowMapping, ColumnPage dataChunk, + public void fillMeasureVector(int[] filteredRowId, ColumnPage dataChunk, ColumnVectorInfo info) { int offset = info.offset; int len = offset + info.size; @@ -69,13 +69,13 @@ public void fillMeasureVectorForFilter(int[] rowMapping, ColumnPage dataChunk, BitSet nullBitSet = dataChunk.getNullBits(); if (nullBitSet.isEmpty()) { for (int i = offset; i < len; i++) { - int currentRow = rowMapping[i]; + int currentRow = filteredRowId[i]; vector.putInt(vectorOffset, (int)dataChunk.getLong(currentRow)); vectorOffset++; } } else { for (int i = offset; i < len; i++) { - int currentRow = rowMapping[i]; + int currentRow = filteredRowId[i]; if (nullBitSet.get(currentRow)) { vector.putNull(vectorOffset); } else { @@ -117,7 +117,7 @@ public void fillMeasureVector(ColumnPage dataChunk, ColumnVectorInfo info) { } @Override - public void fillMeasureVectorForFilter(int[] rowMapping, + public void fillMeasureVector(int[] filteredRowId, ColumnPage dataChunk, ColumnVectorInfo info) { int offset = info.offset; int len = offset + info.size; @@ -126,13 +126,13 @@ public void fillMeasureVectorForFilter(int[] rowMapping, BitSet nullBitSet = dataChunk.getNullBits(); if (nullBitSet.isEmpty()) { for (int i = offset; i < len; i++) { - int currentRow = rowMapping[i]; + int currentRow = filteredRowId[i]; vector.putBoolean(vectorOffset, dataChunk.getBoolean(currentRow)); vectorOffset++; } } else { for (int i = offset; i < len; i++) { - int currentRow = rowMapping[i]; + int currentRow = filteredRowId[i]; if (nullBitSet.get(currentRow)) { vector.putNull(vectorOffset); } else { @@ -171,7 +171,7 @@ public void fillMeasureVector(ColumnPage dataChunk, ColumnVectorInfo info) { } @Override - public void fillMeasureVectorForFilter(int[] rowMapping, ColumnPage dataChunk, + public void fillMeasureVector(int[] filteredRowId, ColumnPage dataChunk, ColumnVectorInfo info) { int offset = info.offset; int len = offset + info.size; @@ -180,13 +180,13 @@ public void fillMeasureVectorForFilter(int[] rowMapping, ColumnPage dataChunk, BitSet nullBitSet = dataChunk.getNullBits(); if (nullBitSet.isEmpty()) { for (int i = offset; i < len; i++) { - int currentRow = rowMapping[i]; + int currentRow = filteredRowId[i]; vector.putShort(vectorOffset, (short) dataChunk.getLong(currentRow)); vectorOffset++; } } else { for (int i = offset; i < len; i++) { - int currentRow = rowMapping[i]; + int currentRow = filteredRowId[i]; if (nullBitSet.get(currentRow)) { vector.putNull(vectorOffset); } else { @@ -225,7 +225,7 @@ public void fillMeasureVector(ColumnPage dataChunk, ColumnVectorInfo info) { } @Override - public void fillMeasureVectorForFilter(int[] rowMapping, ColumnPage dataChunk, + public void fillMeasureVector(int[] filteredRowId, ColumnPage dataChunk, ColumnVectorInfo info) { int offset = info.offset; int len = offset + info.size; @@ -234,13 +234,13 @@ public void fillMeasureVectorForFilter(int[] rowMapping, ColumnPage dataChunk, BitSet nullBitSet = dataChunk.getNullBits(); if (nullBitSet.isEmpty()) { for (int i = offset; i < len; i++) { - int currentRow = rowMapping[i]; + int currentRow = filteredRowId[i]; vector.putLong(vectorOffset, dataChunk.getLong(currentRow)); vectorOffset++; } } else { for (int i = offset; i < len; i++) { - int currentRow = rowMapping[i]; + int currentRow = filteredRowId[i]; if (nullBitSet.get(currentRow)) { vector.putNull(vectorOffset); } else { @@ -279,7 +279,7 @@ public void fillMeasureVector(ColumnPage dataChunk, ColumnVectorInfo info) { } @Override - public void fillMeasureVectorForFilter(int[] rowMapping, ColumnPage dataChunk, + public void fillMeasureVector(int[] filteredRowId, ColumnPage dataChunk, ColumnVectorInfo info) { int offset = info.offset; int len = offset + info.size; @@ -288,7 +288,7 @@ public void fillMeasureVectorForFilter(int[] rowMapping, ColumnPage dataChunk, int precision = info.measure.getMeasure().getPrecision(); BitSet nullBitSet = dataChunk.getNullBits(); for (int i = offset; i < len; i++) { - int currentRow = rowMapping[i]; + int currentRow = filteredRowId[i]; if (nullBitSet.get(currentRow)) { vector.putNull(vectorOffset); } else { @@ -330,7 +330,7 @@ public void fillMeasureVector(ColumnPage dataChunk, ColumnVectorInfo info) { } @Override - public void fillMeasureVectorForFilter(int[] rowMapping, ColumnPage dataChunk, + public void fillMeasureVector(int[] filteredRowId, ColumnPage dataChunk, ColumnVectorInfo info) { int offset = info.offset; int len = offset + info.size; @@ -339,13 +339,13 @@ public void fillMeasureVectorForFilter(int[] rowMapping, ColumnPage dataChunk, BitSet nullBitSet = dataChunk.getNullBits(); if (nullBitSet.isEmpty()) { for (int i = offset; i < len; i++) { - int currentRow = rowMapping[i]; + int currentRow = filteredRowId[i]; vector.putDouble(vectorOffset, dataChunk.getDouble(currentRow)); vectorOffset++; } } else { for (int i = offset; i < len; i++) { - int currentRow = rowMapping[i]; + int currentRow = filteredRowId[i]; if (nullBitSet.get(currentRow)) { vector.putNull(vectorOffset); } else { diff --git a/core/src/main/java/org/apache/carbondata/core/scan/scanner/BlockletScanner.java b/core/src/main/java/org/apache/carbondata/core/scan/scanner/BlockletScanner.java index 0ed0d43ef3a..0a410325ce9 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/scanner/BlockletScanner.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/scanner/BlockletScanner.java @@ -18,9 +18,10 @@ import java.io.IOException; +import org.apache.carbondata.core.datastore.DataRefNode; import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException; -import org.apache.carbondata.core.scan.processor.BlocksChunkHolder; -import org.apache.carbondata.core.scan.result.AbstractScannedResult; +import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks; +import org.apache.carbondata.core.scan.result.BlockletScannedResult; /** * Interface for processing the block @@ -30,31 +31,26 @@ public interface BlockletScanner { /** * Checks whether this blocklet required to scan or not based on min max of each blocklet. - * @param blocksChunkHolder + * @param dataBlock * @return * @throws IOException */ - boolean isScanRequired(BlocksChunkHolder blocksChunkHolder) throws IOException; + boolean isScanRequired(DataRefNode dataBlock); /** * Below method will used to process the block data and get the scanned result * - * @param blocksChunkHolder block chunk which holds the block data + * @param rawBlockletColumnChunks block chunk which holds the block data * @return scannerResult * result after processing */ - AbstractScannedResult scanBlocklet(BlocksChunkHolder blocksChunkHolder) + BlockletScannedResult scanBlocklet(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException, FilterUnsupportedException; /** * Just reads the blocklet from file, does not uncompress it. - * @param blocksChunkHolder + * @param rawBlockletColumnChunks */ - void readBlocklet(BlocksChunkHolder blocksChunkHolder) throws IOException; + void readBlocklet(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException; - /** - * In case if there is no filter satisfies. - * @return AbstractScannedResult - */ - AbstractScannedResult createEmptyResult(); } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/FilterScanner.java b/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java similarity index 60% rename from core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/FilterScanner.java rename to core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java index e77093bf6df..1c73d638b70 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/FilterScanner.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java @@ -21,8 +21,9 @@ import java.util.BitSet; import org.apache.carbondata.core.constants.CarbonCommonConstants; -import org.apache.carbondata.core.datastore.FileHolder; -import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk; +import org.apache.carbondata.core.datastore.DataRefNode; +import org.apache.carbondata.core.datastore.FileReader; +import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage; import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk; import org.apache.carbondata.core.datastore.page.ColumnPage; @@ -30,10 +31,9 @@ import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException; import org.apache.carbondata.core.scan.filter.executer.FilterExecuter; import org.apache.carbondata.core.scan.filter.executer.ImplicitColumnFilterExecutor; -import org.apache.carbondata.core.scan.processor.BlocksChunkHolder; -import org.apache.carbondata.core.scan.result.AbstractScannedResult; +import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks; +import org.apache.carbondata.core.scan.result.BlockletScannedResult; import org.apache.carbondata.core.scan.result.impl.FilterQueryScannedResult; -import org.apache.carbondata.core.scan.scanner.AbstractBlockletScanner; import org.apache.carbondata.core.stats.QueryStatistic; import org.apache.carbondata.core.stats.QueryStatisticsConstants; import org.apache.carbondata.core.stats.QueryStatisticsModel; @@ -43,13 +43,13 @@ /** * Below class will be used for filter query processing - * this class will be first apply the filter then it will read the block if + * this class will be first apply the filter then it will read the column page if * required and return the scanned result */ -public class FilterScanner extends AbstractBlockletScanner { +public class BlockletFilterScanner extends BlockletFullScanner { /** - * filter tree + * filter executer to evaluate filter condition */ private FilterExecuter filterExecuter; /** @@ -67,9 +67,9 @@ public class FilterScanner extends AbstractBlockletScanner { private boolean useBitSetPipeLine; - public FilterScanner(BlockExecutionInfo blockExecutionInfo, + public BlockletFilterScanner(BlockExecutionInfo blockExecutionInfo, QueryStatisticsModel queryStatisticsModel) { - super(blockExecutionInfo); + super(blockExecutionInfo, queryStatisticsModel); // to check whether min max is enabled or not String minMaxEnableValue = CarbonProperties.getInstance() .getProperty(CarbonCommonConstants.CARBON_QUERY_MIN_MAX_ENABLED, @@ -92,48 +92,47 @@ public FilterScanner(BlockExecutionInfo blockExecutionInfo, /** * Below method will be used to process the block * - * @param blocksChunkHolder block chunk holder which holds the data + * @param rawBlockletColumnChunks block chunk holder which holds the data * @throws FilterUnsupportedException */ - @Override public AbstractScannedResult scanBlocklet(BlocksChunkHolder blocksChunkHolder) + @Override + public BlockletScannedResult scanBlocklet(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException, FilterUnsupportedException { - return fillScannedResult(blocksChunkHolder); + return executeFilter(rawBlockletColumnChunks); } - @Override public boolean isScanRequired(BlocksChunkHolder blocksChunkHolder) throws IOException { + @Override + public boolean isScanRequired(DataRefNode dataBlock) { // adding statistics for number of pages QueryStatistic totalPagesScanned = queryStatisticsModel.getStatisticsTypeAndObjMap() .get(QueryStatisticsConstants.TOTAL_PAGE_SCANNED); totalPagesScanned.addCountStatistic(QueryStatisticsConstants.TOTAL_PAGE_SCANNED, - totalPagesScanned.getCount() + blocksChunkHolder.getDataBlock().numberOfPages()); + totalPagesScanned.getCount() + dataBlock.numberOfPages()); // apply min max if (isMinMaxEnabled) { BitSet bitSet = null; // check for implicit include filter instance if (filterExecuter instanceof ImplicitColumnFilterExecutor) { - String blockletId = blockExecutionInfo.getBlockId() + CarbonCommonConstants.FILE_SEPARATOR - + blocksChunkHolder.getDataBlock().blockletId(); + String blockletId = blockExecutionInfo.getBlockIdString() + + CarbonCommonConstants.FILE_SEPARATOR + dataBlock.blockletIndex(); bitSet = ((ImplicitColumnFilterExecutor) filterExecuter) .isFilterValuesPresentInBlockOrBlocklet( - blocksChunkHolder.getDataBlock().getColumnsMaxValue(), - blocksChunkHolder.getDataBlock().getColumnsMinValue(), blockletId); + dataBlock.getColumnsMaxValue(), + dataBlock.getColumnsMinValue(), blockletId); } else { bitSet = this.filterExecuter - .isScanRequired(blocksChunkHolder.getDataBlock().getColumnsMaxValue(), - blocksChunkHolder.getDataBlock().getColumnsMinValue()); - } - if (bitSet.isEmpty()) { - CarbonUtil.freeMemory(blocksChunkHolder.getDimensionRawDataChunk(), - blocksChunkHolder.getMeasureRawDataChunk()); - return false; + .isScanRequired(dataBlock.getColumnsMaxValue(), + dataBlock.getColumnsMinValue()); } + return !bitSet.isEmpty(); } return true; } - @Override public void readBlocklet(BlocksChunkHolder blocksChunkHolder) throws IOException { + @Override + public void readBlocklet(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException { long startTime = System.currentTimeMillis(); - this.filterExecuter.readBlocks(blocksChunkHolder); + this.filterExecuter.readColumnChunks(rawBlockletColumnChunks); // adding statistics for carbon read time QueryStatistic readTime = queryStatisticsModel.getStatisticsTypeAndObjMap() .get(QueryStatisticsConstants.READ_BLOCKlET_TIME); @@ -154,22 +153,23 @@ public FilterScanner(BlockExecutionInfo blockExecutionInfo, * read those blocks again, this is to avoid reading of same blocks which was already read * 5. Set the blocks and filter indexes to result * - * @param blocksChunkHolder + * @param rawBlockletColumnChunks * @throws FilterUnsupportedException */ - private AbstractScannedResult fillScannedResult(BlocksChunkHolder blocksChunkHolder) + private BlockletScannedResult executeFilter(RawBlockletColumnChunks rawBlockletColumnChunks) throws FilterUnsupportedException, IOException { long startTime = System.currentTimeMillis(); QueryStatistic totalBlockletStatistic = queryStatisticsModel.getStatisticsTypeAndObjMap() .get(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM); totalBlockletStatistic.addCountStatistic(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM, totalBlockletStatistic.getCount() + 1); - // apply filter on actual data - BitSetGroup bitSetGroup = this.filterExecuter.applyFilter(blocksChunkHolder, useBitSetPipeLine); - // if indexes is empty then return with empty result + // apply filter on actual data, for each page + BitSetGroup bitSetGroup = this.filterExecuter.applyFilter(rawBlockletColumnChunks, + useBitSetPipeLine); + // if filter result is empty then return with empty result if (bitSetGroup.isEmpty()) { - CarbonUtil.freeMemory(blocksChunkHolder.getDimensionRawDataChunk(), - blocksChunkHolder.getMeasureRawDataChunk()); + CarbonUtil.freeMemory(rawBlockletColumnChunks.getDimensionRawColumnChunks(), + rawBlockletColumnChunks.getMeasureRawColumnChunks()); QueryStatistic scanTime = queryStatisticsModel.getStatisticsTypeAndObjMap() .get(QueryStatisticsConstants.SCAN_BLOCKlET_TIME); @@ -183,10 +183,10 @@ private AbstractScannedResult fillScannedResult(BlocksChunkHolder blocksChunkHol return createEmptyResult(); } - AbstractScannedResult scannedResult = new FilterQueryScannedResult(blockExecutionInfo); + BlockletScannedResult scannedResult = new FilterQueryScannedResult(blockExecutionInfo); scannedResult.setBlockletId( - blockExecutionInfo.getBlockId() + CarbonCommonConstants.FILE_SEPARATOR + blocksChunkHolder - .getDataBlock().blockletId()); + blockExecutionInfo.getBlockIdString() + CarbonCommonConstants.FILE_SEPARATOR + + rawBlockletColumnChunks.getDataBlock().blockletIndex()); // valid scanned blocklet QueryStatistic validScannedBlockletStatistic = queryStatisticsModel.getStatisticsTypeAndObjMap() .get(QueryStatisticsConstants.VALID_SCAN_BLOCKLET_NUM); @@ -202,116 +202,119 @@ private AbstractScannedResult fillScannedResult(BlocksChunkHolder blocksChunkHol .get(QueryStatisticsConstants.PAGE_SCANNED); scannedPages.addCountStatistic(QueryStatisticsConstants.PAGE_SCANNED, scannedPages.getCount() + bitSetGroup.getScannedPages()); - int[] rowCount = new int[bitSetGroup.getNumberOfPages()]; - // get the row indexes from bot set - int[][] indexesGroup = new int[bitSetGroup.getNumberOfPages()][]; - for (int k = 0; k < indexesGroup.length; k++) { - BitSet bitSet = bitSetGroup.getBitSet(k); + int[] pageFilteredRowCount = new int[bitSetGroup.getNumberOfPages()]; + // get the row indexes from bit set for each page + int[][] pageFilteredRowId = new int[bitSetGroup.getNumberOfPages()][]; + int numPages = pageFilteredRowId.length; + for (int pageId = 0; pageId < numPages; pageId++) { + BitSet bitSet = bitSetGroup.getBitSet(pageId); if (bitSet != null && !bitSet.isEmpty()) { - int[] indexes = new int[bitSet.cardinality()]; + int[] matchedRowId = new int[bitSet.cardinality()]; int index = 0; for (int i = bitSet.nextSetBit(0); i >= 0; i = bitSet.nextSetBit(i + 1)) { - indexes[index++] = i; + matchedRowId[index++] = i; } - rowCount[k] = indexes.length; - indexesGroup[k] = indexes; + pageFilteredRowCount[pageId] = matchedRowId.length; + pageFilteredRowId[pageId] = matchedRowId; } } - FileHolder fileReader = blocksChunkHolder.getFileReader(); - int[][] allSelectedDimensionBlocksIndexes = - blockExecutionInfo.getAllSelectedDimensionBlocksIndexes(); + long dimensionReadTime = System.currentTimeMillis(); - DimensionRawColumnChunk[] projectionListDimensionChunk = blocksChunkHolder.getDataBlock() - .getDimensionChunks(fileReader, allSelectedDimensionBlocksIndexes); dimensionReadTime = System.currentTimeMillis() - dimensionReadTime; + FileReader fileReader = rawBlockletColumnChunks.getFileReader(); + + DimensionRawColumnChunk[] dimensionRawColumnChunks = - new DimensionRawColumnChunk[blockExecutionInfo.getTotalNumberDimensionBlock()]; + new DimensionRawColumnChunk[blockExecutionInfo.getTotalNumberDimensionToRead()]; + int numDimensionChunks = dimensionRawColumnChunks.length; // read dimension chunk blocks from file which is not present - for (int i = 0; i < dimensionRawColumnChunks.length; i++) { - if (null != blocksChunkHolder.getDimensionRawDataChunk()[i]) { - dimensionRawColumnChunks[i] = blocksChunkHolder.getDimensionRawDataChunk()[i]; - } + for (int chunkIndex = 0; chunkIndex < numDimensionChunks; chunkIndex++) { + dimensionRawColumnChunks[chunkIndex] = + rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]; } - for (int i = 0; i < allSelectedDimensionBlocksIndexes.length; i++) { - for (int j = allSelectedDimensionBlocksIndexes[i][0]; - j <= allSelectedDimensionBlocksIndexes[i][1]; j++) { - dimensionRawColumnChunks[j] = projectionListDimensionChunk[j]; - } + int[][] allSelectedDimensionColumnIndexRange = + blockExecutionInfo.getAllSelectedDimensionColumnIndexRange(); + DimensionRawColumnChunk[] projectionListDimensionChunk = rawBlockletColumnChunks.getDataBlock() + .readDimensionChunks(fileReader, allSelectedDimensionColumnIndexRange); + for (int[] columnIndexRange : allSelectedDimensionColumnIndexRange) { + System.arraycopy(projectionListDimensionChunk, columnIndexRange[0], + dimensionRawColumnChunks, columnIndexRange[0], + columnIndexRange[1] + 1 - columnIndexRange[0]); } - long dimensionReadTime1 = System.currentTimeMillis(); - /** + + /* * in case projection if the projected dimension are not loaded in the dimensionColumnDataChunk * then loading them */ int[] projectionListDimensionIndexes = blockExecutionInfo.getProjectionListDimensionIndexes(); - int projectionListDimensionIndexesLength = projectionListDimensionIndexes.length; - for (int i = 0; i < projectionListDimensionIndexesLength; i++) { - if (null == dimensionRawColumnChunks[projectionListDimensionIndexes[i]]) { - dimensionRawColumnChunks[projectionListDimensionIndexes[i]] = - blocksChunkHolder.getDataBlock() - .getDimensionChunk(fileReader, projectionListDimensionIndexes[i]); + for (int projectionListDimensionIndex : projectionListDimensionIndexes) { + if (null == dimensionRawColumnChunks[projectionListDimensionIndex]) { + dimensionRawColumnChunks[projectionListDimensionIndex] = + rawBlockletColumnChunks.getDataBlock().readDimensionChunk( + fileReader, projectionListDimensionIndex); + } + } + + DimensionColumnPage[][] dimensionColumnPages = + new DimensionColumnPage[numDimensionChunks][numPages]; + for (int chunkIndex = 0; chunkIndex < numDimensionChunks; chunkIndex++) { + if (dimensionRawColumnChunks[chunkIndex] != null) { + for (int pageId = 0; pageId < numPages; pageId++) { + dimensionColumnPages[chunkIndex][pageId] = + dimensionRawColumnChunks[chunkIndex].decodeColumnPage(pageId); + } } } - dimensionReadTime += (System.currentTimeMillis() - dimensionReadTime1); - dimensionReadTime1 = System.currentTimeMillis(); + + MeasureRawColumnChunk[] measureRawColumnChunks = - new MeasureRawColumnChunk[blockExecutionInfo.getTotalNumberOfMeasureBlock()]; - int[][] allSelectedMeasureBlocksIndexes = - blockExecutionInfo.getAllSelectedMeasureBlocksIndexes(); - MeasureRawColumnChunk[] projectionListMeasureChunk = blocksChunkHolder.getDataBlock() - .getMeasureChunks(fileReader, allSelectedMeasureBlocksIndexes); - dimensionReadTime += System.currentTimeMillis() - dimensionReadTime1; + new MeasureRawColumnChunk[blockExecutionInfo.getTotalNumberOfMeasureToRead()]; + int numMeasureChunks = measureRawColumnChunks.length; + // read the measure chunk blocks which is not present - for (int i = 0; i < measureRawColumnChunks.length; i++) { - if (null != blocksChunkHolder.getMeasureRawDataChunk()[i]) { - measureRawColumnChunks[i] = blocksChunkHolder.getMeasureRawDataChunk()[i]; + for (int chunkIndex = 0; chunkIndex < numMeasureChunks; chunkIndex++) { + if (null != rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) { + measureRawColumnChunks[chunkIndex] = + rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]; } } - for (int i = 0; i < allSelectedMeasureBlocksIndexes.length; i++) { - for (int j = allSelectedMeasureBlocksIndexes[i][0]; - j <= allSelectedMeasureBlocksIndexes[i][1]; j++) { - measureRawColumnChunks[j] = projectionListMeasureChunk[j]; - } + + int[][] allSelectedMeasureColumnIndexRange = + blockExecutionInfo.getAllSelectedMeasureIndexRange(); + MeasureRawColumnChunk[] projectionListMeasureChunk = rawBlockletColumnChunks.getDataBlock() + .readMeasureChunks(fileReader, allSelectedMeasureColumnIndexRange); + for (int[] columnIndexRange : allSelectedMeasureColumnIndexRange) { + System.arraycopy(projectionListMeasureChunk, columnIndexRange[0], measureRawColumnChunks, + columnIndexRange[0], columnIndexRange[1] + 1 - columnIndexRange[0]); } - dimensionReadTime1 = System.currentTimeMillis(); - /** + /* * in case projection if the projected measure are not loaded in the ColumnPage * then loading them */ int[] projectionListMeasureIndexes = blockExecutionInfo.getProjectionListMeasureIndexes(); - int projectionListMeasureIndexesLength = projectionListMeasureIndexes.length; - for (int i = 0; i < projectionListMeasureIndexesLength; i++) { - if (null == measureRawColumnChunks[projectionListMeasureIndexes[i]]) { - measureRawColumnChunks[projectionListMeasureIndexes[i]] = blocksChunkHolder.getDataBlock() - .getMeasureChunk(fileReader, projectionListMeasureIndexes[i]); + for (int projectionListMeasureIndex : projectionListMeasureIndexes) { + if (null == measureRawColumnChunks[projectionListMeasureIndex]) { + measureRawColumnChunks[projectionListMeasureIndex] = rawBlockletColumnChunks.getDataBlock() + .readMeasureChunk(fileReader, projectionListMeasureIndex); } } - dimensionReadTime += System.currentTimeMillis() - dimensionReadTime1; - DimensionColumnDataChunk[][] dimensionColumnDataChunks = - new DimensionColumnDataChunk[dimensionRawColumnChunks.length][indexesGroup.length]; - ColumnPage[][] columnPages = - new ColumnPage[measureRawColumnChunks.length][indexesGroup.length]; - for (int i = 0; i < dimensionRawColumnChunks.length; i++) { - if (dimensionRawColumnChunks[i] != null) { - for (int j = 0; j < indexesGroup.length; j++) { - dimensionColumnDataChunks[i][j] = dimensionRawColumnChunks[i].convertToDimColDataChunk(j); + ColumnPage[][] measureColumnPages = new ColumnPage[numMeasureChunks][numPages]; + for (int chunkIndex = 0; chunkIndex < numMeasureChunks; chunkIndex++) { + if (measureRawColumnChunks[chunkIndex] != null) { + for (int pageId = 0; pageId < numPages; pageId++) { + measureColumnPages[chunkIndex][pageId] = + measureRawColumnChunks[chunkIndex].decodeColumnPage(pageId); } } } - for (int i = 0; i < measureRawColumnChunks.length; i++) { - if (measureRawColumnChunks[i] != null) { - for (int j = 0; j < indexesGroup.length; j++) { - columnPages[i][j] = measureRawColumnChunks[i].convertToColumnPage(j); - } - } - } - scannedResult.setDimensionChunks(dimensionColumnDataChunks); - scannedResult.setIndexes(indexesGroup); - scannedResult.setMeasureChunks(columnPages); + + scannedResult.setDimensionColumnPages(dimensionColumnPages); + scannedResult.setPageFilteredRowId(pageFilteredRowId); + scannedResult.setMeasureColumnPages(measureColumnPages); scannedResult.setDimRawColumnChunks(dimensionRawColumnChunks); scannedResult.setMsrRawColumnChunks(measureRawColumnChunks); - scannedResult.setNumberOfRows(rowCount); + scannedResult.setPageFilteredRowCount(pageFilteredRowCount); // adding statistics for carbon scan time QueryStatistic scanTime = queryStatisticsModel.getStatisticsTypeAndObjMap() .get(QueryStatisticsConstants.SCAN_BLOCKlET_TIME); diff --git a/core/src/main/java/org/apache/carbondata/core/scan/scanner/AbstractBlockletScanner.java b/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFullScanner.java similarity index 63% rename from core/src/main/java/org/apache/carbondata/core/scan/scanner/AbstractBlockletScanner.java rename to core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFullScanner.java index bf26ca32580..f0211dcff49 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/scanner/AbstractBlockletScanner.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFullScanner.java @@ -14,47 +14,54 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.carbondata.core.scan.scanner; +package org.apache.carbondata.core.scan.scanner.impl; import java.io.IOException; import org.apache.carbondata.core.constants.CarbonCommonConstants; import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants; -import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk; +import org.apache.carbondata.core.datastore.DataRefNode; +import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage; import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk; import org.apache.carbondata.core.datastore.page.ColumnPage; import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo; import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException; -import org.apache.carbondata.core.scan.processor.BlocksChunkHolder; -import org.apache.carbondata.core.scan.result.AbstractScannedResult; +import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks; +import org.apache.carbondata.core.scan.result.BlockletScannedResult; import org.apache.carbondata.core.scan.result.impl.NonFilterQueryScannedResult; +import org.apache.carbondata.core.scan.scanner.BlockletScanner; import org.apache.carbondata.core.stats.QueryStatistic; import org.apache.carbondata.core.stats.QueryStatisticsConstants; import org.apache.carbondata.core.stats.QueryStatisticsModel; /** - * Blocklet scanner class to process the block + * Blocklet scanner to do full scan of a blocklet, + * returning all projection and filter column chunks */ -public abstract class AbstractBlockletScanner implements BlockletScanner { +public class BlockletFullScanner implements BlockletScanner { /** * block execution info */ protected BlockExecutionInfo blockExecutionInfo; - public QueryStatisticsModel queryStatisticsModel; + private QueryStatisticsModel queryStatisticsModel; - private AbstractScannedResult emptyResult; + private BlockletScannedResult emptyResult; - public AbstractBlockletScanner(BlockExecutionInfo tableBlockExecutionInfos) { + public BlockletFullScanner(BlockExecutionInfo tableBlockExecutionInfos, + QueryStatisticsModel queryStatisticsModel) { this.blockExecutionInfo = tableBlockExecutionInfos; + this.queryStatisticsModel = queryStatisticsModel; } - @Override public AbstractScannedResult scanBlocklet(BlocksChunkHolder blocksChunkHolder) + @Override + public BlockletScannedResult scanBlocklet( + RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException, FilterUnsupportedException { long startTime = System.currentTimeMillis(); - AbstractScannedResult scannedResult = new NonFilterQueryScannedResult(blockExecutionInfo); + BlockletScannedResult scannedResult = new NonFilterQueryScannedResult(blockExecutionInfo); QueryStatistic totalBlockletStatistic = queryStatisticsModel.getStatisticsTypeAndObjMap() .get(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM); totalBlockletStatistic.addCountStatistic(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM, @@ -68,52 +75,53 @@ public AbstractBlockletScanner(BlockExecutionInfo tableBlockExecutionInfos) { QueryStatistic validPages = queryStatisticsModel.getStatisticsTypeAndObjMap() .get(QueryStatisticsConstants.VALID_PAGE_SCANNED); validPages.addCountStatistic(QueryStatisticsConstants.VALID_PAGE_SCANNED, - validPages.getCount() + blocksChunkHolder.getDataBlock().numberOfPages()); + validPages.getCount() + rawBlockletColumnChunks.getDataBlock().numberOfPages()); // adding statistics for number of pages QueryStatistic totalPagesScanned = queryStatisticsModel.getStatisticsTypeAndObjMap() .get(QueryStatisticsConstants.TOTAL_PAGE_SCANNED); totalPagesScanned.addCountStatistic(QueryStatisticsConstants.TOTAL_PAGE_SCANNED, - totalPagesScanned.getCount() + blocksChunkHolder.getDataBlock().numberOfPages()); + totalPagesScanned.getCount() + rawBlockletColumnChunks.getDataBlock().numberOfPages()); scannedResult.setBlockletId( - blockExecutionInfo.getBlockId() + CarbonCommonConstants.FILE_SEPARATOR + blocksChunkHolder - .getDataBlock().blockletId()); + blockExecutionInfo.getBlockIdString() + CarbonCommonConstants.FILE_SEPARATOR + + rawBlockletColumnChunks.getDataBlock().blockletIndex()); if (!blockExecutionInfo.isPrefetchBlocklet()) { - readBlocklet(blocksChunkHolder); + readBlocklet(rawBlockletColumnChunks); } DimensionRawColumnChunk[] dimensionRawColumnChunks = - blocksChunkHolder.getDimensionRawDataChunk(); - DimensionColumnDataChunk[][] dimensionColumnDataChunks = - new DimensionColumnDataChunk[dimensionRawColumnChunks.length][blocksChunkHolder + rawBlockletColumnChunks.getDimensionRawColumnChunks(); + DimensionColumnPage[][] dimensionColumnDataChunks = + new DimensionColumnPage[dimensionRawColumnChunks.length][rawBlockletColumnChunks .getDataBlock().numberOfPages()]; - MeasureRawColumnChunk[] measureRawColumnChunks = blocksChunkHolder.getMeasureRawDataChunk(); - ColumnPage[][] columnPages = - new ColumnPage[measureRawColumnChunks.length][blocksChunkHolder.getDataBlock() + MeasureRawColumnChunk[] measureRawColumnChunks = + rawBlockletColumnChunks.getMeasureRawColumnChunks(); + ColumnPage[][] measureColumnPages = + new ColumnPage[measureRawColumnChunks.length][rawBlockletColumnChunks.getDataBlock() .numberOfPages()]; - scannedResult.setDimensionChunks(dimensionColumnDataChunks); - scannedResult.setMeasureChunks(columnPages); + scannedResult.setDimensionColumnPages(dimensionColumnDataChunks); + scannedResult.setMeasureColumnPages(measureColumnPages); scannedResult.setDimRawColumnChunks(dimensionRawColumnChunks); scannedResult.setMsrRawColumnChunks(measureRawColumnChunks); if (blockExecutionInfo.isPrefetchBlocklet()) { for (int i = 0; i < dimensionRawColumnChunks.length; i++) { if (dimensionRawColumnChunks[i] != null) { - dimensionColumnDataChunks[i] = dimensionRawColumnChunks[i].convertToDimColDataChunks(); + dimensionColumnDataChunks[i] = dimensionRawColumnChunks[i].decodeAllColumnPages(); } } for (int i = 0; i < measureRawColumnChunks.length; i++) { if (measureRawColumnChunks[i] != null) { - columnPages[i] = measureRawColumnChunks[i].convertToColumnPage(); + measureColumnPages[i] = measureRawColumnChunks[i].decodeAllColumnPages(); } } } int[] numberOfRows = null; - if (blockExecutionInfo.getAllSelectedDimensionBlocksIndexes().length > 0) { + if (blockExecutionInfo.getAllSelectedDimensionColumnIndexRange().length > 0) { for (int i = 0; i < dimensionRawColumnChunks.length; i++) { if (dimensionRawColumnChunks[i] != null) { numberOfRows = dimensionRawColumnChunks[i].getRowCount(); break; } } - } else if (blockExecutionInfo.getAllSelectedMeasureBlocksIndexes().length > 0) { + } else if (blockExecutionInfo.getAllSelectedMeasureIndexRange().length > 0) { for (int i = 0; i < measureRawColumnChunks.length; i++) { if (measureRawColumnChunks[i] != null) { numberOfRows = measureRawColumnChunks[i].getRowCount(); @@ -124,19 +132,19 @@ public AbstractBlockletScanner(BlockExecutionInfo tableBlockExecutionInfos) { // count(*) case there would not be any dimensions are measures selected. if (numberOfRows == null) { - numberOfRows = new int[blocksChunkHolder.getDataBlock().numberOfPages()]; + numberOfRows = new int[rawBlockletColumnChunks.getDataBlock().numberOfPages()]; for (int i = 0; i < numberOfRows.length; i++) { numberOfRows[i] = CarbonV3DataFormatConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT; } - int lastPageSize = blocksChunkHolder.getDataBlock().nodeSize() + int lastPageSize = rawBlockletColumnChunks.getDataBlock().numRows() % CarbonV3DataFormatConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT; ; if (lastPageSize > 0) { numberOfRows[numberOfRows.length - 1] = lastPageSize; } } - scannedResult.setNumberOfRows(numberOfRows); + scannedResult.setPageFilteredRowCount(numberOfRows); if (!blockExecutionInfo.isPrefetchBlocklet()) { scannedResult.fillDataChunks(); } @@ -148,16 +156,18 @@ public AbstractBlockletScanner(BlockExecutionInfo tableBlockExecutionInfos) { return scannedResult; } - @Override public void readBlocklet(BlocksChunkHolder blocksChunkHolder) throws IOException { + @Override + public void readBlocklet(RawBlockletColumnChunks rawBlockletColumnChunks) + throws IOException { long startTime = System.currentTimeMillis(); - DimensionRawColumnChunk[] dimensionRawColumnChunks = blocksChunkHolder.getDataBlock() - .getDimensionChunks(blocksChunkHolder.getFileReader(), - blockExecutionInfo.getAllSelectedDimensionBlocksIndexes()); - blocksChunkHolder.setDimensionRawDataChunk(dimensionRawColumnChunks); - MeasureRawColumnChunk[] measureRawColumnChunks = blocksChunkHolder.getDataBlock() - .getMeasureChunks(blocksChunkHolder.getFileReader(), - blockExecutionInfo.getAllSelectedMeasureBlocksIndexes()); - blocksChunkHolder.setMeasureRawDataChunk(measureRawColumnChunks); + DimensionRawColumnChunk[] dimensionRawColumnChunks = rawBlockletColumnChunks.getDataBlock() + .readDimensionChunks(rawBlockletColumnChunks.getFileReader(), + blockExecutionInfo.getAllSelectedDimensionColumnIndexRange()); + rawBlockletColumnChunks.setDimensionRawColumnChunks(dimensionRawColumnChunks); + MeasureRawColumnChunk[] measureRawColumnChunks = rawBlockletColumnChunks.getDataBlock() + .readMeasureChunks(rawBlockletColumnChunks.getFileReader(), + blockExecutionInfo.getAllSelectedMeasureIndexRange()); + rawBlockletColumnChunks.setMeasureRawColumnChunks(measureRawColumnChunks); // adding statistics for carbon read time QueryStatistic readTime = queryStatisticsModel.getStatisticsTypeAndObjMap() .get(QueryStatisticsConstants.READ_BLOCKlET_TIME); @@ -165,16 +175,16 @@ public AbstractBlockletScanner(BlockExecutionInfo tableBlockExecutionInfos) { readTime.getCount() + (System.currentTimeMillis() - startTime)); } - @Override public AbstractScannedResult createEmptyResult() { + BlockletScannedResult createEmptyResult() { if (emptyResult == null) { emptyResult = new NonFilterQueryScannedResult(blockExecutionInfo); - emptyResult.setNumberOfRows(new int[0]); - emptyResult.setIndexes(new int[0][]); + emptyResult.setPageFilteredRowCount(new int[0]); + emptyResult.setPageFilteredRowId(new int[0][]); } return emptyResult; } - @Override public boolean isScanRequired(BlocksChunkHolder blocksChunkHolder) throws IOException { + @Override public boolean isScanRequired(DataRefNode dataBlock) { // For non filter it is always true return true; } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/NonFilterScanner.java b/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/NonFilterScanner.java deleted file mode 100644 index 1373ed588db..00000000000 --- a/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/NonFilterScanner.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.scan.scanner.impl; - -import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo; -import org.apache.carbondata.core.scan.scanner.AbstractBlockletScanner; -import org.apache.carbondata.core.stats.QueryStatisticsModel; - -/** - * Non filter processor which will be used for non filter query - * In case of non filter query we just need to read all the blocks requested in the - * query and pass it to scanned result - */ -public class NonFilterScanner extends AbstractBlockletScanner { - - public NonFilterScanner(BlockExecutionInfo blockExecutionInfo, - QueryStatisticsModel queryStatisticsModel) { - super(blockExecutionInfo); - super.queryStatisticsModel = queryStatisticsModel; - } -} diff --git a/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java b/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java index 2f981b57479..6faae03e195 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java @@ -30,22 +30,17 @@ public class ByteArrayWrapper implements Comparable, Serializa * to store key which is generated using * key generator */ - protected byte[] dictionaryKey; + private byte[] dictionaryKey; /** * to store no dictionary column data */ - protected byte[][] complexTypesKeys; + private byte[][] complexTypesKeys; /** * to store no dictionary column data */ - protected byte[][] noDictionaryKeys; - - /** - * contains value of implicit columns in byte array format - */ - protected byte[] implicitColumnByteArray; + private byte[][] noDictionaryKeys; public ByteArrayWrapper() { } @@ -90,16 +85,6 @@ public byte[][] getNoDictionaryKeys() { return this.noDictionaryKeys; } - /** - * to get the no dictionary column data - * - * @param index of the no dictionary key - * @return no dictionary key for the index - */ - public byte[] getComplexTypeByIndex(int index) { - return this.complexTypesKeys[index]; - } - /** * to generate the hash code */ @@ -200,13 +185,6 @@ public byte[] getComplexTypeByIndex(int index) { return compareTo; } - /** - * @return the complexTypesKeys - */ - public byte[][] getComplexTypesKeys() { - return complexTypesKeys; - } - /** * @param complexTypesKeys the complexTypesKeys to set */ @@ -214,17 +192,4 @@ public void setComplexTypesKeys(byte[][] complexTypesKeys) { this.complexTypesKeys = complexTypesKeys; } - /** - * @return - */ - public byte[] getImplicitColumnByteArray() { - return implicitColumnByteArray; - } - - /** - * @param implicitColumnByteArray - */ - public void setImplicitColumnByteArray(byte[] implicitColumnByteArray) { - this.implicitColumnByteArray = implicitColumnByteArray; - } } diff --git a/core/src/main/java/org/apache/carbondata/core/stats/PartitionStatistic.java b/core/src/main/java/org/apache/carbondata/core/stats/PartitionStatistic.java deleted file mode 100644 index 8a37d01cbce..00000000000 --- a/core/src/main/java/org/apache/carbondata/core/stats/PartitionStatistic.java +++ /dev/null @@ -1,23 +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.stats; - -import java.io.Serializable; - -public class PartitionStatistic implements Serializable { - -} \ No newline at end of file diff --git a/core/src/main/java/org/apache/carbondata/core/stats/QueryStatisticsModel.java b/core/src/main/java/org/apache/carbondata/core/stats/QueryStatisticsModel.java index 55f088279ef..ed60d37e313 100644 --- a/core/src/main/java/org/apache/carbondata/core/stats/QueryStatisticsModel.java +++ b/core/src/main/java/org/apache/carbondata/core/stats/QueryStatisticsModel.java @@ -37,7 +37,4 @@ public Map getStatisticsTypeAndObjMap() { return statisticsTypeAndObjMap; } - public void setStatisticsTypeAndObjMap(Map statisticsTypeAndObjMap) { - this.statisticsTypeAndObjMap = statisticsTypeAndObjMap; - } } diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java index b6a9e3681cf..d6671b41a78 100644 --- a/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java +++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java @@ -344,15 +344,6 @@ public void setMajorCompacted(String majorCompacted) { this.majorCompacted = majorCompacted; } - /** - * To get isDeleted property. - * - * @return isDeleted - */ - public String getIsDeleted() { - return isDeleted; - } - /** * To set isDeleted property. * diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java index ab849ce366c..49d494abc3a 100755 --- a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java +++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java @@ -678,22 +678,6 @@ public String getUpdateStatusFileName(LoadMetadataDetails[] segmentList) { return ""; } - /** - * getting the task numbers present in the segment. - * @param segmentId - * @return - */ - public List getUpdatedTasksDetailsForSegment(String segmentId, SegmentUpdateStatusManager - updateStatusManager) { - List taskList = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - List list = updateStatusManager.getUpdateDeltaFiles(segmentId); - for (String eachFileName : list) { - taskList.add(CarbonTablePath.DataFileUtil.getTaskNo(eachFileName)); - } - return taskList; - } - - public static class ValidAndInvalidSegmentsInfo { private final List listOfValidSegments; private final List listOfValidUpdatedSegments; diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java index 64c13da988e..92af9a196b8 100644 --- a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java +++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java @@ -48,7 +48,6 @@ import org.apache.carbondata.core.mutate.SegmentUpdateDetails; import org.apache.carbondata.core.mutate.TupleIdEnum; import org.apache.carbondata.core.mutate.UpdateVO; -import org.apache.carbondata.core.reader.CarbonDeleteFilesDataReader; import org.apache.carbondata.core.util.CarbonUtil; import org.apache.carbondata.core.util.path.CarbonStorePath; import org.apache.carbondata.core.util.path.CarbonTablePath; @@ -158,14 +157,6 @@ public LoadMetadataDetails[] getLoadMetadataDetails() { return segmentDetails; } - /** - * - * @param loadMetadataDetails - */ - public void setLoadMetadataDetails(LoadMetadataDetails[] loadMetadataDetails) { - this.segmentDetails = loadMetadataDetails; - } - /** * Returns the UpdateStatus Details. * @return @@ -192,18 +183,6 @@ public ICarbonLock getTableUpdateStatusLock() { LockUsage.TABLE_UPDATE_STATUS_LOCK); } - /** - * Returns all delete delta files of specified block - * - * @param tupleId - * @return - * @throws Exception - */ - public List getDeleteDeltaFiles(String tupleId) throws Exception { - return getDeltaFiles(tupleId, CarbonCommonConstants.DELETE_DELTA_FILE_EXT); - } - - /** * Returns all update delta files of specified Segment. * @@ -266,20 +245,6 @@ public List getUpdateDeltaFiles(final String segmentId) { return updatedDeltaFilesList; } - /** - * Returns all deleted records of specified block - * - * @param tupleId - * @return - * @throws Exception - */ - public Map getDeleteDeltaDataFromAllFiles(String tupleId) throws Exception { - List deltaFiles = getDeltaFiles(tupleId, CarbonCommonConstants.DELETE_DELTA_FILE_EXT); - CarbonDeleteFilesDataReader dataReader = new CarbonDeleteFilesDataReader(); - String blockletId = CarbonUpdateUtil.getRequiredFieldFromTID(tupleId, TupleIdEnum.BLOCKLET_ID); - return dataReader.getDeleteDataFromAllFiles(deltaFiles, blockletId); - } - /** * Below method will be used to get all the delete delta files based on block name * @@ -795,41 +760,6 @@ public void writeLoadDetailsIntoFile(List listOfSegmentUpd } - /** - * compares passed time stamp with status file delete timestamp and - * returns latest timestamp from status file if both are not equal - * returns null otherwise - * - * @param completeBlockName - * @param timestamp - * @return - */ - public String getTimestampForRefreshCache(String completeBlockName, String timestamp) { - long cacheTimestamp = 0; - if (null != timestamp) { - cacheTimestamp = CarbonUpdateUtil.getTimeStampAsLong(timestamp); - } - String blockName = CarbonTablePath.addDataPartPrefix(CarbonUpdateUtil.getBlockName( - CarbonUpdateUtil.getRequiredFieldFromTID(completeBlockName, TupleIdEnum.BLOCK_ID))); - String segmentId = - CarbonUpdateUtil.getRequiredFieldFromTID(completeBlockName, TupleIdEnum.SEGMENT_ID); - SegmentUpdateDetails[] listOfSegmentUpdateDetailsArray = - readLoadMetadata(); - for (SegmentUpdateDetails block : listOfSegmentUpdateDetailsArray) { - if (segmentId.equalsIgnoreCase(block.getSegmentName()) && - block.getBlockName().equalsIgnoreCase(blockName) && - !CarbonUpdateUtil.isBlockInvalid(block.getSegmentStatus())) { - long deleteTimestampFromStatusFile = block.getDeleteDeltaEndTimeAsLong(); - if (Long.compare(deleteTimestampFromStatusFile, cacheTimestamp) == 0) { - return null; - } else { - return block.getDeleteDeltaEndTimestamp(); - } - } - } - return null; - } - /** * This method closes the streams * @@ -849,85 +779,7 @@ private void closeStreams(Closeable... streams) { } } } - /** - * Get the invalid tasks in that segment. - * @param segmentId - * @return - */ - public List getInvalidBlockList(String segmentId) { - - // get the original fact file timestamp from the table status file. - List listOfInvalidBlocks = new ArrayList(); - SegmentStatusManager ssm = new SegmentStatusManager(absoluteTableIdentifier); - CarbonTablePath carbonTablePath = CarbonStorePath - .getCarbonTablePath(absoluteTableIdentifier.getTablePath(), - absoluteTableIdentifier.getCarbonTableIdentifier()); - LoadMetadataDetails[] segmentDetails = - ssm.readLoadMetadata(carbonTablePath.getMetadataDirectoryPath()); - long timestampOfOriginalFacts = 0; - - String startTimestampOfUpdate = "" ; - String endTimestampOfUpdate = ""; - - for (LoadMetadataDetails segment : segmentDetails) { - // find matching segment and return timestamp. - if (segment.getLoadName().equalsIgnoreCase(segmentId)) { - timestampOfOriginalFacts = segment.getLoadStartTime(); - startTimestampOfUpdate = segment.getUpdateDeltaStartTimestamp(); - endTimestampOfUpdate = segment.getUpdateDeltaEndTimestamp(); - } - } - - if (startTimestampOfUpdate.isEmpty()) { - return listOfInvalidBlocks; - - } - - // now after getting the original fact timestamp, what ever is remaining - // files need to cross check it with table status file. - // filter out the fact files. - - String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(segmentId); - CarbonFile segDir = - FileFactory.getCarbonFile(segmentPath, FileFactory.getFileType(segmentPath)); - - final Long endTimeStampFinal = CarbonUpdateUtil.getTimeStampAsLong(endTimestampOfUpdate); - final Long startTimeStampFinal = CarbonUpdateUtil.getTimeStampAsLong(startTimestampOfUpdate); - final Long timeStampOriginalFactFinal = - timestampOfOriginalFacts; - - CarbonFile[] files = segDir.listFiles(new CarbonFileFilter() { - - @Override public boolean accept(CarbonFile pathName) { - String fileName = pathName.getName(); - if (fileName.endsWith(CarbonCommonConstants.UPDATE_DELTA_FILE_EXT)) { - String firstPart = fileName.substring(0, fileName.indexOf('.')); - - long timestamp = Long.parseLong(firstPart - .substring(firstPart.lastIndexOf(CarbonCommonConstants.HYPHEN) + 1, - firstPart.length())); - if (Long.compare(timestamp, endTimeStampFinal) <= 0 - && Long.compare(timestamp, startTimeStampFinal) >= 0) { - return false; - } - if (Long.compare(timestamp, timeStampOriginalFactFinal) == 0) { - return false; - } - // take the rest of files as they are invalid. - return true; - } - return false; - } - }); - - // gather the task numbers. - for (CarbonFile updateFiles : files) { - listOfInvalidBlocks.add(updateFiles.getName()); - } - - return listOfInvalidBlocks; - } /** * Returns the invalid timestamp range of a segment. * @param segmentId @@ -953,12 +805,11 @@ public UpdateVO getInvalidTimestampRange(String segmentId) { } /** * - * @param segmentId * @param block * @param needCompleteList * @return */ - public CarbonFile[] getDeleteDeltaInvalidFilesList(final String segmentId, + public CarbonFile[] getDeleteDeltaInvalidFilesList( final SegmentUpdateDetails block, final boolean needCompleteList, CarbonFile[] allSegmentFiles, boolean isAbortedFile) { @@ -1004,12 +855,11 @@ public CarbonFile[] getDeleteDeltaInvalidFilesList(final String segmentId, /** * - * @param blockName * @param allSegmentFiles * @return */ - public CarbonFile[] getAllBlockRelatedFiles(String blockName, CarbonFile[] allSegmentFiles, - String actualBlockName) { + public CarbonFile[] getAllBlockRelatedFiles(CarbonFile[] allSegmentFiles, + String actualBlockName) { List files = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); for (CarbonFile eachFile : allSegmentFiles) { diff --git a/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java b/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java index 94a041a25fe..b74c27994d0 100644 --- a/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java +++ b/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java @@ -378,7 +378,6 @@ protected SegmentInfo getSegmentInfo(org.apache.carbondata.format.SegmentInfo se cardinality[i] = segmentInfo.getColumn_cardinalities().get(i); } info.setColumnCardinality(cardinality); - info.setNumberOfColumns(segmentInfo.getNum_cols()); return info; } 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 52305bda352..0cc783ea5db 100644 --- a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java +++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java @@ -48,10 +48,10 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants; import org.apache.carbondata.core.constants.CarbonLoadOptionConstants; import org.apache.carbondata.core.datamap.Segment; -import org.apache.carbondata.core.datastore.FileHolder; +import org.apache.carbondata.core.datastore.FileReader; import org.apache.carbondata.core.datastore.block.AbstractIndex; import org.apache.carbondata.core.datastore.block.TableBlockInfo; -import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk; +import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage; import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk; import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk; import org.apache.carbondata.core.datastore.columnar.ColumnGroupModel; @@ -82,7 +82,7 @@ import org.apache.carbondata.core.mutate.UpdateVO; import org.apache.carbondata.core.reader.ThriftReader; import org.apache.carbondata.core.reader.ThriftReader.TBaseCreator; -import org.apache.carbondata.core.scan.model.QueryDimension; +import org.apache.carbondata.core.scan.model.ProjectionDimension; import org.apache.carbondata.core.statusmanager.LoadMetadataDetails; import org.apache.carbondata.core.statusmanager.SegmentStatus; import org.apache.carbondata.core.statusmanager.SegmentStatusManager; @@ -248,16 +248,13 @@ public static int getIncrementedCardinality(int dimCardinality) { public static ColumnGroupModel getColGroupModel(int[][] columnGroups) { int[] columnSplit = new int[columnGroups.length]; int noOfColumnStore = columnSplit.length; - boolean[] columnarStore = new boolean[noOfColumnStore]; for (int i = 0; i < columnGroups.length; i++) { columnSplit[i] = columnGroups[i].length; - columnarStore[i] = columnGroups[i].length <= 1; } ColumnGroupModel colGroupModel = new ColumnGroupModel(); colGroupModel.setNoOfColumnStore(noOfColumnStore); colGroupModel.setColumnSplit(columnSplit); - colGroupModel.setColumnarStore(columnarStore); colGroupModel.setColumnGroup(columnGroups); return colGroupModel; } @@ -418,7 +415,7 @@ public static void deleteFiles(File[] intermediateFiles) throws IOException { } } - public static int getFirstIndexUsingBinarySearch(DimensionColumnDataChunk dimColumnDataChunk, + public static int getFirstIndexUsingBinarySearch(DimensionColumnPage dimColumnDataChunk, int low, int high, byte[] compareValue, boolean matchUpLimit) { int cmpResult = 0; while (high >= low) { @@ -457,7 +454,7 @@ public static int getFirstIndexUsingBinarySearch(DimensionColumnDataChunk dimCol * @return the compareValue's range index in the dimColumnDataChunk */ public static int[] getRangeIndexUsingBinarySearch( - DimensionColumnDataChunk dimColumnDataChunk, int low, int high, byte[] compareValue) { + DimensionColumnPage dimColumnDataChunk, int low, int high, byte[] compareValue) { int[] rangeIndex = new int[2]; int cmpResult = 0; @@ -551,7 +548,7 @@ public static int binarySearch(byte[][] filterValues, int low, int high, * @return index value */ public static int nextLesserValueToTarget(int currentIndex, - DimensionColumnDataChunk dimColumnDataChunk, byte[] compareValue) { + DimensionColumnPage dimColumnDataChunk, byte[] compareValue) { while (currentIndex - 1 >= 0 && dimColumnDataChunk.compareTo(currentIndex - 1, compareValue) >= 0) { --currentIndex; @@ -571,7 +568,7 @@ public static int nextLesserValueToTarget(int currentIndex, * @return index value */ public static int nextGreaterValueToTarget(int currentIndex, - DimensionColumnDataChunk dimColumnDataChunk, byte[] compareValue, int numerOfRows) { + DimensionColumnPage dimColumnDataChunk, byte[] compareValue, int numerOfRows) { while (currentIndex + 1 < numerOfRows && dimColumnDataChunk.compareTo(currentIndex + 1, compareValue) <= 0) { ++currentIndex; @@ -934,7 +931,7 @@ public static boolean hasDataType(DataType dataType, DataType[] dataTypes) { return false; } - public static boolean[] getDictionaryEncodingArray(QueryDimension[] queryDimensions) { + public static boolean[] getDictionaryEncodingArray(ProjectionDimension[] queryDimensions) { boolean[] dictionaryEncodingArray = new boolean[queryDimensions.length]; for (int i = 0; i < queryDimensions.length; i++) { dictionaryEncodingArray[i] = @@ -943,7 +940,7 @@ public static boolean[] getDictionaryEncodingArray(QueryDimension[] queryDimensi return dictionaryEncodingArray; } - public static boolean[] getDirectDictionaryEncodingArray(QueryDimension[] queryDimensions) { + public static boolean[] getDirectDictionaryEncodingArray(ProjectionDimension[] queryDimensions) { boolean[] dictionaryEncodingArray = new boolean[queryDimensions.length]; for (int i = 0; i < queryDimensions.length; i++) { dictionaryEncodingArray[i] = @@ -952,7 +949,7 @@ public static boolean[] getDirectDictionaryEncodingArray(QueryDimension[] queryD return dictionaryEncodingArray; } - public static boolean[] getImplicitColumnArray(QueryDimension[] queryDimensions) { + public static boolean[] getImplicitColumnArray(ProjectionDimension[] queryDimensions) { boolean[] implicitColumnArray = new boolean[queryDimensions.length]; for (int i = 0; i < queryDimensions.length; i++) { implicitColumnArray[i] = queryDimensions[i].getDimension().hasEncoding(Encoding.IMPLICIT); @@ -960,7 +957,7 @@ public static boolean[] getImplicitColumnArray(QueryDimension[] queryDimensions) return implicitColumnArray; } - public static boolean[] getComplexDataTypeArray(QueryDimension[] queryDimensions) { + public static boolean[] getComplexDataTypeArray(ProjectionDimension[] queryDimensions) { boolean[] dictionaryEncodingArray = new boolean[queryDimensions.length]; for (int i = 0; i < queryDimensions.length; i++) { dictionaryEncodingArray[i] = @@ -1008,7 +1005,6 @@ private static DataFileFooter getDataFileFooter(TableBlockInfo tableBlockInfo, fileFooter.setColumnInTable(schema); SegmentInfo segmentInfo = new SegmentInfo(); segmentInfo.setColumnCardinality(detailInfo.getDimLens()); - segmentInfo.setNumberOfColumns(detailInfo.getRowCount()); fileFooter.setSegmentInfo(segmentInfo); return fileFooter; } @@ -1048,7 +1044,7 @@ public static int getNumberOfDimensionColumns(List columnSchemaLis * @return */ public static long calculateMetaSize(TableBlockInfo tableBlockInfo) throws IOException { - FileHolder fileReader = null; + FileReader fileReader = null; try { long completeBlockLength = tableBlockInfo.getBlockLength(); long footerPointer = completeBlockLength - 8; @@ -2183,19 +2179,19 @@ public static boolean usePreviousFilterBitsetGroup(boolean usePrvBitSetGroup, /** * Below method will be used to check filter value is present in the data chunk or not * @param filterValues - * @param dimensionColumnDataChunk + * @param dimensionColumnPage * @param low * @param high * @param chunkRowIndex * @return */ public static int isFilterPresent(byte[][] filterValues, - DimensionColumnDataChunk dimensionColumnDataChunk, int low, int high, int chunkRowIndex) { + DimensionColumnPage dimensionColumnPage, int low, int high, int chunkRowIndex) { int compareResult = 0; int mid = 0; while (low <= high) { mid = (low + high) >>> 1; - compareResult = dimensionColumnDataChunk.compareTo(chunkRowIndex, filterValues[mid]); + compareResult = dimensionColumnPage.compareTo(chunkRowIndex, filterValues[mid]); if (compareResult < 0) { high = mid - 1; } else if (compareResult > 0) { diff --git a/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java b/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java index e61b4779285..d665379fbee 100644 --- a/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java +++ b/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java @@ -21,7 +21,7 @@ import java.util.Iterator; import java.util.List; -import org.apache.carbondata.core.datastore.FileHolder; +import org.apache.carbondata.core.datastore.FileReader; import org.apache.carbondata.core.datastore.block.TableBlockInfo; import org.apache.carbondata.core.datastore.impl.FileFactory; import org.apache.carbondata.core.metadata.ColumnarFormatVersion; @@ -45,7 +45,7 @@ public class DataFileFooterConverter extends AbstractDataFileFooterConverter { @Override public DataFileFooter readDataFileFooter(TableBlockInfo tableBlockInfo) throws IOException { DataFileFooter dataFileFooter = new DataFileFooter(); - FileHolder fileReader = null; + FileReader fileReader = null; try { long completeBlockLength = tableBlockInfo.getBlockLength(); long footerPointer = completeBlockLength - 8; @@ -123,7 +123,7 @@ private BlockletInfo getBlockletInfo( } @Override public List getSchema(TableBlockInfo tableBlockInfo) throws IOException { - FileHolder fileReader = null; + FileReader fileReader = null; List columnSchemaList = new ArrayList(); try { long completeBlockLength = tableBlockInfo.getBlockLength(); diff --git a/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriter.java b/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriter.java index 74fd09ac533..afacc0b874d 100644 --- a/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriter.java +++ b/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriter.java @@ -35,17 +35,6 @@ public interface CarbonDictionaryWriter extends Closeable { */ void write(String value) throws IOException; - /** - * write method that accepts one value at a time - * This method can be used when data is huge and memory is les. In that - * case data can be stored to a file and an iterator can iterate over it and - * pass one value at a time - * - * @param value unique dictionary value - * @throws IOException if an I/O error occurs - */ - void write(byte[] value) throws IOException; - /** * write method that accepts list of byte arrays as value * This can be used when data is less, then string can be converted diff --git a/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java b/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java index 82baccc3be2..53411e9d1e9 100644 --- a/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java @@ -144,7 +144,7 @@ public CarbonDictionaryWriterImpl( * @param value unique dictionary value * @throws IOException if an I/O error occurs */ - @Override public void write(byte[] value) throws IOException { + private void write(byte[] value) throws IOException { if (isFirstTime) { init(); isFirstTime = false; diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java index 8bf8a073450..08dd7910ee2 100644 --- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java +++ b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java @@ -87,7 +87,7 @@ public class DictionaryCacheLoaderImplTest { return 9999; } }; - dictionaryCacheLoader.load(dictionaryInfo, columnIdentifier, 0L, 2L, true); + dictionaryCacheLoader.load(dictionaryInfo, 0L, 2L, true); assertEquals(dictionaryInfo.getDictionaryChunks().getSize(), 4); } @@ -97,7 +97,7 @@ public class DictionaryCacheLoaderImplTest { return 10000; } }; - dictionaryCacheLoader.load(dictionaryInfo, columnIdentifier, 0L, 2L, true); + dictionaryCacheLoader.load(dictionaryInfo, 0L, 2L, true); assertEquals(dictionaryInfo.getDictionaryChunks().getSize(), 2); } } diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileHolderImplUnitTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileReaderImplUnitTest.java similarity index 96% rename from core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileHolderImplUnitTest.java rename to core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileReaderImplUnitTest.java index 5c51c87f0b2..da61a94e87a 100644 --- a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileHolderImplUnitTest.java +++ b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileReaderImplUnitTest.java @@ -22,7 +22,7 @@ import java.io.IOException; import java.io.OutputStreamWriter; -import org.apache.carbondata.core.datastore.impl.DFSFileHolderImpl; +import org.apache.carbondata.core.datastore.impl.DFSFileReaderImpl; import mockit.Mock; import mockit.MockUp; @@ -37,16 +37,16 @@ import static org.hamcrest.CoreMatchers.is; import static org.junit.Assert.assertThat; -public class DFSFileHolderImplUnitTest { +public class DFSFileReaderImplUnitTest { - private static DFSFileHolderImpl dfsFileHolder; + private static DFSFileReaderImpl dfsFileHolder; private static String fileName; private static String fileNameWithEmptyContent; private static File file; private static File fileWithEmptyContent; @BeforeClass public static void setup() { - dfsFileHolder = new DFSFileHolderImpl(); + dfsFileHolder = new DFSFileReaderImpl(); file = new File("Test.carbondata"); fileWithEmptyContent = new File("TestEXception.carbondata"); diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileHolderImplUnitTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileReaderImplUnitTest.java similarity index 94% rename from core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileHolderImplUnitTest.java rename to core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileReaderImplUnitTest.java index ed50d63ba88..a6d3235325f 100644 --- a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileHolderImplUnitTest.java +++ b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileReaderImplUnitTest.java @@ -22,7 +22,7 @@ import java.io.IOException; import java.io.OutputStreamWriter; -import org.apache.carbondata.core.datastore.impl.FileHolderImpl; +import org.apache.carbondata.core.datastore.impl.FileReaderImpl; import mockit.Mock; import mockit.MockUp; @@ -38,18 +38,18 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; -public class FileHolderImplUnitTest { +public class FileReaderImplUnitTest { - private static FileHolderImpl fileHolder; - private static FileHolderImpl fileHolderWithCapacity; + private static FileReaderImpl fileHolder; + private static FileReaderImpl fileHolderWithCapacity; private static String fileName; private static String fileNameWithEmptyContent; private static File file; private static File fileWithEmptyContent; @BeforeClass public static void setup() { - fileHolder = new FileHolderImpl(); - fileHolderWithCapacity = new FileHolderImpl(50); + fileHolder = new FileReaderImpl(); + fileHolderWithCapacity = new FileReaderImpl(50); file = new File("Test.carbondata"); fileWithEmptyContent = new File("TestEXception.carbondata"); diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java b/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java index 9c435539536..20036ec86d9 100644 --- a/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java +++ b/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java @@ -84,7 +84,7 @@ public class SegmentPropertiesTest extends TestCase { assertTrue(true); } - @Test public void testBlockMetadataHasProperDimensionBlockMapping() { + @Test public void testBlockMetadataHasProperDimensionChunkMapping() { Map dimensionOrdinalToBlockMapping = new HashMap(); dimensionOrdinalToBlockMapping.put(0, 0); dimensionOrdinalToBlockMapping.put(1, 1); @@ -97,7 +97,7 @@ public class SegmentPropertiesTest extends TestCase { dimensionOrdinalToBlockMapping.put(8, 5); dimensionOrdinalToBlockMapping.put(9, 6); Map dimensionOrdinalToBlockMappingActual = - blockMetadataInfos.getDimensionOrdinalToBlockMapping(); + blockMetadataInfos.getDimensionOrdinalToChunkMapping(); assertEquals(dimensionOrdinalToBlockMapping.size(), dimensionOrdinalToBlockMappingActual.size()); Iterator> iterator = @@ -112,12 +112,12 @@ public class SegmentPropertiesTest extends TestCase { assertTrue(true); } - @Test public void testBlockMetadataHasProperMeasureBlockMapping() { + @Test public void testBlockMetadataHasProperMeasureChunkMapping() { Map measureOrdinalToBlockMapping = new HashMap(); measureOrdinalToBlockMapping.put(0, 0); measureOrdinalToBlockMapping.put(1, 1); Map measureOrdinalToBlockMappingActual = - blockMetadataInfos.getMeasuresOrdinalToBlockMapping(); + blockMetadataInfos.getMeasuresOrdinalToChunkMapping(); assertEquals(measureOrdinalToBlockMapping.size(), measureOrdinalToBlockMappingActual.size()); Iterator> iterator = measureOrdinalToBlockMapping.entrySet().iterator(); while (iterator.hasNext()) { diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunkTest.java b/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunkTest.java index bdb83cd8bf8..54b66a60e1a 100644 --- a/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunkTest.java +++ b/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunkTest.java @@ -37,7 +37,7 @@ public class ColumnGroupDimensionDataChunkTest { - static ColumnGroupDimensionDataChunk columnGroupDimensionDataChunk; + static ColumnGroupDimensionColumnPage columnGroupDimensionDataChunk; static KeyGenerator keyGenerator; @BeforeClass public static void setup() { @@ -56,7 +56,7 @@ public class ColumnGroupDimensionDataChunkTest { position += keyGenerator.getKeySizeInBytes(); } columnGroupDimensionDataChunk = - new ColumnGroupDimensionDataChunk(data, keyGenerator.getKeySizeInBytes(), 3); + new ColumnGroupDimensionColumnPage(data, keyGenerator.getKeySizeInBytes(), 3); } @Test public void fillChunkDataTest() { @@ -64,7 +64,7 @@ public class ColumnGroupDimensionDataChunkTest { ordinals.add(1); KeyStructureInfo keyStructureInfo = getKeyStructureInfo(ordinals, keyGenerator); byte[] buffer = new byte[1]; - columnGroupDimensionDataChunk.fillChunkData(buffer, 0, 1, keyStructureInfo); + columnGroupDimensionDataChunk.fillRawData(1, 0, buffer, keyStructureInfo); assertEquals(buffer[0], 2); } @@ -81,7 +81,7 @@ public class ColumnGroupDimensionDataChunkTest { ordinals.add(2); KeyStructureInfo keyStructureInfo = getKeyStructureInfo(ordinals, keyGenerator); keyStructureInfo.setMdkeyQueryDimensionOrdinal(new int[] { 2 }); - int res = columnGroupDimensionDataChunk.fillConvertedChunkData(2, 2, row, keyStructureInfo); + int res = columnGroupDimensionDataChunk.fillSurrogateKey(2, 2, row, keyStructureInfo); Assert.assertTrue(Arrays.equals(row, expected)); } diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunkTest.java b/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunkTest.java index 04b7a80a1f4..f327ef662ef 100644 --- a/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunkTest.java +++ b/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunkTest.java @@ -30,7 +30,7 @@ public class FixedLengthDimensionDataChunkTest { - static FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk; + static FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk; static byte[] data; @BeforeClass public static void setup() { @@ -40,7 +40,7 @@ public class FixedLengthDimensionDataChunkTest { int invertedIndexReverse[] = { 1, 0, 5, 7, 8 }; fixedLengthDimensionDataChunk = - new FixedLengthDimensionDataChunk(data, invertedIndex, invertedIndexReverse, 5, 4); + new FixedLengthDimensionColumnPage(data, invertedIndex, invertedIndexReverse, 5, 4); } @Test public void fillChunkDataTest() { @@ -48,7 +48,7 @@ public class FixedLengthDimensionDataChunkTest { int[] maskByteRanges = { 1, 2, 4, 6, 5 }; keyStructureInfo.setMaskByteRanges(maskByteRanges); keyStructureInfo.setMaxKey("1234567".getBytes()); - int res = fixedLengthDimensionDataChunk.fillChunkData(data, 0, 0, keyStructureInfo); + int res = fixedLengthDimensionDataChunk.fillRawData(0, 0, data, keyStructureInfo); int expectedResult = 4 ; assertEquals(res, expectedResult); } @@ -62,7 +62,7 @@ public class FixedLengthDimensionDataChunkTest { @Test public void fillConvertedChunkDataTest() { int[] row = { 1, 2, 4, 6 }; KeyStructureInfo keyStructureInfo = new KeyStructureInfo(); - int res = fixedLengthDimensionDataChunk.fillConvertedChunkData(1, 0, row, keyStructureInfo); + int res = fixedLengthDimensionDataChunk.fillSurrogateKey(1, 0, row, keyStructureInfo); int expectedResult = 1; assertEquals(res, expectedResult); } diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/impl/btree/BTreeBlockFinderTest.java b/core/src/test/java/org/apache/carbondata/core/datastore/impl/btree/BTreeBlockFinderTest.java index d874037f453..7f7719cfa58 100644 --- a/core/src/test/java/org/apache/carbondata/core/datastore/impl/btree/BTreeBlockFinderTest.java +++ b/core/src/test/java/org/apache/carbondata/core/datastore/impl/btree/BTreeBlockFinderTest.java @@ -76,9 +76,9 @@ public class BTreeBlockFinderTest extends TestCase { buffer.array(); IndexKey key = new IndexKey(null, buffer.array()); DataRefNode findFirstBlock = finder.findFirstDataBlock(dataBlock, key); - assertEquals(1, findFirstBlock.nodeNumber()); + assertEquals(1, findFirstBlock.nodeIndex()); DataRefNode findLastBlock = finder.findLastDataBlock(dataBlock, key); - assertEquals(1, findLastBlock.nodeNumber()); + assertEquals(1, findLastBlock.nodeIndex()); } @Test public void testBtreeSearchIsWorkingAndGivingPorperBlockletWithNoDictionary() { @@ -96,9 +96,9 @@ public class BTreeBlockFinderTest extends TestCase { buffer.array(); IndexKey key = new IndexKey(null, buffer.array()); DataRefNode findFirstBlock = finder.findFirstDataBlock(dataBlock, key); - assertEquals(0, findFirstBlock.nodeNumber()); + assertEquals(0, findFirstBlock.nodeIndex()); DataRefNode findLastBlock = finder.findLastDataBlock(dataBlock, key); - assertEquals(0, findLastBlock.nodeNumber()); + assertEquals(0, findLastBlock.nodeIndex()); } @Test public void testBtreeSearchIsWorkingAndGivingPorperBlockletWithDictionaryKey1() @@ -118,9 +118,9 @@ public class BTreeBlockFinderTest extends TestCase { IndexKey key = new IndexKey(multiDimKeyVarLengthGenerator.generateKey(new int[] { 1, 1 }), null); DataRefNode findFirstBlock = finder.findFirstDataBlock(dataBlock, key); - assertEquals(0, findFirstBlock.nodeNumber()); + assertEquals(0, findFirstBlock.nodeIndex()); DataRefNode findLastBlock = finder.findLastDataBlock(dataBlock, key); - assertEquals(0, findLastBlock.nodeNumber()); + assertEquals(0, findLastBlock.nodeIndex()); } @Test public void testBtreeSearchIsWorkingAndGivingPorperBlockletWithDictionaryKey2() @@ -141,9 +141,9 @@ public class BTreeBlockFinderTest extends TestCase { new IndexKey(multiDimKeyVarLengthGenerator.generateKey(new int[] { 0, 0 }), null); DataRefNode findFirstBlock = finder.findFirstDataBlock(dataBlock, key); - assertEquals(0, findFirstBlock.nodeNumber()); + assertEquals(0, findFirstBlock.nodeIndex()); DataRefNode findLastBlock = finder.findLastDataBlock(dataBlock, key); - assertEquals(0, findLastBlock.nodeNumber()); + assertEquals(0, findLastBlock.nodeIndex()); } /** @@ -169,9 +169,9 @@ public class BTreeBlockFinderTest extends TestCase { new IndexKey(multiDimKeyVarLengthGenerator.generateKey(new int[] { 10001, 10001 }), null); DataRefNode findFirstBlock = finder.findFirstDataBlock(dataBlock, key); - assertEquals(99, findFirstBlock.nodeNumber()); + assertEquals(99, findFirstBlock.nodeIndex()); DataRefNode findLastBlock = finder.findLastDataBlock(dataBlock, key); - assertEquals(99, findLastBlock.nodeNumber()); + assertEquals(99, findLastBlock.nodeIndex()); } private List getDataFileFooterList() { diff --git a/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollectorTest.java b/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollectorTest.java index 85bdfd8c6fc..be9141013f5 100644 --- a/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollectorTest.java +++ b/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollectorTest.java @@ -50,32 +50,32 @@ public class RawBasedResultCollectorTest { // aggregatorInfo.setDefaultValues(new Object[] { 1, 2, 3, 4 }); // aggregatorInfo.setMeasureDataTypes( // new DataType[] { DataTypes.INT, DataTypes.TIMESTAMP, DataTypes.INT, DataTypes.INT }); -// QueryMeasure queryMeasure1 = new QueryMeasure("QMCol1"); +// ProjectionMeasure queryMeasure1 = new ProjectionMeasure("QMCol1"); // queryMeasure1.setQueryOrder(1); -// QueryMeasure queryMeasure2 = new QueryMeasure("QMCol2"); +// ProjectionMeasure queryMeasure2 = new ProjectionMeasure("QMCol2"); // queryMeasure1.setQueryOrder(2); -// QueryMeasure queryMeasure3 = new QueryMeasure("QMCol3"); +// ProjectionMeasure queryMeasure3 = new ProjectionMeasure("QMCol3"); // queryMeasure1.setQueryOrder(3); -// QueryMeasure queryMeasure4 = new QueryMeasure("QMCol4"); +// ProjectionMeasure queryMeasure4 = new ProjectionMeasure("QMCol4"); // queryMeasure1.setQueryOrder(4); -// QueryDimension queryDimension1 = new QueryDimension("QDCol1"); +// ProjectionDimension queryDimension1 = new ProjectionDimension("QDCol1"); // queryDimension1.setQueryOrder(1); // ColumnSchema columnSchema = new ColumnSchema(); // queryDimension1.setDimension(new CarbonDimension(columnSchema, 0, 0, 0, 0)); -// QueryDimension queryDimension2 = new QueryDimension("QDCol2"); +// ProjectionDimension queryDimension2 = new ProjectionDimension("QDCol2"); // queryDimension2.setQueryOrder(2); // queryDimension2.setDimension(new CarbonDimension(columnSchema, 1, 1, 1, 1)); -// QueryDimension queryDimension3 = new QueryDimension("QDCol3"); +// ProjectionDimension queryDimension3 = new ProjectionDimension("QDCol3"); // queryDimension3.setQueryOrder(3); // queryDimension3.setDimension(new CarbonDimension(columnSchema, 2, 0, 0, 0)); -// QueryDimension queryDimension4 = new QueryDimension("QDCol4"); +// ProjectionDimension queryDimension4 = new ProjectionDimension("QDCol4"); // queryDimension4.setQueryOrder(4); // queryDimension4.setDimension(new CarbonDimension(columnSchema, 3, 0, 0, 0)); -// blockExecutionInfo.setQueryDimensions( -// new QueryDimension[] { queryDimension1, queryDimension2, queryDimension3, +// blockExecutionInfo.setProjectionDimensions( +// new ProjectionDimension[] { queryDimension1, queryDimension2, queryDimension3, // queryDimension4 }); -// blockExecutionInfo.setQueryMeasures( -// new QueryMeasure[] { queryMeasure1, queryMeasure2, queryMeasure3, queryMeasure4 }); +// blockExecutionInfo.setProjectionMeasures( +// new ProjectionMeasure[] { queryMeasure1, queryMeasure2, queryMeasure3, queryMeasure4 }); // blockExecutionInfo.setFixedKeyUpdateRequired(true); // blockExecutionInfo.setMeasureInfo(aggregatorInfo); // blockExecutionInfo.setMaskedByteForBlock(new int[] { 1, 2 }); @@ -99,7 +99,7 @@ public class RawBasedResultCollectorTest { // return new byte[][] { { 1, 2 }, { 1, 2 } }; // } // -// @SuppressWarnings("unused") @Mock public ColumnPage getMeasureChunk(int ordinal) { +// @SuppressWarnings("unused") @Mock public ColumnPage readMeasureChunk(int ordinal) { // ColumnPage ColumnPage = new ColumnPage(); // PresenceMeta presenceMeta = new PresenceMeta(); // BitSet bitSet = new BitSet(); @@ -120,10 +120,10 @@ public class RawBasedResultCollectorTest { // } // }; // -// AbstractScannedResult abstractScannedResult = +// BlockletScannedResult abstractScannedResult = // new NonFilterQueryScannedResult(blockExecutionInfo); -// abstractScannedResult.setNumberOfRows(2); -// List result = rawBasedResultCollector.collectData(abstractScannedResult, 2); +// abstractScannedResult.setPageFilteredRowCount(2); +// List result = rawBasedResultCollector.collectResultInRow(abstractScannedResult, 2); // int expectedResult = 2; // assertThat(result.size(), is(equalTo(expectedResult))); // } diff --git a/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java b/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java index 45e86f244fb..e8b75b7d92b 100644 --- a/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java +++ b/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java @@ -23,7 +23,7 @@ import org.apache.carbondata.core.datastore.block.SegmentProperties; import org.apache.carbondata.core.datastore.block.SegmentPropertiesTestUtil; import org.apache.carbondata.core.keygenerator.KeyGenException; -import org.apache.carbondata.core.scan.model.QueryDimension; +import org.apache.carbondata.core.scan.model.ProjectionDimension; import junit.framework.TestCase; import mockit.Mock; @@ -45,9 +45,8 @@ public class QueryUtilTest extends TestCase { @Test public void testGetMaskedByteRangeGivingProperMaksedByteRange() { - QueryDimension dimension = - new QueryDimension(segmentProperties.getDimensions().get(0).getColName()); - dimension.setDimension(segmentProperties.getDimensions().get(0)); + ProjectionDimension dimension = + new ProjectionDimension(segmentProperties.getDimensions().get(0)); int[] maskedByteRange = QueryUtil .getMaskedByteRange(Arrays.asList(dimension), segmentProperties.getDimensionKeyGenerator()); int[] expectedMaskedByteRange = { 0 }; @@ -57,11 +56,10 @@ public class QueryUtilTest extends TestCase { } @Test public void testGetMaskedByteRangeGivingProperMaksedByteRangeOnlyForDictionaryKey() { - List dimensions = new ArrayList(); + List dimensions = new ArrayList(); for (int i = 0; i < 2; i++) { - QueryDimension dimension = - new QueryDimension(segmentProperties.getDimensions().get(i).getColName()); - dimension.setDimension(segmentProperties.getDimensions().get(i)); + ProjectionDimension dimension = + new ProjectionDimension(segmentProperties.getDimensions().get(i)); dimensions.add(dimension); } int[] maskedByteRange = @@ -84,11 +82,10 @@ public class QueryUtilTest extends TestCase { } @Test public void testGetMaxKeyBasedOnDimensions() { - List dimensions = new ArrayList(); + List dimensions = new ArrayList(); for (int i = 0; i < 2; i++) { - QueryDimension dimension = - new QueryDimension(segmentProperties.getDimensions().get(i).getColName()); - dimension.setDimension(segmentProperties.getDimensions().get(i)); + ProjectionDimension dimension = + new ProjectionDimension(segmentProperties.getDimensions().get(i)); dimensions.add(dimension); } byte[] maxKeyBasedOnDimensions = null; @@ -115,10 +112,8 @@ public class QueryUtilTest extends TestCase { } @Test public void testGetMaksedByte() { - QueryDimension dimension = - new QueryDimension(segmentProperties.getDimensions().get(0).getColName()); - dimension.setDimension(segmentProperties.getDimensions().get(0)); - dimension.setDimension(segmentProperties.getDimensions().get(0)); + ProjectionDimension dimension = + new ProjectionDimension(segmentProperties.getDimensions().get(0)); int[] maskedByteRange = QueryUtil .getMaskedByteRange(Arrays.asList(dimension), segmentProperties.getDimensionKeyGenerator()); int[] maskedByte = QueryUtil @@ -203,18 +198,16 @@ public class QueryUtilTest extends TestCase { } @Test public void testGetSortDimensionIndexes() { - List sortedDimensions = new ArrayList(); + List sortedDimensions = new ArrayList(); for (int i = 0; i < 2; i++) { - QueryDimension dimension = - new QueryDimension(segmentProperties.getDimensions().get(i).getColName()); - dimension.setDimension(segmentProperties.getDimensions().get(i)); + ProjectionDimension dimension = + new ProjectionDimension(segmentProperties.getDimensions().get(i)); sortedDimensions.add(dimension); } - List queryDimensions = new ArrayList(); + List queryDimensions = new ArrayList(); for (int i = 0; i < 2; i++) { - QueryDimension dimension = - new QueryDimension(segmentProperties.getDimensions().get(i).getColName()); - dimension.setDimension(segmentProperties.getDimensions().get(i)); + ProjectionDimension dimension = + new ProjectionDimension(segmentProperties.getDimensions().get(i)); queryDimensions.add(dimension); } byte[] actualValue = QueryUtil.getSortDimensionIndexes(sortedDimensions, queryDimensions); diff --git a/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java b/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java index 90fcb741a1f..163580dbe57 100644 --- a/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java +++ b/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java @@ -28,8 +28,8 @@ import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema; import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo; import org.apache.carbondata.core.scan.executor.infos.MeasureInfo; -import org.apache.carbondata.core.scan.model.QueryDimension; -import org.apache.carbondata.core.scan.model.QueryMeasure; +import org.apache.carbondata.core.scan.model.ProjectionDimension; +import org.apache.carbondata.core.scan.model.ProjectionMeasure; import org.junit.Test; @@ -79,27 +79,22 @@ public class RestructureUtilTest { List tableComplexDimensions = Arrays.asList(tableComplexDimension1, tableComplexDimension2); - QueryDimension queryDimension1 = new QueryDimension("Id"); - queryDimension1.setDimension(tableBlockDimension1); - QueryDimension queryDimension2 = new QueryDimension("Name"); - queryDimension2.setDimension(tableComplexDimension2); - QueryDimension queryDimension3 = new QueryDimension("Address"); - queryDimension3.setDimension(new CarbonDimension(columnSchema5, 3, 3, 3, 3)); - QueryMeasure queryMeasure1 = new QueryMeasure("Age"); - QueryMeasure queryMeasure2 = new QueryMeasure("Salary"); - queryMeasure1.setMeasure(new CarbonMeasure(columnSchema3, 2)); - queryMeasure2.setMeasure(new CarbonMeasure(columnSchema4, 4)); - List queryMeasures = Arrays.asList(queryMeasure1, queryMeasure2); + ProjectionDimension queryDimension1 = new ProjectionDimension(tableBlockDimension1); + ProjectionDimension queryDimension2 = new ProjectionDimension(tableComplexDimension2); + ProjectionDimension queryDimension3 = new ProjectionDimension(new CarbonDimension(columnSchema5, 3, 3, 3, 3)); + ProjectionMeasure queryMeasure1 = new ProjectionMeasure(new CarbonMeasure(columnSchema3, 2)); + ProjectionMeasure queryMeasure2 = new ProjectionMeasure(new CarbonMeasure(columnSchema4, 4)); + List queryMeasures = Arrays.asList(queryMeasure1, queryMeasure2); - List queryDimensions = + List queryDimensions = Arrays.asList(queryDimension1, queryDimension2, queryDimension3); - List result = null; + List result = null; result = RestructureUtil .createDimensionInfoAndGetCurrentBlockQueryDimension(blockExecutionInfo, queryDimensions, tableBlockDimensions, tableComplexDimensions, queryMeasures.size()); List resultDimension = new ArrayList<>(result.size()); - for (QueryDimension queryDimension : result) { + for (ProjectionDimension queryDimension : result) { resultDimension.add(queryDimension.getDimension()); } assertThat(resultDimension, @@ -126,13 +121,10 @@ public class RestructureUtilTest { carbonMeasure3.getColumnSchema().setDefaultValue("3".getBytes()); List currentBlockMeasures = Arrays.asList(carbonMeasure1, carbonMeasure2); - QueryMeasure queryMeasure1 = new QueryMeasure("Id"); - queryMeasure1.setMeasure(carbonMeasure1); - QueryMeasure queryMeasure2 = new QueryMeasure("Name"); - queryMeasure2.setMeasure(carbonMeasure2); - QueryMeasure queryMeasure3 = new QueryMeasure("Age"); - queryMeasure3.setMeasure(carbonMeasure3); - List queryMeasures = Arrays.asList(queryMeasure1, queryMeasure2, queryMeasure3); + ProjectionMeasure queryMeasure1 = new ProjectionMeasure(carbonMeasure1); + ProjectionMeasure queryMeasure2 = new ProjectionMeasure(carbonMeasure2); + ProjectionMeasure queryMeasure3 = new ProjectionMeasure(carbonMeasure3); + List queryMeasures = Arrays.asList(queryMeasure1, queryMeasure2, queryMeasure3); BlockExecutionInfo blockExecutionInfo = new BlockExecutionInfo(); RestructureUtil.createMeasureInfoAndGetCurrentBlockQueryMeasures(blockExecutionInfo, queryMeasures, currentBlockMeasures); diff --git a/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java b/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java index 565da04e076..9c7c26cd863 100644 --- a/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java +++ b/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java @@ -328,7 +328,7 @@ public void testCheckIfExpressionContainsColumnWithExpressionNotInstanceOfColumn return "test"; } }; - assertTrue(FilterUtil.getFilterListForRS(expression, columnExpression, defaultValues, + assertTrue(FilterUtil.getFilterListForRS(expression, defaultValues, defaultSurrogate) instanceof ColumnFilterInfo); } diff --git a/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImplTest.java b/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImplTest.java index e3ae42cad1d..11c49804589 100644 --- a/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImplTest.java +++ b/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImplTest.java @@ -18,12 +18,12 @@ import java.util.BitSet; -import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk; +import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage; import org.apache.carbondata.core.util.CarbonUtil; public class ExcludeFilterExecuterImplTest extends IncludeFilterExecuterImplTest { - @Override public BitSet setFilterdIndexToBitSetNew(DimensionColumnDataChunk dimColumnDataChunk, + @Override public BitSet setFilterdIndexToBitSetNew(DimensionColumnPage dimColumnDataChunk, int numerOfRows, byte[][] filterValues) { BitSet bitSet = new BitSet(numerOfRows); bitSet.flip(0, numerOfRows); @@ -46,7 +46,7 @@ public class ExcludeFilterExecuterImplTest extends IncludeFilterExecuterImplTest return bitSet; } - @Override public BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimColumnDataChunk, + @Override public BitSet setFilterdIndexToBitSet(DimensionColumnPage dimColumnDataChunk, int numerOfRows, byte[][] filterValues) { BitSet bitSet = new BitSet(numerOfRows); bitSet.flip(0, numerOfRows); diff --git a/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImplTest.java b/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImplTest.java index 29dda5287bc..1bfa875a5b8 100644 --- a/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImplTest.java +++ b/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImplTest.java @@ -18,8 +18,8 @@ import java.util.BitSet; -import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk; -import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionDataChunk; +import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage; +import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionColumnPage; import org.apache.carbondata.core.util.CarbonUtil; import org.junit.Assert; @@ -38,22 +38,22 @@ public void setUp() throws Exception { } - public BitSet setFilterdIndexToBitSetNew(DimensionColumnDataChunk dimensionColumnDataChunk, + public BitSet setFilterdIndexToBitSetNew(DimensionColumnPage dimensionColumnPage, int numerOfRows, byte[][] filterValues) { BitSet bitSet = new BitSet(numerOfRows); - if (dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) { + if (dimensionColumnPage instanceof FixedLengthDimensionColumnPage) { // byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys(); if (filterValues.length > 1) { for (int i = 0; i < numerOfRows; i++) { int index = CarbonUtil.binarySearch(filterValues, 0, filterValues.length - 1, - dimensionColumnDataChunk.getChunkData(i)); + dimensionColumnPage.getChunkData(i)); if (index >= 0) { bitSet.set(i); } } } else if (filterValues.length == 1) { for (int i = 0; i < numerOfRows; i++) { - if (dimensionColumnDataChunk.compareTo(i, filterValues[0]) == 0) { + if (dimensionColumnPage.compareTo(i, filterValues[0]) == 0) { bitSet.set(i); } } @@ -62,14 +62,14 @@ public BitSet setFilterdIndexToBitSetNew(DimensionColumnDataChunk dimensionColum return bitSet; } - public BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk, int numerOfRows, + public BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage, int numerOfRows, byte[][] filterValues) { BitSet bitSet = new BitSet(numerOfRows); - if (dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) { + if (dimensionColumnPage instanceof FixedLengthDimensionColumnPage) { // byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys(); for (int k = 0; k < filterValues.length; k++) { for (int j = 0; j < numerOfRows; j++) { - if (dimensionColumnDataChunk.compareTo(j, filterValues[k]) == 0) { + if (dimensionColumnPage.compareTo(j, filterValues[k]) == 0) { bitSet.set(j); } } @@ -163,7 +163,7 @@ private void comparePerformance(int dataChunkSize, int filteredValueCnt, } System.out.println("dimColumnSize: " + dimColumnSize); - FixedLengthDimensionDataChunk dimensionColumnDataChunk; + FixedLengthDimensionColumnPage dimensionColumnDataChunk; DimColumnExecuterFilterInfo dim = new DimColumnExecuterFilterInfo(); byte[] dataChunk = new byte[dataChunkSize * dimColumnSize]; @@ -183,7 +183,7 @@ private void comparePerformance(int dataChunkSize, int filteredValueCnt, } dim.setFilterKeys(filterKeys); - dimensionColumnDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null, + dimensionColumnDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null, dataChunkSize, dimColumnSize); // repeat query and compare 2 result between old code and new optimized code @@ -194,7 +194,7 @@ private void comparePerformance(int dataChunkSize, int filteredValueCnt, oldTime = oldTime + System.currentTimeMillis() - start; start = System.currentTimeMillis(); - BitSet bitNew = this.setFilterdIndexToBitSetNew((FixedLengthDimensionDataChunk) dimensionColumnDataChunk, dataChunkSize, + BitSet bitNew = this.setFilterdIndexToBitSetNew((FixedLengthDimensionColumnPage) dimensionColumnDataChunk, dataChunkSize, filterKeys); newTime = newTime + System.currentTimeMillis() - start; @@ -212,7 +212,7 @@ private void comparePerformance(int dataChunkSize, int filteredValueCnt, } - private BitSet setFilterdIndexToBitSetWithColumnIndexOld(FixedLengthDimensionDataChunk dimensionColumnDataChunk, + private BitSet setFilterdIndexToBitSetWithColumnIndexOld(FixedLengthDimensionColumnPage dimensionColumnDataChunk, int numerOfRows, byte[][] filterValues) { BitSet bitSet = new BitSet(numerOfRows); int start = 0; @@ -243,7 +243,7 @@ private BitSet setFilterdIndexToBitSetWithColumnIndexOld(FixedLengthDimensionDat return bitSet; } - private BitSet setFilterdIndexToBitSetWithColumnIndexNew(FixedLengthDimensionDataChunk dimensionColumnDataChunk, + private BitSet setFilterdIndexToBitSetWithColumnIndexNew(FixedLengthDimensionColumnPage dimensionColumnDataChunk, int numerOfRows, byte[][] filterValues) { BitSet bitSet = new BitSet(numerOfRows); int startIndex = 0; @@ -281,7 +281,7 @@ public void testRangBinarySearch() { int filteredValueCnt = 800; // column dictionary size int dimColumnSize = 2; - FixedLengthDimensionDataChunk dimensionColumnDataChunk; + FixedLengthDimensionColumnPage dimensionColumnDataChunk; DimColumnExecuterFilterInfo dim = new DimColumnExecuterFilterInfo(); byte[] dataChunk = new byte[dataChunkSize * dimColumnSize]; @@ -303,7 +303,7 @@ public void testRangBinarySearch() { } dim.setFilterKeys(filterKeys); - dimensionColumnDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null, + dimensionColumnDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null, dataChunk.length / dimColumnSize, dimColumnSize); // initial to run diff --git a/core/src/test/java/org/apache/carbondata/core/scan/result/BatchResultTest.java b/core/src/test/java/org/apache/carbondata/core/scan/result/RowBatchTest.java similarity index 81% rename from core/src/test/java/org/apache/carbondata/core/scan/result/BatchResultTest.java rename to core/src/test/java/org/apache/carbondata/core/scan/result/RowBatchTest.java index 541ce480412..123d64e22e8 100644 --- a/core/src/test/java/org/apache/carbondata/core/scan/result/BatchResultTest.java +++ b/core/src/test/java/org/apache/carbondata/core/scan/result/RowBatchTest.java @@ -29,38 +29,38 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; -public class BatchResultTest { - private static BatchResult batchResult; +public class RowBatchTest { + private static RowBatch rowBatch; private static List rowsList = new ArrayList(2); @BeforeClass public static void setUp() { - batchResult = new BatchResult(); + rowBatch = new RowBatch(); rowsList.add(0, new Integer[] { 1, 2 }); rowsList.add(1, new Integer[] { 3 }); } @Test public void testNext() throws NoSuchElementException { - BatchResult rows = new BatchResult(); + RowBatch rows = new RowBatch(); rows.setRows(rowsList); Object[] result = rows.next(); Assert.assertTrue(result.equals(rowsList.get(0))); } @Test(expected = NoSuchElementException.class) public void testNextWithNoSuchElementException() { - BatchResult rows = new BatchResult(); + RowBatch rows = new RowBatch(); List emptyList = new ArrayList(2); rows.setRows(emptyList); rows.next(); } @Test public void testGetRows() { - new MockUp() { + new MockUp() { @Mock public void $init() { //to be left blank } }; - BatchResult batchResult = new BatchResult(); - List list = batchResult.getRows(); + RowBatch rowBatch = new RowBatch(); + List list = rowBatch.getRows(); assertNull("Number of rows is null", list); } @@ -68,16 +68,16 @@ public class BatchResultTest { List list = new ArrayList<>(); list.add(0, new Integer[] { 1, 2 }); list.add(1, new Integer[] { 1, 2 }); - batchResult.setRows(list); - boolean result = batchResult.hasNext(); + rowBatch.setRows(list); + boolean result = rowBatch.hasNext(); Assert.assertTrue(result); } @Test public void testGetRawRow() { List list = new ArrayList<>(); list.add(0, new Integer[] { 1, 2 }); - batchResult.setRows(list); - Object[] actualValue = batchResult.getRawRow(0); + rowBatch.setRows(list); + Object[] actualValue = rowBatch.getRawRow(0); Assert.assertTrue(list.get(0) == actualValue); } @@ -85,8 +85,8 @@ public class BatchResultTest { List list = new ArrayList<>(); list.add(0, new Integer[] { 1, 2 }); list.add(1, new Integer[] { 1, 2 }); - batchResult.setRows(list); - int actualValue = batchResult.getSize(); + rowBatch.setRows(list); + int actualValue = rowBatch.getSize(); int expectedValue = 2; assertEquals(expectedValue, actualValue); } diff --git a/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java b/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java index f4450e30711..5f8d1999247 100644 --- a/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java +++ b/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java @@ -31,7 +31,7 @@ import org.apache.carbondata.core.datamap.Segment; import org.apache.carbondata.core.datastore.block.TableBlockInfo; -import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionDataChunk; +import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionColumnPage; import org.apache.carbondata.core.datastore.columnar.ColumnGroupModel; import org.apache.carbondata.core.datastore.filesystem.LocalCarbonFile; import org.apache.carbondata.core.datastore.impl.FileFactory; @@ -45,7 +45,7 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension; import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure; import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema; -import org.apache.carbondata.core.scan.model.QueryDimension; +import org.apache.carbondata.core.scan.model.ProjectionDimension; import mockit.Mock; import mockit.MockUp; @@ -267,8 +267,8 @@ public void testToDeleteFoldersAndFilesSintlyWithInterruptedException() @Test public void testToGetNextLesserValue() { byte[] dataChunks = { 5, 6, 7, 8, 9 }; byte[] compareValues = { 7 }; - FixedLengthDimensionDataChunk fixedLengthDataChunk = - new FixedLengthDimensionDataChunk(dataChunks, null, null, 5, 1); + FixedLengthDimensionColumnPage fixedLengthDataChunk = + new FixedLengthDimensionColumnPage(dataChunks, null, null, 5, 1); int result = CarbonUtil.nextLesserValueToTarget(2, fixedLengthDataChunk, compareValues); assertEquals(result, 1); } @@ -276,8 +276,8 @@ public void testToDeleteFoldersAndFilesSintlyWithInterruptedException() @Test public void testToGetNextLesserValueToTarget() { byte[] dataChunks = { 7, 7, 7, 8, 9 }; byte[] compareValues = { 7 }; - FixedLengthDimensionDataChunk fixedLengthDataChunk = - new FixedLengthDimensionDataChunk(dataChunks, null, null, 5, 1); + FixedLengthDimensionColumnPage fixedLengthDataChunk = + new FixedLengthDimensionColumnPage(dataChunks, null, null, 5, 1); int result = CarbonUtil.nextLesserValueToTarget(2, fixedLengthDataChunk, compareValues); assertEquals(result, -1); } @@ -285,8 +285,8 @@ public void testToDeleteFoldersAndFilesSintlyWithInterruptedException() @Test public void testToGetnextGreaterValue() { byte[] dataChunks = { 5, 6, 7, 8, 9 }; byte[] compareValues = { 7 }; - FixedLengthDimensionDataChunk fixedLengthDataChunk = - new FixedLengthDimensionDataChunk(dataChunks, null, null, 5, 1); + FixedLengthDimensionColumnPage fixedLengthDataChunk = + new FixedLengthDimensionColumnPage(dataChunks, null, null, 5, 1); int result = CarbonUtil.nextGreaterValueToTarget(2, fixedLengthDataChunk, compareValues, 5); assertEquals(result, 3); } @@ -302,8 +302,8 @@ public void testToDeleteFoldersAndFilesSintlyWithInterruptedException() @Test public void testToGetnextGreaterValueToTarget() { byte[] dataChunks = { 5, 6, 7, 7, 7 }; byte[] compareValues = { 7 }; - FixedLengthDimensionDataChunk fixedLengthDataChunk = - new FixedLengthDimensionDataChunk(dataChunks, null, null, 5, 1); + FixedLengthDimensionColumnPage fixedLengthDataChunk = + new FixedLengthDimensionColumnPage(dataChunks, null, null, 5, 1); int result = CarbonUtil.nextGreaterValueToTarget(2, fixedLengthDataChunk, compareValues, 5); assertEquals(result, 5); } @@ -525,23 +525,23 @@ public void testToDeleteFoldersAndFilesSintlyWithInterruptedException() } @Test public void testToGetDictionaryEncodingArray() { - QueryDimension column1 = new QueryDimension("Column1"); - QueryDimension column2 = new QueryDimension("Column2"); ColumnSchema column1Schema = new ColumnSchema(); ColumnSchema column2Schema = new ColumnSchema(); column1Schema.setColumnName("Column1"); List encoding = new ArrayList<>(); encoding.add(Encoding.DICTIONARY); column1Schema.setEncodingList(encoding); - column1.setDimension(new CarbonDimension(column1Schema, 1, 1, 1, 1)); + ProjectionDimension + column1 = new ProjectionDimension(new CarbonDimension(column1Schema, 1, 1, 1, 1)); column2Schema.setColumnName("Column2"); List encoding2 = new ArrayList<>(); encoding2.add(Encoding.DELTA); column2Schema.setEncodingList(encoding2); - column2.setDimension(new CarbonDimension(column2Schema, 1, 1, 1, 1)); + ProjectionDimension + column2 = new ProjectionDimension(new CarbonDimension(column2Schema, 1, 1, 1, 1)); - QueryDimension[] queryDimensions = { column1, column2 }; + ProjectionDimension[] queryDimensions = { column1, column2 }; boolean[] dictionaryEncoding = CarbonUtil.getDictionaryEncodingArray(queryDimensions); boolean[] expectedDictionaryEncoding = { true, false }; @@ -551,23 +551,23 @@ public void testToDeleteFoldersAndFilesSintlyWithInterruptedException() } @Test public void testToGetDirectDictionaryEncodingArray() { - QueryDimension column1 = new QueryDimension("Column1"); - QueryDimension column2 = new QueryDimension("Column2"); ColumnSchema column1Schema = new ColumnSchema(); ColumnSchema column2Schema = new ColumnSchema(); column1Schema.setColumnName("Column1"); List encoding = new ArrayList<>(); encoding.add(Encoding.DIRECT_DICTIONARY); column1Schema.setEncodingList(encoding); - column1.setDimension(new CarbonDimension(column1Schema, 1, 1, 1, 1)); + ProjectionDimension + column1 = new ProjectionDimension(new CarbonDimension(column1Schema, 1, 1, 1, 1)); column2Schema.setColumnName("Column2"); List encoding2 = new ArrayList<>(); encoding2.add(Encoding.DELTA); column2Schema.setEncodingList(encoding2); - column2.setDimension(new CarbonDimension(column2Schema, 1, 1, 1, 1)); + ProjectionDimension + column2 = new ProjectionDimension(new CarbonDimension(column2Schema, 1, 1, 1, 1)); - QueryDimension[] queryDimensions = { column1, column2 }; + ProjectionDimension[] queryDimensions = { column1, column2 }; boolean[] dictionaryEncoding = CarbonUtil.getDirectDictionaryEncodingArray(queryDimensions); boolean[] expectedDictionaryEncoding = { true, false }; @@ -577,19 +577,19 @@ public void testToDeleteFoldersAndFilesSintlyWithInterruptedException() } @Test public void testToGetComplexDataTypeArray() { - QueryDimension column1 = new QueryDimension("Column1"); - QueryDimension column2 = new QueryDimension("Column2"); ColumnSchema column1Schema = new ColumnSchema(); ColumnSchema column2Schema = new ColumnSchema(); column1Schema.setColumnName("Column1"); column1Schema.setDataType(DataTypes.DATE); - column1.setDimension(new CarbonDimension(column1Schema, 1, 1, 1, 1)); + ProjectionDimension + column1 = new ProjectionDimension(new CarbonDimension(column1Schema, 1, 1, 1, 1)); column2Schema.setColumnName("Column2"); column2Schema.setDataType(DataTypes.createDefaultArrayType()); - column2.setDimension(new CarbonDimension(column2Schema, 1, 1, 1, 1)); + ProjectionDimension + column2 = new ProjectionDimension(new CarbonDimension(column2Schema, 1, 1, 1, 1)); - QueryDimension[] queryDimensions = { column1, column2 }; + ProjectionDimension[] queryDimensions = { column1, column2 }; boolean[] dictionaryEncoding = CarbonUtil.getComplexDataTypeArray(queryDimensions); boolean[] expectedDictionaryEncoding = { false, true }; @@ -806,8 +806,8 @@ public DataInputStream getDataInputStream(String path, FileFactory.FileType file @Test public void testToGetFirstIndexUsingBinarySearchWithCompareTo1() { byte[] dataChunks = { 10, 20, 30, 40, 50, 60 }; byte[] compareValue = { 5 }; - FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk = - new FixedLengthDimensionDataChunk(dataChunks, null, null, 6, 1); + FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk = + new FixedLengthDimensionColumnPage(dataChunks, null, null, 6, 1); int result = CarbonUtil .getFirstIndexUsingBinarySearch(fixedLengthDimensionDataChunk, 1, 3, compareValue, false); assertEquals(-2, result); @@ -816,8 +816,8 @@ public DataInputStream getDataInputStream(String path, FileFactory.FileType file @Test public void testToGetFirstIndexUsingBinarySearchWithCompareToLessThan0() { byte[] dataChunks = { 10, 20, 30, 40, 50, 60 }; byte[] compareValue = { 30 }; - FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk = - new FixedLengthDimensionDataChunk(dataChunks, null, null, 6, 1); + FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk = + new FixedLengthDimensionColumnPage(dataChunks, null, null, 6, 1); int result = CarbonUtil .getFirstIndexUsingBinarySearch(fixedLengthDimensionDataChunk, 1, 3, compareValue, false); assertEquals(2, result); @@ -826,8 +826,8 @@ public DataInputStream getDataInputStream(String path, FileFactory.FileType file @Test public void testToGetFirstIndexUsingBinarySearchWithCompareTo0() { byte[] dataChunks = { 10, 10, 10, 40, 50, 60 }; byte[] compareValue = { 10 }; - FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk = - new FixedLengthDimensionDataChunk(dataChunks, null, null, 6, 1); + FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk = + new FixedLengthDimensionColumnPage(dataChunks, null, null, 6, 1); int result = CarbonUtil .getFirstIndexUsingBinarySearch(fixedLengthDimensionDataChunk, 1, 3, compareValue, false); assertEquals(0, result); @@ -836,8 +836,8 @@ public DataInputStream getDataInputStream(String path, FileFactory.FileType file @Test public void testToGetFirstIndexUsingBinarySearchWithMatchUpLimitTrue() { byte[] dataChunks = { 10, 10, 10, 40, 50, 60 }; byte[] compareValue = { 10 }; - FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk = - new FixedLengthDimensionDataChunk(dataChunks, null, null, 6, 1); + FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk = + new FixedLengthDimensionColumnPage(dataChunks, null, null, 6, 1); int result = CarbonUtil .getFirstIndexUsingBinarySearch(fixedLengthDimensionDataChunk, 1, 3, compareValue, true); assertEquals(2, result); @@ -847,13 +847,13 @@ public DataInputStream getDataInputStream(String path, FileFactory.FileType file public void testBinaryRangeSearch() { byte[] dataChunk = new byte[10]; - FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk; + FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk; byte[] keyWord = new byte[1]; int[] range; dataChunk = "abbcccddddeffgggh".getBytes(); byte[][] dataArr = new byte[dataChunk.length / keyWord.length][keyWord.length]; - fixedLengthDimensionDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null, + fixedLengthDimensionDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null, dataChunk.length / keyWord.length, keyWord.length); for (int ii = 0; ii < dataChunk.length / keyWord.length; ii++) { @@ -885,7 +885,7 @@ public void testBinaryRangeSearch() { assertRangeIndex(dataArr, dataChunk, fixedLengthDimensionDataChunk, keyWord, expectRangeIndex); dataChunk = "ab".getBytes(); - fixedLengthDimensionDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null, + fixedLengthDimensionDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null, dataChunk.length / keyWord.length, keyWord.length); keyWord[0] = Byte.valueOf("97"); @@ -899,7 +899,7 @@ public void testBinaryRangeSearch() { assertEquals(1, range[1]); dataChunk = "aabb".getBytes(); - fixedLengthDimensionDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null, + fixedLengthDimensionDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null, dataChunk.length / keyWord.length, keyWord.length); keyWord[0] = Byte.valueOf("97"); @@ -913,7 +913,7 @@ public void testBinaryRangeSearch() { assertEquals(3, range[1]); dataChunk = "a".getBytes(); - fixedLengthDimensionDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null, + fixedLengthDimensionDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null, dataChunk.length / keyWord.length, keyWord.length); keyWord[0] = Byte.valueOf("97"); @@ -922,7 +922,7 @@ public void testBinaryRangeSearch() { assertEquals(0, range[1]); dataChunk = "aa".getBytes(); - fixedLengthDimensionDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null, + fixedLengthDimensionDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null, dataChunk.length / keyWord.length, keyWord.length); keyWord[0] = Byte.valueOf("97"); @@ -931,7 +931,7 @@ public void testBinaryRangeSearch() { assertEquals(1, range[1]); dataChunk = "aabbbbbbbbbbcc".getBytes(); - fixedLengthDimensionDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null, + fixedLengthDimensionDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null, dataChunk.length / keyWord.length, keyWord.length); keyWord[0] = Byte.valueOf("98"); range = CarbonUtil.getRangeIndexUsingBinarySearch(fixedLengthDimensionDataChunk, 0, dataChunk.length - 1, keyWord); @@ -944,14 +944,14 @@ public void testBinaryRangeSearch() { public void IndexUsingBinarySearchLengthTwo() { byte[] dataChunk = new byte[10]; - FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk; + FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk; byte[] keyWord = new byte[2]; dataChunk = "aabbbbbbbbbbcc".getBytes(); byte[][] dataArr = new byte[dataChunk.length / keyWord.length][keyWord.length]; - fixedLengthDimensionDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null, + fixedLengthDimensionDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null, dataChunk.length / keyWord.length, keyWord.length); for (int ii = 0; ii < dataChunk.length / keyWord.length; ii++) { @@ -986,14 +986,14 @@ public void IndexUsingBinarySearchLengthTwo() { public void IndexUsingBinarySearchLengthThree() { byte[] dataChunk = new byte[10]; - FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk; + FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk; byte[] keyWord = new byte[3]; dataChunk = "aaabbbbbbbbbccc".getBytes(); byte[][] dataArr = new byte[dataChunk.length / keyWord.length][keyWord.length]; - fixedLengthDimensionDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null, + fixedLengthDimensionDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null, dataChunk.length / keyWord.length, keyWord.length); for (int ii = 0; ii < dataChunk.length / keyWord.length; ii++) { @@ -1101,7 +1101,7 @@ private String generateString(int length) { } private void assertRangeIndex(byte[][] dataArr, byte[] dataChunk, - FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk, byte[] keyWord, int[] expectRangeIndex) { + FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk, byte[] keyWord, int[] expectRangeIndex) { int[] range; range = CarbonUtil.getRangeIndexUsingBinarySearch(fixedLengthDimensionDataChunk, 0, (dataChunk.length - 1) / keyWord.length, keyWord); diff --git a/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java b/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java index de64c0a93bb..e5069949753 100644 --- a/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java +++ b/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java @@ -25,10 +25,10 @@ import java.util.Arrays; import java.util.List; -import org.apache.carbondata.core.datastore.FileHolder; +import org.apache.carbondata.core.datastore.FileReader; import org.apache.carbondata.core.datastore.block.TableBlockInfo; import org.apache.carbondata.core.datastore.impl.FileFactory; -import org.apache.carbondata.core.datastore.impl.FileHolderImpl; +import org.apache.carbondata.core.datastore.impl.FileReaderImpl; import org.apache.carbondata.core.metadata.ColumnarFormatVersion; import org.apache.carbondata.core.metadata.blocklet.DataFileFooter; import org.apache.carbondata.core.metadata.blocklet.SegmentInfo; @@ -229,13 +229,13 @@ public DataInputStream getDataInputStream(String path, FileFactory.FileType file } @SuppressWarnings("unused") @Mock - public FileHolder getFileHolder(FileFactory.FileType fileType) { - return new FileHolderImpl(); + public FileReader getFileHolder(FileFactory.FileType fileType) { + return new FileReaderImpl(); } }; - new MockUp() { + new MockUp() { @SuppressWarnings("unused") @Mock public long readLong(String filePath, long offset) { return 1; } @@ -249,7 +249,6 @@ public FileHolder getFileHolder(FileFactory.FileType fileType) { SegmentInfo segmentInfo = new SegmentInfo(); int[] arr = { 1, 2, 3 }; segmentInfo.setColumnCardinality(arr); - segmentInfo.setNumberOfColumns(segmentInfo1.getNum_cols()); dataFileFooter.setNumberOfRows(3); dataFileFooter.setSegmentInfo(segmentInfo); TableBlockInfo info = new TableBlockInfo("/file.carbondata", 1, "0", new String[0], 1, ColumnarFormatVersion.V1, null); 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 4c9a7846f51..4fb5dccf99d 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 @@ -36,7 +36,6 @@ import org.apache.carbondata.core.scan.expression.logical.TrueExpression; 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.intf.FilterOptimizerBasic; import org.apache.carbondata.core.scan.filter.optimizer.RangeFilterOptmizer; import mockit.Deencapsulation; @@ -102,7 +101,7 @@ public boolean checkBothTrees(Expression a, Expression b) { new LessThanEqualToExpression(new ColumnExpression("a", DataTypes.STRING), new LiteralExpression("20", DataTypes.STRING))), new TrueExpression(null)); FilterOptimizer rangeFilterOptimizer = - new RangeFilterOptmizer(new FilterOptimizerBasic(), inputFilter); + new RangeFilterOptmizer(inputFilter); rangeFilterOptimizer.optimizeFilter(); result = checkBothTrees(inputFilter, output); Assert.assertTrue(result); @@ -143,7 +142,7 @@ public boolean checkBothTrees(Expression a, Expression b) { new LessThanEqualToExpression(new ColumnExpression("a", DataTypes.STRING), new LiteralExpression("05", DataTypes.STRING))); FilterOptimizer rangeFilterOptimizer = - new RangeFilterOptmizer(new FilterOptimizerBasic(), inputFilter); + new RangeFilterOptmizer(inputFilter); rangeFilterOptimizer.optimizeFilter(); result = checkBothTrees(inputFilter, output); // no change @@ -218,7 +217,7 @@ public boolean checkBothTrees(Expression a, Expression b) { Expression Andb3 = new AndExpression(Andb2, new TrueExpression(null)); FilterOptimizer rangeFilterOptimizer = - new RangeFilterOptmizer(new FilterOptimizerBasic(), inputFilter); + new RangeFilterOptmizer(inputFilter); rangeFilterOptimizer.optimizeFilter(); result = checkBothTrees(inputFilter, new AndExpression(Andb3, new TrueExpression(null))); // no change @@ -302,7 +301,7 @@ public boolean checkBothTrees(Expression a, Expression b) { Expression Orb3 = new OrExpression(Orb2, lessThanb2); FilterOptimizer rangeFilterOptimizer = - new RangeFilterOptmizer(new FilterOptimizerBasic(), inputFilter); + new RangeFilterOptmizer(inputFilter); rangeFilterOptimizer.optimizeFilter(); result = checkBothTrees(inputFilter, new OrExpression(Orb3, lessThanb1)); // no change diff --git a/core/src/test/java/org/apache/carbondata/scanner/impl/FilterScannerTest.java b/core/src/test/java/org/apache/carbondata/scanner/impl/FilterScannerTest.java deleted file mode 100644 index 94c3f68b357..00000000000 --- a/core/src/test/java/org/apache/carbondata/scanner/impl/FilterScannerTest.java +++ /dev/null @@ -1,160 +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.scanner.impl; - -import static junit.framework.TestCase.assertEquals; - -public class FilterScannerTest { -// -// private static FilterScanner filterScanner; -// private static BlockletIndex blockletIndex; -// private static BlockletMinMaxIndex blockletMinMaxIndex; -// private static BTreeBuilderInfo bTreeBuilderInfo; -// private static DataFileFooter dataFileFooter; -// -// @BeforeClass public static void setUp() { -// BlockExecutionInfo blockExecutionInfo = new BlockExecutionInfo(); -// FilterExecuter filterExecutor = new AndFilterExecuterImpl(null, null); -// blockExecutionInfo.setFilterExecuterTree(filterExecutor); -// blockExecutionInfo.setFixedLengthKeySize(1); -// blockExecutionInfo.setNoDictionaryBlockIndexes(new int[] { 1, 2 }); -// blockExecutionInfo.setDictionaryColumnBlockIndex(new int[] { 1 }); -// blockExecutionInfo.setColumnGroupToKeyStructureInfo(new HashMap()); -// blockExecutionInfo.setComplexDimensionInfoMap(new HashMap()); -// blockExecutionInfo.setComplexColumnParentBlockIndexes(new int[] { 1 }); -// blockExecutionInfo.setQueryDimensions(new QueryDimension[] { new QueryDimension("Col1") }); -// blockExecutionInfo.setAllSelectedDimensionBlocksIndexes(new int[][] { { 0, 0 } }); -// blockExecutionInfo.setAllSelectedMeasureBlocksIndexes(new int[][] { { 0, 0 } }); -// blockExecutionInfo.setTotalNumberOfMeasureBlock(1); -// blockExecutionInfo.setTotalNumberDimensionBlock(1); -// QueryStatisticsModel queryStatisticsModel = new QueryStatisticsModel(); -// QueryStatistic queryStatistic = new QueryStatistic(); -// queryStatistic.addCountStatistic(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM, 1); -// Map statisticsTypeAndObjMap = new HashMap<>(); -// statisticsTypeAndObjMap.put(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM, queryStatistic); -// statisticsTypeAndObjMap.put(QueryStatisticsConstants.VALID_SCAN_BLOCKLET_NUM, queryStatistic); -// queryStatisticsModel.setStatisticsTypeAndObjMap(statisticsTypeAndObjMap); -// QueryStatisticsRecorder queryStatisticsRecorder = new QueryStatisticsRecorderImpl("1"); -// queryStatisticsModel.setRecorder(queryStatisticsRecorder); -// filterScanner = new FilterScanner(blockExecutionInfo, queryStatisticsModel); -// blockletIndex = new BlockletIndex(); -// blockletMinMaxIndex = new BlockletMinMaxIndex(); -// blockletMinMaxIndex.setMinValues(new byte[][] { { 1, 2 } }); -// blockletMinMaxIndex.setMaxValues(new byte[][] { { 10, 12 } }); -// blockletIndex.setMinMaxIndex(blockletMinMaxIndex); -// dataFileFooter = new DataFileFooter(); -// dataFileFooter.setBlockletIndex(blockletIndex); -// bTreeBuilderInfo = new BTreeBuilderInfo(Arrays.asList(dataFileFooter), new int[] { 1 }); -// } -// -// @Test public void testToScanBlockletWithEmptyBitSet() throws QueryExecutionException { -// new MockUp() { -// @SuppressWarnings("unused") @Mock -// public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) { -// return new BitSet(); -// } -// }; -// BlocksChunkHolder blocksChunkHolder = new BlocksChunkHolder(1, 1); -// DataRefNode dataRefNode = new BlockBTreeLeafNode(bTreeBuilderInfo, 0, 1); -// blocksChunkHolder.setDataBlock(dataRefNode); -// AbstractScannedResult abstractScannedResult = filterScanner.scanBlocklet(blocksChunkHolder); -// assertEquals(0, abstractScannedResult.numberOfOutputRows()); -// } -// -// @Test public void testToScanBlockletWithNonEmptyBitSet() throws QueryExecutionException { -// new MockUp() { -// @SuppressWarnings("unused") @Mock -// public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) { -// BitSet bitSet = new BitSet(); -// bitSet.set(1); -// bitSet.set(2); -// bitSet.set(1); -// return bitSet; -// } -// -// @SuppressWarnings("unused") @Mock -// public BitSet applyFilter(BlocksChunkHolder blockChunkHolder) -// throws FilterUnsupportedException { -// BitSet bitSet = new BitSet(); -// bitSet.set(1); -// bitSet.set(2); -// bitSet.set(1); -// return bitSet; -// } -// }; -// DataRefNode dataRefNode = new MockUp() { -// @Mock @SuppressWarnings("unused") DimensionColumnDataChunk[] getDimensionChunks( -// FileHolder fileReader, int[][] blockIndexes) { -// DimensionColumnDataChunk[] dimensionChunkAttributes = -// { new ColumnGroupDimensionDataChunk(null, null) }; -// return dimensionChunkAttributes; -// } -// -// @Mock @SuppressWarnings("unused") ColumnPage[] getMeasureChunks( -// FileHolder fileReader, int[][] blockIndexes) { -// -// ColumnPage[] ColumnPages = { new ColumnPage() }; -// return ColumnPages; -// } -// }.getMockInstance(); -// -// BlocksChunkHolder blocksChunkHolder = new BlocksChunkHolder(1, 1); -// blocksChunkHolder.setDataBlock(dataRefNode); -// DimensionChunkAttributes dimensionChunkAttributes = new DimensionChunkAttributes(); -// DimensionColumnDataChunk dimensionColumnDataChunk = -// new FixedLengthDimensionDataChunk(new byte[] { 0, 1 }, dimensionChunkAttributes); -// blocksChunkHolder.setDimensionRawDataChunk(new DimensionColumnDataChunk[] -// -// { dimensionColumnDataChunk }); -// ColumnPage ColumnPage = new ColumnPage(); -// blocksChunkHolder.setMeasureDataChunk(new ColumnPage[] -// -// { ColumnPage }); -// FileHolder fileHolder = new DFSFileHolderImpl(); -// blocksChunkHolder.setFileReader(fileHolder); -// AbstractScannedResult abstractScannedResult = filterScanner.scanBlocklet(blocksChunkHolder); -// -// assertEquals(2, abstractScannedResult.numberOfOutputRows()); -// } -// -// @Test(expected = QueryExecutionException.class) public void testToScanBlockletWithException() -// throws QueryExecutionException { -// new MockUp() { -// @SuppressWarnings("unused") @Mock -// public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) { -// BitSet bitSet = new BitSet(); -// bitSet.set(1); -// bitSet.set(2); -// bitSet.set(1); -// return bitSet; -// } -// -// @SuppressWarnings("unused") @Mock -// public BitSet applyFilter(BlocksChunkHolder blockChunkHolder) -// throws FilterUnsupportedException { -// throw new FilterUnsupportedException("Filter unsupported"); -// } -// }; -// BlocksChunkHolder blocksChunkHolder = new BlocksChunkHolder(1, 1); -// BTreeBuilderInfo bTreeBuilderInfo = -// new BTreeBuilderInfo(Arrays.asList(dataFileFooter), new int[] { 1 }); -// DataRefNode dataRefNode = new BlockBTreeLeafNode(bTreeBuilderInfo, 0, 1); -// blocksChunkHolder.setDataBlock(dataRefNode); -// filterScanner.scanBlocklet(blocksChunkHolder); -// } - -} diff --git a/dev/findbugs-exclude.xml b/dev/findbugs-exclude.xml index 1520cd41eac..b19db85c9b1 100644 --- a/dev/findbugs-exclude.xml +++ b/dev/findbugs-exclude.xml @@ -31,7 +31,7 @@ - + diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala index 76afcbfd80e..7a1532759a5 100644 --- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala +++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala @@ -62,7 +62,7 @@ object CarbonSessionExample { spark.sql( s""" - | SELECT * + | SELECT charField, stringField, intField | FROM carbon_table | WHERE stringfield = 'spark' AND decimalField > 40 """.stripMargin).show() diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java index fdec2679e63..5a20d7e0585 100644 --- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java +++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java @@ -58,7 +58,6 @@ import org.apache.carbondata.core.scan.filter.SingleTableProvider; import org.apache.carbondata.core.scan.filter.TableProvider; import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf; -import org.apache.carbondata.core.scan.model.CarbonQueryPlan; import org.apache.carbondata.core.scan.model.QueryModel; import org.apache.carbondata.core.stats.QueryStatistic; import org.apache.carbondata.core.stats.QueryStatisticsConstants; @@ -112,11 +111,11 @@ public class CarbonTableInputFormat extends FileInputFormat { // comma separated list of input segment numbers public static final String INPUT_SEGMENT_NUMBERS = "mapreduce.input.carboninputformat.segmentnumbers"; - public static final String VALIDATE_INPUT_SEGMENT_IDs = + private static final String VALIDATE_INPUT_SEGMENT_IDs = "mapreduce.input.carboninputformat.validsegments"; // comma separated list of input files public static final String INPUT_FILES = "mapreduce.input.carboninputformat.files"; - public static final String ALTER_PARTITION_ID = "mapreduce.input.carboninputformat.partitionid"; + private static final String ALTER_PARTITION_ID = "mapreduce.input.carboninputformat.partitionid"; private static final Log LOG = LogFactory.getLog(CarbonTableInputFormat.class); private static final String FILTER_PREDICATE = "mapreduce.input.carboninputformat.filter.predicate"; @@ -127,7 +126,7 @@ public class CarbonTableInputFormat extends FileInputFormat { private static final String DATA_MAP_DSTR = "mapreduce.input.carboninputformat.datamapdstr"; public static final String DATABASE_NAME = "mapreduce.input.carboninputformat.databaseName"; public static final String TABLE_NAME = "mapreduce.input.carboninputformat.tableName"; - public static final String PARTITIONS_TO_PRUNE = + private static final String PARTITIONS_TO_PRUNE = "mapreduce.input.carboninputformat.partitions.to.prune"; public static final String UPADTE_T = "mapreduce.input.carboninputformat.partitions.to.prune"; @@ -340,7 +339,8 @@ public AbsoluteTableIdentifier getAbsoluteTableIdentifier(Configuration configur * @return List list of CarbonInputSplit * @throws IOException */ - @Override public List getSplits(JobContext job) throws IOException { + @Override + public List getSplits(JobContext job) throws IOException { AbsoluteTableIdentifier identifier = getAbsoluteTableIdentifier(job.getConfiguration()); LoadMetadataDetails[] loadMetadataDetails = SegmentStatusManager .readTableStatusFile(CarbonTablePath.getTableStatusFilePath(identifier.getTablePath())); @@ -811,28 +811,29 @@ private CarbonInputSplit convertToCarbonInputSplit(ExtendedBlocklet blocklet) th return split; } - @Override public RecordReader createRecordReader(InputSplit inputSplit, + @Override + public RecordReader createRecordReader(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException { Configuration configuration = taskAttemptContext.getConfiguration(); - QueryModel queryModel = getQueryModel(inputSplit, taskAttemptContext); + QueryModel queryModel = createQueryModel(inputSplit, taskAttemptContext); CarbonReadSupport readSupport = getReadSupportClass(configuration); return new CarbonRecordReader(queryModel, readSupport); } - public QueryModel getQueryModel(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) + public QueryModel createQueryModel(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException { Configuration configuration = taskAttemptContext.getConfiguration(); CarbonTable carbonTable = getOrCreateCarbonTable(configuration); TableProvider tableProvider = new SingleTableProvider(carbonTable); - // getting the table absoluteTableIdentifier from the carbonTable - // to avoid unnecessary deserialization - AbsoluteTableIdentifier identifier = carbonTable.getAbsoluteTableIdentifier(); // query plan includes projection column - String projection = getColumnProjection(configuration); - CarbonQueryPlan queryPlan = CarbonInputFormatUtil.createQueryPlan(carbonTable, projection); - QueryModel queryModel = QueryModel.createModel(identifier, queryPlan, carbonTable, - getDataTypeConverter(configuration)); + String projectionString = getColumnProjection(configuration); + String[] projectionColumnNames = null; + if (projectionString != null) { + projectionColumnNames = projectionString.split(","); + } + QueryModel queryModel = carbonTable.createQueryWithProjection( + projectionColumnNames, getDataTypeConverter(configuration)); // set the filter to the query model in order to filter blocklet before scan Expression filter = getFilterPredicates(configuration); @@ -887,7 +888,8 @@ public CarbonReadSupport getReadSupportClass(Configuration configuration) { return readSupport; } - @Override protected boolean isSplitable(JobContext context, Path filename) { + @Override + protected boolean isSplitable(JobContext context, Path filename) { try { // Don't split the file if it is local file system FileSystem fileSystem = filename.getFileSystem(context.getConfiguration()); @@ -900,16 +902,6 @@ public CarbonReadSupport getReadSupportClass(Configuration configuration) { return true; } - /** - * required to be moved to core - * - * @return updateExtension - */ - private String getUpdateExtension() { - // TODO: required to modify when supporting update, mostly will be update timestamp - return "update"; - } - /** * return valid segment to access */ diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java index a590a5bd959..0fe0cbfe54b 100644 --- a/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java +++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java @@ -176,9 +176,7 @@ private List getDataBlocksOfSegment(JobContext job, filterredBlocks = filterExpressionProcessor.getFilterredBlocks( abstractIndex.getDataRefNode(), resolver, - abstractIndex, - identifier - ); + abstractIndex); } resultFilterredBlocks.addAll(filterredBlocks); } diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java index 95a7af02b43..1e227c4d51b 100644 --- a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java +++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java @@ -156,7 +156,7 @@ public class CarbonStreamRecordReader extends RecordReader { hadoopConf = context.getConfiguration(); if (model == null) { CarbonTableInputFormat format = new CarbonTableInputFormat(); - model = format.getQueryModel(split, context); + model = format.createQueryModel(split, context); } carbonTable = model.getTable(); List dimensions = diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/BlockLevelTraverser.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/BlockLevelTraverser.java index 89a4a9a0a7c..2f288619ead 100644 --- a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/BlockLevelTraverser.java +++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/BlockLevelTraverser.java @@ -67,7 +67,7 @@ public long getBlockRowMapping(AbstractIndex abstractIndex, Map bl blockName = CarbonTablePath.getCarbonDataFileName(blockName); blockName = blockName + CarbonTablePath.getCarbonDataExtension(); - long rowCount = currentBlock.nodeSize(); + long rowCount = currentBlock.numRows(); String key = CarbonUpdateUtil.getSegmentBlockNameKey(segId, blockName); diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java index 056c27bd572..9f8c5ec7fa4 100644 --- a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java +++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java @@ -26,18 +26,12 @@ import org.apache.carbondata.core.datastore.impl.FileFactory; import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier; import org.apache.carbondata.core.metadata.schema.table.CarbonTable; -import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension; -import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure; import org.apache.carbondata.core.scan.expression.Expression; import org.apache.carbondata.core.scan.filter.FilterExpressionProcessor; import org.apache.carbondata.core.scan.filter.TableProvider; import org.apache.carbondata.core.scan.filter.intf.FilterOptimizer; -import org.apache.carbondata.core.scan.filter.intf.FilterOptimizerBasic; import org.apache.carbondata.core.scan.filter.optimizer.RangeFilterOptmizer; import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf; -import org.apache.carbondata.core.scan.model.CarbonQueryPlan; -import org.apache.carbondata.core.scan.model.QueryDimension; -import org.apache.carbondata.core.scan.model.QueryMeasure; import org.apache.carbondata.core.scan.model.QueryModel; import org.apache.carbondata.hadoop.api.CarbonTableInputFormat; @@ -52,45 +46,14 @@ */ public class CarbonInputFormatUtil { - public static CarbonQueryPlan createQueryPlan(CarbonTable carbonTable, String columnString) { - String[] columns = null; - if (columnString != null) { - columns = columnString.split(","); - } - String factTableName = carbonTable.getTableName(); - CarbonQueryPlan plan = new CarbonQueryPlan(carbonTable.getDatabaseName(), factTableName); - // fill dimensions - // If columns are null, set all dimensions and measures - int i = 0; - if (columns != null) { - for (String column : columns) { - CarbonDimension dimensionByName = carbonTable.getDimensionByName(factTableName, column); - if (dimensionByName != null) { - addQueryDimension(plan, i, dimensionByName); - i++; - } else { - CarbonMeasure measure = carbonTable.getMeasureByName(factTableName, column); - if (measure == null) { - throw new RuntimeException(column + " column not found in the table " + factTableName); - } - addQueryMeasure(plan, i, measure); - i++; - } - } - } - - plan.setQueryId(System.nanoTime() + ""); - return plan; - } - public static CarbonTableInputFormat createCarbonInputFormat( AbsoluteTableIdentifier identifier, Job job) throws IOException { CarbonTableInputFormat carbonInputFormat = new CarbonTableInputFormat<>(); - carbonInputFormat.setDatabaseName(job.getConfiguration(), - identifier.getCarbonTableIdentifier().getDatabaseName()); - carbonInputFormat - .setTableName(job.getConfiguration(), identifier.getCarbonTableIdentifier().getTableName()); + CarbonTableInputFormat.setDatabaseName( + job.getConfiguration(), identifier.getCarbonTableIdentifier().getDatabaseName()); + CarbonTableInputFormat.setTableName( + job.getConfiguration(), identifier.getCarbonTableIdentifier().getTableName()); FileInputFormat.addInputPath(job, new Path(identifier.getTablePath())); return carbonInputFormat; } @@ -98,30 +61,16 @@ public static CarbonTableInputFormat createCarbonInputFormat( public static CarbonTableInputFormat createCarbonTableInputFormat( AbsoluteTableIdentifier identifier, List partitionId, Job job) throws IOException { CarbonTableInputFormat carbonTableInputFormat = new CarbonTableInputFormat<>(); - carbonTableInputFormat.setPartitionIdList(job.getConfiguration(), partitionId); - carbonTableInputFormat.setDatabaseName(job.getConfiguration(), - identifier.getCarbonTableIdentifier().getDatabaseName()); - carbonTableInputFormat - .setTableName(job.getConfiguration(), identifier.getCarbonTableIdentifier().getTableName()); + CarbonTableInputFormat.setPartitionIdList( + job.getConfiguration(), partitionId); + CarbonTableInputFormat.setDatabaseName( + job.getConfiguration(), identifier.getCarbonTableIdentifier().getDatabaseName()); + CarbonTableInputFormat.setTableName( + job.getConfiguration(), identifier.getCarbonTableIdentifier().getTableName()); FileInputFormat.addInputPath(job, new Path(identifier.getTablePath())); return carbonTableInputFormat; } - private static void addQueryMeasure(CarbonQueryPlan plan, int order, CarbonMeasure measure) { - QueryMeasure queryMeasure = new QueryMeasure(measure.getColName()); - queryMeasure.setQueryOrder(order); - queryMeasure.setMeasure(measure); - plan.addMeasure(queryMeasure); - } - - private static void addQueryDimension(CarbonQueryPlan plan, int order, - CarbonDimension dimension) { - QueryDimension queryDimension = new QueryDimension(dimension.getColName()); - queryDimension.setQueryOrder(order); - queryDimension.setDimension(dimension); - plan.addDimension(queryDimension); - } - public static void processFilterExpression(Expression filterExpression, CarbonTable carbonTable, boolean[] isFilterDimensions, boolean[] isFilterMeasures) { QueryModel.processFilterExpression(carbonTable, filterExpression, isFilterDimensions, @@ -130,7 +79,7 @@ public static void processFilterExpression(Expression filterExpression, CarbonTa if (null != filterExpression) { // Optimize Filter Expression and fit RANGE filters is conditions apply. FilterOptimizer rangeFilterOptimizer = - new RangeFilterOptmizer(new FilterOptimizerBasic(), filterExpression); + new RangeFilterOptmizer(filterExpression); rangeFilterOptimizer.optimizeFilter(); } } diff --git a/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java b/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java index f109e1c9e42..1b57f9313eb 100644 --- a/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java +++ b/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java @@ -30,7 +30,6 @@ import org.apache.carbondata.core.scan.filter.SingleTableProvider; import org.apache.carbondata.core.scan.filter.TableProvider; import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf; -import org.apache.carbondata.core.scan.model.CarbonQueryPlan; import org.apache.carbondata.core.scan.model.QueryModel; import org.apache.carbondata.core.util.DataTypeConverterImpl; import org.apache.carbondata.hadoop.CarbonInputSplit; @@ -140,11 +139,11 @@ private QueryModel getQueryModel(Configuration configuration, String path) AbsoluteTableIdentifier identifier = carbonTable.getAbsoluteTableIdentifier(); - String projection = getProjection(configuration, carbonTable, + String projectionString = getProjection(configuration, carbonTable, identifier.getCarbonTableIdentifier().getTableName()); - CarbonQueryPlan queryPlan = CarbonInputFormatUtil.createQueryPlan(carbonTable, projection); - QueryModel queryModel = - QueryModel.createModel(identifier, queryPlan, carbonTable, new DataTypeConverterImpl()); + String[] projectionColumns = projectionString.split(","); + QueryModel queryModel = carbonTable.createQueryWithProjection( + projectionColumns, new DataTypeConverterImpl()); // set the filter to the query model in order to filter blocklet before scan Expression filter = getFilterPredicates(configuration); CarbonInputFormatUtil.processFilterExpression(filter, carbonTable, null, null); diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataPageSource.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataPageSource.java index 1679f29e51c..5f1f90a80e4 100644 --- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataPageSource.java +++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataPageSource.java @@ -54,7 +54,7 @@ class CarbondataPageSource implements ConnectorPageSource { private final List types; private final PageBuilder pageBuilder; private boolean closed; - private CarbonVectorizedRecordReader vectorReader; + private PrestoCarbonVectorizedRecordReader vectorReader; private CarbonDictionaryDecodeReadSupport readSupport; private long sizeOfData = 0; diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordCursor.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordCursor.java index c614fa9c093..5772fbf8183 100755 --- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordCursor.java +++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordCursor.java @@ -54,7 +54,7 @@ public class CarbondataRecordCursor implements RecordCursor { private CarbondataSplit split; private CarbonDictionaryDecodeReadSupport readSupport; private Tuple3[] dictionary; - CarbonVectorizedRecordReader vectorizedRecordReader; + PrestoCarbonVectorizedRecordReader vectorizedRecordReader; private long totalBytes; private long nanoStart; @@ -63,7 +63,7 @@ public class CarbondataRecordCursor implements RecordCursor { public CarbondataRecordCursor(CarbonDictionaryDecodeReadSupport readSupport, - CarbonVectorizedRecordReader vectorizedRecordReader, + PrestoCarbonVectorizedRecordReader vectorizedRecordReader, List columnHandles, CarbondataSplit split) { this.vectorizedRecordReader = vectorizedRecordReader; @@ -194,7 +194,7 @@ private void checkFieldType(int field, Type expected) { //todo delete cache from readSupport } - public CarbonVectorizedRecordReader getVectorizedRecordReader() { + public PrestoCarbonVectorizedRecordReader getVectorizedRecordReader() { return vectorizedRecordReader; } diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSet.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSet.java index 0f8fe8727e4..286ff0e06c9 100755 --- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSet.java +++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSet.java @@ -78,8 +78,8 @@ public CarbondataRecordSet(CarbonTable carbonTable, ConnectorSession session, readSupport .initialize(queryModel.getProjectionColumns(), queryModel.getTable()); CarbonIterator iterator = queryExecutor.execute(queryModel); - CarbonVectorizedRecordReader vectorReader = - new CarbonVectorizedRecordReader(queryExecutor, queryModel, + PrestoCarbonVectorizedRecordReader vectorReader = + new PrestoCarbonVectorizedRecordReader(queryExecutor, queryModel, (AbstractDetailQueryResultIterator) iterator); return new CarbondataRecordCursor(readSupport, vectorReader, columns, split); } catch (QueryExecutionException e) { diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java index f039daf4b8a..5a2f831c5ff 100755 --- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java +++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java @@ -105,7 +105,7 @@ public CarbondataRecordSetProvider(CarbondataConnectorId connectorId, CarbonTabl new TaskAttemptContextImpl(jobConf, new TaskAttemptID("", 1, TaskType.MAP, 0, 0)); CarbonInputSplit carbonInputSplit = CarbonLocalInputSplit.convertSplit(carbondataSplit.getLocalInputSplit()); - queryModel = carbonTableInputFormat.getQueryModel(carbonInputSplit, hadoopAttemptContext); + queryModel = carbonTableInputFormat.createQueryModel(carbonInputSplit, hadoopAttemptContext); queryModel.setVectorReader(true); } catch (IOException e) { throw new RuntimeException("Unable to get the Query Model ", e); diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorizedRecordReader.java b/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoCarbonVectorizedRecordReader.java similarity index 88% rename from integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorizedRecordReader.java rename to integration/presto/src/main/java/org/apache/carbondata/presto/PrestoCarbonVectorizedRecordReader.java index 9a8f8c5e76e..a1907dbe46f 100644 --- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorizedRecordReader.java +++ b/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoCarbonVectorizedRecordReader.java @@ -33,8 +33,8 @@ import org.apache.carbondata.core.scan.executor.QueryExecutor; import org.apache.carbondata.core.scan.executor.QueryExecutorFactory; import org.apache.carbondata.core.scan.executor.exception.QueryExecutionException; -import org.apache.carbondata.core.scan.model.QueryDimension; -import org.apache.carbondata.core.scan.model.QueryMeasure; +import org.apache.carbondata.core.scan.model.ProjectionDimension; +import org.apache.carbondata.core.scan.model.ProjectionMeasure; import org.apache.carbondata.core.scan.model.QueryModel; import org.apache.carbondata.core.scan.result.iterator.AbstractDetailQueryResultIterator; import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector; @@ -51,7 +51,7 @@ * A specialized RecordReader that reads into InternalRows or ColumnarBatches directly using the * carbondata column APIs and fills the data directly into columns. */ -class CarbonVectorizedRecordReader extends AbstractRecordReader { +class PrestoCarbonVectorizedRecordReader extends AbstractRecordReader { private int batchIdx = 0; @@ -72,7 +72,7 @@ class CarbonVectorizedRecordReader extends AbstractRecordReader { private QueryExecutor queryExecutor; - public CarbonVectorizedRecordReader(QueryExecutor queryExecutor, QueryModel queryModel, AbstractDetailQueryResultIterator iterator) { + public PrestoCarbonVectorizedRecordReader(QueryExecutor queryExecutor, QueryModel queryModel, AbstractDetailQueryResultIterator iterator) { this.queryModel = queryModel; this.iterator = iterator; this.queryExecutor = queryExecutor; @@ -164,39 +164,39 @@ public CarbonVectorizedRecordReader(QueryExecutor queryExecutor, QueryModel quer */ private void initBatch() { - List queryDimension = queryModel.getQueryDimension(); - List queryMeasures = queryModel.getQueryMeasures(); + List queryDimension = queryModel.getProjectionDimensions(); + List queryMeasures = queryModel.getProjectionMeasures(); StructField[] fields = new StructField[queryDimension.size() + queryMeasures.size()]; for (int i = 0; i < queryDimension.size(); i++) { - QueryDimension dim = queryDimension.get(i); + ProjectionDimension dim = queryDimension.get(i); if (dim.getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) { DirectDictionaryGenerator generator = DirectDictionaryKeyGeneratorFactory .getDirectDictionaryGenerator(dim.getDimension().getDataType()); - fields[dim.getQueryOrder()] = new StructField(dim.getColumnName(), + fields[dim.getOrdinal()] = new StructField(dim.getColumnName(), generator.getReturnType()); } else if (!dim.getDimension().hasEncoding(Encoding.DICTIONARY)) { - fields[dim.getQueryOrder()] = new StructField(dim.getColumnName(), + fields[dim.getOrdinal()] = new StructField(dim.getColumnName(), dim.getDimension().getDataType()); } else if (dim.getDimension().isComplex()) { - fields[dim.getQueryOrder()] = new StructField(dim.getColumnName(), + fields[dim.getOrdinal()] = new StructField(dim.getColumnName(), dim.getDimension().getDataType()); } else { - fields[dim.getQueryOrder()] = new StructField(dim.getColumnName(), + fields[dim.getOrdinal()] = new StructField(dim.getColumnName(), DataTypes.INT); } } - for (QueryMeasure msr : queryMeasures) { + for (ProjectionMeasure msr : queryMeasures) { DataType dataType = msr.getMeasure().getDataType(); if (dataType == DataTypes.BOOLEAN || dataType == DataTypes.SHORT || dataType == DataTypes.INT || dataType == DataTypes.LONG) { - fields[msr.getQueryOrder()] = + fields[msr.getOrdinal()] = new StructField(msr.getColumnName(), msr.getMeasure().getDataType()); } else if (DataTypes.isDecimal(dataType)) { - fields[msr.getQueryOrder()] = + fields[msr.getOrdinal()] = new StructField(msr.getColumnName(), msr.getMeasure().getDataType()); } else { - fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(), DataTypes.DOUBLE); + fields[msr.getOrdinal()] = new StructField(msr.getColumnName(), DataTypes.DOUBLE); } } diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala index ab3ab5dc922..3c70619bab2 100644 --- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala +++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala @@ -171,7 +171,8 @@ class CarbonMergerRDD[K, V]( LOGGER.info(s"Restructured block exists: $restructuredBlockExists") DataTypeUtil.setDataTypeConverter(new SparkDataTypeConverterImpl) exec = new CarbonCompactionExecutor(segmentMapping, segmentProperties, - carbonTable, dataFileMetadataSegMapping, restructuredBlockExists) + carbonTable, dataFileMetadataSegMapping, restructuredBlockExists, + new SparkDataTypeConverterImpl) // fire a query and get the results. var result2: java.util.List[RawResultIterator] = null diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala index 772f7027151..97be1fba832 100644 --- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala +++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala @@ -336,7 +336,7 @@ class CarbonScanRDD( TaskMetricsMap.getInstance().registerThreadCallback() inputMetricsStats.initBytesReadCallback(context, inputSplit) val iterator = if (inputSplit.getAllSplits.size() > 0) { - val model = format.getQueryModel(inputSplit, attemptContext) + val model = format.createQueryModel(inputSplit, attemptContext) // get RecordReader by FileFormat val reader: RecordReader[Void, Object] = inputSplit.getFileFormat match { case FileFormat.ROW_V1 => diff --git a/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java b/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java index 7d42130b23a..432d50a044d 100644 --- a/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java +++ b/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java @@ -40,7 +40,7 @@ class ColumnarVectorWrapper implements CarbonColumnVector { private DataType blockDataType; - public ColumnarVectorWrapper(ColumnVector columnVector, boolean[] filteredRows) { + ColumnarVectorWrapper(ColumnVector columnVector, boolean[] filteredRows) { this.columnVector = columnVector; this.filteredRows = filteredRows; this.dataType = CarbonScalaUtil.convertSparkToCarbonDataType(columnVector.dataType()); diff --git a/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java b/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java index 5d927df8450..73da87850d4 100644 --- a/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java +++ b/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java @@ -35,8 +35,8 @@ import org.apache.carbondata.core.scan.executor.QueryExecutor; import org.apache.carbondata.core.scan.executor.QueryExecutorFactory; import org.apache.carbondata.core.scan.executor.exception.QueryExecutionException; -import org.apache.carbondata.core.scan.model.QueryDimension; -import org.apache.carbondata.core.scan.model.QueryMeasure; +import org.apache.carbondata.core.scan.model.ProjectionDimension; +import org.apache.carbondata.core.scan.model.ProjectionMeasure; import org.apache.carbondata.core.scan.model.QueryModel; import org.apache.carbondata.core.scan.result.iterator.AbstractDetailQueryResultIterator; import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector; @@ -100,7 +100,8 @@ public VectorizedCarbonRecordReader(QueryModel queryModel, InputMetricsStats inp /** * Implementation of RecordReader API. */ - @Override public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) + @Override + public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException, UnsupportedOperationException { // The input split can contain single HDFS block or multiple blocks, so firstly get all the // blocks and then set them in the query model. @@ -145,7 +146,8 @@ public VectorizedCarbonRecordReader(QueryModel queryModel, InputMetricsStats inp } } - @Override public void close() throws IOException { + @Override + public void close() throws IOException { logStatistics(rowCount, queryModel.getStatisticsRecorder()); if (columnarBatch != null) { columnarBatch.close(); @@ -165,10 +167,13 @@ public VectorizedCarbonRecordReader(QueryModel queryModel, InputMetricsStats inp } } - @Override public boolean nextKeyValue() throws IOException, InterruptedException { + @Override + public boolean nextKeyValue() throws IOException, InterruptedException { resultBatch(); - if (returnColumnarBatch) return nextBatch(); + if (returnColumnarBatch) { + return nextBatch(); + } if (batchIdx >= numBatched) { if (!nextBatch()) return false; @@ -177,7 +182,8 @@ public VectorizedCarbonRecordReader(QueryModel queryModel, InputMetricsStats inp return true; } - @Override public Object getCurrentValue() throws IOException, InterruptedException { + @Override + public Object getCurrentValue() throws IOException, InterruptedException { if (returnColumnarBatch) { int value = columnarBatch.numValidRows(); rowCount += value; @@ -190,11 +196,13 @@ public VectorizedCarbonRecordReader(QueryModel queryModel, InputMetricsStats inp return columnarBatch.getRow(batchIdx - 1); } - @Override public Void getCurrentKey() throws IOException, InterruptedException { + @Override + public Void getCurrentKey() throws IOException, InterruptedException { return null; } - @Override public float getProgress() throws IOException, InterruptedException { + @Override + public float getProgress() throws IOException, InterruptedException { // TODO : Implement it based on total number of rows it is going to retrive. return 0; } @@ -206,44 +214,44 @@ public VectorizedCarbonRecordReader(QueryModel queryModel, InputMetricsStats inp */ private void initBatch(MemoryMode memMode) { - List queryDimension = queryModel.getQueryDimension(); - List queryMeasures = queryModel.getQueryMeasures(); + List queryDimension = queryModel.getProjectionDimensions(); + List queryMeasures = queryModel.getProjectionMeasures(); StructField[] fields = new StructField[queryDimension.size() + queryMeasures.size()]; for (int i = 0; i < queryDimension.size(); i++) { - QueryDimension dim = queryDimension.get(i); + ProjectionDimension dim = queryDimension.get(i); if (dim.getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) { DirectDictionaryGenerator generator = DirectDictionaryKeyGeneratorFactory .getDirectDictionaryGenerator(dim.getDimension().getDataType()); - fields[dim.getQueryOrder()] = new StructField(dim.getColumnName(), + fields[dim.getOrdinal()] = new StructField(dim.getColumnName(), CarbonScalaUtil.convertCarbonToSparkDataType(generator.getReturnType()), true, null); } else if (!dim.getDimension().hasEncoding(Encoding.DICTIONARY)) { - fields[dim.getQueryOrder()] = new StructField(dim.getColumnName(), + fields[dim.getOrdinal()] = new StructField(dim.getColumnName(), CarbonScalaUtil.convertCarbonToSparkDataType(dim.getDimension().getDataType()), true, null); } else if (dim.getDimension().isComplex()) { - fields[dim.getQueryOrder()] = new StructField(dim.getColumnName(), + fields[dim.getOrdinal()] = new StructField(dim.getColumnName(), CarbonScalaUtil.convertCarbonToSparkDataType(dim.getDimension().getDataType()), true, null); } else { - fields[dim.getQueryOrder()] = new StructField(dim.getColumnName(), + fields[dim.getOrdinal()] = new StructField(dim.getColumnName(), CarbonScalaUtil.convertCarbonToSparkDataType(DataTypes.INT), true, null); } } for (int i = 0; i < queryMeasures.size(); i++) { - QueryMeasure msr = queryMeasures.get(i); + ProjectionMeasure msr = queryMeasures.get(i); DataType dataType = msr.getMeasure().getDataType(); if (dataType == DataTypes.BOOLEAN || dataType == DataTypes.SHORT || dataType == DataTypes.INT || dataType == DataTypes.LONG) { - fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(), + fields[msr.getOrdinal()] = new StructField(msr.getColumnName(), CarbonScalaUtil.convertCarbonToSparkDataType(msr.getMeasure().getDataType()), true, null); } else if (DataTypes.isDecimal(dataType)) { - fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(), + fields[msr.getOrdinal()] = new StructField(msr.getColumnName(), new DecimalType(msr.getMeasure().getPrecision(), msr.getMeasure().getScale()), true, null); } else { - fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(), + fields[msr.getOrdinal()] = new StructField(msr.getColumnName(), CarbonScalaUtil.convertCarbonToSparkDataType(DataTypes.DOUBLE), true, null); } } @@ -261,9 +269,8 @@ private void initBatch() { initBatch(DEFAULT_MEMORY_MODE); } - private ColumnarBatch resultBatch() { + private void resultBatch() { if (columnarBatch == null) initBatch(); - return columnarBatch; } /* diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionExecutor.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionExecutor.java index f51ced36a32..6a401d811e8 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionExecutor.java +++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionExecutor.java @@ -34,20 +34,16 @@ import org.apache.carbondata.core.datastore.block.TaskBlockInfo; import org.apache.carbondata.core.metadata.blocklet.DataFileFooter; import org.apache.carbondata.core.metadata.schema.table.CarbonTable; -import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension; -import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure; import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema; import org.apache.carbondata.core.scan.executor.QueryExecutor; import org.apache.carbondata.core.scan.executor.QueryExecutorFactory; import org.apache.carbondata.core.scan.executor.exception.QueryExecutionException; -import org.apache.carbondata.core.scan.model.QueryDimension; -import org.apache.carbondata.core.scan.model.QueryMeasure; import org.apache.carbondata.core.scan.model.QueryModel; -import org.apache.carbondata.core.scan.result.BatchResult; +import org.apache.carbondata.core.scan.result.RowBatch; import org.apache.carbondata.core.scan.result.iterator.RawResultIterator; import org.apache.carbondata.core.util.CarbonProperties; import org.apache.carbondata.core.util.CarbonUtil; -import org.apache.carbondata.core.util.DataTypeUtil; +import org.apache.carbondata.core.util.DataTypeConverter; /** * Executor class for executing the query on the selected segments to be merged. @@ -70,6 +66,9 @@ public class CarbonCompactionExecutor { */ private boolean restructuredBlockExists; + // converter for UTF8String and decimal conversion + private DataTypeConverter dataTypeConverter; + /** * Constructor * @@ -82,13 +81,14 @@ public class CarbonCompactionExecutor { public CarbonCompactionExecutor(Map segmentMapping, SegmentProperties segmentProperties, CarbonTable carbonTable, Map> dataFileMetadataSegMapping, - boolean restructuredBlockExists) { + boolean restructuredBlockExists, DataTypeConverter dataTypeConverter) { this.segmentMapping = segmentMapping; this.destinationSegProperties = segmentProperties; this.carbonTable = carbonTable; this.dataFileMetadataSegMapping = dataFileMetadataSegMapping; this.restructuredBlockExists = restructuredBlockExists; - queryExecutorList = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); + this.queryExecutorList = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); + this.dataTypeConverter = dataTypeConverter; } /** @@ -100,7 +100,9 @@ public List processTableBlocks() throws QueryExecutionExcepti List resultList = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); List list = null; - queryModel = prepareQueryModel(list); + queryModel = carbonTable.createQueryModelWithProjectAllColumns(dataTypeConverter); + queryModel.setReadPageByPage(enablePageLevelReaderForCompaction()); + queryModel.setForcedDetailRawQuery(true); // iterate each seg ID for (Map.Entry taskMap : segmentMapping.entrySet()) { String segmentId = taskMap.getKey(); @@ -156,7 +158,7 @@ private SegmentProperties getSourceSegmentProperties(List listMe * @param blockList * @return */ - private CarbonIterator executeBlockList(List blockList) + private CarbonIterator executeBlockList(List blockList) throws QueryExecutionException, IOException { queryModel.setTableBlockInfos(blockList); QueryExecutor queryExecutor = QueryExecutorFactory.getQueryExecutor(queryModel); @@ -194,48 +196,6 @@ private void clearDictionaryFromQueryModel() { } } - /** - * Preparing of the query model. - * - * @param blockList - * @return - */ - private QueryModel prepareQueryModel(List blockList) { - QueryModel model = new QueryModel(); - model.setTableBlockInfos(blockList); - model.setForcedDetailRawQuery(true); - model.setFilterExpressionResolverTree(null); - model.setConverter(DataTypeUtil.getDataTypeConverter()); - model.setReadPageByPage(enablePageLevelReaderForCompaction()); - - List dims = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - - List dimensions = - carbonTable.getDimensionByTableName(carbonTable.getTableName()); - for (CarbonDimension dim : dimensions) { - // check if dimension is deleted - QueryDimension queryDimension = new QueryDimension(dim.getColName()); - queryDimension.setDimension(dim); - dims.add(queryDimension); - } - model.setQueryDimension(dims); - - List msrs = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - List measures = - carbonTable.getMeasureByTableName(carbonTable.getTableName()); - for (CarbonMeasure carbonMeasure : measures) { - // check if measure is deleted - QueryMeasure queryMeasure = new QueryMeasure(carbonMeasure.getColName()); - queryMeasure.setMeasure(carbonMeasure); - msrs.add(queryMeasure); - } - model.setQueryMeasures(msrs); - model.setQueryId(System.nanoTime() + ""); - model.setAbsoluteTableIdentifier(carbonTable.getAbsoluteTableIdentifier()); - model.setTable(carbonTable); - return model; - } - /** * Whether to enable page level reader for compaction or not. */ diff --git a/processing/src/main/java/org/apache/carbondata/processing/partition/impl/QueryPartitionHelper.java b/processing/src/main/java/org/apache/carbondata/processing/partition/impl/QueryPartitionHelper.java index 79e9e5a8b27..b6f12a56294 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/partition/impl/QueryPartitionHelper.java +++ b/processing/src/main/java/org/apache/carbondata/processing/partition/impl/QueryPartitionHelper.java @@ -23,7 +23,6 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants; import org.apache.carbondata.core.metadata.schema.table.CarbonTable; -import org.apache.carbondata.core.scan.model.CarbonQueryPlan; import org.apache.carbondata.processing.partition.DataPartitioner; import org.apache.carbondata.processing.partition.Partition; @@ -46,9 +45,8 @@ public static QueryPartitionHelper getInstance() { /** * Get partitions applicable for query based on filters applied in query */ - public List getPartitionsForQuery(CarbonQueryPlan queryPlan) { - String tableUniqueName = - CarbonTable.buildUniqueName(queryPlan.getDatabaseName(), queryPlan.getTableName()); + public List getPartitionsForQuery(String databaseName, String tableName) { + String tableUniqueName = CarbonTable.buildUniqueName(databaseName, tableName); DataPartitioner dataPartitioner = partitionerMap.get(tableUniqueName); diff --git a/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/AbstractCarbonQueryExecutor.java b/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/AbstractCarbonQueryExecutor.java index 36e022bab4e..01db4f6fb35 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/AbstractCarbonQueryExecutor.java +++ b/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/AbstractCarbonQueryExecutor.java @@ -18,7 +18,6 @@ package org.apache.carbondata.processing.partition.spliter; import java.io.IOException; -import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -26,19 +25,14 @@ import org.apache.carbondata.common.logging.LogService; import org.apache.carbondata.common.logging.LogServiceFactory; import org.apache.carbondata.core.cache.dictionary.Dictionary; -import org.apache.carbondata.core.constants.CarbonCommonConstants; import org.apache.carbondata.core.datastore.block.TableBlockInfo; import org.apache.carbondata.core.datastore.block.TaskBlockInfo; import org.apache.carbondata.core.metadata.schema.table.CarbonTable; -import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension; -import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure; import org.apache.carbondata.core.scan.executor.QueryExecutor; import org.apache.carbondata.core.scan.executor.QueryExecutorFactory; import org.apache.carbondata.core.scan.executor.exception.QueryExecutionException; -import org.apache.carbondata.core.scan.model.QueryDimension; -import org.apache.carbondata.core.scan.model.QueryMeasure; import org.apache.carbondata.core.scan.model.QueryModel; -import org.apache.carbondata.core.scan.result.BatchResult; +import org.apache.carbondata.core.scan.result.RowBatch; import org.apache.carbondata.core.util.CarbonUtil; public abstract class AbstractCarbonQueryExecutor { @@ -47,8 +41,8 @@ public abstract class AbstractCarbonQueryExecutor { LogServiceFactory.getLogService(AbstractCarbonQueryExecutor.class.getName()); protected CarbonTable carbonTable; protected QueryModel queryModel; - protected QueryExecutor queryExecutor; - protected Map segmentMapping; + private QueryExecutor queryExecutor; + Map segmentMapping; /** * get executor and execute the query model. @@ -56,53 +50,13 @@ public abstract class AbstractCarbonQueryExecutor { * @param blockList * @return */ - protected CarbonIterator executeBlockList(List blockList) + CarbonIterator executeBlockList(List blockList) throws QueryExecutionException, IOException { queryModel.setTableBlockInfos(blockList); this.queryExecutor = QueryExecutorFactory.getQueryExecutor(queryModel); return queryExecutor.execute(queryModel); } - /** - * Preparing of the query model. - * - * @param blockList - * @return - */ - protected QueryModel prepareQueryModel(List blockList) { - QueryModel model = new QueryModel(); - model.setTableBlockInfos(blockList); - model.setForcedDetailRawQuery(true); - model.setFilterExpressionResolverTree(null); - - List dims = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - - List dimensions = - carbonTable.getDimensionByTableName(carbonTable.getTableName()); - for (CarbonDimension dim : dimensions) { - // check if dimension is deleted - QueryDimension queryDimension = new QueryDimension(dim.getColName()); - queryDimension.setDimension(dim); - dims.add(queryDimension); - } - model.setQueryDimension(dims); - - List msrs = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - List measures = - carbonTable.getMeasureByTableName(carbonTable.getTableName()); - for (CarbonMeasure carbonMeasure : measures) { - // check if measure is deleted - QueryMeasure queryMeasure = new QueryMeasure(carbonMeasure.getColName()); - queryMeasure.setMeasure(carbonMeasure); - msrs.add(queryMeasure); - } - model.setQueryMeasures(msrs); - model.setQueryId(System.nanoTime() + ""); - model.setAbsoluteTableIdentifier(carbonTable.getAbsoluteTableIdentifier()); - model.setTable(carbonTable); - return model; - } - /** * Below method will be used * for cleanup diff --git a/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/CarbonSplitExecutor.java b/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/CarbonSplitExecutor.java index 6afec0b8116..b18207d02a9 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/CarbonSplitExecutor.java +++ b/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/CarbonSplitExecutor.java @@ -31,6 +31,7 @@ import org.apache.carbondata.core.metadata.schema.table.CarbonTable; import org.apache.carbondata.core.scan.executor.exception.QueryExecutionException; import org.apache.carbondata.core.scan.result.iterator.PartitionSpliterRawResultIterator; +import org.apache.carbondata.core.util.DataTypeConverterImpl; /** * Used to read carbon blocks when add/split partition @@ -48,7 +49,8 @@ public CarbonSplitExecutor(Map segmentMapping, CarbonTabl public List processDataBlocks(String segmentId) throws QueryExecutionException, IOException { List list = null; - queryModel = prepareQueryModel(list); + queryModel = carbonTable.createQueryModelWithProjectAllColumns(new DataTypeConverterImpl()); + queryModel.setForcedDetailRawQuery(true); List resultList = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); TaskBlockInfo taskBlockInfo = segmentMapping.get(segmentId); diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonQueryUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonQueryUtil.java index ec91472ee8a..4abdf3ccbee 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonQueryUtil.java +++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonQueryUtil.java @@ -24,7 +24,7 @@ import java.util.Map; import org.apache.carbondata.core.constants.CarbonCommonConstants; -import org.apache.carbondata.core.scan.model.CarbonQueryPlan; +import org.apache.carbondata.core.scan.model.QueryProjection; import org.apache.carbondata.processing.partition.Partition; import org.apache.carbondata.processing.partition.impl.DefaultLoadBalancer; import org.apache.carbondata.processing.partition.impl.PartitionMultiFileImpl; @@ -46,7 +46,7 @@ private CarbonQueryUtil() { * It creates the one split for each region server. */ public static synchronized TableSplit[] getTableSplits(String databaseName, String tableName, - CarbonQueryPlan queryPlan) { + QueryProjection queryPlan) { //Just create splits depends on locations of region servers List allPartitions = null; @@ -55,7 +55,7 @@ public static synchronized TableSplit[] getTableSplits(String databaseName, Stri QueryPartitionHelper.getInstance().getAllPartitions(databaseName, tableName); } else { allPartitions = - QueryPartitionHelper.getInstance().getPartitionsForQuery(queryPlan); + QueryPartitionHelper.getInstance().getPartitionsForQuery(databaseName, tableName); } TableSplit[] splits = new TableSplit[allPartitions.size()]; for (int i = 0; i < splits.length; i++) { diff --git a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala index 5e0fce50be7..578f0cda5f3 100644 --- a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala +++ b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala @@ -157,7 +157,7 @@ class StreamHandoffRDD[K, V]( CarbonTableInputFormat.setTableInfo(hadoopConf, carbonTable.getTableInfo) val attemptContext = new TaskAttemptContextImpl(hadoopConf, attemptId) val format = new CarbonTableInputFormat[Array[Object]]() - val model = format.getQueryModel(inputSplit, attemptContext) + val model = format.createQueryModel(inputSplit, attemptContext) val inputFormat = new CarbonStreamInputFormat val streamReader = inputFormat.createRecordReader(inputSplit, attemptContext) .asInstanceOf[CarbonStreamRecordReader]