in core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java [435:610]
private BlockExecutionInfo getBlockExecutionInfoForBlock(QueryModel queryModel,
AbstractIndex blockIndex, int numberOfBlockletToScan, String filePath,
String[] deleteDeltaFiles, Segment segment) {
BlockExecutionInfo blockExecutionInfo = new BlockExecutionInfo();
SegmentProperties segmentProperties = blockIndex.getSegmentProperties();
// set actual query dimensions and measures. It may differ in case of restructure scenarios
RestructureUtil.actualProjectionOfSegment(blockExecutionInfo, queryModel, segmentProperties);
// below is to get only those dimension in query which is present in the
// table block
List<ProjectionDimension> projectDimensions = RestructureUtil
.createDimensionInfoAndGetCurrentBlockQueryDimension(blockExecutionInfo,
blockExecutionInfo.getActualQueryDimensions(), segmentProperties.getDimensions(),
segmentProperties.getComplexDimensions(),
blockExecutionInfo.getActualQueryMeasures().length,
queryModel.getTable().getTableInfo().isTransactionalTable(), queryModel);
boolean isStandardTable = CarbonUtil.isStandardCarbonTable(queryModel.getTable());
String blockId = CarbonUtil
.getBlockId(queryModel.getAbsoluteTableIdentifier(), filePath, segment.getSegmentNo(),
queryModel.getTable().getTableInfo().isTransactionalTable(),
isStandardTable, queryModel.getTable().isHivePartitionTable());
blockExecutionInfo.setBlockId(CarbonTablePath.getShortBlockId(blockId));
blockExecutionInfo.setDeleteDeltaFilePath(deleteDeltaFiles);
blockExecutionInfo.setStartBlockletIndex(0);
blockExecutionInfo.setNumberOfBlockletToScan(numberOfBlockletToScan);
blockExecutionInfo.setProjectionDimensions(projectDimensions
.toArray(new ProjectionDimension[0]));
// get measures present in the current block
List<ProjectionMeasure> projectionMeasures = RestructureUtil
.createMeasureInfoAndGetCurrentBlockQueryMeasures(blockExecutionInfo,
blockExecutionInfo.getActualQueryMeasures(), segmentProperties.getMeasures(),
queryModel.getTable().getTableInfo().isTransactionalTable(), queryModel);
blockExecutionInfo.setProjectionMeasures(
projectionMeasures.toArray(new ProjectionMeasure[projectionMeasures.size()]));
blockExecutionInfo.setDataBlock(blockIndex);
// setting whether raw record query or not
blockExecutionInfo.setRawRecordDetailQuery(queryModel.isForcedDetailRawQuery());
// total number dimension
blockExecutionInfo
.setTotalNumberDimensionToRead(
segmentProperties.getDimensionOrdinalToChunkMapping().size());
blockExecutionInfo.setReadOnlyDelta(queryModel.isReadOnlyDelta());
if (queryModel.isReadPageByPage()) {
blockExecutionInfo.setPrefetchBlocklet(false);
LOGGER.info("Query prefetch is: false, read page by page");
} else {
LOGGER.info("Query prefetch is: " + queryModel.isPreFetchData());
blockExecutionInfo.setPrefetchBlocklet(queryModel.isPreFetchData());
}
// In case of fg index it should not go to direct fill.
boolean fgIndexPathPresent = false;
for (TableBlockInfo blockInfo : queryModel.getTableBlockInfos()) {
fgIndexPathPresent = blockInfo.getIndexWriterPath() != null;
if (fgIndexPathPresent) {
queryModel.setDirectVectorFill(false);
break;
}
}
blockExecutionInfo.setDirectVectorFill(queryModel.isDirectVectorFill());
blockExecutionInfo.setTotalNumberOfMeasureToRead(
segmentProperties.getMeasuresOrdinalToChunkMapping().size());
blockExecutionInfo.setComplexDimensionInfoMap(
QueryUtil.getComplexDimensionsMap(
projectDimensions,
segmentProperties.getDimensionOrdinalToChunkMapping(),
queryProperties.complexFilterDimension));
if (null != queryModel.getIndexFilter()) {
FilterResolverIntf filterResolverIntf;
if (!filePath.startsWith(queryModel.getTable().getTablePath())) {
filterResolverIntf = queryModel.getIndexFilter().getExternalSegmentResolver();
} else {
// loading the filter executor tree for filter evaluation
filterResolverIntf = queryModel.getIndexFilter().getResolver();
}
blockExecutionInfo.setFilterExecutorTree(
FilterUtil.getFilterExecutorTree(filterResolverIntf, segmentProperties,
blockExecutionInfo.getComplexDimensionInfoMap(), false));
}
// expression measure
List<CarbonMeasure> expressionMeasures =
new ArrayList<CarbonMeasure>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
// setting all the dimension chunk indexes to be read from file
int numberOfElementToConsider = 0;
// list of dimensions to be projected
Set<Integer> allProjectionListDimensionIndexes = new LinkedHashSet<>();
// create a list of filter dimensions present in the current block
Set<CarbonDimension> currentBlockFilterDimensions =
getCurrentBlockFilterDimensions(queryProperties.complexFilterDimension, segmentProperties);
int[] dimensionChunkIndexes = QueryUtil.getDimensionChunkIndexes(projectDimensions,
segmentProperties.getDimensionOrdinalToChunkMapping(),
currentBlockFilterDimensions, allProjectionListDimensionIndexes);
ReusableDataBuffer[] dimensionBuffer = new ReusableDataBuffer[reusableDimensionBufferSize];
for (int i = 0; i < dimensionBuffer.length; i++) {
dimensionBuffer[i] = new ReusableDataBuffer();
}
blockExecutionInfo.setDimensionReusableDataBuffer(dimensionBuffer);
int numberOfColumnToBeReadInOneIO = Integer.parseInt(CarbonProperties.getInstance()
.getProperty(CarbonV3DataFormatConstants.NUMBER_OF_COLUMN_TO_READ_IN_IO,
CarbonV3DataFormatConstants.NUMBER_OF_COLUMN_TO_READ_IN_IO_DEFAULT_VALUE));
if (dimensionChunkIndexes.length > 0) {
numberOfElementToConsider = dimensionChunkIndexes[dimensionChunkIndexes.length - 1]
== segmentProperties.getBlockToDimensionOrdinalMapping().size() - 1 ?
dimensionChunkIndexes.length - 1 :
dimensionChunkIndexes.length;
blockExecutionInfo.setAllSelectedDimensionColumnIndexRange(
CarbonUtil.getRangeIndex(dimensionChunkIndexes, numberOfElementToConsider,
numberOfColumnToBeReadInOneIO));
} else {
blockExecutionInfo.setAllSelectedDimensionColumnIndexRange(new int[0][0]);
}
// get the list of updated filter measures present in the current block
Set<CarbonMeasure> filterMeasures =
getCurrentBlockFilterMeasures(queryProperties.filterMeasures, segmentProperties);
// list of measures to be projected
List<Integer> allProjectionListMeasureIndexes = new ArrayList<>();
int[] measureChunkIndexes = QueryUtil.getMeasureChunkIndexes(
projectionMeasures, expressionMeasures,
segmentProperties.getMeasuresOrdinalToChunkMapping(), filterMeasures,
allProjectionListMeasureIndexes);
ReusableDataBuffer[] measureBuffer =
new ReusableDataBuffer[allProjectionListMeasureIndexes.size()];
for (int i = 0; i < measureBuffer.length; i++) {
measureBuffer[i] = new ReusableDataBuffer();
}
blockExecutionInfo.setMeasureReusableDataBuffer(measureBuffer);
if (measureChunkIndexes.length > 0) {
numberOfElementToConsider = measureChunkIndexes[measureChunkIndexes.length - 1]
== segmentProperties.getMeasures().size() - 1 ?
measureChunkIndexes.length - 1 :
measureChunkIndexes.length;
// setting all the measure chunk indexes to be read from file
blockExecutionInfo.setAllSelectedMeasureIndexRange(
CarbonUtil.getRangeIndex(
measureChunkIndexes, numberOfElementToConsider,
numberOfColumnToBeReadInOneIO));
} else {
blockExecutionInfo.setAllSelectedMeasureIndexRange(new int[0][0]);
}
// setting the indexes of list of dimension in projection list
blockExecutionInfo.setProjectionListDimensionIndexes(ArrayUtils.toPrimitive(
allProjectionListDimensionIndexes
.toArray(new Integer[allProjectionListDimensionIndexes.size()])));
// setting the indexes of list of measures in projection list
blockExecutionInfo.setProjectionListMeasureIndexes(ArrayUtils.toPrimitive(
allProjectionListMeasureIndexes
.toArray(new Integer[allProjectionListMeasureIndexes.size()])));
// setting the size of fixed key column (dictionary column)
blockExecutionInfo
.setFixedLengthKeySize(getKeySize(projectDimensions, segmentProperties));
List<Integer> dictionaryColumnChunkIndex = new ArrayList<Integer>();
List<Integer> noDictionaryColumnChunkIndex = new ArrayList<Integer>();
// get the block index to be read from file for query dimension
// for both dictionary columns and no dictionary columns
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
if (!queryModel.isForcedDetailRawQuery()) {
Arrays.sort(queryDictionaryColumnChunkIndexes);
}
blockExecutionInfo.setDictionaryColumnChunkIndex(queryDictionaryColumnChunkIndexes);
// setting the no dictionary column block indexes
blockExecutionInfo.setNoDictionaryColumnChunkIndexes(ArrayUtils.toPrimitive(
noDictionaryColumnChunkIndex.toArray(new Integer[noDictionaryColumnChunkIndex.size()])));
blockExecutionInfo.setComplexColumnParentBlockIndexes(
getComplexDimensionParentBlockIndexes(projectDimensions));
blockExecutionInfo.setVectorBatchCollector(queryModel.isVectorReader());
DataTypeUtil.setDataTypeConverter(queryModel.getConverter());
blockExecutionInfo.setRequiredRowId(queryModel.isRequiredRowId());
return blockExecutionInfo;
}