diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java index 367d00a0fba8..70dadd2f2498 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java @@ -128,7 +128,16 @@ protected BrokerResponse handleRequest(long requestId, String query, SqlNodeAndO Long timeoutMsFromQueryOption = QueryOptionsUtils.getTimeoutMs(queryOptions); queryTimeoutMs = timeoutMsFromQueryOption != null ? timeoutMsFromQueryOption : _brokerTimeoutMs; database = DatabaseUtils.extractDatabaseFromQueryRequest(queryOptions, httpHeaders); - QueryEnvironment queryEnvironment = new QueryEnvironment(database, _tableCache, _workerManager); + boolean inferPartitionHint = _config.getProperty(CommonConstants.Broker.CONFIG_OF_INFER_PARTITION_HINT, + CommonConstants.Broker.DEFAULT_INFER_PARTITION_HINT); + //@formatter:off + QueryEnvironment queryEnvironment = new QueryEnvironment(QueryEnvironment.configBuilder() + .database(database) + .tableCache(_tableCache) + .workerManager(_workerManager) + .defaultInferPartitionHint(inferPartitionHint) + .build()); + //@formatter:on switch (sqlNodeAndOptions.getSqlNode().getKind()) { case EXPLAIN: boolean askServers = QueryOptionsUtils.isExplainAskingServers(queryOptions) diff --git a/pinot-clients/pinot-java-client/pom.xml b/pinot-clients/pinot-java-client/pom.xml index 19eeabd7b9dd..4678af3e4f5e 100644 --- a/pinot-clients/pinot-java-client/pom.xml +++ b/pinot-clients/pinot-java-client/pom.xml @@ -61,4 +61,13 @@ test + + + + pinot-fastdev + + none + + + diff --git a/pinot-clients/pinot-jdbc-client/pom.xml b/pinot-clients/pinot-jdbc-client/pom.xml index d9da07e24590..4dbc070ff367 100644 --- a/pinot-clients/pinot-jdbc-client/pom.xml +++ b/pinot-clients/pinot-jdbc-client/pom.xml @@ -74,5 +74,11 @@ package + + pinot-fastdev + + none + + diff --git a/pinot-common/pom.xml b/pinot-common/pom.xml index 109c3a264927..b11c09cf777f 100644 --- a/pinot-common/pom.xml +++ b/pinot-common/pom.xml @@ -340,6 +340,12 @@ package + + pinot-fastdev + + none + + - - maven-shade-plugin - - - package - - shade - - - - - com - ${shadeBase}.com - - com.google.protobuf.** - com.google.common.** - - - - - - - net.alchim31.maven scala-maven-plugin diff --git a/pinot-connectors/pinot-spark-3-connector/pom.xml b/pinot-connectors/pinot-spark-3-connector/pom.xml index e8393c78c8fb..39881b39547a 100644 --- a/pinot-connectors/pinot-spark-3-connector/pom.xml +++ b/pinot-connectors/pinot-spark-3-connector/pom.xml @@ -35,6 +35,43 @@ + + build-shaded-jar + + + + + maven-shade-plugin + + + package + + shade + + + + + com + ${shadeBase}.com + + com.google.protobuf.** + com.google.common.** + + + + + + + + + + + + pinot-fastdev + + none + + scala-2.12 @@ -62,29 +99,6 @@ - - maven-shade-plugin - - - package - - shade - - - - - com - ${shadeBase}.com - - com.google.protobuf.** - com.google.common.** - - - - - - - net.alchim31.maven scala-maven-plugin diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/IdealStateGroupCommitTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/IdealStateGroupCommitTest.java index ffe39764a41b..40304d14c6a0 100644 --- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/IdealStateGroupCommitTest.java +++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/IdealStateGroupCommitTest.java @@ -18,6 +18,9 @@ */ package org.apache.pinot.controller.helix; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.function.Function; @@ -32,60 +35,93 @@ import org.slf4j.LoggerFactory; import org.testng.Assert; import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; public class IdealStateGroupCommitTest { - private static final Logger LOGGER = LoggerFactory.getLogger(IdealStateGroupCommit.class); + private static final Logger LOGGER = LoggerFactory.getLogger(IdealStateGroupCommitTest.class); private static final ControllerTest TEST_INSTANCE = ControllerTest.getInstance(); - private static final String TABLE_NAME = "potato_OFFLINE"; - private static final int NUM_UPDATES = 2400; + private static final String TABLE_NAME_PREFIX = "potato_"; + private static final int NUM_PROCESSORS = 100; + private static final int NUM_UPDATES = 2000; + private static final int NUM_TABLES = 20; + + private ExecutorService _executorService; @BeforeClass public void setUp() throws Exception { TEST_INSTANCE.setupSharedStateAndValidate(); + _executorService = Executors.newFixedThreadPool(100); + } + + @BeforeMethod + public void beforeTest() { + for (int i = 0; i < NUM_UPDATES; i++) { + String tableName = TABLE_NAME_PREFIX + i + "_OFFLINE"; + IdealState idealState = new IdealState(tableName); + idealState.setStateModelDefRef("OnlineOffline"); + idealState.setRebalanceMode(IdealState.RebalanceMode.CUSTOMIZED); + idealState.setReplicas("1"); + idealState.setNumPartitions(0); + TEST_INSTANCE.getHelixAdmin().addResource(TEST_INSTANCE.getHelixClusterName(), tableName, idealState); + ControllerMetrics.get().removeTableMeter(tableName, ControllerMeter.IDEAL_STATE_UPDATE_SUCCESS); + } + } - IdealState idealState = new IdealState(TABLE_NAME); - idealState.setStateModelDefRef("OnlineOffline"); - idealState.setRebalanceMode(IdealState.RebalanceMode.CUSTOMIZED); - idealState.setReplicas("1"); - idealState.setNumPartitions(0); - TEST_INSTANCE.getHelixAdmin() - .addResource(TEST_INSTANCE.getHelixClusterName(), TABLE_NAME, idealState); + @AfterMethod + public void afterTest() { + for (int i = 0; i < NUM_UPDATES; i++) { + String tableName = TABLE_NAME_PREFIX + i + "_OFFLINE"; + TEST_INSTANCE.getHelixAdmin().dropResource(TEST_INSTANCE.getHelixClusterName(), tableName); + } } @AfterClass public void tearDown() { + _executorService.shutdown(); TEST_INSTANCE.cleanup(); } - @Test + @Test(invocationCount = 5) public void testGroupCommit() throws InterruptedException { - final IdealStateGroupCommit commit = new IdealStateGroupCommit(); - ExecutorService newFixedThreadPool = Executors.newFixedThreadPool(400); + List groupCommitList = new ArrayList<>(); + for (int i = 0; i < NUM_PROCESSORS; i++) { + groupCommitList.add(new IdealStateGroupCommit()); + } for (int i = 0; i < NUM_UPDATES; i++) { - Runnable runnable = new IdealStateUpdater(TEST_INSTANCE.getHelixManager(), commit, TABLE_NAME, i); - newFixedThreadPool.submit(runnable); + for (int j = 0; j < NUM_TABLES; j++) { + String tableName = TABLE_NAME_PREFIX + j + "_OFFLINE"; + IdealStateGroupCommit commit = groupCommitList.get(new Random().nextInt(NUM_PROCESSORS)); + Runnable runnable = new IdealStateUpdater(TEST_INSTANCE.getHelixManager(), commit, tableName, i); + _executorService.submit(runnable); + } } - IdealState idealState = HelixHelper.getTableIdealState(TEST_INSTANCE.getHelixManager(), TABLE_NAME); - while (idealState.getNumPartitions() < NUM_UPDATES) { - Thread.sleep(500); - idealState = HelixHelper.getTableIdealState(TEST_INSTANCE.getHelixManager(), TABLE_NAME); + for (int i = 0; i < NUM_TABLES; i++) { + String tableName = TABLE_NAME_PREFIX + i + "_OFFLINE"; + IdealState idealState = HelixHelper.getTableIdealState(TEST_INSTANCE.getHelixManager(), tableName); + while (idealState.getNumPartitions() < NUM_UPDATES) { + Thread.sleep(500); + idealState = HelixHelper.getTableIdealState(TEST_INSTANCE.getHelixManager(), tableName); + } + Assert.assertEquals(idealState.getNumPartitions(), NUM_UPDATES); + ControllerMetrics controllerMetrics = ControllerMetrics.get(); + long idealStateUpdateSuccessCount = + controllerMetrics.getMeteredTableValue(tableName, ControllerMeter.IDEAL_STATE_UPDATE_SUCCESS).count(); + Assert.assertTrue(idealStateUpdateSuccessCount < NUM_UPDATES); + LOGGER.info("{} IdealState update are successfully commited with {} times zk updates.", NUM_UPDATES, + idealStateUpdateSuccessCount); } - Assert.assertEquals(idealState.getNumPartitions(), NUM_UPDATES); - ControllerMetrics controllerMetrics = ControllerMetrics.get(); - long idealStateUpdateSuccessCount = - controllerMetrics.getMeteredTableValue(TABLE_NAME, ControllerMeter.IDEAL_STATE_UPDATE_SUCCESS).count(); - Assert.assertTrue(idealStateUpdateSuccessCount < NUM_UPDATES); - LOGGER.info("{} IdealState update are successfully commited with {} times zk updates.", NUM_UPDATES, - idealStateUpdateSuccessCount); } } class IdealStateUpdater implements Runnable { + private static final Logger LOGGER = LoggerFactory.getLogger(IdealStateGroupCommitTest.class); + private final HelixManager _helixManager; private final IdealStateGroupCommit _commit; private final String _tableName; @@ -100,13 +136,22 @@ public IdealStateUpdater(HelixManager helixManager, IdealStateGroupCommit commit @Override public void run() { - _commit.commit(_helixManager, _tableName, new Function() { + Function updater = new Function() { @Override public IdealState apply(IdealState idealState) { idealState.setPartitionState("test_id" + _i, "test_id" + _i, "ONLINE"); return idealState; } - }, RetryPolicies.exponentialBackoffRetryPolicy(5, 1000L, 2.0f), false); - HelixHelper.getTableIdealState(_helixManager, _tableName); + }; + + while (true) { + try { + if (_commit.commit(_helixManager, _tableName, updater, RetryPolicies.noDelayRetryPolicy(1), false) != null) { + break; + } + } catch (Throwable e) { + LOGGER.warn("IdealState updater {} failed to commit.", _i, e); + } + } } } diff --git a/pinot-core/pom.xml b/pinot-core/pom.xml index 87048865c2e2..368df3f4024a 100644 --- a/pinot-core/pom.xml +++ b/pinot-core/pom.xml @@ -177,5 +177,11 @@ package + + pinot-fastdev + + none + + diff --git a/pinot-core/src/main/java/org/apache/pinot/core/operator/query/FilteredGroupByOperator.java b/pinot-core/src/main/java/org/apache/pinot/core/operator/query/FilteredGroupByOperator.java index 97ce0e4f6bb5..5c82827cb4d8 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/operator/query/FilteredGroupByOperator.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/operator/query/FilteredGroupByOperator.java @@ -124,33 +124,24 @@ protected GroupByResultsBlock getNextBlock() { // Perform aggregation group-by on all the blocks DefaultGroupByExecutor groupByExecutor; - if (groupKeyGenerator == null) { - // The group key generator should be shared across all AggregationFunctions so that agg results can be - // aligned. Given that filtered aggregations are stored as an iterable of iterables so that all filtered aggs - // with the same filter can share transform blocks, rather than a singular flat iterable in the case where - // aggs are all non-filtered, sharing a GroupKeyGenerator across all aggs cannot be accomplished by allowing - // the GroupByExecutor to have sole ownership of the GroupKeyGenerator. Therefore, we allow constructing a - // GroupByExecutor with a pre-existing GroupKeyGenerator so that the GroupKeyGenerator can be shared across - // loop iterations i.e. across all aggs. - if (aggregationInfo.isUseStarTree()) { - groupByExecutor = - new StarTreeGroupByExecutor(_queryContext, aggregationFunctions, _groupByExpressions, projectOperator); - } else { - groupByExecutor = - new DefaultGroupByExecutor(_queryContext, aggregationFunctions, _groupByExpressions, projectOperator); - } - groupKeyGenerator = groupByExecutor.getGroupKeyGenerator(); + + if (aggregationInfo.isUseStarTree()) { + groupByExecutor = + new StarTreeGroupByExecutor(_queryContext, aggregationFunctions, _groupByExpressions, projectOperator, + groupKeyGenerator); } else { - if (aggregationInfo.isUseStarTree()) { - groupByExecutor = - new StarTreeGroupByExecutor(_queryContext, aggregationFunctions, _groupByExpressions, projectOperator, - groupKeyGenerator); - } else { - groupByExecutor = - new DefaultGroupByExecutor(_queryContext, aggregationFunctions, _groupByExpressions, projectOperator, - groupKeyGenerator); - } + groupByExecutor = + new DefaultGroupByExecutor(_queryContext, aggregationFunctions, _groupByExpressions, projectOperator, + groupKeyGenerator); } + // The group key generator should be shared across all AggregationFunctions so that agg results can be + // aligned. Given that filtered aggregations are stored as an iterable of iterables so that all filtered aggs + // with the same filter can share transform blocks, rather than a singular flat iterable in the case where + // aggs are all non-filtered, sharing a GroupKeyGenerator across all aggs cannot be accomplished by allowing + // the GroupByExecutor to have sole ownership of the GroupKeyGenerator. Therefore, we allow constructing a + // GroupByExecutor with a pre-existing GroupKeyGenerator so that the GroupKeyGenerator can be shared across + // loop iterations i.e. across all aggs. + groupKeyGenerator = groupByExecutor.getGroupKeyGenerator(); int numDocsScanned = 0; ValueBlock valueBlock; diff --git a/pinot-core/src/main/java/org/apache/pinot/core/operator/timeseries/TimeSeriesAggregationOperator.java b/pinot-core/src/main/java/org/apache/pinot/core/operator/timeseries/TimeSeriesAggregationOperator.java index b55605589b2d..a47a05e83af7 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/operator/timeseries/TimeSeriesAggregationOperator.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/operator/timeseries/TimeSeriesAggregationOperator.java @@ -105,6 +105,9 @@ protected TimeSeriesResultsBlock getNextBlock() { case LONG: tagValues[i] = ArrayUtils.toObject(blockValSet.getLongValuesSV()); break; + case INT: + tagValues[i] = ArrayUtils.toObject(blockValSet.getIntValuesSV()); + break; default: throw new NotImplementedException("Can't handle types other than string and long"); } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/AggregationFunctionUtils.java b/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/AggregationFunctionUtils.java index 95020d0a34e1..962fc7425984 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/AggregationFunctionUtils.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/AggregationFunctionUtils.java @@ -39,6 +39,7 @@ import org.apache.pinot.common.request.context.FilterContext; import org.apache.pinot.common.request.context.predicate.Predicate; import org.apache.pinot.common.utils.DataSchema.ColumnDataType; +import org.apache.pinot.common.utils.config.QueryOptionsUtils; import org.apache.pinot.core.common.BlockValSet; import org.apache.pinot.core.common.ObjectSerDeUtils; import org.apache.pinot.core.operator.BaseProjectOperator; @@ -384,7 +385,14 @@ public static List buildFilteredAggregationInfos(SegmentContext } } - if (!nonFilteredFunctions.isEmpty()) { + if (!nonFilteredFunctions.isEmpty() || ((queryContext.getGroupByExpressions() != null) + && !QueryOptionsUtils.isFilteredAggregationsSkipEmptyGroups(queryContext.getQueryOptions()))) { + // If there are no non-filtered aggregation functions for a group by query, we still add a new AggregationInfo + // with an empty AggregationFunction array and the main query filter so that the GroupByExecutor will compute all + // the groups (from the result of applying the main query filter) but no unnecessary additional aggregation will + // be done since the AggregationFunction array is empty. However, if the query option to skip empty groups is + // enabled, we don't do this in order to avoid unnecessary computation of empty groups (which can be very + // expensive if the main filter has high selectivity). AggregationFunction[] aggregationFunctions = nonFilteredFunctions.toArray(new AggregationFunction[0]); aggregationInfos.add( buildAggregationInfo(segmentContext, queryContext, aggregationFunctions, mainFilter, mainFilterOperator, diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/NumericalFilterOptimizer.java b/pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/NumericalFilterOptimizer.java index 7d2336a1a030..961dfa3b71a6 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/NumericalFilterOptimizer.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/optimizer/filter/NumericalFilterOptimizer.java @@ -24,7 +24,6 @@ import org.apache.pinot.common.request.Expression; import org.apache.pinot.common.request.ExpressionType; import org.apache.pinot.common.request.Function; -import org.apache.pinot.common.request.Literal; import org.apache.pinot.spi.data.FieldSpec; import org.apache.pinot.spi.data.FieldSpec.DataType; import org.apache.pinot.spi.data.Schema; @@ -32,34 +31,44 @@ /** - * Numerical expressions of form "column literal", where operator can be '=', '!=', '>', '>=', '<', or '<=', - * can compare a column of one datatype (say INT) with a literal of different datatype (say DOUBLE). These expressions - * can not be evaluated on the Server. Hence, we rewrite such expressions into an equivalent expression whose LHS and - * RHS are of the same datatype. - * + * Numerical expressions of the form "column [operator] literal", where operator can be '=', '!=', '>', '>=', '<', '<=', + * or 'BETWEEN' can compare a column of one datatype (say INT) with a literal of different datatype (say DOUBLE). These + * expressions can not be evaluated on the Server. Hence, we rewrite such expressions into an equivalent expression + * whose LHS and RHS are of the same datatype. + *

* Simple predicate examples: - * 1) WHERE "intColumn = 5.0" gets rewritten to "WHERE intColumn = 5" - * 2) WHERE "intColumn != 5.0" gets rewritten to "WHERE intColumn != 5" - * 3) WHERE "intColumn = 5.5" gets rewritten to "WHERE false" because INT values can not match 5.5. - * 4) WHERE "intColumn = 3000000000" gets rewritten to "WHERE false" because INT values can not match 3000000000. - * 5) WHERE "intColumn != 3000000000" gets rewritten to "WHERE true" because INT values always not equal to 3000000000. - * 6) WHERE "intColumn < 5.1" gets rewritten to "WHERE intColumn <= 5" - * 7) WHERE "intColumn > -3E9" gets rewritten to "WHERE true" because int values are always greater than -3E9. - * + *

    + *
  1. "WHERE intColumn = 5.0" gets rewritten to "WHERE intColumn = 5" + *
  2. "WHERE intColumn != 5.0" gets rewritten to "WHERE intColumn != 5" + *
  3. "WHERE intColumn = 5.5" gets rewritten to "WHERE false" because INT values can not match 5.5. + *
  4. "WHERE intColumn = 3000000000" gets rewritten to "WHERE false" because INT values can not match 3000000000. + *
  5. "WHERE intColumn != 3000000000" gets rewritten to "WHERE true" because INT values always not equal to + * 3000000000. + *
  6. "WHERE intColumn < 5.1" gets rewritten to "WHERE intColumn <= 5" + *
  7. "WHERE intColumn > -3E9" gets rewritten to "WHERE true" because int values are always greater than -3E9. + *
  8. "WHERE intColumn BETWEEN 2.5 AND 7.5" gets rewritten to "WHERE intColumn BETWEEN 3 AND 7" + *
  9. "WHERE intColumn BETWEEN 5.5 AND 3000000000" gets rewritten to "WHERE intColumn BETWEEN 6 AND 2147483647" since + * 3000000000 is greater than Integer.MAX_VALUE. + *
  10. "WHERE intColumn BETWEEN 10 AND 0" gets rewritten to "WHERE false" because lower bound is greater than upper + * bound. + *
+ *

* Compound predicate examples: - * 8) WHERE "intColumn1 = 5.5 AND intColumn2 = intColumn3" + *

    + *
  1. "WHERE intColumn1 = 5.5 AND intColumn2 = intColumn3" * rewrite to "WHERE false AND intColumn2 = intColumn3" * rewrite to "WHERE intColumn2 = intColumn3" - * 9) WHERE "intColumn1 != 5.5 OR intColumn2 = 5000000000" (5000000000 is out of bounds for integer column) + *
  2. "WHERE intColumn1 != 5.5 OR intColumn2 = 5000000000" (5000000000 is out of bounds for integer column) * rewrite to "WHERE true OR false" * rewrite to "WHERE true" * rewrite to query without any WHERE clause. - * + *
+ *

* When entire predicate gets rewritten to false (Example 3 above), the query will not return any data. Hence, it is * better for the Broker itself to return an empty response rather than sending the query to servers for further * evaluation. - * - * TODO: Add support for BETWEEN, IN, and NOT IN operators. + *

+ * TODO: Add support for IN, and NOT IN operators. */ public class NumericalFilterOptimizer extends BaseAndOrBooleanFilterOptimizer { @@ -74,33 +83,39 @@ boolean canBeOptimized(Expression filterExpression, @Nullable Schema schema) { Expression optimizeChild(Expression filterExpression, @Nullable Schema schema) { Function function = filterExpression.getFunctionCall(); FilterKind kind = FilterKind.valueOf(function.getOperator()); + + if (!kind.isRange() && kind != FilterKind.EQUALS && kind != FilterKind.NOT_EQUALS) { + return filterExpression; + } + + List operands = function.getOperands(); + // Verify that LHS is a numeric column and RHS is a literal before rewriting. + Expression lhs = operands.get(0); + Expression rhs = operands.get(1); + + DataType dataType = getDataType(lhs, schema); + if (dataType == null || !dataType.isNumeric() || !rhs.isSetLiteral()) { + // No rewrite here + return filterExpression; + } + switch (kind) { - case IS_NULL: - case IS_NOT_NULL: - // No need to try to optimize IS_NULL and IS_NOT_NULL operations on numerical columns. - break; - default: - List operands = function.getOperands(); - // Verify that LHS is a numeric column and RHS is a numeric literal before rewriting. - Expression lhs = operands.get(0); - Expression rhs = operands.get(1); - if (isNumericLiteral(rhs)) { - DataType dataType = getDataType(lhs, schema); - if (dataType != null && dataType.isNumeric()) { - switch (kind) { - case EQUALS: - case NOT_EQUALS: - return rewriteEqualsExpression(filterExpression, kind, dataType, rhs); - case GREATER_THAN: - case GREATER_THAN_OR_EQUAL: - case LESS_THAN: - case LESS_THAN_OR_EQUAL: - return rewriteRangeExpression(filterExpression, kind, dataType, rhs); - default: - break; - } - } + case BETWEEN: { + return rewriteBetweenExpression(filterExpression, dataType); + } + case EQUALS: + case NOT_EQUALS: + case GREATER_THAN: + case GREATER_THAN_OR_EQUAL: + case LESS_THAN: + case LESS_THAN_OR_EQUAL: { + if (kind.isRange()) { + return rewriteRangeExpression(filterExpression, kind, dataType, rhs); + } else { + return rewriteEqualsExpression(filterExpression, kind, dataType, rhs); } + } + default: break; } return filterExpression; @@ -346,6 +361,159 @@ private static Expression rewriteRangeExpression(Expression range, FilterKind ki return range; } + /** + * Rewrite expressions of the form "column BETWEEN lower AND upper" to ensure that lower and upper bounds are the same + * datatype as the column (or can be cast to the same datatype in the server). + */ + private static Expression rewriteBetweenExpression(Expression between, DataType dataType) { + // TODO: Consider unifying logic with rewriteRangeExpression + List operands = between.getFunctionCall().getOperands(); + Expression lower = operands.get(1); + Expression upper = operands.get(2); + + // The BETWEEN filter predicate currently only supports literals as lower and upper bounds, but we're still checking + // here just in case. + if (lower.isSetLiteral()) { + switch (lower.getLiteral().getSetField()) { + case LONG_VALUE: { + long actual = lower.getLiteral().getLongValue(); + // Other data types can be converted on the server side. + if (dataType == DataType.INT) { + if (actual > Integer.MAX_VALUE) { + // Lower bound literal value is greater than the bounds of INT. + return getExpressionFromBoolean(false); + } + if (actual < Integer.MIN_VALUE) { + lower.getLiteral().setIntValue(Integer.MIN_VALUE); + } + } + break; + } + case DOUBLE_VALUE: { + double actual = lower.getLiteral().getDoubleValue(); + + switch (dataType) { + case INT: { + if (actual > Integer.MAX_VALUE) { + // Lower bound literal value is greater than the bounds of INT. + return getExpressionFromBoolean(false); + } + if (actual < Integer.MIN_VALUE) { + lower.getLiteral().setIntValue(Integer.MIN_VALUE); + } else { + // Double value is in int range + int converted = (int) actual; + int comparison = BigDecimal.valueOf(converted).compareTo(BigDecimal.valueOf(actual)); + if (comparison >= 0) { + lower.getLiteral().setIntValue(converted); + } else { + lower.getLiteral().setIntValue(converted + 1); + } + } + break; + } + case LONG: { + if (actual > Long.MAX_VALUE) { + // Lower bound literal value is greater than the bounds of LONG. + return getExpressionFromBoolean(false); + } + if (actual < Long.MIN_VALUE) { + lower.getLiteral().setLongValue(Long.MIN_VALUE); + } else { + // Double value is in long range + long converted = (long) actual; + int comparison = BigDecimal.valueOf(converted).compareTo(BigDecimal.valueOf(actual)); + if (comparison >= 0) { + lower.getLiteral().setLongValue(converted); + } else { + lower.getLiteral().setLongValue(converted + 1); + } + } + break; + } + default: + // For other numeric data types, the double literal can be converted on the server side. + break; + } + break; + } + default: + break; + } + } + + if (upper.isSetLiteral()) { + switch (upper.getLiteral().getSetField()) { + case LONG_VALUE: { + long actual = upper.getLiteral().getLongValue(); + // Other data types can be converted on the server side. + if (dataType == DataType.INT) { + if (actual < Integer.MIN_VALUE) { + // Upper bound literal value is lesser than the bounds of INT. + return getExpressionFromBoolean(false); + } + if (actual > Integer.MAX_VALUE) { + upper.getLiteral().setIntValue(Integer.MAX_VALUE); + } + } + break; + } + case DOUBLE_VALUE: { + double actual = upper.getLiteral().getDoubleValue(); + + switch (dataType) { + case INT: { + if (actual < Integer.MIN_VALUE) { + // Upper bound literal value is lesser than the bounds of INT. + return getExpressionFromBoolean(false); + } + if (actual > Integer.MAX_VALUE) { + upper.getLiteral().setIntValue(Integer.MAX_VALUE); + } else { + // Double value is in int range + int converted = (int) actual; + int comparison = BigDecimal.valueOf(converted).compareTo(BigDecimal.valueOf(actual)); + if (comparison <= 0) { + upper.getLiteral().setIntValue(converted); + } else { + upper.getLiteral().setIntValue(converted - 1); + } + } + break; + } + case LONG: { + if (actual < Long.MIN_VALUE) { + // Upper bound literal value is lesser than the bounds of LONG. + return getExpressionFromBoolean(false); + } + if (actual > Long.MAX_VALUE) { + upper.getLiteral().setLongValue(Long.MAX_VALUE); + } else { + // Double value is in long range + long converted = (long) actual; + int comparison = BigDecimal.valueOf(converted).compareTo(BigDecimal.valueOf(actual)); + if (comparison <= 0) { + upper.getLiteral().setLongValue(converted); + } else { + upper.getLiteral().setLongValue(converted - 1); + } + } + break; + } + default: + // For other numeric data types, the double literal can be converted on the server side. + break; + } + break; + } + default: + break; + } + } + + return between; + } + /** * Helper function to rewrite range operator of a range expression. * @param range Range expression. @@ -356,9 +524,9 @@ private static void rewriteRangeOperator(Expression range, FilterKind kind, int if (comparison > 0) { // Literal value is greater than the converted value, so rewrite: // "column > literal" to "column > converted" - // "column >= literal" to "column >= converted" + // "column >= literal" to "column > converted" // "column < literal" to "column <= converted" - // "column <= literal" to "column < converted" + // "column <= literal" to "column <= converted" if (kind == FilterKind.GREATER_THAN || kind == FilterKind.GREATER_THAN_OR_EQUAL) { range.getFunctionCall().setOperator(FilterKind.GREATER_THAN.name()); } else if (kind == FilterKind.LESS_THAN || kind == FilterKind.LESS_THAN_OR_EQUAL) { @@ -413,21 +581,4 @@ private static DataType getDataType(Expression expression, Schema schema) { } return null; } - - /** @return true if expression is a numeric literal; otherwise, false. */ - private static boolean isNumericLiteral(Expression expression) { - if (expression.getType() == ExpressionType.LITERAL) { - Literal._Fields type = expression.getLiteral().getSetField(); - switch (type) { - case INT_VALUE: - case LONG_VALUE: - case FLOAT_VALUE: - case DOUBLE_VALUE: - return true; - default: - break; - } - } - return false; - } } diff --git a/pinot-core/src/test/java/org/apache/pinot/core/query/optimizer/filter/NumericalFilterOptimizerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/query/optimizer/filter/NumericalFilterOptimizerTest.java index e947740e2423..363ef8887db2 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/query/optimizer/filter/NumericalFilterOptimizerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/query/optimizer/filter/NumericalFilterOptimizerTest.java @@ -295,6 +295,74 @@ public void testRangeRewrites() { + ".223372036854776E18>)]))"); } + @Test + public void testBetweenRewrites() { + // Test LONG literal with INT column. + Assert.assertEquals(rewrite("SELECT * FROM testTable WHERE intColumn BETWEEN 3000000000 AND 4000000000"), + "Expression(type:LITERAL, literal:)"); + + Assert.assertEquals(rewrite("SELECT * FROM testTable WHERE intColumn BETWEEN -4000000000 AND -3000000000"), + "Expression(type:LITERAL, literal:)"); + + // No rewrite + Assert.assertEquals(rewrite("SELECT * FROM testTable WHERE intColumn BETWEEN -2000000000 AND 2000000000"), + "Expression(type:FUNCTION, functionCall:Function(operator:BETWEEN, operands:[Expression(type:IDENTIFIER, " + + "identifier:Identifier(name:intColumn)), Expression(type:LITERAL, literal:), Expression(type:LITERAL, literal:)]))"); + + // Test INT column with DOUBLE lower bound greater than Integer.MAX_VALUE. + Assert.assertEquals(rewrite("SELECT * FROM testTable WHERE intColumn BETWEEN 3000000000.0 AND 4000000000.0"), + "Expression(type:LITERAL, literal:)"); + // Test INT column with DOUBLE upper bound lesser than Integer.MIN_VALUE. + Assert.assertEquals(rewrite("SELECT * FROM testTable WHERE intColumn BETWEEN -4000000000.0 AND -3000000000.0"), + "Expression(type:LITERAL, literal:)"); + // Test INT column with LONG lower bound lesser than Integer.MIN_VALUE. + Assert.assertEquals(rewrite("SELECT * FROM testTable WHERE intColumn BETWEEN -4000000000 AND 0"), + "Expression(type:FUNCTION, functionCall:Function(operator:BETWEEN, operands:[Expression(type:IDENTIFIER, " + + "identifier:Identifier(name:intColumn)), Expression(type:LITERAL, literal:), Expression(type:LITERAL, literal:)]))"); + // Test INT column with LONG upper bound greater than Integer.MAX_VALUE. + Assert.assertEquals(rewrite("SELECT * FROM testTable WHERE intColumn BETWEEN 0 AND 4000000000"), + "Expression(type:FUNCTION, functionCall:Function(operator:BETWEEN, operands:[Expression(type:IDENTIFIER, " + + "identifier:Identifier(name:intColumn)), Expression(type:LITERAL, literal:), Expression(type:LITERAL, literal:)]))"); + + // Test LONG column with DOUBLE lower bound greater than Long.MAX_VALUE. + Assert.assertEquals( + rewrite("SELECT * FROM testTable WHERE longColumn BETWEEN 9323372036854775808.0 AND 9323372036854775809.0"), + "Expression(type:LITERAL, literal:)"); + // Test LONG column with DOUBLE upper bound lesser than Long.MIN_VALUE. + Assert.assertEquals( + rewrite("SELECT * FROM testTable WHERE longColumn BETWEEN -9323372036854775809.0 AND -9323372036854775808.0"), + "Expression(type:LITERAL, literal:)"); + // Test LONG column with DOUBLE lower bound lesser than Long.MIN_VALUE. + Assert.assertEquals(rewrite("SELECT * FROM testTable WHERE longColumn BETWEEN -9323372036854775809.0 AND 0"), + "Expression(type:FUNCTION, functionCall:Function(operator:BETWEEN, operands:[Expression(type:IDENTIFIER, " + + "identifier:Identifier(name:longColumn)), Expression(type:LITERAL, literal:), Expression(type:LITERAL, literal:)]))"); + // Test LONG column with DOUBLE upper bound greater than Long.MAX_VALUE. + Assert.assertEquals(rewrite("SELECT * FROM testTable WHERE longColumn BETWEEN 0 AND 9323372036854775808.0"), + "Expression(type:FUNCTION, functionCall:Function(operator:BETWEEN, operands:[Expression(type:IDENTIFIER, " + + "identifier:Identifier(name:longColumn)), Expression(type:LITERAL, literal:), " + + "Expression(type:LITERAL, literal:)]))"); + + // Test DOUBLE literal rewrite for INT and LONG columns. + Assert.assertEquals(rewrite("SELECT * FROM testTable WHERE intColumn BETWEEN 2.5 AND 7.5"), + "Expression(type:FUNCTION, functionCall:Function(operator:BETWEEN, operands:[Expression(type:IDENTIFIER, " + + "identifier:Identifier(name:intColumn)), Expression(type:LITERAL, literal:), " + + "Expression(type:LITERAL, literal:)]))"); + + Assert.assertEquals(rewrite("SELECT * FROM testTable WHERE longColumn BETWEEN 2.1 AND 7.9"), + "Expression(type:FUNCTION, functionCall:Function(operator:BETWEEN, operands:[Expression(type:IDENTIFIER, " + + "identifier:Identifier(name:longColumn)), Expression(type:LITERAL, literal:), " + + "Expression(type:LITERAL, literal:)]))"); + + Assert.assertEquals(rewrite("SELECT * FROM testTable WHERE intColumn BETWEEN -5.0 AND -3.5"), + "Expression(type:FUNCTION, functionCall:Function(operator:BETWEEN, operands:[Expression(type:IDENTIFIER, " + + "identifier:Identifier(name:intColumn)), Expression(type:LITERAL, literal:), " + + "Expression(type:LITERAL, literal:)]))"); + } + @Test public void testNull() { // Test column IS NOT NULL. diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/FilteredAggregationsTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/FilteredAggregationsTest.java index 02a9a05395ff..e253dce452f5 100644 --- a/pinot-core/src/test/java/org/apache/pinot/queries/FilteredAggregationsTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/queries/FilteredAggregationsTest.java @@ -41,6 +41,7 @@ import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.data.readers.GenericRow; import org.apache.pinot.spi.data.readers.RecordReader; +import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.builder.TableConfigBuilder; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; @@ -394,9 +395,9 @@ public void testGroupBy() { @Test public void testGroupByMultipleColumns() { - String filterQuery = - "SELECT SUM(INT_COL) FILTER(WHERE INT_COL > 25000) testSum FROM MyTable GROUP BY BOOLEAN_COL, STRING_COL " - + "ORDER BY BOOLEAN_COL, STRING_COL"; + String filterQuery = "SET " + CommonConstants.Broker.Request.QueryOptionKey.FILTERED_AGGREGATIONS_SKIP_EMPTY_GROUPS + + "=true; SELECT SUM(INT_COL) FILTER(WHERE INT_COL > 25000) testSum FROM MyTable GROUP BY BOOLEAN_COL, " + + "STRING_COL ORDER BY BOOLEAN_COL, STRING_COL"; String nonFilterQuery = "SELECT SUM(INT_COL) testSum FROM MyTable WHERE INT_COL > 25000 GROUP BY BOOLEAN_COL, STRING_COL " + "ORDER BY BOOLEAN_COL, STRING_COL"; diff --git a/pinot-distribution/pom.xml b/pinot-distribution/pom.xml index 126d7727baae..9c8f1f054963 100644 --- a/pinot-distribution/pom.xml +++ b/pinot-distribution/pom.xml @@ -166,5 +166,12 @@ + + + pinot-fastdev + + none + + diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineExplainIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineExplainIntegrationTest.java index 3c8e931616e3..8303a583d382 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineExplainIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineExplainIntegrationTest.java @@ -83,7 +83,7 @@ public void simpleQuery() { + "PinotLogicalExchange(distribution=[broadcast])\n" + " LeafStageCombineOperator(table=[mytable])\n" + " StreamingInstanceResponse\n" - + " StreamingCombineSelect(repeated=[12])\n" + + " StreamingCombineSelect\n" + " SelectStreaming(table=[mytable], totalDocs=[115545])\n" + " Transform(expressions=[['1']])\n" + " Project(columns=[[]])\n" @@ -97,77 +97,70 @@ public void simpleQueryVerbose() { explainVerbose("SELECT 1 FROM mytable", //@formatter:off "Execution Plan\n" - + "IntermediateCombine\n" - + " Alternative(servers=[1])\n" - + " PinotLogicalExchange(distribution=[broadcast])\n" - + " LeafStageCombineOperator(table=[mytable])\n" - + " StreamingInstanceResponse\n" - + " StreamingCombineSelect\n" - + " SelectStreaming(segment=[any], table=[mytable], totalDocs=[any])\n" - + " Transform(expressions=[['1']])\n" - + " Project(columns=[[]])\n" - + " DocIdSet(maxDocs=[10000])\n" - + " FilterMatchEntireSegment(numDocs=[any])\n" - + " SelectStreaming(segment=[any], table=[mytable], totalDocs=[any])\n" - + " Transform(expressions=[['1']])\n" - + " Project(columns=[[]])\n" - + " DocIdSet(maxDocs=[10000])\n" - + " FilterMatchEntireSegment(numDocs=[any])\n" - + " SelectStreaming(segment=[any], table=[mytable], totalDocs=[any])\n" - + " Transform(expressions=[['1']])\n" - + " Project(columns=[[]])\n" - + " DocIdSet(maxDocs=[10000])\n" - + " FilterMatchEntireSegment(numDocs=[any])\n" - + " SelectStreaming(segment=[any], table=[mytable], totalDocs=[any])\n" - + " Transform(expressions=[['1']])\n" - + " Project(columns=[[]])\n" - + " DocIdSet(maxDocs=[10000])\n" - + " FilterMatchEntireSegment(numDocs=[any])\n" - + " SelectStreaming(segment=[any], table=[mytable], totalDocs=[any])\n" - + " Transform(expressions=[['1']])\n" - + " Project(columns=[[]])\n" - + " DocIdSet(maxDocs=[10000])\n" - + " FilterMatchEntireSegment(numDocs=[any])\n" - + " SelectStreaming(segment=[any], table=[mytable], totalDocs=[any])\n" - + " Transform(expressions=[['1']])\n" - + " Project(columns=[[]])\n" - + " DocIdSet(maxDocs=[10000])\n" - + " FilterMatchEntireSegment(numDocs=[any])\n" - + " Alternative(servers=[1])\n" - + " PinotLogicalExchange(distribution=[broadcast])\n" - + " LeafStageCombineOperator(table=[mytable])\n" - + " StreamingInstanceResponse\n" - + " StreamingCombineSelect\n" - + " SelectStreaming(segment=[any], table=[mytable], totalDocs=[any])\n" - + " Transform(expressions=[['1']])\n" - + " Project(columns=[[]])\n" - + " DocIdSet(maxDocs=[10000])\n" - + " FilterMatchEntireSegment(numDocs=[any])\n" - + " SelectStreaming(segment=[any], table=[mytable], totalDocs=[any])\n" - + " Transform(expressions=[['1']])\n" - + " Project(columns=[[]])\n" - + " DocIdSet(maxDocs=[10000])\n" - + " FilterMatchEntireSegment(numDocs=[any])\n" - + " SelectStreaming(segment=[any], table=[mytable], totalDocs=[any])\n" - + " Transform(expressions=[['1']])\n" - + " Project(columns=[[]])\n" - + " DocIdSet(maxDocs=[10000])\n" - + " FilterMatchEntireSegment(numDocs=[any])\n" - + " SelectStreaming(segment=[any], table=[mytable], totalDocs=[any])\n" - + " Transform(expressions=[['1']])\n" - + " Project(columns=[[]])\n" - + " DocIdSet(maxDocs=[10000])\n" - + " FilterMatchEntireSegment(numDocs=[any])\n" - + " SelectStreaming(segment=[any], table=[mytable], totalDocs=[any])\n" - + " Transform(expressions=[['1']])\n" - + " Project(columns=[[]])\n" - + " DocIdSet(maxDocs=[10000])\n" - + " FilterMatchEntireSegment(numDocs=[any])\n" - + " SelectStreaming(segment=[any], table=[mytable], totalDocs=[any])\n" - + " Transform(expressions=[['1']])\n" - + " Project(columns=[[]])\n" - + " DocIdSet(maxDocs=[10000])\n" - + " FilterMatchEntireSegment(numDocs=[any])\n"); + + "PinotLogicalExchange(distribution=[broadcast])\n" + + " LeafStageCombineOperator(table=[mytable])\n" + + " StreamingInstanceResponse\n" + + " StreamingCombineSelect\n" + + " SelectStreaming(segment=[any], table=[mytable], totalDocs=[any])\n" + + " Transform(expressions=[['1']])\n" + + " Project(columns=[[]])\n" + + " DocIdSet(maxDocs=[10000])\n" + + " FilterMatchEntireSegment(numDocs=[any])\n" + + " SelectStreaming(segment=[any], table=[mytable], totalDocs=[any])\n" + + " Transform(expressions=[['1']])\n" + + " Project(columns=[[]])\n" + + " DocIdSet(maxDocs=[10000])\n" + + " FilterMatchEntireSegment(numDocs=[any])\n" + + " SelectStreaming(segment=[any], table=[mytable], totalDocs=[any])\n" + + " Transform(expressions=[['1']])\n" + + " Project(columns=[[]])\n" + + " DocIdSet(maxDocs=[10000])\n" + + " FilterMatchEntireSegment(numDocs=[any])\n" + + " SelectStreaming(segment=[any], table=[mytable], totalDocs=[any])\n" + + " Transform(expressions=[['1']])\n" + + " Project(columns=[[]])\n" + + " DocIdSet(maxDocs=[10000])\n" + + " FilterMatchEntireSegment(numDocs=[any])\n" + + " SelectStreaming(segment=[any], table=[mytable], totalDocs=[any])\n" + + " Transform(expressions=[['1']])\n" + + " Project(columns=[[]])\n" + + " DocIdSet(maxDocs=[10000])\n" + + " FilterMatchEntireSegment(numDocs=[any])\n" + + " SelectStreaming(segment=[any], table=[mytable], totalDocs=[any])\n" + + " Transform(expressions=[['1']])\n" + + " Project(columns=[[]])\n" + + " DocIdSet(maxDocs=[10000])\n" + + " FilterMatchEntireSegment(numDocs=[any])\n" + + " SelectStreaming(segment=[any], table=[mytable], totalDocs=[any])\n" + + " Transform(expressions=[['1']])\n" + + " Project(columns=[[]])\n" + + " DocIdSet(maxDocs=[10000])\n" + + " FilterMatchEntireSegment(numDocs=[any])\n" + + " SelectStreaming(segment=[any], table=[mytable], totalDocs=[any])\n" + + " Transform(expressions=[['1']])\n" + + " Project(columns=[[]])\n" + + " DocIdSet(maxDocs=[10000])\n" + + " FilterMatchEntireSegment(numDocs=[any])\n" + + " SelectStreaming(segment=[any], table=[mytable], totalDocs=[any])\n" + + " Transform(expressions=[['1']])\n" + + " Project(columns=[[]])\n" + + " DocIdSet(maxDocs=[10000])\n" + + " FilterMatchEntireSegment(numDocs=[any])\n" + + " SelectStreaming(segment=[any], table=[mytable], totalDocs=[any])\n" + + " Transform(expressions=[['1']])\n" + + " Project(columns=[[]])\n" + + " DocIdSet(maxDocs=[10000])\n" + + " FilterMatchEntireSegment(numDocs=[any])\n" + + " SelectStreaming(segment=[any], table=[mytable], totalDocs=[any])\n" + + " Transform(expressions=[['1']])\n" + + " Project(columns=[[]])\n" + + " DocIdSet(maxDocs=[10000])\n" + + " FilterMatchEntireSegment(numDocs=[any])\n" + + " SelectStreaming(segment=[any], table=[mytable], totalDocs=[any])\n" + + " Transform(expressions=[['1']])\n" + + " Project(columns=[[]])\n" + + " DocIdSet(maxDocs=[10000])\n" + + " FilterMatchEntireSegment(numDocs=[any])\n"); //@formatter:on } diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java index 48cd23ea7d91..b450612d45e5 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineIntegrationTest.java @@ -140,11 +140,6 @@ protected void setupTenants() throws IOException { } -// @Override -// protected boolean useMultiStageQueryEngine() { -// return true; -// } - @BeforeMethod @Override public void resetMultiStage() { @@ -1043,6 +1038,46 @@ public void testMVNumericCastInFilter() throws Exception { assertEquals(jsonNode.get("resultTable").get("rows").get(0).get(0).asInt(), 15482); } + @Test + public void testFilteredAggregationWithNoValueMatchingAggregationFilterDefault() + throws Exception { + // Use a hint to ensure that the aggregation will not be pushed to the leaf stage, so that we can test the + // MultistageGroupByExecutor + String sqlQuery = "SELECT /*+ aggOptions(is_skip_leaf_stage_group_by='true') */" + + "AirlineID, COUNT(*) FILTER (WHERE Origin = 'garbage') FROM mytable WHERE AirlineID > 20000 GROUP BY " + + "AirlineID"; + JsonNode result = postQuery(sqlQuery); + assertNoError(result); + // Ensure that result set is not empty + assertTrue(result.get("numRowsResultSet").asInt() > 0); + + // Ensure that the count is 0 for all groups (because the aggregation filter does not match any rows) + JsonNode rows = result.get("resultTable").get("rows"); + for (int i = 0; i < rows.size(); i++) { + assertEquals(rows.get(i).get(1).asInt(), 0); + // Ensure that the main filter was applied + assertTrue(rows.get(i).get(0).asInt() > 20000); + } + } + + @Test + public void testFilteredAggregationWithNoValueMatchingAggregationFilterWithOption() + throws Exception { + // Use a hint to ensure that the aggregation will not be pushed to the leaf stage, so that we can test the + // MultistageGroupByExecutor + String sqlQuery = "SET " + CommonConstants.Broker.Request.QueryOptionKey.FILTERED_AGGREGATIONS_SKIP_EMPTY_GROUPS + + "=true; SELECT /*+ aggOptions(is_skip_leaf_stage_group_by='true') */" + + "AirlineID, COUNT(*) FILTER (WHERE Origin = 'garbage') FROM mytable WHERE AirlineID > 20000 GROUP BY " + + "AirlineID"; + + JsonNode result = postQuery(sqlQuery); + assertNoError(result); + + // Result set will be empty since the aggregation filter does not match any rows, and we've set the query option to + // skip empty groups + assertEquals(result.get("numRowsResultSet").asInt(), 0); + } + @Override protected String getTableName() { return _tableName; diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java index 25a75352f723..0fa4868179ca 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java @@ -3722,4 +3722,44 @@ public void testSkipIndexes(boolean useMultiStageQueryEngine) updateTableConfig(tableConfig); reloadAllSegments(TEST_UPDATED_RANGE_INDEX_QUERY, true, numTotalDocs); } + + @Test(dataProvider = "useBothQueryEngines") + public void testFilteredAggregationWithNoValueMatchingAggregationFilterDefault(boolean useMultiStageQueryEngine) + throws Exception { + setUseMultiStageQueryEngine(useMultiStageQueryEngine); + + String sqlQuery = + "SELECT AirlineID, COUNT(*) FILTER (WHERE Origin = 'garbage') FROM mytable WHERE AirlineID > 20000 GROUP BY " + + "AirlineID"; + + JsonNode result = postQuery(sqlQuery); + assertNoError(result); + + // Ensure that result set is not empty since all groups should be computed by default + assertTrue(result.get("numRowsResultSet").asInt() > 0); + + // Ensure that the count is 0 for all groups (because the aggregation filter does not match any rows) + JsonNode rows = result.get("resultTable").get("rows"); + for (int i = 0; i < rows.size(); i++) { + assertEquals(rows.get(i).get(1).asInt(), 0); + // Ensure that the main filter was applied + assertTrue(rows.get(i).get(0).asInt() > 20000); + } + } + + @Test(dataProvider = "useBothQueryEngines") + public void testFilteredAggregationWithNoValueMatchingAggregationFilterWithOption(boolean useMultiStageQueryEngine) + throws Exception { + setUseMultiStageQueryEngine(useMultiStageQueryEngine); + String sqlQuery = + "SET " + CommonConstants.Broker.Request.QueryOptionKey.FILTERED_AGGREGATIONS_SKIP_EMPTY_GROUPS + "=true; " + + "SELECT AirlineID, COUNT(*) FILTER (WHERE Origin = 'garbage') FROM mytable WHERE AirlineID > 20000 " + + "GROUP BY AirlineID"; + JsonNode result = postQuery(sqlQuery); + assertNoError(result); + + // Result set will be empty since the aggregation filter does not match any rows, and we've set the option to skip + // empty groups + assertEquals(result.get("numRowsResultSet").asInt(), 0); + } } diff --git a/pinot-perf/pom.xml b/pinot-perf/pom.xml index 8638af7bd543..af314abdc49b 100644 --- a/pinot-perf/pom.xml +++ b/pinot-perf/pom.xml @@ -192,28 +192,46 @@ - - org.apache.maven.plugins - maven-shade-plugin - - - - package - - shade - - - - - - org.openjdk.jmh.Main - - - benchmarks - - - - + + + build-shaded-jar + + true + + + + + org.apache.maven.plugins + maven-shade-plugin + + + + package + + shade + + + + + + org.openjdk.jmh.Main + + + benchmarks + + + + + + + + + pinot-fastdev + + none + + + diff --git a/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkColumnValueSegmentPruner.java b/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkColumnValueSegmentPruner.java index d73da4bbbac5..f261c6231bae 100644 --- a/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkColumnValueSegmentPruner.java +++ b/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkColumnValueSegmentPruner.java @@ -23,10 +23,8 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.function.LongSupplier; @@ -43,7 +41,6 @@ import org.apache.pinot.segment.spi.IndexSegment; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.segment.spi.index.startree.AggregationFunctionColumnPair; -import org.apache.pinot.spi.config.table.BloomFilterConfig; import org.apache.pinot.spi.config.table.FieldConfig; import org.apache.pinot.spi.config.table.StarTreeIndexConfig; import org.apache.pinot.spi.config.table.TableConfig; @@ -79,7 +76,6 @@ @Measurement(iterations = 5, time = 2) @State(Scope.Benchmark) public class BenchmarkColumnValueSegmentPruner { - public static final String QUERY_1 = "SELECT * FROM MyTable WHERE SORTED_COL IN (1, 2, 3, 4)"; @Param({"10"}) @@ -113,26 +109,32 @@ public static void main(String[] args) private static final String NO_INDEX_INT_COL_NAME = "NO_INDEX_INT_COL"; private static final String NO_INDEX_STRING_COL = "NO_INDEX_STRING_COL"; private static final String LOW_CARDINALITY_STRING_COL = "LOW_CARDINALITY_STRING_COL"; - + private static final List FIELD_CONFIGS = new ArrayList<>(); + private static final TableConfig TABLE_CONFIG = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME) + .setInvertedIndexColumns(List.of(INT_COL_NAME, LOW_CARDINALITY_STRING_COL)).setFieldConfigList(FIELD_CONFIGS) + .setNoDictionaryColumns(List.of(RAW_INT_COL_NAME, RAW_STRING_COL_NAME)).setSortedColumn(SORTED_COL_NAME) + .setVarLengthDictionaryColumns(Collections.singletonList(SORTED_COL_NAME)) + .setRangeIndexColumns(Arrays.asList(INT_COL_NAME, LOW_CARDINALITY_STRING_COL)) + .setBloomFilterColumns(Collections.singletonList(SORTED_COL_NAME)).setStarTreeIndexConfigs( + Collections.singletonList(new StarTreeIndexConfig(List.of(SORTED_COL_NAME, INT_COL_NAME), null, + Collections.singletonList( + new AggregationFunctionColumnPair(AggregationFunctionType.SUM, RAW_INT_COL_NAME).toColumnName()), + null, Integer.MAX_VALUE))).build(); + private static final Schema SCHEMA = new Schema.SchemaBuilder().setSchemaName(TABLE_NAME) + .addSingleValueDimension(SORTED_COL_NAME, FieldSpec.DataType.INT) + .addSingleValueDimension(NO_INDEX_INT_COL_NAME, FieldSpec.DataType.INT) + .addSingleValueDimension(RAW_INT_COL_NAME, FieldSpec.DataType.INT) + .addSingleValueDimension(INT_COL_NAME, FieldSpec.DataType.INT) + .addSingleValueDimension(RAW_STRING_COL_NAME, FieldSpec.DataType.STRING) + .addSingleValueDimension(NO_INDEX_STRING_COL, FieldSpec.DataType.STRING) + .addSingleValueDimension(LOW_CARDINALITY_STRING_COL, FieldSpec.DataType.STRING).build(); @Setup public void setUp() throws Exception { _supplier = Distribution.createLongSupplier(42, _scenario); FileUtils.deleteQuietly(INDEX_DIR); - - IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig(); - - Set invertedIndexCols = new HashSet<>(); - invertedIndexCols.add(INT_COL_NAME); - invertedIndexCols.add(LOW_CARDINALITY_STRING_COL); - - Map bloomFilterConfigMap = new HashMap<>(); - bloomFilterConfigMap.put(SORTED_COL_NAME, new BloomFilterConfig(BloomFilterConfig.DEFAULT_FPP, 10000, false)); - - indexLoadingConfig.setRangeIndexColumns(invertedIndexCols); - indexLoadingConfig.setInvertedIndexColumns(invertedIndexCols); - indexLoadingConfig.setBloomFilterConfigs(bloomFilterConfigMap); + IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig(TABLE_CONFIG, SCHEMA); _indexSegments = new ArrayList<>(); for (int i = 0; i < _numSegments; i++) { @@ -178,30 +180,7 @@ private List createTestData(int numRows) { private void buildSegment(String segmentName) throws Exception { List rows = createTestData(_numRows); - List fieldConfigs = new ArrayList<>(); - - TableConfig tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME) - .setInvertedIndexColumns(Collections.singletonList(INT_COL_NAME)) - .setFieldConfigList(fieldConfigs) - .setNoDictionaryColumns(Arrays.asList(RAW_INT_COL_NAME, RAW_STRING_COL_NAME)) - .setSortedColumn(SORTED_COL_NAME) - .setVarLengthDictionaryColumns(Collections.singletonList(SORTED_COL_NAME)) - .setBloomFilterColumns(Collections.singletonList(SORTED_COL_NAME)) - .setStarTreeIndexConfigs(Collections.singletonList( - new StarTreeIndexConfig(Arrays.asList(SORTED_COL_NAME, INT_COL_NAME), null, Collections.singletonList( - new AggregationFunctionColumnPair(AggregationFunctionType.SUM, RAW_INT_COL_NAME).toColumnName()), null, - Integer.MAX_VALUE))) - .build(); - Schema schema = new Schema.SchemaBuilder().setSchemaName(TABLE_NAME) - .addSingleValueDimension(SORTED_COL_NAME, FieldSpec.DataType.INT) - .addSingleValueDimension(NO_INDEX_INT_COL_NAME, FieldSpec.DataType.INT) - .addSingleValueDimension(RAW_INT_COL_NAME, FieldSpec.DataType.INT) - .addSingleValueDimension(INT_COL_NAME, FieldSpec.DataType.INT) - .addSingleValueDimension(RAW_STRING_COL_NAME, FieldSpec.DataType.STRING) - .addSingleValueDimension(NO_INDEX_STRING_COL, FieldSpec.DataType.STRING) - .addSingleValueDimension(LOW_CARDINALITY_STRING_COL, FieldSpec.DataType.STRING) - .build(); - SegmentGeneratorConfig config = new SegmentGeneratorConfig(tableConfig, schema); + SegmentGeneratorConfig config = new SegmentGeneratorConfig(TABLE_CONFIG, SCHEMA); config.setOutDir(INDEX_DIR.getPath()); config.setTableName(TABLE_NAME); config.setSegmentName(segmentName); diff --git a/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkNativeAndLuceneBasedLike.java b/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkNativeAndLuceneBasedLike.java index 4ed28c746f56..548f7d12baa6 100644 --- a/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkNativeAndLuceneBasedLike.java +++ b/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkNativeAndLuceneBasedLike.java @@ -21,9 +21,7 @@ import java.io.File; import java.util.ArrayList; import java.util.Collections; -import java.util.HashSet; import java.util.List; -import java.util.Set; import java.util.concurrent.TimeUnit; import org.apache.commons.io.FileUtils; import org.apache.pinot.core.common.Operator; @@ -81,6 +79,9 @@ public class BenchmarkNativeAndLuceneBasedLike { private static final String URL_COL = "URL_COL"; private static final String INT_COL_NAME = "INT_COL"; private static final String NO_INDEX_STRING_COL_NAME = "NO_INDEX_COL"; + private TableConfig _tableConfig; + private List _fieldConfigs = new ArrayList<>(); + private Schema _schema; @Param({"LUCENE", "NATIVE"}) private FSTType _fstType; @@ -105,11 +106,8 @@ public void setUp() _queryContext = QueryContextConverterUtils.getQueryContext(_query); FileUtils.deleteQuietly(INDEX_DIR); buildSegment(_fstType); - IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig(); - Set fstIndexCols = new HashSet<>(); - fstIndexCols.add(DOMAIN_NAMES_COL); - indexLoadingConfig.setFSTIndexColumns(fstIndexCols); - indexLoadingConfig.setFSTIndexType(_fstType); + + IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig(_tableConfig, _schema); _indexSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, SEGMENT_NAME), indexLoadingConfig); } @@ -145,19 +143,18 @@ private List createTestData(int numRows) { private void buildSegment(FSTType fstType) throws Exception { List rows = createTestData(_numRows); - List fieldConfigs = new ArrayList<>(); - fieldConfigs.add( + _fieldConfigs.add( new FieldConfig(DOMAIN_NAMES_COL, FieldConfig.EncodingType.DICTIONARY, FieldConfig.IndexType.FST, null, null)); - fieldConfigs + _fieldConfigs .add(new FieldConfig(URL_COL, FieldConfig.EncodingType.DICTIONARY, FieldConfig.IndexType.FST, null, null)); - TableConfig tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME) - .setInvertedIndexColumns(Collections.singletonList(DOMAIN_NAMES_COL)).setFieldConfigList(fieldConfigs).build(); - Schema schema = new Schema.SchemaBuilder().setSchemaName(TABLE_NAME) + _tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME) + .setInvertedIndexColumns(Collections.singletonList(DOMAIN_NAMES_COL)).setFieldConfigList(_fieldConfigs).build(); + _schema = new Schema.SchemaBuilder().setSchemaName(TABLE_NAME) .addSingleValueDimension(DOMAIN_NAMES_COL, FieldSpec.DataType.STRING) .addSingleValueDimension(URL_COL, FieldSpec.DataType.STRING) .addSingleValueDimension(NO_INDEX_STRING_COL_NAME, FieldSpec.DataType.STRING) .addMetric(INT_COL_NAME, FieldSpec.DataType.INT).build(); - SegmentGeneratorConfig config = new SegmentGeneratorConfig(tableConfig, schema); + SegmentGeneratorConfig config = new SegmentGeneratorConfig(_tableConfig, _schema); config.setOutDir(INDEX_DIR.getPath()); config.setTableName(TABLE_NAME); config.setSegmentName(SEGMENT_NAME); diff --git a/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkOrderByQueries.java b/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkOrderByQueries.java index 2c72421893fa..e627550969ab 100644 --- a/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkOrderByQueries.java +++ b/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkOrderByQueries.java @@ -23,10 +23,8 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.function.LongSupplier; @@ -94,6 +92,28 @@ public static void main(String[] args) private static final String NO_INDEX_INT_COL_NAME = "NO_INDEX_INT_COL"; private static final String NO_INDEX_STRING_COL = "NO_INDEX_STRING_COL"; private static final String LOW_CARDINALITY_STRING_COL = "LOW_CARDINALITY_STRING_COL"; + private static final List FIELD_CONFIGS = new ArrayList<>(); + + private static final TableConfig TABLE_CONFIG = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME) + .setInvertedIndexColumns(List.of(INT_COL_NAME, LOW_CARDINALITY_STRING_COL)) + .setFieldConfigList(FIELD_CONFIGS) + .setNoDictionaryColumns(List.of(RAW_INT_COL_NAME, RAW_STRING_COL_NAME)) + .setSortedColumn(SORTED_COL_NAME) + .setRangeIndexColumns(List.of(INT_COL_NAME, LOW_CARDINALITY_STRING_COL)) + .setStarTreeIndexConfigs(Collections.singletonList( + new StarTreeIndexConfig(Arrays.asList(SORTED_COL_NAME, INT_COL_NAME), null, Collections.singletonList( + new AggregationFunctionColumnPair(AggregationFunctionType.SUM, RAW_INT_COL_NAME).toColumnName()), null, + Integer.MAX_VALUE))) + .build(); + private static final Schema SCHEMA = new Schema.SchemaBuilder().setSchemaName(TABLE_NAME) + .addSingleValueDimension(SORTED_COL_NAME, FieldSpec.DataType.INT) + .addSingleValueDimension(NO_INDEX_INT_COL_NAME, FieldSpec.DataType.INT) + .addSingleValueDimension(RAW_INT_COL_NAME, FieldSpec.DataType.INT) + .addSingleValueDimension(INT_COL_NAME, FieldSpec.DataType.INT) + .addSingleValueDimension(RAW_STRING_COL_NAME, FieldSpec.DataType.STRING) + .addSingleValueDimension(NO_INDEX_STRING_COL, FieldSpec.DataType.STRING) + .addSingleValueDimension(LOW_CARDINALITY_STRING_COL, FieldSpec.DataType.STRING) + .build(); @Param("1500000") private int _numRows; @@ -115,14 +135,8 @@ public void setUp() buildSegment(FIRST_SEGMENT_NAME); buildSegment(SECOND_SEGMENT_NAME); - IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig(); - Set invertedIndexCols = new HashSet<>(); - invertedIndexCols.add(INT_COL_NAME); - invertedIndexCols.add(LOW_CARDINALITY_STRING_COL); - - indexLoadingConfig.setRangeIndexColumns(invertedIndexCols); - indexLoadingConfig.setInvertedIndexColumns(invertedIndexCols); + IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig(TABLE_CONFIG, SCHEMA); ImmutableSegment firstImmutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, FIRST_SEGMENT_NAME), indexLoadingConfig); @@ -167,28 +181,7 @@ private List createTestData(int numRows) { private void buildSegment(String segmentName) throws Exception { List rows = createTestData(_numRows); - List fieldConfigs = new ArrayList<>(); - - TableConfig tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME) - .setInvertedIndexColumns(Collections.singletonList(INT_COL_NAME)) - .setFieldConfigList(fieldConfigs) - .setNoDictionaryColumns(Arrays.asList(RAW_INT_COL_NAME, RAW_STRING_COL_NAME)) - .setSortedColumn(SORTED_COL_NAME) - .setStarTreeIndexConfigs(Collections.singletonList( - new StarTreeIndexConfig(Arrays.asList(SORTED_COL_NAME, INT_COL_NAME), null, Collections.singletonList( - new AggregationFunctionColumnPair(AggregationFunctionType.SUM, RAW_INT_COL_NAME).toColumnName()), null, - Integer.MAX_VALUE))) - .build(); - Schema schema = new Schema.SchemaBuilder().setSchemaName(TABLE_NAME) - .addSingleValueDimension(SORTED_COL_NAME, FieldSpec.DataType.INT) - .addSingleValueDimension(NO_INDEX_INT_COL_NAME, FieldSpec.DataType.INT) - .addSingleValueDimension(RAW_INT_COL_NAME, FieldSpec.DataType.INT) - .addSingleValueDimension(INT_COL_NAME, FieldSpec.DataType.INT) - .addSingleValueDimension(RAW_STRING_COL_NAME, FieldSpec.DataType.STRING) - .addSingleValueDimension(NO_INDEX_STRING_COL, FieldSpec.DataType.STRING) - .addSingleValueDimension(LOW_CARDINALITY_STRING_COL, FieldSpec.DataType.STRING) - .build(); - SegmentGeneratorConfig config = new SegmentGeneratorConfig(tableConfig, schema); + SegmentGeneratorConfig config = new SegmentGeneratorConfig(TABLE_CONFIG, SCHEMA); config.setOutDir(INDEX_DIR.getPath()); config.setTableName(TABLE_NAME); config.setSegmentName(segmentName); diff --git a/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkQueries.java b/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkQueries.java index d5002c09b674..3b3a58c1da8a 100644 --- a/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkQueries.java +++ b/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkQueries.java @@ -23,10 +23,8 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.function.LongSupplier; @@ -94,6 +92,25 @@ public static void main(String[] args) private static final String NO_INDEX_INT_COL_NAME = "NO_INDEX_INT_COL"; private static final String NO_INDEX_STRING_COL = "NO_INDEX_STRING_COL"; private static final String LOW_CARDINALITY_STRING_COL = "LOW_CARDINALITY_STRING_COL"; + private static final List FIELD_CONFIGS = new ArrayList<>(); + + private static final TableConfig TABLE_CONFIG = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME) + .setInvertedIndexColumns(List.of(INT_COL_NAME, LOW_CARDINALITY_STRING_COL)).setFieldConfigList(FIELD_CONFIGS) + .setNoDictionaryColumns(List.of(RAW_INT_COL_NAME, RAW_STRING_COL_NAME)).setSortedColumn(SORTED_COL_NAME) + .setRangeIndexColumns(List.of(INT_COL_NAME, LOW_CARDINALITY_STRING_COL)).setStarTreeIndexConfigs( + Collections.singletonList(new StarTreeIndexConfig(List.of(SORTED_COL_NAME, INT_COL_NAME), null, + Collections.singletonList( + new AggregationFunctionColumnPair(AggregationFunctionType.SUM, RAW_INT_COL_NAME).toColumnName()), + null, Integer.MAX_VALUE))).build(); + private static final Schema SCHEMA = new Schema.SchemaBuilder().setSchemaName(TABLE_NAME) + .addSingleValueDimension(SORTED_COL_NAME, FieldSpec.DataType.INT) + .addSingleValueDimension(NO_INDEX_INT_COL_NAME, FieldSpec.DataType.INT) + .addSingleValueDimension(RAW_INT_COL_NAME, FieldSpec.DataType.INT) + .addSingleValueDimension(INT_COL_NAME, FieldSpec.DataType.INT) + .addSingleValueDimension(RAW_STRING_COL_NAME, FieldSpec.DataType.STRING) + .addSingleValueDimension(NO_INDEX_STRING_COL, FieldSpec.DataType.STRING) + .addSingleValueDimension(LOW_CARDINALITY_STRING_COL, FieldSpec.DataType.STRING) + .build(); public static final String FILTERED_QUERY = "SELECT SUM(INT_COL) FILTER(WHERE INT_COL > 123 AND INT_COL < 599999)," + "MAX(INT_COL) FILTER(WHERE INT_COL > 123 AND INT_COL < 599999) " @@ -190,15 +207,8 @@ public void setUp() buildSegment(FIRST_SEGMENT_NAME); buildSegment(SECOND_SEGMENT_NAME); - IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig(); - - Set invertedIndexCols = new HashSet<>(); - invertedIndexCols.add(INT_COL_NAME); - invertedIndexCols.add(LOW_CARDINALITY_STRING_COL); - - indexLoadingConfig.setRangeIndexColumns(invertedIndexCols); - indexLoadingConfig.setInvertedIndexColumns(invertedIndexCols); + IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig(TABLE_CONFIG, SCHEMA); ImmutableSegment firstImmutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, FIRST_SEGMENT_NAME), indexLoadingConfig); ImmutableSegment secondImmutableSegment = @@ -240,28 +250,7 @@ private List createTestData(int numRows) { private void buildSegment(String segmentName) throws Exception { List rows = createTestData(_numRows); - List fieldConfigs = new ArrayList<>(); - - TableConfig tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME) - .setInvertedIndexColumns(Collections.singletonList(INT_COL_NAME)) - .setFieldConfigList(fieldConfigs) - .setNoDictionaryColumns(Arrays.asList(RAW_INT_COL_NAME, RAW_STRING_COL_NAME)) - .setSortedColumn(SORTED_COL_NAME) - .setStarTreeIndexConfigs(Collections.singletonList( - new StarTreeIndexConfig(Arrays.asList(SORTED_COL_NAME, INT_COL_NAME), null, Collections.singletonList( - new AggregationFunctionColumnPair(AggregationFunctionType.SUM, RAW_INT_COL_NAME).toColumnName()), null, - Integer.MAX_VALUE))) - .build(); - Schema schema = new Schema.SchemaBuilder().setSchemaName(TABLE_NAME) - .addSingleValueDimension(SORTED_COL_NAME, FieldSpec.DataType.INT) - .addSingleValueDimension(NO_INDEX_INT_COL_NAME, FieldSpec.DataType.INT) - .addSingleValueDimension(RAW_INT_COL_NAME, FieldSpec.DataType.INT) - .addSingleValueDimension(INT_COL_NAME, FieldSpec.DataType.INT) - .addSingleValueDimension(RAW_STRING_COL_NAME, FieldSpec.DataType.STRING) - .addSingleValueDimension(NO_INDEX_STRING_COL, FieldSpec.DataType.STRING) - .addSingleValueDimension(LOW_CARDINALITY_STRING_COL, FieldSpec.DataType.STRING) - .build(); - SegmentGeneratorConfig config = new SegmentGeneratorConfig(tableConfig, schema); + SegmentGeneratorConfig config = new SegmentGeneratorConfig(TABLE_CONFIG, SCHEMA); config.setOutDir(INDEX_DIR.getPath()); config.setTableName(TABLE_NAME); config.setSegmentName(segmentName); diff --git a/pinot-perf/src/main/java/org/apache/pinot/perf/StringDictionaryPerfTest.java b/pinot-perf/src/main/java/org/apache/pinot/perf/StringDictionaryPerfTest.java index 0344e4827f2a..9594e2a5343b 100644 --- a/pinot-perf/src/main/java/org/apache/pinot/perf/StringDictionaryPerfTest.java +++ b/pinot-perf/src/main/java/org/apache/pinot/perf/StringDictionaryPerfTest.java @@ -60,11 +60,15 @@ public class StringDictionaryPerfTest { "Kurtosis", "Variance", "BufferSize" }; private static final Joiner COMMA_JOINER = Joiner.on(","); + private static final TableConfig TABLE_CONFIG = + new TableConfigBuilder(TableType.OFFLINE).setOnHeapDictionaryColumns(List.of(COLUMN_NAME)).setTableName("test") + .build(); private final DescriptiveStatistics _statistics = new DescriptiveStatistics(); private String[] _inputStrings; private File _indexDir; private int _dictLength; + private Schema _schema; /** * Helper method to build a segment: @@ -75,19 +79,18 @@ public class StringDictionaryPerfTest { */ private void buildSegment(int dictLength) throws Exception { - Schema schema = new Schema(); + _schema = new Schema(); String segmentName = "perfTestSegment" + System.currentTimeMillis(); _indexDir = new File(TMP_DIR + File.separator + segmentName); _indexDir.deleteOnExit(); FieldSpec fieldSpec = new DimensionFieldSpec(COLUMN_NAME, FieldSpec.DataType.STRING, true); - schema.addField(fieldSpec); - TableConfig tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName("test").build(); + _schema.addField(fieldSpec); _dictLength = dictLength; _inputStrings = new String[dictLength]; - SegmentGeneratorConfig config = new SegmentGeneratorConfig(tableConfig, schema); + SegmentGeneratorConfig config = new SegmentGeneratorConfig(TABLE_CONFIG, _schema); config.setOutDir(_indexDir.getParent()); config.setFormat(FileFormat.AVRO); config.setSegmentName(segmentName); @@ -151,11 +154,7 @@ private String[] perfTestGetValues(int numGetValues) Runtime r = Runtime.getRuntime(); System.gc(); long oldMemory = r.totalMemory() - r.freeMemory(); - IndexLoadingConfig defaultIndexLoadingConfig = new IndexLoadingConfig(); - defaultIndexLoadingConfig.setReadMode(ReadMode.heap); - Set columnNames = new HashSet<>(); - columnNames.add(COLUMN_NAME); - defaultIndexLoadingConfig.setOnHeapDictionaryColumns(columnNames); + IndexLoadingConfig defaultIndexLoadingConfig = new IndexLoadingConfig(TABLE_CONFIG, _schema); ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(_indexDir, defaultIndexLoadingConfig); Dictionary dictionary = immutableSegment.getDictionary(COLUMN_NAME); diff --git a/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-hadoop/pom.xml b/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-hadoop/pom.xml index 85e261670be1..6bbb98902dfb 100644 --- a/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-hadoop/pom.xml +++ b/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-hadoop/pom.xml @@ -62,4 +62,13 @@ test + + + + pinot-fastdev + + none + + + diff --git a/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-hadoop/src/main/java/org/apache/pinot/plugin/ingestion/batch/hadoop/HadoopSegmentGenerationJobRunner.java b/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-hadoop/src/main/java/org/apache/pinot/plugin/ingestion/batch/hadoop/HadoopSegmentGenerationJobRunner.java index f63f42418767..188757bb94a8 100644 --- a/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-hadoop/src/main/java/org/apache/pinot/plugin/ingestion/batch/hadoop/HadoopSegmentGenerationJobRunner.java +++ b/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-hadoop/src/main/java/org/apache/pinot/plugin/ingestion/batch/hadoop/HadoopSegmentGenerationJobRunner.java @@ -349,7 +349,11 @@ protected void addMapperJarToDistributedCache(Job job, PinotFS outputDirFS, URI throws Exception { File ourJar = new File(getClass().getProtectionDomain().getCodeSource().getLocation().toURI()); Path distributedCacheJar = new Path(stagingDirURI.toString(), ourJar.getName()); - outputDirFS.copyFromLocalDir(ourJar, distributedCacheJar.toUri()); + if (ourJar.isDirectory()) { + outputDirFS.copyFromLocalDir(ourJar, distributedCacheJar.toUri()); + } else { + outputDirFS.copyFromLocalFile(ourJar, distributedCacheJar.toUri()); + } job.addFileToClassPath(distributedCacheJar); } diff --git a/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-spark-2.4/pom.xml b/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-spark-2.4/pom.xml index d05a23a973fe..9eb28d2f4980 100644 --- a/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-spark-2.4/pom.xml +++ b/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-spark-2.4/pom.xml @@ -109,4 +109,13 @@ stax2-api + + + + pinot-fastdev + + none + + + diff --git a/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-spark-3/pom.xml b/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-spark-3/pom.xml index 098072b84674..74c1dc278eb5 100644 --- a/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-spark-3/pom.xml +++ b/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-spark-3/pom.xml @@ -73,4 +73,13 @@ test + + + + pinot-fastdev + + none + + + diff --git a/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-standalone/pom.xml b/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-standalone/pom.xml index e1be6d57487a..85051371b754 100644 --- a/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-standalone/pom.xml +++ b/pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-standalone/pom.xml @@ -48,4 +48,13 @@ + + + + pinot-fastdev + + none + + + diff --git a/pinot-plugins/pinot-environment/pinot-azure/pom.xml b/pinot-plugins/pinot-environment/pinot-azure/pom.xml index 43a7bd76e9fd..c18d3e6636a1 100644 --- a/pinot-plugins/pinot-environment/pinot-azure/pom.xml +++ b/pinot-plugins/pinot-environment/pinot-azure/pom.xml @@ -40,4 +40,13 @@ provided + + + + pinot-fastdev + + none + + + diff --git a/pinot-plugins/pinot-file-system/pinot-adls/pom.xml b/pinot-plugins/pinot-file-system/pinot-adls/pom.xml index 968dd5c2f72e..2e04826af13f 100644 --- a/pinot-plugins/pinot-file-system/pinot-adls/pom.xml +++ b/pinot-plugins/pinot-file-system/pinot-adls/pom.xml @@ -43,4 +43,13 @@ azure-identity + + + + pinot-fastdev + + none + + + diff --git a/pinot-plugins/pinot-file-system/pinot-gcs/pom.xml b/pinot-plugins/pinot-file-system/pinot-gcs/pom.xml index c9356b24a2c5..4c3fa581cce6 100644 --- a/pinot-plugins/pinot-file-system/pinot-gcs/pom.xml +++ b/pinot-plugins/pinot-file-system/pinot-gcs/pom.xml @@ -89,4 +89,13 @@ jersey-server + + + + pinot-fastdev + + none + + + diff --git a/pinot-plugins/pinot-file-system/pinot-hdfs/pom.xml b/pinot-plugins/pinot-file-system/pinot-hdfs/pom.xml index a3871d8590cc..e167c3afe282 100644 --- a/pinot-plugins/pinot-file-system/pinot-hdfs/pom.xml +++ b/pinot-plugins/pinot-file-system/pinot-hdfs/pom.xml @@ -52,4 +52,13 @@ hadoop-shaded-protobuf_3_21 + + + + pinot-fastdev + + none + + + diff --git a/pinot-plugins/pinot-file-system/pinot-s3/pom.xml b/pinot-plugins/pinot-file-system/pinot-s3/pom.xml index 60c8bcf5ecff..8d35b42124bc 100644 --- a/pinot-plugins/pinot-file-system/pinot-s3/pom.xml +++ b/pinot-plugins/pinot-file-system/pinot-s3/pom.xml @@ -70,5 +70,11 @@ package + + pinot-fastdev + + none + + diff --git a/pinot-plugins/pinot-input-format/pinot-avro-base/pom.xml b/pinot-plugins/pinot-input-format/pinot-avro-base/pom.xml index 7ef902819e87..0c36701406d7 100644 --- a/pinot-plugins/pinot-input-format/pinot-avro-base/pom.xml +++ b/pinot-plugins/pinot-input-format/pinot-avro-base/pom.xml @@ -35,4 +35,13 @@ package + + + pinot-fastdev + + none + + + + diff --git a/pinot-plugins/pinot-input-format/pinot-avro/pom.xml b/pinot-plugins/pinot-input-format/pinot-avro/pom.xml index 02a9150c36ea..274b956e2628 100644 --- a/pinot-plugins/pinot-input-format/pinot-avro/pom.xml +++ b/pinot-plugins/pinot-input-format/pinot-avro/pom.xml @@ -41,4 +41,13 @@ pinot-avro-base + + + + pinot-fastdev + + none + + + diff --git a/pinot-plugins/pinot-input-format/pinot-clp-log/pom.xml b/pinot-plugins/pinot-input-format/pinot-clp-log/pom.xml index 041a07dad656..0962c8fd7226 100644 --- a/pinot-plugins/pinot-input-format/pinot-clp-log/pom.xml +++ b/pinot-plugins/pinot-input-format/pinot-clp-log/pom.xml @@ -46,4 +46,13 @@ clp-ffi + + + + pinot-fastdev + + none + + + diff --git a/pinot-plugins/pinot-input-format/pinot-confluent-avro/pom.xml b/pinot-plugins/pinot-input-format/pinot-confluent-avro/pom.xml index d22290bd05ae..ced2f80669a0 100644 --- a/pinot-plugins/pinot-input-format/pinot-confluent-avro/pom.xml +++ b/pinot-plugins/pinot-input-format/pinot-confluent-avro/pom.xml @@ -58,4 +58,13 @@ kafka-avro-serializer + + + + pinot-fastdev + + none + + + diff --git a/pinot-plugins/pinot-input-format/pinot-csv/pom.xml b/pinot-plugins/pinot-input-format/pinot-csv/pom.xml index cf7b0d143f32..c2c0cb1f2358 100644 --- a/pinot-plugins/pinot-input-format/pinot-csv/pom.xml +++ b/pinot-plugins/pinot-input-format/pinot-csv/pom.xml @@ -41,4 +41,13 @@ commons-csv + + + + pinot-fastdev + + none + + + diff --git a/pinot-plugins/pinot-input-format/pinot-json/pom.xml b/pinot-plugins/pinot-input-format/pinot-json/pom.xml index 94c5e5274250..f3313c4a9a00 100644 --- a/pinot-plugins/pinot-input-format/pinot-json/pom.xml +++ b/pinot-plugins/pinot-input-format/pinot-json/pom.xml @@ -34,4 +34,13 @@ ${basedir}/../../.. package + + + + pinot-fastdev + + none + + + diff --git a/pinot-plugins/pinot-input-format/pinot-orc/pom.xml b/pinot-plugins/pinot-input-format/pinot-orc/pom.xml index 26f3134d13c5..07d0350fdfad 100644 --- a/pinot-plugins/pinot-input-format/pinot-orc/pom.xml +++ b/pinot-plugins/pinot-input-format/pinot-orc/pom.xml @@ -81,4 +81,13 @@ protobuf-java + + + + pinot-fastdev + + none + + + diff --git a/pinot-plugins/pinot-input-format/pinot-parquet/pom.xml b/pinot-plugins/pinot-input-format/pinot-parquet/pom.xml index fe837b81898b..550b3951d286 100644 --- a/pinot-plugins/pinot-input-format/pinot-parquet/pom.xml +++ b/pinot-plugins/pinot-input-format/pinot-parquet/pom.xml @@ -73,4 +73,13 @@ ${hadoop.dependencies.scope} + + + + pinot-fastdev + + none + + + diff --git a/pinot-plugins/pinot-input-format/pinot-protobuf/pom.xml b/pinot-plugins/pinot-input-format/pinot-protobuf/pom.xml index 86ee2160de1a..0558d5a9585f 100644 --- a/pinot-plugins/pinot-input-format/pinot-protobuf/pom.xml +++ b/pinot-plugins/pinot-input-format/pinot-protobuf/pom.xml @@ -97,4 +97,13 @@ + + + + pinot-fastdev + + none + + + diff --git a/pinot-plugins/pinot-input-format/pinot-thrift/pom.xml b/pinot-plugins/pinot-input-format/pinot-thrift/pom.xml index 2cafc5005980..57e9539f7824 100644 --- a/pinot-plugins/pinot-input-format/pinot-thrift/pom.xml +++ b/pinot-plugins/pinot-input-format/pinot-thrift/pom.xml @@ -40,4 +40,13 @@ libthrift + + + + pinot-fastdev + + none + + + diff --git a/pinot-plugins/pinot-metrics/pinot-dropwizard/pom.xml b/pinot-plugins/pinot-metrics/pinot-dropwizard/pom.xml index 2c3a5b0891bb..1f6b389aa1ec 100644 --- a/pinot-plugins/pinot-metrics/pinot-dropwizard/pom.xml +++ b/pinot-plugins/pinot-metrics/pinot-dropwizard/pom.xml @@ -54,4 +54,13 @@ metrics-jmx + + + + pinot-fastdev + + none + + + diff --git a/pinot-plugins/pinot-metrics/pinot-yammer/pom.xml b/pinot-plugins/pinot-metrics/pinot-yammer/pom.xml index 56ea6c57c670..d783f46ab467 100644 --- a/pinot-plugins/pinot-metrics/pinot-yammer/pom.xml +++ b/pinot-plugins/pinot-metrics/pinot-yammer/pom.xml @@ -41,4 +41,13 @@ metrics-core + + + + pinot-fastdev + + none + + + diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/pom.xml b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/pom.xml index 12ea77db0453..d03f55654358 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/pom.xml +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/pom.xml @@ -62,4 +62,13 @@ test + + + + pinot-fastdev + + none + + + diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-3.0/pom.xml b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-3.0/pom.xml index cd1b020ad07d..001e843ad92c 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-3.0/pom.xml +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-3.0/pom.xml @@ -75,4 +75,13 @@ test + + + + pinot-fastdev + + none + + + diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-base/pom.xml b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-base/pom.xml index 268faa1c57d1..26bf56add08f 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kafka-base/pom.xml +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kafka-base/pom.xml @@ -60,4 +60,13 @@ test + + + + pinot-fastdev + + none + + + diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/pom.xml b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/pom.xml index 6b14b7e01356..46c9b3f2fdd1 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/pom.xml +++ b/pinot-plugins/pinot-stream-ingestion/pinot-kinesis/pom.xml @@ -78,5 +78,11 @@ package + + pinot-fastdev + + none + + diff --git a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/pom.xml b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/pom.xml index 9021edb2d6f1..fcb6a45268f3 100644 --- a/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/pom.xml +++ b/pinot-plugins/pinot-stream-ingestion/pinot-pulsar/pom.xml @@ -58,4 +58,13 @@ test + + + + pinot-fastdev + + none + + + diff --git a/pinot-query-planner/pom.xml b/pinot-query-planner/pom.xml index 60fd5369d6ce..408c7bdfc2ad 100644 --- a/pinot-query-planner/pom.xml +++ b/pinot-query-planner/pom.xml @@ -49,6 +49,10 @@ org.codehaus.janino commons-compiler + + org.immutables + value-annotations + org.testng diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintOptions.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintOptions.java index 431d741e4c32..f88908a74f78 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintOptions.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintOptions.java @@ -87,9 +87,28 @@ public static class JoinHintOptions { } public static class TableHintOptions { + /** + * Indicates how many partitions the table must be partitioned by. + * This must be equal to the partition count of the table in + * {@code tableIndexConfig.segmentPartitionConfig.columnPartitionMap}. + */ public static final String PARTITION_KEY = "partition_key"; + /** + * The function to use to partition the table. + * This must be equal to {@code functionName} in {@code tableIndexConfig.segmentPartitionConfig.columnPartitionMap}. + */ public static final String PARTITION_FUNCTION = "partition_function"; + /** + * The size of each partition. + * This must be equal to {@code numPartition} in {@code tableIndexConfig.segmentPartitionConfig.columnPartitionMap}. + */ public static final String PARTITION_SIZE = "partition_size"; + /** + * The number of workers per partition. + * + * How many threads to use in the following stage after partition is joined. + * When partition info is set, each partition is processed as a separate query in the leaf stage. + */ public static final String PARTITION_PARALLELISM = "partition_parallelism"; } } diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintStrategyTable.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintStrategyTable.java index f26fe026a49a..9de47caea8c7 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintStrategyTable.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintStrategyTable.java @@ -19,9 +19,11 @@ package org.apache.pinot.calcite.rel.hint; import java.util.List; +import java.util.function.Predicate; import javax.annotation.Nullable; import org.apache.calcite.rel.hint.HintPredicates; import org.apache.calcite.rel.hint.HintStrategyTable; +import org.apache.calcite.rel.hint.Hintable; import org.apache.calcite.rel.hint.RelHint; import org.apache.pinot.spi.utils.BooleanUtils; @@ -38,6 +40,29 @@ private PinotHintStrategyTable() { .hintStrategy(PinotHintOptions.JOIN_HINT_OPTIONS, HintPredicates.JOIN) .hintStrategy(PinotHintOptions.TABLE_HINT_OPTIONS, HintPredicates.TABLE_SCAN).build(); + + /** + * Get the first hint that has the specified name. + */ + @Nullable + public static RelHint getHint(Hintable hintable, String hintName) { + return hintable.getHints().stream() + .filter(relHint -> relHint.hintName.equals(hintName)) + .findFirst() + .orElse(null); + } + + /** + * Get the first hint that satisfies the predicate. + */ + @Nullable + public static RelHint getHint(Hintable hintable, Predicate predicate) { + return hintable.getHints().stream() + .filter(predicate) + .findFirst() + .orElse(null); + } + /** * Check if a hint-able {@link org.apache.calcite.rel.RelNode} contains a specific {@link RelHint} by name. * diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotImplicitTableHintRule.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotImplicitTableHintRule.java new file mode 100644 index 000000000000..ff5260eef999 --- /dev/null +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotImplicitTableHintRule.java @@ -0,0 +1,223 @@ +/** + * 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.pinot.calcite.rel.rules; + +import java.util.ArrayList; +import java.util.Map; +import javax.annotation.Nullable; +import org.apache.calcite.plan.RelOptRuleCall; +import org.apache.calcite.plan.RelRule; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.hint.RelHint; +import org.apache.calcite.rel.logical.LogicalTableScan; +import org.apache.pinot.calcite.rel.hint.PinotHintOptions; +import org.apache.pinot.calcite.rel.hint.PinotHintStrategyTable; +import org.apache.pinot.query.planner.logical.RelToPlanNodeConverter; +import org.apache.pinot.query.routing.WorkerManager; +import org.immutables.value.Value; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Value.Enclosing +public class PinotImplicitTableHintRule extends RelRule { + + private static final Logger LOGGER = LoggerFactory.getLogger(PinotImplicitTableHintRule.class); + private final WorkerManager _workerManager; + + private PinotImplicitTableHintRule(Config config) { + super(config); + _workerManager = config.getWorkerManager(); + } + + public static PinotImplicitTableHintRule withWorkerManager(WorkerManager workerManager) { + return new PinotImplicitTableHintRule(ImmutablePinotImplicitTableHintRule.Config.builder() + .operandSupplier(b0 -> b0.operand(LogicalTableScan.class).anyInputs()) + .workerManager(workerManager) + .build() + ); + } + + @Override + public boolean matches(RelOptRuleCall call) { + LogicalTableScan tableScan = call.rel(0); + + RelHint explicitHint = getTableOptionHint(tableScan); + + if (explicitHint == null) { + return true; + } + // we don't want to apply this rule if the explicit hint is complete + Map kvOptions = explicitHint.kvOptions; + return kvOptions.containsKey(PinotHintOptions.TableHintOptions.PARTITION_KEY) + && kvOptions.containsKey(PinotHintOptions.TableHintOptions.PARTITION_FUNCTION) + && kvOptions.containsKey(PinotHintOptions.TableHintOptions.PARTITION_SIZE); + } + + @Override + public void onMatch(RelOptRuleCall call) { + LogicalTableScan tableScan = call.rel(0); + + String tableName = RelToPlanNodeConverter.getTableNameFromTableScan(tableScan); + @Nullable + TableOptions implicitTableOptions = _workerManager.inferTableOptions(tableName); + if (implicitTableOptions == null) { + return; + } + + @Nullable + RelHint explicitHint = getTableOptionHint(tableScan); + TableOptions tableOptions = calculateTableOptions(explicitHint, implicitTableOptions, tableScan); + RelNode newRel = withNewTableOptions(tableScan, tableOptions); + call.transformTo(newRel); + } + + /** + * Get the table option hint from the table scan, if any. + */ + @Nullable + private static RelHint getTableOptionHint(LogicalTableScan tableScan) { + return PinotHintStrategyTable.getHint(tableScan, PinotHintOptions.TABLE_HINT_OPTIONS); + } + + /** + * Returns a new node which is a copy of the given table scan with the new table options hint. + */ + private static RelNode withNewTableOptions(LogicalTableScan tableScan, TableOptions tableOptions) { + ArrayList newHints = new ArrayList<>(tableScan.getHints()); + + newHints.removeIf(relHint -> relHint.hintName.equals(PinotHintOptions.TABLE_HINT_OPTIONS)); + + RelHint.Builder builder = RelHint.builder(PinotHintOptions.TABLE_HINT_OPTIONS) + .hintOption(PinotHintOptions.TableHintOptions.PARTITION_KEY, tableOptions.getPartitionKey()) + .hintOption(PinotHintOptions.TableHintOptions.PARTITION_FUNCTION, tableOptions.getPartitionFunction()) + .hintOption(PinotHintOptions.TableHintOptions.PARTITION_SIZE, String.valueOf(tableOptions.getPartitionSize())); + + if (tableOptions.getPartitionParallelism() != null) { + builder.hintOption(PinotHintOptions.TableHintOptions.PARTITION_PARALLELISM, + String.valueOf(tableOptions.getPartitionParallelism())); + } + + newHints.add(builder.build()); + + return tableScan.withHints(newHints); + } + + /** + * Creates a new table options hint based on the given table partition info and the explicit hint, if any. + * + * Any explicit hint will override the implicit hint obtained from the table partition info. + */ + private static TableOptions calculateTableOptions( + @Nullable RelHint relHint, TableOptions implicitTableOptions, LogicalTableScan tableScan) { + if (relHint == null) { + return implicitTableOptions; + } + + // there is a hint, check fill default data and obtain the partition parallelism if supplied + Map kvOptions = relHint.kvOptions; + + ImmutableTableOptions newTableOptions = ImmutableTableOptions.copyOf(implicitTableOptions); + newTableOptions = overridePartitionKey(newTableOptions, tableScan, kvOptions); + newTableOptions = overridePartitionFunction(newTableOptions, tableScan, kvOptions); + newTableOptions = overridePartitionSize(newTableOptions, tableScan, kvOptions); + newTableOptions = overridePartitionParallelism(newTableOptions, tableScan, kvOptions); + + return newTableOptions; + } + + /** + * Returns a table options hint with the partition key overridden by the hint, if any. + */ + private static ImmutableTableOptions overridePartitionKey(ImmutableTableOptions base, LogicalTableScan tableScan, + Map kvOptions) { + String partitionKey = kvOptions.get(kvOptions.get(PinotHintOptions.TableHintOptions.PARTITION_KEY)); + if (partitionKey == null || partitionKey.equals(base.getPartitionKey())) { + return base; + } + LOGGER.debug("Override implicit table hint for {} with explicit partition key: {}", tableScan, partitionKey); + return base.withPartitionKey(partitionKey); + } + + /** + * Returns a table options hint with the partition function overridden by the hint, if any. + */ + private static ImmutableTableOptions overridePartitionFunction(ImmutableTableOptions base, LogicalTableScan tableScan, + Map kvOptions) { + String partitionFunction = kvOptions.get(kvOptions.get(PinotHintOptions.TableHintOptions.PARTITION_FUNCTION)); + if (partitionFunction == null || partitionFunction.equals(base.getPartitionFunction())) { + return base; + } + LOGGER.debug("Override implicit table hint for {} with explicit partition function: {}", tableScan, + partitionFunction); + return base.withPartitionFunction(partitionFunction); + } + + /** + * Returns a table options hint with the partition parallelism overridden by the hint, if any. + */ + private static ImmutableTableOptions overridePartitionParallelism(ImmutableTableOptions base, + LogicalTableScan tableScan, Map kvOptions) { + String partitionParallelismStr = kvOptions.get(PinotHintOptions.TableHintOptions.PARTITION_PARALLELISM); + if (partitionParallelismStr == null) { + return base; + } + try { + int partitionParallelism = Integer.parseInt(partitionParallelismStr); + LOGGER.debug("Override implicit table hint for {} with explicit partition parallelism: {}", tableScan, + partitionParallelism); + return base.withPartitionParallelism(partitionParallelism); + } catch (NumberFormatException e) { + throw new IllegalArgumentException( + "Invalid partition parallelism: " + partitionParallelismStr + " for table: " + tableScan); + } + } + + /** + * Returns a table options hint with the partition size overridden by the hint, if any. + */ + private static ImmutableTableOptions overridePartitionSize(ImmutableTableOptions base, LogicalTableScan tableScan, + Map kvOptions) { + String partitionSizeStr = kvOptions.get(PinotHintOptions.TableHintOptions.PARTITION_SIZE); + if (partitionSizeStr == null) { + return base; + } + try { + int explicitPartitionSize = Integer.parseInt(partitionSizeStr); + if (explicitPartitionSize == base.getPartitionSize()) { + return base; + } + LOGGER.debug("Override implicit table hint for {} with explicit partition size: {}", tableScan, + explicitPartitionSize); + return base.withPartitionSize(explicitPartitionSize); + } catch (NumberFormatException e) { + throw new IllegalArgumentException("Invalid partition size: " + partitionSizeStr + " for table: " + tableScan); + } + } + + @Value.Immutable + public interface Config extends RelRule.Config { + @Nullable + WorkerManager getWorkerManager(); + + @Override + default PinotImplicitTableHintRule toRule() { + return new PinotImplicitTableHintRule(this); + } + } +} diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/TableOptions.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/TableOptions.java new file mode 100644 index 000000000000..dfcbaa8ae3e1 --- /dev/null +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/TableOptions.java @@ -0,0 +1,38 @@ +/** + * 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.pinot.calcite.rel.rules; + +import javax.annotation.Nullable; +import org.immutables.value.Value; + + +/** + * An internal interface used to generate the table options hint. + */ +@Value.Immutable +public interface TableOptions { + String getPartitionKey(); + + String getPartitionFunction(); + + int getPartitionSize(); + + @Nullable + Integer getPartitionParallelism(); +} diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java index f14adb55a5b0..1681d41c9479 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/QueryEnvironment.java @@ -22,6 +22,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Properties; import java.util.Set; import javax.annotation.Nullable; @@ -51,6 +52,7 @@ import org.apache.calcite.tools.FrameworkConfig; import org.apache.calcite.tools.Frameworks; import org.apache.calcite.tools.RelBuilder; +import org.apache.pinot.calcite.rel.rules.PinotImplicitTableHintRule; import org.apache.pinot.calcite.rel.rules.PinotQueryRuleSets; import org.apache.pinot.calcite.rel.rules.PinotRelDistributionTraitRule; import org.apache.pinot.calcite.rel.rules.PinotRuleUtils; @@ -74,9 +76,11 @@ import org.apache.pinot.query.routing.WorkerManager; import org.apache.pinot.query.type.TypeFactory; import org.apache.pinot.query.validate.BytesCastVisitor; +import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.sql.parsers.CalciteSqlParser; import org.apache.pinot.sql.parsers.SqlNodeAndOptions; import org.apache.pinot.sql.parsers.parser.SqlPhysicalExplain; +import org.immutables.value.Value; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -85,7 +89,14 @@ * The {@code QueryEnvironment} contains the main entrypoint for query planning. * *

It provide the higher level entry interface to convert a SQL string into a {@link DispatchableSubPlan}. + * It is also used to execute some static analysis on the query like to determine if it can be compiled or get the + * tables involved in the query. */ + + //TODO: We should consider splitting this class in two: One that is used for parsing and one that is used for + // executing queries. This would allow us to remove the worker manager from the parsing environment and therefore + // make sure there is a worker manager when executing queries. +@Value.Enclosing public class QueryEnvironment { private static final Logger LOGGER = LoggerFactory.getLogger(QueryEnvironment.class); private static final CalciteConnectionConfig CONNECTION_CONFIG; @@ -100,26 +111,64 @@ public class QueryEnvironment { private final FrameworkConfig _config; private final CalciteCatalogReader _catalogReader; private final HepProgram _optProgram; - private final HepProgram _traitProgram; - - // Pinot extensions - private final TableCache _tableCache; - private final WorkerManager _workerManager; + private final Config _envConfig; - public QueryEnvironment(String database, TableCache tableCache, @Nullable WorkerManager workerManager) { - PinotCatalog catalog = new PinotCatalog(tableCache, database); + public QueryEnvironment(Config config) { + _envConfig = config; + String database = config.getDatabase(); + PinotCatalog catalog = new PinotCatalog(config.getTableCache(), database); CalciteSchema rootSchema = CalciteSchema.createRootSchema(false, false, database, catalog); _config = Frameworks.newConfigBuilder().traitDefs().operatorTable(PinotOperatorTable.instance()) .defaultSchema(rootSchema.plus()).sqlToRelConverterConfig(PinotRuleUtils.PINOT_SQL_TO_REL_CONFIG).build(); _catalogReader = new CalciteCatalogReader(rootSchema, List.of(database), _typeFactory, CONNECTION_CONFIG); _optProgram = getOptProgram(); - _traitProgram = getTraitProgram(); - _tableCache = tableCache; - _workerManager = workerManager; } - private PlannerContext getPlannerContext() { - return new PlannerContext(_config, _catalogReader, _typeFactory, _optProgram, _traitProgram); + public QueryEnvironment(String database, TableCache tableCache, @Nullable WorkerManager workerManager) { + this(configBuilder() + .database(database) + .tableCache(tableCache) + .workerManager(workerManager) + .build()); + } + + /** + * Returns a planner context that can be used to either parse, explain or execute a query. + */ + private PlannerContext getPlannerContext(SqlNodeAndOptions sqlNodeAndOptions) { + WorkerManager workerManager = getWorkerManager(sqlNodeAndOptions); + HepProgram traitProgram = getTraitProgram(workerManager); + return new PlannerContext(_config, _catalogReader, _typeFactory, _optProgram, traitProgram); + } + + @Nullable + private WorkerManager getWorkerManager(SqlNodeAndOptions sqlNodeAndOptions) { + String inferPartitionHint = sqlNodeAndOptions.getOptions() + .get(CommonConstants.Broker.Request.QueryOptionKey.INFER_PARTITION_HINT); + WorkerManager workerManager = _envConfig.getWorkerManager(); + + if (inferPartitionHint == null) { + return _envConfig.defaultInferPartitionHint() ? workerManager : null; + } + switch (inferPartitionHint.toLowerCase()) { + case "true": + Objects.requireNonNull(workerManager, "WorkerManager is required in order to infer partition hint"); + return workerManager; + case "false": + return null; + default: + throw new RuntimeException("Invalid value for query option '" + + CommonConstants.Broker.Request.QueryOptionKey.INFER_PARTITION_HINT + "': " + + inferPartitionHint); + } + } + + /** + * Returns the planner context that should be used only for parsing queries. + */ + private PlannerContext getParsingPlannerContext() { + HepProgram traitProgram = getTraitProgram(null); + return new PlannerContext(_config, _catalogReader, _typeFactory, _optProgram, traitProgram); } /** @@ -135,7 +184,7 @@ private PlannerContext getPlannerContext() { * @return QueryPlannerResult containing the dispatchable query plan and the relRoot. */ public QueryPlannerResult planQuery(String sqlQuery, SqlNodeAndOptions sqlNodeAndOptions, long requestId) { - try (PlannerContext plannerContext = getPlannerContext()) { + try (PlannerContext plannerContext = getPlannerContext(sqlNodeAndOptions)) { plannerContext.setOptions(sqlNodeAndOptions.getOptions()); RelRoot relRoot = compileQuery(sqlNodeAndOptions.getSqlNode(), plannerContext); // TODO: current code only assume one SubPlan per query, but we should support multiple SubPlans per query. @@ -150,6 +199,11 @@ public QueryPlannerResult planQuery(String sqlQuery, SqlNodeAndOptions sqlNodeAn } } + @VisibleForTesting + public DispatchableSubPlan planQuery(String sqlQuery) { + return planQuery(sqlQuery, CalciteSqlParser.compileToSqlNodeAndOptions(sqlQuery), 0).getQueryPlan(); + } + /** * Explain a SQL query. * @@ -165,7 +219,7 @@ public QueryPlannerResult planQuery(String sqlQuery, SqlNodeAndOptions sqlNodeAn */ public QueryPlannerResult explainQuery(String sqlQuery, SqlNodeAndOptions sqlNodeAndOptions, long requestId, @Nullable AskingServerStageExplainer.OnServerExplainer onServerExplainer) { - try (PlannerContext plannerContext = getPlannerContext()) { + try (PlannerContext plannerContext = getPlannerContext(sqlNodeAndOptions)) { SqlExplain explain = (SqlExplain) sqlNodeAndOptions.getSqlNode(); plannerContext.setOptions(sqlNodeAndOptions.getOptions()); RelRoot relRoot = compileQuery(explain.getExplicandum(), plannerContext); @@ -209,11 +263,6 @@ public QueryPlannerResult explainQuery(String sqlQuery, SqlNodeAndOptions sqlNod } } - @VisibleForTesting - public DispatchableSubPlan planQuery(String sqlQuery) { - return planQuery(sqlQuery, CalciteSqlParser.compileToSqlNodeAndOptions(sqlQuery), 0).getQueryPlan(); - } - @VisibleForTesting public String explainQuery(String sqlQuery, long requestId) { SqlNodeAndOptions sqlNodeAndOptions = CalciteSqlParser.compileToSqlNodeAndOptions(sqlQuery); @@ -222,7 +271,7 @@ public String explainQuery(String sqlQuery, long requestId) { } public List getTableNamesForQuery(String sqlQuery) { - try (PlannerContext plannerContext = getPlannerContext()) { + try (PlannerContext plannerContext = getParsingPlannerContext()) { SqlNode sqlNode = CalciteSqlParser.compileToSqlNodeAndOptions(sqlQuery).getSqlNode(); if (sqlNode.getKind().equals(SqlKind.EXPLAIN)) { sqlNode = ((SqlExplain) sqlNode).getExplicandum(); @@ -239,7 +288,7 @@ public List getTableNamesForQuery(String sqlQuery) { * Returns whether the query can be successfully compiled in this query environment */ public boolean canCompileQuery(String query) { - try (PlannerContext plannerContext = getPlannerContext()) { + try (PlannerContext plannerContext = getParsingPlannerContext()) { SqlNode sqlNode = CalciteSqlParser.compileToSqlNodeAndOptions(query).getSqlNode(); if (sqlNode.getKind().equals(SqlKind.EXPLAIN)) { sqlNode = ((SqlExplain) sqlNode).getExplicandum(); @@ -353,7 +402,7 @@ private DispatchableSubPlan toDispatchableSubPlan(RelRoot relRoot, PlannerContex @Nullable TransformationTracker.Builder tracker) { SubPlan plan = PinotLogicalQueryPlanner.makePlan(relRoot, tracker); PinotDispatchPlanner pinotDispatchPlanner = - new PinotDispatchPlanner(plannerContext, _workerManager, requestId, _tableCache); + new PinotDispatchPlanner(plannerContext, _envConfig.getWorkerManager(), requestId, _envConfig.getTableCache()); return pinotDispatchPlanner.createDispatchableSubPlan(plan); } @@ -385,7 +434,7 @@ private static HepProgram getOptProgram() { return hepProgramBuilder.build(); } - private static HepProgram getTraitProgram() { + private static HepProgram getTraitProgram(@Nullable WorkerManager workerManager) { HepProgramBuilder hepProgramBuilder = new HepProgramBuilder(); // Set the match order as BOTTOM_UP. @@ -398,8 +447,44 @@ private static HepProgram getTraitProgram() { } // apply RelDistribution trait to all nodes + if (workerManager != null) { + hepProgramBuilder.addRuleInstance(PinotImplicitTableHintRule.withWorkerManager(workerManager)); + } hepProgramBuilder.addRuleInstance(PinotRelDistributionTraitRule.INSTANCE); return hepProgramBuilder.build(); } + + public static ImmutableQueryEnvironment.Config.Builder configBuilder() { + return ImmutableQueryEnvironment.Config.builder(); + } + + @Value.Immutable + public interface Config { + String getDatabase(); + + @Nullable // In theory nullable only in tests. We should fix LiteralOnlyBrokerRequestTest to not need this. + TableCache getTableCache(); + + /** + * Whether to apply partition hint by default or not. + * + * This is treated as the default value for the broker and it is expected to be obtained from a Pinot configuration. + * This default value can be always overridden at query level by the query option + * {@link CommonConstants.Broker.Request.QueryOptionKey#INFER_PARTITION_HINT}. + */ + @Value.Default + default boolean defaultInferPartitionHint() { + return CommonConstants.Broker.DEFAULT_INFER_PARTITION_HINT; + } + + /** + * Returns the worker manager. + * + * This is used whenever the query needs to be executed, but can be null when the QueryEnvironment will be used + * just to execute some static analysis on the query like parsing it or getting the tables involved in the query. + */ + @Nullable + WorkerManager getWorkerManager(); + } } diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/explain/ExplainNodeSimplifier.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/explain/ExplainNodeSimplifier.java index e7a1b8d48d03..17b0b02fe1ed 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/explain/ExplainNodeSimplifier.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/explain/ExplainNodeSimplifier.java @@ -21,10 +21,7 @@ import com.google.common.base.CaseFormat; import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.List; -import java.util.Map; -import org.apache.pinot.common.proto.Plan; import org.apache.pinot.core.query.reduce.ExplainPlanDataTableReducer; import org.apache.pinot.query.planner.plannode.AggregateNode; import org.apache.pinot.query.planner.plannode.ExchangeNode; @@ -54,10 +51,8 @@ *

  • Its title must contain the text {@code Combine}
  • * * - * Also nodes with only one input are not simplifiable by definition. - * - * Simplified nodes will have a new attribute {@code repeated} that will contain the number of times the node was - * repeated. + * The simplification process merges the inputs of the node into a single node. + * As a corollary, nodes with only one input are already simplified by definition. */ public class ExplainNodeSimplifier { private static final Logger LOGGER = LoggerFactory.getLogger(ExplainNodeSimplifier.class); @@ -73,8 +68,6 @@ public static PlanNode simplifyNode(PlanNode root) { } private static class Visitor implements PlanNodeVisitor { - private static final String REPEAT_ATTRIBUTE_KEY = "repeated"; - private PlanNode defaultNode(PlanNode node) { List inputs = node.getInputs(); List newInputs = simplifyChildren(inputs); @@ -159,13 +152,8 @@ public PlanNode visitExplained(ExplainedNode node, Void context) { } child1 = merged; } - Map attributes = new HashMap<>(node.getAttributes()); - Plan.ExplainNode.AttributeValue repeatedValue = Plan.ExplainNode.AttributeValue.newBuilder() - .setLong(simplifiedChildren.size()) - .build(); - attributes.put(REPEAT_ATTRIBUTE_KEY, repeatedValue); return new ExplainedNode(node.getStageId(), node.getDataSchema(), node.getNodeHint(), - Collections.singletonList(child1), node.getTitle(), attributes); + Collections.singletonList(child1), node.getTitle(), node.getAttributes()); } private List simplifyChildren(List children) { diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/explain/PlanNodeMerger.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/explain/PlanNodeMerger.java index 1541a8b6742a..aa2e44173b4e 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/explain/PlanNodeMerger.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/explain/PlanNodeMerger.java @@ -18,14 +18,18 @@ */ package org.apache.pinot.query.planner.explain; +import com.google.common.base.CaseFormat; import com.google.common.collect.Streams; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; import javax.annotation.Nullable; import org.apache.pinot.common.proto.Plan; import org.apache.pinot.core.operator.ExplainAttributeBuilder; +import org.apache.pinot.core.query.reduce.ExplainPlanDataTableReducer; import org.apache.pinot.query.planner.plannode.AggregateNode; import org.apache.pinot.query.planner.plannode.ExchangeNode; import org.apache.pinot.query.planner.plannode.ExplainedNode; @@ -93,6 +97,8 @@ public static PlanNode mergePlans(PlanNode plan1, PlanNode plan2, boolean verbos } private static class Visitor implements PlanNodeVisitor { + public static final String COMBINE + = CaseFormat.UPPER_UNDERSCORE.to(CaseFormat.UPPER_CAMEL, ExplainPlanDataTableReducer.COMBINE); private final boolean _verbose; public Visitor(boolean verbose) { @@ -441,7 +447,12 @@ public PlanNode visitExplained(ExplainedNode node, PlanNode context) { Map selfAttributes = node.getAttributes(); Map otherAttributes = otherNode.getAttributes(); - List children = mergeChildren(node, context); + List children; + if (node.getTitle().contains(COMBINE)) { + children = mergeCombineChildren(node, otherNode); + } else { + children = mergeChildren(node, context); + } if (children == null) { return null; } @@ -486,23 +497,23 @@ public PlanNode visitExplained(ExplainedNode node, PlanNode context) { if (selfValue.hasLong() && otherValue.hasLong()) { // If both are long, add them attributeBuilder.putLong(selfEntry.getKey(), selfValue.getLong() + otherValue.getLong()); } else { // Otherwise behave as if they are idempotent - if (!Objects.equals(otherValue, selfEntry.getValue())) { + if (!Objects.equals(otherValue, selfValue)) { return null; } - attributeBuilder.putAttribute(selfEntry.getKey(), selfEntry.getValue()); + attributeBuilder.putAttribute(selfEntry.getKey(), selfValue); } break; } case IDEMPOTENT: { - if (!Objects.equals(otherValue, selfEntry.getValue())) { + if (!Objects.equals(otherValue, selfValue)) { return null; } - attributeBuilder.putAttribute(selfEntry.getKey(), selfEntry.getValue()); + attributeBuilder.putAttribute(selfEntry.getKey(), selfValue); break; } case IGNORABLE: { - if (Objects.equals(otherValue, selfEntry.getValue())) { - attributeBuilder.putAttribute(selfEntry.getKey(), selfEntry.getValue()); + if (Objects.equals(otherValue, selfValue)) { + attributeBuilder.putAttribute(selfEntry.getKey(), selfValue); } else if (_verbose) { // If mode is verbose, we will not merge the nodes when an ignorable attribute is different return null; @@ -518,13 +529,49 @@ public PlanNode visitExplained(ExplainedNode node, PlanNode context) { } for (Map.Entry otherEntry : otherAttributes.entrySet()) { Plan.ExplainNode.AttributeValue selfValue = selfAttributes.get(otherEntry.getKey()); - if (selfValue == null) { // otherwise it has already been merged - attributeBuilder.putAttribute(otherEntry.getKey(), otherEntry.getValue()); + if (selfValue != null) { // it has already been merged + continue; + } + switch (otherEntry.getValue().getMergeType()) { + case DEFAULT: + attributeBuilder.putAttribute(otherEntry.getKey(), otherEntry.getValue()); + break; + case IGNORABLE: + if (_verbose) { + return null; + } + break; + case IDEMPOTENT: + case UNRECOGNIZED: + default: + return null; } } return new ExplainedNode(node.getStageId(), node.getDataSchema(), node.getNodeHint(), children, node.getTitle(), attributeBuilder.build()); } } + + private List mergeCombineChildren(ExplainedNode node1, ExplainedNode node2) { + List mergedChildren = new ArrayList<>(node1.getInputs().size() + node2.getInputs().size()); + + Set pendingOn2 = new HashSet<>(node2.getInputs()); + for (PlanNode input1 : node1.getInputs()) { + PlanNode merged = null; + for (PlanNode input2 : pendingOn2) { + merged = mergePlans(input1, input2); + if (merged != null) { + pendingOn2.remove(input2); + break; + } + } + mergedChildren.add(merged != null ? merged : input1); + } + mergedChildren.addAll(pendingOn2); + + mergedChildren.sort(PlanNodeSorter.DefaultComparator.INSTANCE); + + return mergedChildren; + } } } diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/routing/WorkerManager.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/routing/WorkerManager.java index ba3a709fed00..6af598bb3da7 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/routing/WorkerManager.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/routing/WorkerManager.java @@ -33,6 +33,8 @@ import org.apache.calcite.rel.RelDistribution; import org.apache.commons.lang3.tuple.Pair; import org.apache.pinot.calcite.rel.hint.PinotHintOptions; +import org.apache.pinot.calcite.rel.rules.ImmutableTableOptions; +import org.apache.pinot.calcite.rel.rules.TableOptions; import org.apache.pinot.core.routing.RoutingManager; import org.apache.pinot.core.routing.RoutingTable; import org.apache.pinot.core.routing.TablePartitionInfo; @@ -407,8 +409,10 @@ private void assignWorkersToPartitionedLeafFragment(DispatchablePlanMetadata met PinotHintOptions.TableHintOptions.PARTITION_PARALLELISM, partitionParallelism); String tableName = metadata.getScannedTables().get(0); - PartitionTableInfo partitionTableInfo = - getPartitionTableInfo(tableName, partitionKey, numPartitions, partitionFunction); + // calculates the partition table info using the routing manager + PartitionTableInfo partitionTableInfo = calculatePartitionTableInfo(tableName); + // verifies that the partition table obtained from routing manager is compatible with the hint options + checkPartitionInfoMap(partitionTableInfo, tableName, partitionKey, numPartitions, partitionFunction); // Pick one server per partition // NOTE: Pick server based on the request id so that the same server is picked across different table scan when the @@ -440,8 +444,21 @@ private void assignWorkersToPartitionedLeafFragment(DispatchablePlanMetadata met metadata.setPartitionParallelism(partitionParallelism); } - private PartitionTableInfo getPartitionTableInfo(String tableName, String partitionKey, int numPartitions, - String partitionFunction) { + @Nullable + public TableOptions inferTableOptions(String tableName) { + try { + PartitionTableInfo partitionTableInfo = calculatePartitionTableInfo(tableName); + return ImmutableTableOptions.builder() + .partitionFunction(partitionTableInfo._partitionFunction) + .partitionKey(partitionTableInfo._partitionKey) + .partitionSize(partitionTableInfo._numPartitions) + .build(); + } catch (IllegalStateException e) { + return null; + } + } + + private PartitionTableInfo calculatePartitionTableInfo(String tableName) { TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableName); if (tableType == null) { String offlineTableName = TableNameBuilder.OFFLINE.tableNameWithType(tableName); @@ -450,19 +467,27 @@ private PartitionTableInfo getPartitionTableInfo(String tableName, String partit boolean realtimeRoutingExists = _routingManager.routingExists(realtimeTableName); Preconditions.checkState(offlineRoutingExists || realtimeRoutingExists, "Routing doesn't exist for table: %s", tableName); + if (offlineRoutingExists && realtimeRoutingExists) { + TablePartitionInfo offlineTpi = _routingManager.getTablePartitionInfo(offlineTableName); + Preconditions.checkState(offlineTpi != null, "Failed to find table partition info for table: %s", + offlineTableName); + TablePartitionInfo realtimeTpi = _routingManager.getTablePartitionInfo(realtimeTableName); + Preconditions.checkState(realtimeTpi != null, "Failed to find table partition info for table: %s", + realtimeTableName); // For hybrid table, find the common servers for each partition TimeBoundaryInfo timeBoundaryInfo = _routingManager.getTimeBoundaryInfo(offlineTableName); // Ignore OFFLINE side when time boundary info is unavailable if (timeBoundaryInfo == null) { - return getRealtimePartitionTableInfo(realtimeTableName, partitionKey, numPartitions, partitionFunction); + return PartitionTableInfo.fromTablePartitionInfo(realtimeTpi, TableType.REALTIME); } - TablePartitionInfo.PartitionInfo[] offlinePartitionInfoMap = - getTablePartitionInfo(offlineTableName, partitionKey, numPartitions, - partitionFunction).getPartitionInfoMap(); - TablePartitionInfo.PartitionInfo[] realtimePartitionInfoMap = - getTablePartitionInfo(realtimeTableName, partitionKey, numPartitions, - partitionFunction).getPartitionInfoMap(); + + verifyCompatibility(offlineTpi, realtimeTpi); + + TablePartitionInfo.PartitionInfo[] offlinePartitionInfoMap = offlineTpi.getPartitionInfoMap(); + TablePartitionInfo.PartitionInfo[] realtimePartitionInfoMap = realtimeTpi.getPartitionInfoMap(); + + int numPartitions = offlineTpi.getNumPartitions(); PartitionInfo[] partitionInfoMap = new PartitionInfo[numPartitions]; for (int i = 0; i < numPartitions; i++) { TablePartitionInfo.PartitionInfo offlinePartitionInfo = offlinePartitionInfoMap[i]; @@ -484,79 +509,117 @@ private PartitionTableInfo getPartitionTableInfo(String tableName, String partit fullyReplicatedServers.retainAll(realtimePartitionInfo._fullyReplicatedServers); Preconditions.checkState(!fullyReplicatedServers.isEmpty(), "Failed to find fully replicated server for partition: %s in hybrid table: %s", i, tableName); - partitionInfoMap[i] = new PartitionInfo(fullyReplicatedServers, offlinePartitionInfo._segments, - realtimePartitionInfo._segments); + partitionInfoMap[i] = new PartitionInfo( + fullyReplicatedServers, offlinePartitionInfo._segments, realtimePartitionInfo._segments); } - return new PartitionTableInfo(partitionInfoMap, timeBoundaryInfo); + return new PartitionTableInfo(partitionInfoMap, timeBoundaryInfo, offlineTpi.getPartitionColumn(), + numPartitions, offlineTpi.getPartitionFunctionName()); } else if (offlineRoutingExists) { - return getOfflinePartitionTableInfo(offlineTableName, partitionKey, numPartitions, partitionFunction); + return getOfflinePartitionTableInfo(offlineTableName); } else { - return getRealtimePartitionTableInfo(realtimeTableName, partitionKey, numPartitions, partitionFunction); + return getRealtimePartitionTableInfo(realtimeTableName); } } else { if (tableType == TableType.OFFLINE) { - return getOfflinePartitionTableInfo(tableName, partitionKey, numPartitions, partitionFunction); + return getOfflinePartitionTableInfo(tableName); } else { - return getRealtimePartitionTableInfo(tableName, partitionKey, numPartitions, partitionFunction); + return getRealtimePartitionTableInfo(tableName); } } } - private TablePartitionInfo getTablePartitionInfo(String tableNameWithType, String partitionKey, int numPartitions, - String partitionFunction) { - TablePartitionInfo tablePartitionInfo = _routingManager.getTablePartitionInfo(tableNameWithType); - Preconditions.checkState(tablePartitionInfo != null, "Failed to find table partition info for table: %s", - tableNameWithType); - Preconditions.checkState(tablePartitionInfo.getPartitionColumn().equals(partitionKey), + private static void verifyCompatibility(TablePartitionInfo offlineTpi, TablePartitionInfo realtimeTpi) + throws IllegalArgumentException { + Preconditions.checkState(offlineTpi.getPartitionColumn().equals(realtimeTpi.getPartitionColumn()), + "Partition column mismatch for hybrid table %s: %s offline vs %s online", + offlineTpi.getTableNameWithType(), offlineTpi.getPartitionColumn(), realtimeTpi.getPartitionColumn()); + Preconditions.checkState(offlineTpi.getNumPartitions() == realtimeTpi.getNumPartitions(), + "Partition size mismatch for hybrid table %s: %s offline vs %s online", + offlineTpi.getTableNameWithType(), offlineTpi.getNumPartitions(), realtimeTpi.getNumPartitions()); + Preconditions.checkState( + offlineTpi.getPartitionFunctionName().equalsIgnoreCase(realtimeTpi.getPartitionFunctionName()), + "Partition function mismatch for hybrid table %s: %s offline vs %s online", + offlineTpi.getTableNameWithType(), offlineTpi.getPartitionFunctionName(), + realtimeTpi.getPartitionFunctionName()); + } + + /** + * Verifies that the partition info maps from the table partition info are compatible with the information supplied + * as arguments. + */ + private void checkPartitionInfoMap(PartitionTableInfo partitionTableInfo, String tableNameWithType, + String partitionKey, int numPartitions, String partitionFunction) { + Preconditions.checkState(partitionTableInfo._partitionKey.equals(partitionKey), "Partition key: %s does not match partition column: %s for table: %s", partitionKey, - tablePartitionInfo.getPartitionColumn(), tableNameWithType); - Preconditions.checkState(tablePartitionInfo.getNumPartitions() == numPartitions, + partitionTableInfo._partitionKey, tableNameWithType); + Preconditions.checkState(partitionTableInfo._numPartitions == numPartitions, "Partition size mismatch (hint: %s, table: %s) for table: %s", numPartitions, - tablePartitionInfo.getNumPartitions(), tableNameWithType); - Preconditions.checkState(tablePartitionInfo.getPartitionFunctionName().equalsIgnoreCase(partitionFunction), + partitionTableInfo._numPartitions, tableNameWithType); + Preconditions.checkState(partitionTableInfo._partitionFunction.equalsIgnoreCase(partitionFunction), "Partition function mismatch (hint: %s, table: %s) for table %s", partitionFunction, - tablePartitionInfo.getPartitionFunctionName(), tableNameWithType); - Preconditions.checkState(tablePartitionInfo.getSegmentsWithInvalidPartition().isEmpty(), - "Find %s segments with invalid partition for table: %s", - tablePartitionInfo.getSegmentsWithInvalidPartition().size(), tableNameWithType); - return tablePartitionInfo; + partitionTableInfo._partitionFunction, tableNameWithType); } - private PartitionTableInfo getOfflinePartitionTableInfo(String offlineTableName, String partitionKey, - int numPartitions, String partitionFunction) { - TablePartitionInfo.PartitionInfo[] tablePartitionInfoArr = - getTablePartitionInfo(offlineTableName, partitionKey, numPartitions, partitionFunction).getPartitionInfoMap(); - PartitionInfo[] partitionInfoMap = new PartitionInfo[numPartitions]; - for (int i = 0; i < numPartitions; i++) { - TablePartitionInfo.PartitionInfo partitionInfo = tablePartitionInfoArr[i]; - if (partitionInfo != null) { - partitionInfoMap[i] = new PartitionInfo(partitionInfo._fullyReplicatedServers, partitionInfo._segments, null); - } - } - return new PartitionTableInfo(partitionInfoMap, null); + private PartitionTableInfo getOfflinePartitionTableInfo(String offlineTableName) { + TablePartitionInfo offlineTpi = _routingManager.getTablePartitionInfo(offlineTableName); + Preconditions.checkState(offlineTpi != null, "Failed to find table partition info for table: %s", + offlineTableName); + return PartitionTableInfo.fromTablePartitionInfo(offlineTpi, TableType.OFFLINE); } - private PartitionTableInfo getRealtimePartitionTableInfo(String realtimeTableName, String partitionKey, - int numPartitions, String partitionFunction) { - TablePartitionInfo.PartitionInfo[] tablePartitionInfoArr = - getTablePartitionInfo(realtimeTableName, partitionKey, numPartitions, partitionFunction).getPartitionInfoMap(); - PartitionInfo[] partitionInfoMap = new PartitionInfo[numPartitions]; - for (int i = 0; i < numPartitions; i++) { - TablePartitionInfo.PartitionInfo partitionInfo = tablePartitionInfoArr[i]; - if (partitionInfo != null) { - partitionInfoMap[i] = new PartitionInfo(partitionInfo._fullyReplicatedServers, null, partitionInfo._segments); - } - } - return new PartitionTableInfo(partitionInfoMap, null); + private PartitionTableInfo getRealtimePartitionTableInfo(String realtimeTableName) { + TablePartitionInfo realtimeTpi = _routingManager.getTablePartitionInfo(realtimeTableName); + Preconditions.checkState(realtimeTpi != null, "Failed to find table partition info for table: %s", + realtimeTableName); + return PartitionTableInfo.fromTablePartitionInfo(realtimeTpi, TableType.REALTIME); } private static class PartitionTableInfo { final PartitionInfo[] _partitionInfoMap; + @Nullable final TimeBoundaryInfo _timeBoundaryInfo; + final String _partitionKey; + final int _numPartitions; + final String _partitionFunction; - PartitionTableInfo(PartitionInfo[] partitionInfoMap, @Nullable TimeBoundaryInfo timeBoundaryInfo) { + PartitionTableInfo(PartitionInfo[] partitionInfoMap, @Nullable TimeBoundaryInfo timeBoundaryInfo, + String partitionKey, int numPartitions, String partitionFunction) { _partitionInfoMap = partitionInfoMap; _timeBoundaryInfo = timeBoundaryInfo; + _partitionKey = partitionKey; + _numPartitions = numPartitions; + _partitionFunction = partitionFunction; + } + + public static PartitionTableInfo fromTablePartitionInfo( + TablePartitionInfo tablePartitionInfo, TableType tableType) { + if (!tablePartitionInfo.getSegmentsWithInvalidPartition().isEmpty()) { + throw new IllegalStateException("Find " + tablePartitionInfo.getSegmentsWithInvalidPartition().size() + + " segments with invalid partition"); + } + + int numPartitions = tablePartitionInfo.getNumPartitions(); + TablePartitionInfo.PartitionInfo[] tablePartitionInfoMap = tablePartitionInfo.getPartitionInfoMap(); + PartitionInfo[] workerPartitionInfoMap = new PartitionInfo[numPartitions]; + for (int i = 0; i < numPartitions; i++) { + TablePartitionInfo.PartitionInfo partitionInfo = tablePartitionInfoMap[i]; + if (partitionInfo != null) { + switch (tableType) { + case OFFLINE: + workerPartitionInfoMap[i] = + new PartitionInfo(partitionInfo._fullyReplicatedServers, partitionInfo._segments, null); + break; + case REALTIME: + workerPartitionInfoMap[i] = + new PartitionInfo(partitionInfo._fullyReplicatedServers, null, partitionInfo._segments); + break; + default: + throw new IllegalStateException("Unsupported table type: " + tableType); + } + } + } + return new PartitionTableInfo(workerPartitionInfoMap, null, tablePartitionInfo.getPartitionColumn(), + numPartitions, tablePartitionInfo.getPartitionFunctionName()); } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java index 5b80d3506545..a0739162a615 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java @@ -253,7 +253,8 @@ public void processTimeSeriesQuery(String serializedPlan, Map me final Consumer handleErrors = (t) -> { Map errorMetadata = new HashMap<>(); errorMetadata.put(WorkerResponseMetadataKeys.ERROR_TYPE, t.getClass().getSimpleName()); - errorMetadata.put(WorkerResponseMetadataKeys.ERROR_MESSAGE, t.getMessage()); + errorMetadata.put(WorkerResponseMetadataKeys.ERROR_MESSAGE, t.getMessage() == null + ? "Unknown error: no message" : t.getMessage()); responseObserver.onNext(Worker.TimeSeriesResponse.newBuilder().putAllMetadata(errorMetadata).build()); responseObserver.onCompleted(); }; @@ -280,6 +281,7 @@ public void processTimeSeriesQuery(String serializedPlan, Map me } }); } catch (Throwable t) { + LOGGER.error("Error running time-series query", t); handleErrors.accept(t); } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageGroupByExecutor.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageGroupByExecutor.java index 1f3f3a20fc32..41501f69383e 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageGroupByExecutor.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageGroupByExecutor.java @@ -58,6 +58,7 @@ public class MultistageGroupByExecutor { private final AggType _aggType; private final DataSchema _resultSchema; private final int _numGroupsLimit; + private final boolean _filteredAggregationsSkipEmptyGroups; // Group By Result holders for each mode private final GroupByResultHolder[] _aggregateResultHolders; @@ -79,6 +80,10 @@ public MultistageGroupByExecutor(int[] groupKeyIds, AggregationFunction[] aggFun int maxInitialResultHolderCapacity = getMaxInitialResultHolderCapacity(opChainMetadata, nodeHint); _numGroupsLimit = getNumGroupsLimit(opChainMetadata, nodeHint); + // By default, we compute all groups for SQL compliant results. However, we allow overriding this behavior via + // query option for improved performance. + _filteredAggregationsSkipEmptyGroups = QueryOptionsUtils.isFilteredAggregationsSkipEmptyGroups(opChainMetadata); + int numFunctions = aggFunctions.length; if (!aggType.isInputIntermediateFormat()) { _aggregateResultHolders = new GroupByResultHolder[numFunctions]; @@ -241,6 +246,12 @@ private void processAggregate(TransferableBlock block) { aggFunction.aggregateGroupBySV(numMatchedRows, filteredIntKeys, groupByResultHolder, blockValSetMap); } } + if (intKeys == null && !_filteredAggregationsSkipEmptyGroups) { + // _groupIdGenerator should still have all the groups even if there are only filtered aggregates for SQL + // compliant results. However, if the query option to skip empty groups is set, we avoid this step for + // improved performance. + generateGroupByKeys(block); + } } } diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/QueryRunnerTestBase.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/QueryRunnerTestBase.java index e6d6d4342481..8fb1a6c75852 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/QueryRunnerTestBase.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/queries/QueryRunnerTestBase.java @@ -113,8 +113,8 @@ protected QueryEnvironment.QueryPlannerResult planQuery(String sql) { protected QueryDispatcher.QueryResult queryRunner(String sql, boolean trace) { long requestId = REQUEST_ID_GEN.getAndIncrement(); SqlNodeAndOptions sqlNodeAndOptions = CalciteSqlParser.compileToSqlNodeAndOptions(sql); - QueryEnvironment.QueryPlannerResult queryPlannerResult = - _queryEnvironment.planQuery(sql, sqlNodeAndOptions, requestId); + QueryEnvironment.QueryPlannerResult queryPlannerResult = _queryEnvironment.planQuery(sql, sqlNodeAndOptions, + requestId); DispatchableSubPlan dispatchableSubPlan = queryPlannerResult.getQueryPlan(); Map requestMetadataMap = new HashMap<>(); requestMetadataMap.put(CommonConstants.Query.Request.MetadataKeys.REQUEST_ID, String.valueOf(requestId)); diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/timeseries/PhysicalTimeSeriesPlanVisitorTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/timeseries/PhysicalTimeSeriesPlanVisitorTest.java index 929e2669cf6a..8f877aec016d 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/timeseries/PhysicalTimeSeriesPlanVisitorTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/timeseries/PhysicalTimeSeriesPlanVisitorTest.java @@ -54,7 +54,7 @@ public void testCompileQueryContext() { assertEquals(queryContext.getTimeSeriesContext().getTimeColumn(), timeColumn); assertEquals(queryContext.getTimeSeriesContext().getValueExpression().getIdentifier(), "orderCount"); assertEquals(queryContext.getFilter().toString(), - "(cityName = 'Chicago' AND orderTime >= '1000' AND orderTime <= '2000')"); + "(cityName = 'Chicago' AND orderTime >= '1000' AND orderTime < '2000')"); } // Case-2: With offset, complex group-by expression, complex value, and non-empty filter { @@ -75,7 +75,7 @@ public void testCompileQueryContext() { assertEquals(queryContext.getTimeSeriesContext().getValueExpression().toString(), "times(orderCount,'2')"); assertNotNull(queryContext.getFilter()); assertEquals(queryContext.getFilter().toString(), - "(cityName = 'Chicago' AND orderTime >= '990' AND orderTime <= '1990')"); + "(cityName = 'Chicago' AND orderTime >= '990' AND orderTime < '1990')"); } } } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkForwardIndexWriter.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkForwardIndexWriter.java index fadcce827e0a..0a6bd4792569 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkForwardIndexWriter.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkForwardIndexWriter.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.math.BigDecimal; import javax.annotation.concurrent.NotThreadSafe; +import org.apache.pinot.segment.local.utils.ArraySerDeUtils; import org.apache.pinot.segment.spi.compression.ChunkCompressionType; import org.apache.pinot.spi.utils.BigDecimalUtils; @@ -97,54 +98,34 @@ public void putBytes(byte[] value) { writeChunkIfNecessary(); } - // Note: some duplication is tolerated between these overloads for the sake of memory efficiency + @Override + public void putIntMV(int[] values) { + putBytes(ArraySerDeUtils.serializeIntArrayWithLength(values)); + } + + @Override + public void putLongMV(long[] values) { + putBytes(ArraySerDeUtils.serializeLongArrayWithLength(values)); + } + + @Override + public void putFloatMV(float[] values) { + putBytes(ArraySerDeUtils.serializeFloatArrayWithLength(values)); + } + + @Override + public void putDoubleMV(double[] values) { + putBytes(ArraySerDeUtils.serializeDoubleArrayWithLength(values)); + } @Override public void putStringMV(String[] values) { - // the entire String[] will be encoded as a single string, write the header here - _chunkBuffer.putInt(_chunkHeaderOffset, _chunkDataOffSet); - _chunkHeaderOffset += CHUNK_HEADER_ENTRY_ROW_OFFSET_SIZE; - // write all the strings into the data buffer as if it's a single string, - // but with its own embedded header so offsets to strings within the body - // can be located - _chunkBuffer.putInt(_chunkDataOffSet, values.length); - _chunkDataOffSet += Integer.BYTES; - int headerSize = Integer.BYTES * values.length; - int bodyPosition = _chunkDataOffSet + headerSize; - _chunkBuffer.position(bodyPosition); - int bodySize = 0; - for (int i = 0, h = _chunkDataOffSet; i < values.length; i++, h += Integer.BYTES) { - byte[] utf8 = values[i].getBytes(UTF_8); - _chunkBuffer.putInt(h, utf8.length); - _chunkBuffer.put(utf8); - bodySize += utf8.length; - } - _chunkDataOffSet += headerSize + bodySize; - writeChunkIfNecessary(); + putBytes(ArraySerDeUtils.serializeStringArray(values)); } @Override public void putBytesMV(byte[][] values) { - // the entire byte[][] will be encoded as a single string, write the header here - _chunkBuffer.putInt(_chunkHeaderOffset, _chunkDataOffSet); - _chunkHeaderOffset += CHUNK_HEADER_ENTRY_ROW_OFFSET_SIZE; - // write all the byte[]s into the data buffer as if it's a single byte[], - // but with its own embedded header so offsets to byte[]s within the body - // can be located - _chunkBuffer.putInt(_chunkDataOffSet, values.length); - _chunkDataOffSet += Integer.BYTES; - int headerSize = Integer.BYTES * values.length; - int bodyPosition = _chunkDataOffSet + headerSize; - _chunkBuffer.position(bodyPosition); - int bodySize = 0; - for (int i = 0, h = _chunkDataOffSet; i < values.length; i++, h += Integer.BYTES) { - byte[] bytes = values[i]; - _chunkBuffer.putInt(h, bytes.length); - _chunkBuffer.put(bytes); - bodySize += bytes.length; - } - _chunkDataOffSet += headerSize + bodySize; - writeChunkIfNecessary(); + putBytes(ArraySerDeUtils.serializeBytesArray(values)); } private void writeChunkIfNecessary() { diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkForwardIndexWriterV4.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkForwardIndexWriterV4.java index 440808a6b0bd..332c52d0c59f 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkForwardIndexWriterV4.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkForwardIndexWriterV4.java @@ -30,6 +30,7 @@ import javax.annotation.concurrent.NotThreadSafe; import org.apache.commons.io.FileUtils; import org.apache.pinot.segment.local.io.compression.ChunkCompressorFactory; +import org.apache.pinot.segment.local.utils.ArraySerDeUtils; import org.apache.pinot.segment.spi.compression.ChunkCompressionType; import org.apache.pinot.segment.spi.compression.ChunkCompressor; import org.apache.pinot.segment.spi.memory.CleanerUtil; @@ -37,8 +38,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static java.nio.charset.StandardCharsets.UTF_8; - /** * Chunk-based raw (non-dictionary-encoded) forward index writer where each chunk contains variable number of docs, and @@ -77,7 +76,9 @@ public class VarByteChunkForwardIndexWriterV4 implements VarByteChunkWriter { public static final int VERSION = 4; - private static final Logger LOGGER = LoggerFactory.getLogger(VarByteChunkForwardIndexWriterV4.class); + // Use the run-time concrete class to retrieve the logger + protected final Logger _logger = LoggerFactory.getLogger(getClass()); + private static final String DATA_BUFFER_SUFFIX = ".buf"; private final File _dataBuffer; @@ -106,11 +107,16 @@ public VarByteChunkForwardIndexWriterV4(File file, ChunkCompressionType compress writeHeader(_chunkCompressor.compressionType(), chunkSize); } + // Child class must override this class instance method + protected int getVersion() { + return VERSION; + } + private void writeHeader(ChunkCompressionType compressionType, int targetDecompressedChunkSize) throws IOException { // keep metadata BE for backwards compatibility // (e.g. the version needs to be read by a factory which assumes BE) - _output.writeInt(VERSION); + _output.writeInt(getVersion()); _output.writeInt(targetDecompressedChunkSize); _output.writeInt(compressionType.getValue()); // reserve a slot to write the data offset into @@ -145,49 +151,33 @@ public void putBytes(byte[] bytes) { } @Override - public void putStringMV(String[] values) { - // num values + length of each value - int headerSize = Integer.BYTES + Integer.BYTES * values.length; - int size = headerSize; - byte[][] stringBytes = new byte[values.length][]; - for (int i = 0; i < values.length; i++) { - stringBytes[i] = values[i].getBytes(UTF_8); - size += stringBytes[i].length; - } + public void putIntMV(int[] values) { + putBytes(ArraySerDeUtils.serializeIntArrayWithLength(values)); + } - // Format : [numValues][length1][length2]...[lengthN][value1][value2]...[valueN] - byte[] serializedBytes = new byte[size]; - ByteBuffer byteBuffer = ByteBuffer.wrap(serializedBytes); - byteBuffer.putInt(values.length); - byteBuffer.position(headerSize); - for (int i = 0; i < values.length; i++) { - byteBuffer.putInt((i + 1) * Integer.BYTES, stringBytes[i].length); - byteBuffer.put(stringBytes[i]); - } + @Override + public void putLongMV(long[] values) { + putBytes(ArraySerDeUtils.serializeLongArrayWithLength(values)); + } - putBytes(serializedBytes); + @Override + public void putFloatMV(float[] values) { + putBytes(ArraySerDeUtils.serializeFloatArrayWithLength(values)); } @Override - public void putBytesMV(byte[][] values) { - // num values + length of each value - int headerSize = Integer.BYTES + Integer.BYTES * values.length; - int size = headerSize; - for (byte[] value : values) { - size += value.length; - } + public void putDoubleMV(double[] values) { + putBytes(ArraySerDeUtils.serializeDoubleArrayWithLength(values)); + } - // Format : [numValues][length1][length2]...[lengthN][bytes1][bytes2]...[bytesN] - byte[] serializedBytes = new byte[size]; - ByteBuffer byteBuffer = ByteBuffer.wrap(serializedBytes); - byteBuffer.putInt(values.length); - byteBuffer.position(headerSize); - for (int i = 0; i < values.length; i++) { - byteBuffer.putInt((i + 1) * Integer.BYTES, values[i].length); - byteBuffer.put(values[i]); - } + @Override + public void putStringMV(String[] values) { + putBytes(ArraySerDeUtils.serializeStringArray(values)); + } - putBytes(serializedBytes); + @Override + public void putBytesMV(byte[][] values) { + putBytes(ArraySerDeUtils.serializeBytesArray(values)); } private void writeHugeChunk(byte[] bytes) { @@ -287,7 +277,7 @@ private void write(ByteBuffer buffer, boolean huge) { _chunkOffset += compressedSize; _docIdOffset = _nextDocId; } catch (IOException e) { - LOGGER.error("Exception caught while compressing/writing data chunk", e); + _logger.error("Exception caught while compressing/writing data chunk", e); throw new RuntimeException(e); } finally { if (mapped != null) { diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkForwardIndexWriterV5.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkForwardIndexWriterV5.java new file mode 100644 index 000000000000..b96812a05936 --- /dev/null +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkForwardIndexWriterV5.java @@ -0,0 +1,117 @@ +/** + * 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.pinot.segment.local.io.writer.impl; + +import java.io.File; +import java.io.IOException; +import javax.annotation.concurrent.NotThreadSafe; +import org.apache.pinot.segment.local.utils.ArraySerDeUtils; +import org.apache.pinot.segment.spi.compression.ChunkCompressionType; + + +/** + * Forward index writer that extends {@link VarByteChunkForwardIndexWriterV4} and overrides the data layout for + * multi-value fixed byte operations to improve space efficiency. + * + *

    Consider the following multi-value document as an example: {@code [int(1), int(2), int(3)]}. + * The current binary data layout in {@code VarByteChunkForwardIndexWriterV4} is as follows:

    + *
    + *     0x00000010 0x00000003 0x00000001 0x00000002 0x00000003
    + * 
    + * + *
      + *
    1. The first 4 bytes ({@code 0x00000010}) represent the total payload length of the byte array + * containing the multi-value document content, which in this case is 16 bytes.
    2. + * + *
    3. The next 4 bytes ({@code 0x00000003}) represent the number of elements in the multi-value document (i.e., 3) + * .
    4. + * + *
    5. The remaining 12 bytes ({@code 0x00000001 0x00000002 0x00000003}) represent the 3 integer values of the + * multi-value document: 1, 2, and 3.
    6. + *
    + * + *

    In Pinot, the fixed byte raw forward index can only store one specific fixed-length data type: + * {@code int}, {@code long}, {@code float}, or {@code double}. Instead of explicitly storing the number of elements + * for each document for multi-value document, this value can be inferred by:

    + *
    + *     number of elements = buffer payload length / size of data type
    + * 
    + * + *

    If the forward index uses the passthrough chunk compression type (i.e., no compression), we can save + * 4 bytes per document by omitting the explicit element count. This leads to the following space savings:

    + * + *
      + *
    • For documents with 0 elements, we save 50%.
    • + *
    • For documents with 1 element, we save 33%.
    • + *
    • For documents with 2 elements, we save 25%.
    • + *
    • As the number of elements increases, the percentage of space saved decreases.
    • + *
    + * + *

    For forward indexes that use compression to reduce data size, the savings can be even more significant + * in certain cases. This is demonstrated in the unit test {@link VarByteChunkV5Test#validateCompressionRatioIncrease}, + * where ZStandard was used as the chunk compressor. In the test, 1 million short multi-value (MV) documents + * were inserted, following a Gaussian distribution for document lengths. Additionally, the values of each integer + * in the MV documents were somewhat repetitive. Under these conditions, we observed a 50%+ reduction in on-disk + * file size compared to the V4 forward index writer version.

    + * + *

    Note that the {@code VERSION} tag is a {@code static final} class variable set to {@code 5}. Since static + * variables are shadowed in the child class thus associated with the class that defines them, care must be taken to + * ensure that the parent class can correctly observe the child class's {@code VERSION} value at runtime. To handle + * this cleanly and correctly, the {@code getVersion()} method is overridden to return the concrete subclass's + * {@code VERSION} value, ensuring that the correct version number is returned even when using a reference to the + * parent class.

    + * + * @see VarByteChunkForwardIndexWriterV4 + * @see VarByteChunkForwardIndexWriterV5#getVersion() + */ +@NotThreadSafe +public class VarByteChunkForwardIndexWriterV5 extends VarByteChunkForwardIndexWriterV4 { + public static final int VERSION = 5; + + public VarByteChunkForwardIndexWriterV5(File file, ChunkCompressionType compressionType, int chunkSize) + throws IOException { + super(file, compressionType, chunkSize); + } + + // Override the parent class getVersion(); + @Override + public int getVersion() { + return VERSION; + } + + @Override + public void putIntMV(int[] values) { + putBytes(ArraySerDeUtils.serializeIntArrayWithoutLength(values)); + } + + @Override + public void putLongMV(long[] values) { + putBytes(ArraySerDeUtils.serializeLongArrayWithoutLength(values)); + } + + @Override + public void putFloatMV(float[] values) { + putBytes(ArraySerDeUtils.serializeFloatArrayWithoutLength(values)); + } + + @Override + public void putDoubleMV(double[] values) { + putBytes(ArraySerDeUtils.serializeDoubleArrayWithoutLength(values)); + } +} diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkWriter.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkWriter.java index bf3537d67c98..82fbd2b4691e 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkWriter.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/io/writer/impl/VarByteChunkWriter.java @@ -29,6 +29,14 @@ public interface VarByteChunkWriter extends Closeable { void putBytes(byte[] value); + void putIntMV(int[] values); + + void putLongMV(long[] values); + + void putFloatMV(float[] values); + + void putDoubleMV(double[] values); + void putStringMV(String[] values); void putBytesMV(byte[][] values); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java index 749c4cf704a0..b395e5b21cd8 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java @@ -293,8 +293,9 @@ private boolean createDictionaryForColumn(ColumnIndexCreationInfo info, SegmentG FieldIndexConfigs fieldIndexConfigs = config.getIndexConfigsByColName().get(column); if (DictionaryIndexType.ignoreDictionaryOverride(config.isOptimizeDictionary(), - config.isOptimizeDictionaryForMetrics(), config.getNoDictionarySizeRatioThreshold(), spec, fieldIndexConfigs, - info.getDistinctValueCount(), info.getTotalNumberOfEntries())) { + config.isOptimizeDictionaryForMetrics(), config.getNoDictionarySizeRatioThreshold(), + config.getNoDictionaryCardinalityRatioThreshold(), spec, fieldIndexConfigs, info.getDistinctValueCount(), + info.getTotalNumberOfEntries())) { // Ignore overrides and pick from config createDictionary = info.isCreateDictionary(); } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/MultiValueFixedByteRawIndexCreator.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/MultiValueFixedByteRawIndexCreator.java index dcdcb9970516..b8a6bd6daafd 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/MultiValueFixedByteRawIndexCreator.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/MultiValueFixedByteRawIndexCreator.java @@ -20,9 +20,9 @@ import java.io.File; import java.io.IOException; -import java.nio.ByteBuffer; import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriter; import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriterV4; +import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriterV5; import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkWriter; import org.apache.pinot.segment.spi.V1Constants.Indexes; import org.apache.pinot.segment.spi.compression.ChunkCompressionType; @@ -70,23 +70,32 @@ public MultiValueFixedByteRawIndexCreator(File indexFile, ChunkCompressionType c ForwardIndexConfig.DEFAULT_TARGET_DOCS_PER_CHUNK); } - public MultiValueFixedByteRawIndexCreator(File indexFile, ChunkCompressionType compressionType, int totalDocs, DataType valueType, int maxNumberOfMultiValueElements, boolean deriveNumDocsPerChunk, int writerVersion, int targetMaxChunkSizeBytes, int targetDocsPerChunk) throws IOException { - // Store the length followed by the values - int totalMaxLength = Integer.BYTES + (maxNumberOfMultiValueElements * valueType.getStoredType().size()); if (writerVersion < VarByteChunkForwardIndexWriterV4.VERSION) { + // Store the length followed by the values + int totalMaxLength = Integer.BYTES + (maxNumberOfMultiValueElements * valueType.getStoredType().size()); int numDocsPerChunk = deriveNumDocsPerChunk ? Math.max(targetMaxChunkSizeBytes / (totalMaxLength + VarByteChunkForwardIndexWriter.CHUNK_HEADER_ENTRY_ROW_OFFSET_SIZE), 1) : targetDocsPerChunk; _indexWriter = new VarByteChunkForwardIndexWriter(indexFile, compressionType, totalDocs, numDocsPerChunk, totalMaxLength, writerVersion); } else { - int chunkSize = - ForwardIndexUtils.getDynamicTargetChunkSize(totalMaxLength, targetDocsPerChunk, targetMaxChunkSizeBytes); - _indexWriter = new VarByteChunkForwardIndexWriterV4(indexFile, compressionType, chunkSize); + if (writerVersion == VarByteChunkForwardIndexWriterV5.VERSION) { + // Store only the values + int totalMaxLength = maxNumberOfMultiValueElements * valueType.getStoredType().size(); + int chunkSize = + ForwardIndexUtils.getDynamicTargetChunkSize(totalMaxLength, targetDocsPerChunk, targetMaxChunkSizeBytes); + _indexWriter = new VarByteChunkForwardIndexWriterV5(indexFile, compressionType, chunkSize); + } else { + // Store the length followed by the values + int totalMaxLength = Integer.BYTES + (maxNumberOfMultiValueElements * valueType.getStoredType().size()); + int chunkSize = + ForwardIndexUtils.getDynamicTargetChunkSize(totalMaxLength, targetDocsPerChunk, targetMaxChunkSizeBytes); + _indexWriter = new VarByteChunkForwardIndexWriterV4(indexFile, compressionType, chunkSize); + } } _valueType = valueType; } @@ -108,54 +117,22 @@ public DataType getValueType() { @Override public void putIntMV(int[] values) { - byte[] bytes = new byte[Integer.BYTES + values.length * Integer.BYTES]; - ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); - //write the length - byteBuffer.putInt(values.length); - //write the content of each element - for (int value : values) { - byteBuffer.putInt(value); - } - _indexWriter.putBytes(bytes); + _indexWriter.putIntMV(values); } @Override public void putLongMV(long[] values) { - byte[] bytes = new byte[Integer.BYTES + values.length * Long.BYTES]; - ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); - //write the length - byteBuffer.putInt(values.length); - //write the content of each element - for (long value : values) { - byteBuffer.putLong(value); - } - _indexWriter.putBytes(bytes); + _indexWriter.putLongMV(values); } @Override public void putFloatMV(float[] values) { - byte[] bytes = new byte[Integer.BYTES + values.length * Float.BYTES]; - ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); - //write the length - byteBuffer.putInt(values.length); - //write the content of each element - for (float value : values) { - byteBuffer.putFloat(value); - } - _indexWriter.putBytes(bytes); + _indexWriter.putFloatMV(values); } @Override public void putDoubleMV(double[] values) { - byte[] bytes = new byte[Integer.BYTES + values.length * Double.BYTES]; - ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); - //write the length - byteBuffer.putInt(values.length); - //write the content of each element - for (double value : values) { - byteBuffer.putDouble(value); - } - _indexWriter.putBytes(bytes); + _indexWriter.putDoubleMV(values); } @Override diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/dictionary/DictionaryIndexType.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/dictionary/DictionaryIndexType.java index 9df6d24daaad..9fb6dbce87cc 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/dictionary/DictionaryIndexType.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/dictionary/DictionaryIndexType.java @@ -218,11 +218,12 @@ public static boolean shouldUseVarLengthDictionary(FieldSpec.DataType columnStor * This function evaluates whether to override dictionary (i.e use noDictionary) * for a column even when its explicitly configured. This evaluation is for both dimension and metric * column types. + * + * @return true if dictionary should be created, false if noDictionary should be used */ - public static boolean ignoreDictionaryOverride(boolean optimizeDictionary, - boolean optimizeDictionaryForMetrics, double noDictionarySizeRatioThreshold, - FieldSpec fieldSpec, FieldIndexConfigs fieldIndexConfigs, int cardinality, - int totalNumberOfEntries) { + public static boolean ignoreDictionaryOverride(boolean optimizeDictionary, boolean optimizeDictionaryForMetrics, + double noDictionarySizeRatioThreshold, @Nullable Double noDictionaryCardinalityRatioThreshold, + FieldSpec fieldSpec, FieldIndexConfigs fieldIndexConfigs, int cardinality, int totalNumberOfEntries) { // For an inverted index dictionary is required if (fieldIndexConfigs.getConfig(StandardIndexes.inverted()).isEnabled()) { return true; @@ -236,22 +237,38 @@ public static boolean ignoreDictionaryOverride(boolean optimizeDictionary, // Do not create dictionary if index size with dictionary is going to be larger than index size without dictionary // This is done to reduce the cost of dictionary for high cardinality columns // Off by default and needs optimizeDictionary to be set to true - if (fieldSpec.isSingleValueField() && fieldSpec.getDataType().isFixedWidth()) { - // if you can safely enable dictionary, you can ignore overrides - return canSafelyCreateDictionaryWithinThreshold(cardinality, totalNumberOfEntries, - noDictionarySizeRatioThreshold, fieldSpec); + if (fieldSpec.isSingleValueField()) { + return ignoreDictionaryOverrideForSingleValueFields(cardinality, totalNumberOfEntries, + noDictionarySizeRatioThreshold, noDictionaryCardinalityRatioThreshold, fieldSpec); } } + if (optimizeDictionaryForMetrics && !optimizeDictionary && fieldSpec.isSingleValueField() + && fieldSpec.getFieldType() == FieldSpec.FieldType.METRIC) { + return ignoreDictionaryOverrideForSingleValueFields(cardinality, totalNumberOfEntries, + noDictionarySizeRatioThreshold, noDictionaryCardinalityRatioThreshold, fieldSpec); + } + return true; + } - if (optimizeDictionaryForMetrics && !optimizeDictionary) { - if (fieldSpec.isSingleValueField() && fieldSpec.getDataType().isFixedWidth() && fieldSpec.getFieldType() - == FieldSpec.FieldType.METRIC) { + /** + * Hold common logic for ignoring dictionary override for single value fields, used for dim and metric cols + */ + private static boolean ignoreDictionaryOverrideForSingleValueFields(int cardinality, int totalNumberOfEntries, + double noDictionarySizeRatioThreshold, Double noDictionaryCardinalityRatioThreshold, FieldSpec fieldSpec) { + if (fieldSpec.isSingleValueField()) { + if (fieldSpec.getDataType().isFixedWidth()) { // if you can safely enable dictionary, you can ignore overrides return canSafelyCreateDictionaryWithinThreshold(cardinality, totalNumberOfEntries, noDictionarySizeRatioThreshold, fieldSpec); } + // Config not set, default to old behavior of create dictionary for var width cols + if (noDictionaryCardinalityRatioThreshold == null) { + return true; + } + // Variable width type, so create based simply on cardinality threshold since size cannot be calculated easily + return noDictionaryCardinalityRatioThreshold * totalNumberOfEntries > cardinality; } - return true; + return false; } /** diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java index db815761d9ea..cc7201ed985f 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java @@ -21,6 +21,7 @@ import java.util.Arrays; import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriterV4; +import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriterV5; import org.apache.pinot.segment.local.segment.creator.impl.fwd.CLPForwardIndexCreatorV1; import org.apache.pinot.segment.local.segment.index.readers.forward.CLPForwardIndexReaderV1; import org.apache.pinot.segment.local.segment.index.readers.forward.FixedBitMVEntryDictForwardIndexReader; @@ -30,6 +31,7 @@ import org.apache.pinot.segment.local.segment.index.readers.forward.FixedByteChunkSVForwardIndexReader; import org.apache.pinot.segment.local.segment.index.readers.forward.FixedBytePower2ChunkSVForwardIndexReader; import org.apache.pinot.segment.local.segment.index.readers.forward.VarByteChunkForwardIndexReaderV4; +import org.apache.pinot.segment.local.segment.index.readers.forward.VarByteChunkForwardIndexReaderV5; import org.apache.pinot.segment.local.segment.index.readers.forward.VarByteChunkMVForwardIndexReader; import org.apache.pinot.segment.local.segment.index.readers.forward.VarByteChunkSVForwardIndexReader; import org.apache.pinot.segment.local.segment.index.readers.sorted.SortedIndexReaderImpl; @@ -106,7 +108,10 @@ public static ForwardIndexReader createRawIndexReader(PinotDataBuffer dataBuffer : new FixedByteChunkSVForwardIndexReader(dataBuffer, storedType); } - if (version == VarByteChunkForwardIndexWriterV4.VERSION) { + if (version == VarByteChunkForwardIndexWriterV5.VERSION) { + // V5 is the same as V4 except the multi-value docs have implicit value count rather than explicit + return new VarByteChunkForwardIndexReaderV5(dataBuffer, storedType, isSingleValue); + } else if (version == VarByteChunkForwardIndexWriterV4.VERSION) { // V4 reader is common for sv var byte, mv fixed byte and mv var byte return new VarByteChunkForwardIndexReaderV4(dataBuffer, storedType, isSingleValue); } else { diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java index 3b654a7ebc59..21dd9e18c250 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java @@ -303,9 +303,10 @@ Map> computeOperations(SegmentDirectory.Reader segmentRe if (existingColumnMetadata.getFieldSpec().getFieldType() != FieldSpec.FieldType.COMPLEX && DictionaryIndexType.ignoreDictionaryOverride(_tableConfig.getIndexingConfig().isOptimizeDictionary(), _tableConfig.getIndexingConfig().isOptimizeDictionaryForMetrics(), - _tableConfig.getIndexingConfig().getNoDictionarySizeRatioThreshold(), existingColumnMetadata.getFieldSpec(), - _fieldIndexConfigs.get(column), existingColumnMetadata.getCardinality(), - existingColumnMetadata.getTotalNumberOfEntries())) { + _tableConfig.getIndexingConfig().getNoDictionarySizeRatioThreshold(), + _tableConfig.getIndexingConfig().getNoDictionaryCardinalityRatioThreshold(), + existingColumnMetadata.getFieldSpec(), _fieldIndexConfigs.get(column), + existingColumnMetadata.getCardinality(), existingColumnMetadata.getTotalNumberOfEntries())) { columnOperationsMap.put(column, Collections.singletonList(Operation.ENABLE_DICTIONARY)); } } else if (existingHasDict && !newIsDict) { diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/IndexLoadingConfig.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/IndexLoadingConfig.java index 0684fe9097d5..51e33e544453 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/IndexLoadingConfig.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/IndexLoadingConfig.java @@ -83,7 +83,6 @@ public class IndexLoadingConfig { private final Map _noDictionaryConfig = new HashMap<>(); private final Set _varLengthDictionaryColumns = new HashSet<>(); private Set _onHeapDictionaryColumns = new HashSet<>(); - private final Set _forwardIndexDisabledColumns = new HashSet<>(); private Map _bloomFilterConfigs = new HashMap<>(); private boolean _enableDynamicStarTreeCreation; private List _starTreeIndexConfigs; @@ -540,10 +539,6 @@ public Set getOnHeapDictionaryColumns() { return unmodifiable(_onHeapDictionaryColumns); } - public Set getForwardIndexDisabledColumns() { - return unmodifiable(_forwardIndexDisabledColumns); - } - public Map getBloomFilterConfigs() { return unmodifiable(_bloomFilterConfigs); } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java index e655f9e470b9..f6dc1a159b35 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java @@ -411,7 +411,7 @@ protected boolean createColumnV1Indices(String column) } // TODO: Support forward index disabled derived column - if (_indexLoadingConfig.getForwardIndexDisabledColumns().contains(column)) { + if (isForwardIndexDisabled(column)) { LOGGER.warn("Skip creating forward index disabled derived column: {}", column); if (errorOnFailure) { throw new UnsupportedOperationException( @@ -443,8 +443,8 @@ protected boolean createColumnV1Indices(String column) * Check and return whether the forward index is disabled for a given column */ protected boolean isForwardIndexDisabled(String column) { - return _indexLoadingConfig.getForwardIndexDisabledColumns() != null - && _indexLoadingConfig.getForwardIndexDisabledColumns().contains(column); + FieldIndexConfigs fieldIndexConfig = _indexLoadingConfig.getFieldIndexConfig(column); + return fieldIndexConfig != null && fieldIndexConfig.getConfig(StandardIndexes.forward()).isDisabled(); } /** diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/FixedByteChunkMVForwardIndexReader.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/FixedByteChunkMVForwardIndexReader.java index f96ed6e878a6..8e53ecb15639 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/FixedByteChunkMVForwardIndexReader.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/FixedByteChunkMVForwardIndexReader.java @@ -22,6 +22,7 @@ import java.util.List; import javax.annotation.Nullable; import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriter; +import org.apache.pinot.segment.local.utils.ArraySerDeUtils; import org.apache.pinot.segment.spi.memory.PinotDataBuffer; import org.apache.pinot.spi.data.FieldSpec.DataType; @@ -53,92 +54,47 @@ public ChunkReaderContext createContext() { @Override public int getIntMV(int docId, int[] valueBuffer, ChunkReaderContext context) { - ByteBuffer byteBuffer = slice(docId, context); - int numValues = byteBuffer.getInt(); - for (int i = 0; i < numValues; i++) { - valueBuffer[i] = byteBuffer.getInt(); - } - return numValues; + return ArraySerDeUtils.deserializeIntArrayWithLength(slice(docId, context), valueBuffer); } @Override public int[] getIntMV(int docId, ChunkReaderContext context) { - ByteBuffer byteBuffer = slice(docId, context); - int numValues = byteBuffer.getInt(); - int[] valueBuffer = new int[numValues]; - for (int i = 0; i < numValues; i++) { - valueBuffer[i] = byteBuffer.getInt(); - } - return valueBuffer; + return ArraySerDeUtils.deserializeIntArrayWithLength(slice(docId, context)); } @Override public int getLongMV(int docId, long[] valueBuffer, ChunkReaderContext context) { - ByteBuffer byteBuffer = slice(docId, context); - int numValues = byteBuffer.getInt(); - for (int i = 0; i < numValues; i++) { - valueBuffer[i] = byteBuffer.getLong(); - } - return numValues; + return ArraySerDeUtils.deserializeLongArrayWithLength(slice(docId, context), valueBuffer); } @Override public long[] getLongMV(int docId, ChunkReaderContext context) { - ByteBuffer byteBuffer = slice(docId, context); - int numValues = byteBuffer.getInt(); - long[] valueBuffer = new long[numValues]; - for (int i = 0; i < numValues; i++) { - valueBuffer[i] = byteBuffer.getLong(); - } - return valueBuffer; + return ArraySerDeUtils.deserializeLongArrayWithLength(slice(docId, context)); } @Override public int getFloatMV(int docId, float[] valueBuffer, ChunkReaderContext context) { - ByteBuffer byteBuffer = slice(docId, context); - int numValues = byteBuffer.getInt(); - for (int i = 0; i < numValues; i++) { - valueBuffer[i] = byteBuffer.getFloat(); - } - return numValues; + return ArraySerDeUtils.deserializeFloatArrayWithLength(slice(docId, context), valueBuffer); } @Override public float[] getFloatMV(int docId, ChunkReaderContext context) { - ByteBuffer byteBuffer = slice(docId, context); - int numValues = byteBuffer.getInt(); - float[] valueBuffer = new float[numValues]; - for (int i = 0; i < numValues; i++) { - valueBuffer[i] = byteBuffer.getFloat(); - } - return valueBuffer; + return ArraySerDeUtils.deserializeFloatArrayWithLength(slice(docId, context)); } @Override public int getDoubleMV(int docId, double[] valueBuffer, ChunkReaderContext context) { - ByteBuffer byteBuffer = slice(docId, context); - int numValues = byteBuffer.getInt(); - for (int i = 0; i < numValues; i++) { - valueBuffer[i] = byteBuffer.getDouble(); - } - return numValues; + return ArraySerDeUtils.deserializeDoubleArrayWithLength(slice(docId, context), valueBuffer); } @Override public double[] getDoubleMV(int docId, ChunkReaderContext context) { - ByteBuffer byteBuffer = slice(docId, context); - int numValues = byteBuffer.getInt(); - double[] valueBuffer = new double[numValues]; - for (int i = 0; i < numValues; i++) { - valueBuffer[i] = byteBuffer.getDouble(); - } - return valueBuffer; + return ArraySerDeUtils.deserializeDoubleArrayWithLength(slice(docId, context)); } @Override public int getNumValuesMV(int docId, ChunkReaderContext context) { - ByteBuffer byteBuffer = slice(docId, context); - return byteBuffer.getInt(); + return slice(docId, context).getInt(); } private ByteBuffer slice(int docId, ChunkReaderContext context) { diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/VarByteChunkForwardIndexReaderV4.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/VarByteChunkForwardIndexReaderV4.java index 851f5f3e7d7d..cf2a8b4de4da 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/VarByteChunkForwardIndexReaderV4.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/VarByteChunkForwardIndexReaderV4.java @@ -29,6 +29,7 @@ import java.util.Map; import org.apache.pinot.segment.local.io.compression.ChunkCompressorFactory; import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriterV4; +import org.apache.pinot.segment.local.utils.ArraySerDeUtils; import org.apache.pinot.segment.spi.compression.ChunkCompressionType; import org.apache.pinot.segment.spi.compression.ChunkDecompressor; import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader; @@ -46,6 +47,8 @@ * Chunk-based raw (non-dictionary-encoded) forward index reader for values of SV variable length data types * (BIG_DECIMAL, STRING, BYTES), MV fixed length and MV variable length data types. *

    For data layout, please refer to the documentation for {@link VarByteChunkForwardIndexWriterV4} + * + * TODO: Consider reading directly from sliced ByteBuffer instead of copying to byte[] first */ public class VarByteChunkForwardIndexReaderV4 implements ForwardIndexReader { @@ -64,8 +67,7 @@ public class VarByteChunkForwardIndexReaderV4 public VarByteChunkForwardIndexReaderV4(PinotDataBuffer dataBuffer, FieldSpec.DataType storedType, boolean isSingleValue) { - int version = dataBuffer.getInt(0); - Preconditions.checkState(version == VarByteChunkForwardIndexWriterV4.VERSION, "Illegal index version: %s", version); + validateIndexVersion(dataBuffer); _storedType = storedType; _targetDecompressedChunkSize = dataBuffer.getInt(4); _chunkCompressionType = ChunkCompressionType.valueOf(dataBuffer.getInt(8)); @@ -78,6 +80,15 @@ public VarByteChunkForwardIndexReaderV4(PinotDataBuffer dataBuffer, FieldSpec.Da _isSingleValue = isSingleValue; } + public void validateIndexVersion(PinotDataBuffer dataBuffer) { + int version = dataBuffer.getInt(0); + Preconditions.checkState(version == getVersion(), "Illegal index version: %s", version); + } + + public int getVersion() { + return VarByteChunkForwardIndexWriterV4.VERSION; + } + @Override public boolean isDictionaryEncoded() { return false; @@ -130,144 +141,62 @@ public Map getMap(int docId, ReaderContext context) { @Override public int getIntMV(int docId, int[] valueBuffer, VarByteChunkForwardIndexReaderV4.ReaderContext context) { - ByteBuffer byteBuffer = ByteBuffer.wrap(context.getValue(docId)); - int numValues = byteBuffer.getInt(); - for (int i = 0; i < numValues; i++) { - valueBuffer[i] = byteBuffer.getInt(); - } - return numValues; + return ArraySerDeUtils.deserializeIntArrayWithLength(context.getValue(docId), valueBuffer); } @Override public int[] getIntMV(int docId, VarByteChunkForwardIndexReaderV4.ReaderContext context) { - ByteBuffer byteBuffer = ByteBuffer.wrap(context.getValue(docId)); - int numValues = byteBuffer.getInt(); - int[] valueBuffer = new int[numValues]; - for (int i = 0; i < numValues; i++) { - valueBuffer[i] = byteBuffer.getInt(); - } - return valueBuffer; + return ArraySerDeUtils.deserializeIntArrayWithLength(context.getValue(docId)); } @Override public int getLongMV(int docId, long[] valueBuffer, VarByteChunkForwardIndexReaderV4.ReaderContext context) { - ByteBuffer byteBuffer = ByteBuffer.wrap(context.getValue(docId)); - int numValues = byteBuffer.getInt(); - for (int i = 0; i < numValues; i++) { - valueBuffer[i] = byteBuffer.getLong(); - } - return numValues; + return ArraySerDeUtils.deserializeLongArrayWithLength(context.getValue(docId), valueBuffer); } @Override public long[] getLongMV(int docId, VarByteChunkForwardIndexReaderV4.ReaderContext context) { - ByteBuffer byteBuffer = ByteBuffer.wrap(context.getValue(docId)); - int numValues = byteBuffer.getInt(); - long[] valueBuffer = new long[numValues]; - for (int i = 0; i < numValues; i++) { - valueBuffer[i] = byteBuffer.getLong(); - } - return valueBuffer; + return ArraySerDeUtils.deserializeLongArrayWithLength(context.getValue(docId)); } @Override public int getFloatMV(int docId, float[] valueBuffer, VarByteChunkForwardIndexReaderV4.ReaderContext context) { - ByteBuffer byteBuffer = ByteBuffer.wrap(context.getValue(docId)); - int numValues = byteBuffer.getInt(); - for (int i = 0; i < numValues; i++) { - valueBuffer[i] = byteBuffer.getFloat(); - } - return numValues; + return ArraySerDeUtils.deserializeFloatArrayWithLength(context.getValue(docId), valueBuffer); } @Override public float[] getFloatMV(int docId, VarByteChunkForwardIndexReaderV4.ReaderContext context) { - ByteBuffer byteBuffer = ByteBuffer.wrap(context.getValue(docId)); - int numValues = byteBuffer.getInt(); - float[] valueBuffer = new float[numValues]; - for (int i = 0; i < numValues; i++) { - valueBuffer[i] = byteBuffer.getFloat(); - } - return valueBuffer; + return ArraySerDeUtils.deserializeFloatArrayWithLength(context.getValue(docId)); } @Override public int getDoubleMV(int docId, double[] valueBuffer, VarByteChunkForwardIndexReaderV4.ReaderContext context) { - ByteBuffer byteBuffer = ByteBuffer.wrap(context.getValue(docId)); - int numValues = byteBuffer.getInt(); - for (int i = 0; i < numValues; i++) { - valueBuffer[i] = byteBuffer.getDouble(); - } - return numValues; + return ArraySerDeUtils.deserializeDoubleArrayWithLength(context.getValue(docId), valueBuffer); } @Override public double[] getDoubleMV(int docId, VarByteChunkForwardIndexReaderV4.ReaderContext context) { - ByteBuffer byteBuffer = ByteBuffer.wrap(context.getValue(docId)); - int numValues = byteBuffer.getInt(); - double[] valueBuffer = new double[numValues]; - for (int i = 0; i < numValues; i++) { - valueBuffer[i] = byteBuffer.getFloat(); - } - return valueBuffer; + return ArraySerDeUtils.deserializeDoubleArrayWithLength(context.getValue(docId)); } @Override public int getStringMV(int docId, String[] valueBuffer, VarByteChunkForwardIndexReaderV4.ReaderContext context) { - byte[] bytes = context.getValue(docId); - ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); - int numValues = byteBuffer.getInt(); - int offset = (numValues + 1) * Integer.BYTES; - for (int i = 0; i < numValues; i++) { - int length = byteBuffer.getInt((i + 1) * Integer.BYTES); - valueBuffer[i] = new String(bytes, offset, length, StandardCharsets.UTF_8); - offset += length; - } - return numValues; + return ArraySerDeUtils.deserializeStringArray(context.getValue(docId), valueBuffer); } @Override public String[] getStringMV(int docId, VarByteChunkForwardIndexReaderV4.ReaderContext context) { - byte[] bytes = context.getValue(docId); - ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); - int numValues = byteBuffer.getInt(); - int offset = (numValues + 1) * Integer.BYTES; - String[] valueBuffer = new String[numValues]; - for (int i = 0; i < numValues; i++) { - int length = byteBuffer.getInt((i + 1) * Integer.BYTES); - valueBuffer[i] = new String(bytes, offset, length, StandardCharsets.UTF_8); - offset += length; - } - return valueBuffer; + return ArraySerDeUtils.deserializeStringArray(context.getValue(docId)); } @Override public int getBytesMV(int docId, byte[][] valueBuffer, VarByteChunkForwardIndexReaderV4.ReaderContext context) { - ByteBuffer byteBuffer = ByteBuffer.wrap(context.getValue(docId)); - int numValues = byteBuffer.getInt(); - byteBuffer.position((numValues + 1) * Integer.BYTES); - for (int i = 0; i < numValues; i++) { - int length = byteBuffer.getInt((i + 1) * Integer.BYTES); - byte[] bytes = new byte[length]; - byteBuffer.get(bytes, 0, length); - valueBuffer[i] = bytes; - } - return numValues; + return ArraySerDeUtils.deserializeBytesArray(context.getValue(docId), valueBuffer); } @Override public byte[][] getBytesMV(int docId, VarByteChunkForwardIndexReaderV4.ReaderContext context) { - ByteBuffer byteBuffer = ByteBuffer.wrap(context.getValue(docId)); - int numValues = byteBuffer.getInt(); - byteBuffer.position((numValues + 1) * Integer.BYTES); - byte[][] valueBuffer = new byte[numValues][]; - for (int i = 0; i < numValues; i++) { - int length = byteBuffer.getInt((i + 1) * Integer.BYTES); - byte[] bytes = new byte[length]; - byteBuffer.get(bytes, 0, length); - valueBuffer[i] = bytes; - } - return valueBuffer; + return ArraySerDeUtils.deserializeBytesArray(context.getValue(docId)); } @Override diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/VarByteChunkForwardIndexReaderV5.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/VarByteChunkForwardIndexReaderV5.java new file mode 100644 index 000000000000..e72fedfc584e --- /dev/null +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/VarByteChunkForwardIndexReaderV5.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.pinot.segment.local.segment.index.readers.forward; + +import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriterV4; +import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriterV5; +import org.apache.pinot.segment.local.utils.ArraySerDeUtils; +import org.apache.pinot.segment.spi.memory.PinotDataBuffer; +import org.apache.pinot.spi.data.FieldSpec; + + +/** + * Chunk-based raw (non-dictionary-encoded) forward index reader for values of SV variable length data types + * (BIG_DECIMAL, STRING, BYTES), MV fixed length and MV variable length data types. + *

    For data layout, please refer to the documentation for {@link VarByteChunkForwardIndexWriterV4} + */ +public class VarByteChunkForwardIndexReaderV5 extends VarByteChunkForwardIndexReaderV4 { + public VarByteChunkForwardIndexReaderV5(PinotDataBuffer dataBuffer, FieldSpec.DataType storedType, + boolean isSingleValue) { + super(dataBuffer, storedType, isSingleValue); + } + + @Override + public int getVersion() { + return VarByteChunkForwardIndexWriterV5.VERSION; + } + + @Override + public int getIntMV(int docId, int[] valueBuffer, VarByteChunkForwardIndexReaderV4.ReaderContext context) { + return ArraySerDeUtils.deserializeIntArrayWithoutLength(context.getValue(docId), valueBuffer); + } + + @Override + public int[] getIntMV(int docId, VarByteChunkForwardIndexReaderV4.ReaderContext context) { + return ArraySerDeUtils.deserializeIntArrayWithoutLength(context.getValue(docId)); + } + + @Override + public int getLongMV(int docId, long[] valueBuffer, VarByteChunkForwardIndexReaderV4.ReaderContext context) { + return ArraySerDeUtils.deserializeLongArrayWithoutLength(context.getValue(docId), valueBuffer); + } + + @Override + public long[] getLongMV(int docId, VarByteChunkForwardIndexReaderV4.ReaderContext context) { + return ArraySerDeUtils.deserializeLongArrayWithoutLength(context.getValue(docId)); + } + + @Override + public int getFloatMV(int docId, float[] valueBuffer, VarByteChunkForwardIndexReaderV4.ReaderContext context) { + return ArraySerDeUtils.deserializeFloatArrayWithoutLength(context.getValue(docId), valueBuffer); + } + + @Override + public float[] getFloatMV(int docId, VarByteChunkForwardIndexReaderV4.ReaderContext context) { + return ArraySerDeUtils.deserializeFloatArrayWithoutLength(context.getValue(docId)); + } + + @Override + public int getDoubleMV(int docId, double[] valueBuffer, VarByteChunkForwardIndexReaderV4.ReaderContext context) { + return ArraySerDeUtils.deserializeDoubleArrayWithoutLength(context.getValue(docId), valueBuffer); + } + + @Override + public double[] getDoubleMV(int docId, VarByteChunkForwardIndexReaderV4.ReaderContext context) { + return ArraySerDeUtils.deserializeDoubleArrayWithoutLength(context.getValue(docId)); + } +} diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/VarByteChunkMVForwardIndexReader.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/VarByteChunkMVForwardIndexReader.java index 1d133bb896c3..bd7dc5dbb746 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/VarByteChunkMVForwardIndexReader.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/VarByteChunkMVForwardIndexReader.java @@ -19,10 +19,10 @@ package org.apache.pinot.segment.local.segment.index.readers.forward; import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; import java.util.List; import javax.annotation.Nullable; import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriter; +import org.apache.pinot.segment.local.utils.ArraySerDeUtils; import org.apache.pinot.segment.spi.memory.PinotDataBuffer; import org.apache.pinot.spi.data.FieldSpec.DataType; @@ -31,6 +31,8 @@ * Chunk-based multi-value raw (non-dictionary-encoded) forward index reader for values of variable length data type * (STRING, BYTES). *

    For data layout, please refer to the documentation for {@link VarByteChunkForwardIndexWriter} + * + * TODO: Consider reading directly from sliced ByteBuffer instead of copying to byte[] first */ public final class VarByteChunkMVForwardIndexReader extends BaseChunkForwardIndexReader { private static final int ROW_OFFSET_SIZE = VarByteChunkForwardIndexWriter.CHUNK_HEADER_ENTRY_ROW_OFFSET_SIZE; @@ -53,105 +55,28 @@ public ChunkReaderContext createContext() { } @Override - public int getStringMV(final int docId, final String[] valueBuffer, final ChunkReaderContext context) { - byte[] compressedBytes; - if (_isCompressed) { - compressedBytes = getBytesCompressed(docId, context); - } else { - compressedBytes = getBytesUncompressed(docId); - } - ByteBuffer byteBuffer = ByteBuffer.wrap(compressedBytes); - int numValues = byteBuffer.getInt(); - int contentOffset = (numValues + 1) * Integer.BYTES; - for (int i = 0; i < numValues; i++) { - int length = byteBuffer.getInt((i + 1) * Integer.BYTES); - byte[] bytes = new byte[length]; - byteBuffer.position(contentOffset); - byteBuffer.get(bytes, 0, length); - valueBuffer[i] = new String(bytes, StandardCharsets.UTF_8); - contentOffset += length; - } - return numValues; + public int getStringMV(int docId, String[] valueBuffer, ChunkReaderContext context) { + return ArraySerDeUtils.deserializeStringArray(getBytes(docId, context), valueBuffer); } @Override - public String[] getStringMV(final int docId, final ChunkReaderContext context) { - byte[] compressedBytes; - if (_isCompressed) { - compressedBytes = getBytesCompressed(docId, context); - } else { - compressedBytes = getBytesUncompressed(docId); - } - ByteBuffer byteBuffer = ByteBuffer.wrap(compressedBytes); - int numValues = byteBuffer.getInt(); - int contentOffset = (numValues + 1) * Integer.BYTES; - String[] valueBuffer = new String[numValues]; - for (int i = 0; i < numValues; i++) { - int length = byteBuffer.getInt((i + 1) * Integer.BYTES); - byte[] bytes = new byte[length]; - byteBuffer.position(contentOffset); - byteBuffer.get(bytes, 0, length); - valueBuffer[i] = new String(bytes, StandardCharsets.UTF_8); - contentOffset += length; - } - return valueBuffer; + public String[] getStringMV(int docId, ChunkReaderContext context) { + return ArraySerDeUtils.deserializeStringArray(getBytes(docId, context)); } @Override - public int getBytesMV(final int docId, final byte[][] valueBuffer, final ChunkReaderContext context) { - byte[] compressedBytes; - if (_isCompressed) { - compressedBytes = getBytesCompressed(docId, context); - } else { - compressedBytes = getBytesUncompressed(docId); - } - ByteBuffer byteBuffer = ByteBuffer.wrap(compressedBytes); - int numValues = byteBuffer.getInt(); - int contentOffset = (numValues + 1) * Integer.BYTES; - for (int i = 0; i < numValues; i++) { - int length = byteBuffer.getInt((i + 1) * Integer.BYTES); - byte[] bytes = new byte[length]; - byteBuffer.position(contentOffset); - byteBuffer.get(bytes, 0, length); - valueBuffer[i] = bytes; - contentOffset += length; - } - return numValues; + public int getBytesMV(int docId, byte[][] valueBuffer, ChunkReaderContext context) { + return ArraySerDeUtils.deserializeBytesArray(getBytes(docId, context), valueBuffer); } @Override - public byte[][] getBytesMV(final int docId, final ChunkReaderContext context) { - byte[] compressedBytes; - if (_isCompressed) { - compressedBytes = getBytesCompressed(docId, context); - } else { - compressedBytes = getBytesUncompressed(docId); - } - ByteBuffer byteBuffer = ByteBuffer.wrap(compressedBytes); - int numValues = byteBuffer.getInt(); - int contentOffset = (numValues + 1) * Integer.BYTES; - byte[][] valueBuffer = new byte[numValues][]; - for (int i = 0; i < numValues; i++) { - int length = byteBuffer.getInt((i + 1) * Integer.BYTES); - byte[] bytes = new byte[length]; - byteBuffer.position(contentOffset); - byteBuffer.get(bytes, 0, length); - valueBuffer[i] = bytes; - contentOffset += length; - } - return valueBuffer; + public byte[][] getBytesMV(int docId, ChunkReaderContext context) { + return ArraySerDeUtils.deserializeBytesArray(getBytes(docId, context)); } @Override - public int getNumValuesMV(final int docId, final ChunkReaderContext context) { - byte[] compressedBytes; - if (_isCompressed) { - compressedBytes = getBytesCompressed(docId, context); - } else { - compressedBytes = getBytesUncompressed(docId); - } - ByteBuffer byteBuffer = ByteBuffer.wrap(compressedBytes); - return byteBuffer.getInt(); + public int getNumValuesMV(int docId, ChunkReaderContext context) { + return ByteBuffer.wrap(getBytes(docId, context)).getInt(); } @Override diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/ArraySerDeUtils.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/ArraySerDeUtils.java new file mode 100644 index 000000000000..58238a33e068 --- /dev/null +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/ArraySerDeUtils.java @@ -0,0 +1,363 @@ +/** + * 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.pinot.segment.local.utils; + +import java.nio.ByteBuffer; + +import static java.nio.charset.StandardCharsets.UTF_8; + + +public class ArraySerDeUtils { + private ArraySerDeUtils() { + } + + public static byte[] serializeIntArrayWithLength(int[] values) { + byte[] bytes = new byte[Integer.BYTES + values.length * Integer.BYTES]; + ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); + byteBuffer.putInt(values.length); + writeValues(byteBuffer, values); + return bytes; + } + + public static int[] deserializeIntArrayWithLength(byte[] bytes) { + return deserializeIntArrayWithLength(ByteBuffer.wrap(bytes)); + } + + public static int[] deserializeIntArrayWithLength(ByteBuffer byteBuffer) { + int length = byteBuffer.getInt(); + int[] values = new int[length]; + readValues(byteBuffer, values, length); + return values; + } + + public static int deserializeIntArrayWithLength(byte[] bytes, int[] values) { + return deserializeIntArrayWithLength(ByteBuffer.wrap(bytes), values); + } + + public static int deserializeIntArrayWithLength(ByteBuffer byteBuffer, int[] values) { + int length = byteBuffer.getInt(); + readValues(byteBuffer, values, length); + return length; + } + + public static byte[] serializeIntArrayWithoutLength(int[] values) { + byte[] bytes = new byte[values.length * Integer.BYTES]; + ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); + writeValues(byteBuffer, values); + return bytes; + } + + public static int[] deserializeIntArrayWithoutLength(byte[] bytes) { + ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); + int length = bytes.length / Integer.BYTES; + int[] values = new int[length]; + readValues(byteBuffer, values, length); + return values; + } + + public static int deserializeIntArrayWithoutLength(byte[] bytes, int[] values) { + ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); + int length = bytes.length / Integer.BYTES; + readValues(byteBuffer, values, length); + return length; + } + + private static void writeValues(ByteBuffer byteBuffer, int[] values) { + for (int value : values) { + byteBuffer.putInt(value); + } + } + + private static void readValues(ByteBuffer byteBuffer, int[] values, int length) { + for (int i = 0; i < length; i++) { + values[i] = byteBuffer.getInt(); + } + } + + public static byte[] serializeLongArrayWithLength(long[] values) { + byte[] bytes = new byte[Integer.BYTES + values.length * Long.BYTES]; + ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); + byteBuffer.putInt(values.length); + writeValues(byteBuffer, values); + return bytes; + } + + public static long[] deserializeLongArrayWithLength(byte[] bytes) { + return deserializeLongArrayWithLength(ByteBuffer.wrap(bytes)); + } + + public static long[] deserializeLongArrayWithLength(ByteBuffer byteBuffer) { + int length = byteBuffer.getInt(); + long[] values = new long[length]; + readValues(byteBuffer, values, length); + return values; + } + + public static int deserializeLongArrayWithLength(byte[] bytes, long[] values) { + return deserializeLongArrayWithLength(ByteBuffer.wrap(bytes), values); + } + + public static int deserializeLongArrayWithLength(ByteBuffer byteBuffer, long[] values) { + int length = byteBuffer.getInt(); + readValues(byteBuffer, values, length); + return length; + } + + public static byte[] serializeLongArrayWithoutLength(long[] values) { + byte[] bytes = new byte[values.length * Long.BYTES]; + ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); + writeValues(byteBuffer, values); + return bytes; + } + + public static long[] deserializeLongArrayWithoutLength(byte[] bytes) { + ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); + int length = bytes.length / Long.BYTES; + long[] values = new long[length]; + readValues(byteBuffer, values, length); + return values; + } + + public static int deserializeLongArrayWithoutLength(byte[] bytes, long[] values) { + ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); + int length = bytes.length / Long.BYTES; + readValues(byteBuffer, values, length); + return length; + } + + private static void writeValues(ByteBuffer byteBuffer, long[] values) { + for (long value : values) { + byteBuffer.putLong(value); + } + } + + private static void readValues(ByteBuffer byteBuffer, long[] values, int length) { + for (int i = 0; i < length; i++) { + values[i] = byteBuffer.getLong(); + } + } + + public static byte[] serializeFloatArrayWithLength(float[] values) { + byte[] bytes = new byte[Integer.BYTES + values.length * Float.BYTES]; + ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); + byteBuffer.putInt(values.length); + writeValues(byteBuffer, values); + return bytes; + } + + public static float[] deserializeFloatArrayWithLength(byte[] bytes) { + return deserializeFloatArrayWithLength(ByteBuffer.wrap(bytes)); + } + + public static float[] deserializeFloatArrayWithLength(ByteBuffer byteBuffer) { + int length = byteBuffer.getInt(); + float[] values = new float[length]; + readValues(byteBuffer, values, length); + return values; + } + + public static int deserializeFloatArrayWithLength(byte[] bytes, float[] values) { + return deserializeFloatArrayWithLength(ByteBuffer.wrap(bytes), values); + } + + public static int deserializeFloatArrayWithLength(ByteBuffer byteBuffer, float[] values) { + int length = byteBuffer.getInt(); + readValues(byteBuffer, values, length); + return length; + } + + public static byte[] serializeFloatArrayWithoutLength(float[] values) { + byte[] bytes = new byte[values.length * Float.BYTES]; + ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); + writeValues(byteBuffer, values); + return bytes; + } + + public static float[] deserializeFloatArrayWithoutLength(byte[] bytes) { + ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); + int length = bytes.length / Float.BYTES; + float[] values = new float[length]; + readValues(byteBuffer, values, length); + return values; + } + + public static int deserializeFloatArrayWithoutLength(byte[] bytes, float[] values) { + ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); + int length = bytes.length / Float.BYTES; + readValues(byteBuffer, values, length); + return length; + } + + private static void writeValues(ByteBuffer byteBuffer, float[] values) { + for (float value : values) { + byteBuffer.putFloat(value); + } + } + + private static void readValues(ByteBuffer byteBuffer, float[] values, int length) { + for (int i = 0; i < length; i++) { + values[i] = byteBuffer.getFloat(); + } + } + + public static byte[] serializeDoubleArrayWithLength(double[] values) { + byte[] bytes = new byte[Integer.BYTES + values.length * Double.BYTES]; + ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); + byteBuffer.putInt(values.length); + writeValues(byteBuffer, values); + return bytes; + } + + public static double[] deserializeDoubleArrayWithLength(byte[] bytes) { + return deserializeDoubleArrayWithLength(ByteBuffer.wrap(bytes)); + } + + public static double[] deserializeDoubleArrayWithLength(ByteBuffer byteBuffer) { + int length = byteBuffer.getInt(); + double[] values = new double[length]; + readValues(byteBuffer, values, length); + return values; + } + + public static int deserializeDoubleArrayWithLength(byte[] bytes, double[] values) { + return deserializeDoubleArrayWithLength(ByteBuffer.wrap(bytes), values); + } + + public static int deserializeDoubleArrayWithLength(ByteBuffer byteBuffer, double[] values) { + int length = byteBuffer.getInt(); + readValues(byteBuffer, values, length); + return length; + } + + public static byte[] serializeDoubleArrayWithoutLength(double[] values) { + byte[] bytes = new byte[values.length * Double.BYTES]; + ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); + writeValues(byteBuffer, values); + return bytes; + } + + public static double[] deserializeDoubleArrayWithoutLength(byte[] bytes) { + ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); + int length = bytes.length / Double.BYTES; + double[] values = new double[length]; + readValues(byteBuffer, values, length); + return values; + } + + public static int deserializeDoubleArrayWithoutLength(byte[] bytes, double[] values) { + ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); + int length = bytes.length / Double.BYTES; + readValues(byteBuffer, values, length); + return length; + } + + private static void writeValues(ByteBuffer byteBuffer, double[] values) { + for (double value : values) { + byteBuffer.putDouble(value); + } + } + + private static void readValues(ByteBuffer byteBuffer, double[] values, int length) { + for (int i = 0; i < length; i++) { + values[i] = byteBuffer.getDouble(); + } + } + + public static byte[] serializeStringArray(String[] values) { + // Format: [numValues][length1][length2]...[lengthN][value1][value2]...[valueN] + int headerSize = Integer.BYTES + Integer.BYTES * values.length; + int size = headerSize; + byte[][] stringBytes = new byte[values.length][]; + for (int i = 0; i < values.length; i++) { + stringBytes[i] = values[i].getBytes(UTF_8); + size += stringBytes[i].length; + } + return writeValues(stringBytes, size, headerSize); + } + + public static String[] deserializeStringArray(byte[] bytes) { + ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); + int numValues = byteBuffer.getInt(); + String[] values = new String[numValues]; + readValues(bytes, byteBuffer, values, numValues); + return values; + } + + public static int deserializeStringArray(byte[] bytes, String[] values) { + ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); + int numValues = byteBuffer.getInt(); + readValues(bytes, byteBuffer, values, numValues); + return numValues; + } + + public static byte[] serializeBytesArray(byte[][] values) { + // Format: [numValues][length1][length2]...[lengthN][value1][value2]...[valueN] + int headerSize = Integer.BYTES + Integer.BYTES * values.length; + int size = headerSize; + for (byte[] value : values) { + size += value.length; + } + return writeValues(values, size, headerSize); + } + + public static byte[][] deserializeBytesArray(byte[] bytes) { + ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); + int numValues = byteBuffer.getInt(); + byte[][] values = new byte[numValues][]; + readValues(byteBuffer, values, numValues); + return values; + } + + public static int deserializeBytesArray(byte[] bytes, byte[][] values) { + ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); + int numValues = byteBuffer.getInt(); + readValues(byteBuffer, values, numValues); + return numValues; + } + + private static byte[] writeValues(byte[][] values, int size, int headerSize) { + byte[] bytes = new byte[size]; + ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); + byteBuffer.putInt(values.length); + byteBuffer.position(headerSize); + for (int i = 0; i < values.length; i++) { + byteBuffer.putInt((i + 1) * Integer.BYTES, values[i].length); + byteBuffer.put(values[i]); + } + return bytes; + } + + private static void readValues(byte[] bytes, ByteBuffer byteBuffer, String[] values, int numValues) { + int offset = (numValues + 1) * Integer.BYTES; + for (int i = 0; i < numValues; i++) { + int length = byteBuffer.getInt(); + values[i] = new String(bytes, offset, length, UTF_8); + offset += length; + } + } + + private static void readValues(ByteBuffer byteBuffer, byte[][] values, int numValues) { + byteBuffer.position((numValues + 1) * Integer.BYTES); + for (int i = 0; i < numValues; i++) { + int length = byteBuffer.getInt((i + 1) * Integer.BYTES); + values[i] = new byte[length]; + byteBuffer.get(values[i]); + } + } +} diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionaryOptimizerCardinalityTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionaryOptimizerCardinalityTest.java new file mode 100644 index 000000000000..85f57beae7ac --- /dev/null +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionaryOptimizerCardinalityTest.java @@ -0,0 +1,167 @@ +/** + * 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.pinot.segment.local.segment.creator; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Random; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import org.apache.commons.io.FileUtils; +import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader; +import org.apache.pinot.segment.local.segment.creator.impl.SegmentCreationDriverFactory; +import org.apache.pinot.segment.spi.ImmutableSegment; +import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; +import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver; +import org.apache.pinot.segment.spi.creator.SegmentVersion; +import org.apache.pinot.spi.config.table.FieldConfig; +import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; +import org.apache.pinot.spi.data.DimensionFieldSpec; +import org.apache.pinot.spi.data.FieldSpec; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.data.readers.FileFormat; +import org.apache.pinot.spi.utils.ReadMode; +import org.apache.pinot.spi.utils.builder.TableConfigBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + + +public class DictionaryOptimizerCardinalityTest { + + private static final Logger LOGGER = LoggerFactory.getLogger(DictionaryOptimizerCardinalityTest.class); + private static final File INDEX_DIR = new File(DictionaryOptimizerCardinalityTest.class.toString()); + private static File _segmentDirectory; + private static File _csvFile; + + // Test cardinality based dictionary optimization for var-length data type columns + @Test + public void testDictionaryForMixedCardinalitiesStringType() + throws Exception { + + ImmutableSegment heapSegment = ImmutableSegmentLoader.load(_segmentDirectory, ReadMode.heap); + + Schema schema = heapSegment.getSegmentMetadata().getSchema(); + for (FieldSpec fieldSpec : schema.getAllFieldSpecs()) { + // Skip virtual columns + if (fieldSpec.isVirtualColumn()) { + continue; + } + + String columnName = fieldSpec.getName(); + if ("low_cardinality_strings".equals(columnName)) { + Assert.assertTrue(heapSegment.getForwardIndex(columnName).isDictionaryEncoded(), + "Low cardinality columns should be dictionary encoded"); + } + + if ("high_cardinality_strings".equals(columnName)) { + Assert.assertFalse(heapSegment.getForwardIndex(columnName).isDictionaryEncoded(), + "High cardinality columns should be raw encoded"); + } + } + } + + @BeforeClass + private void setup() + throws Exception { + + if (INDEX_DIR.exists()) { + FileUtils.deleteQuietly(INDEX_DIR); + } + INDEX_DIR.mkdirs(); + _csvFile = new File(INDEX_DIR, "data.csv"); + generateCsv(_csvFile, 500); + + IngestionConfig ingestionConfig = new IngestionConfig(); + ingestionConfig.setRowTimeValueCheck(false); + ingestionConfig.setSegmentTimeValueCheck(false); + Schema schema = + new Schema.SchemaBuilder().addSingleValueDimension("low_cardinality_strings", FieldSpec.DataType.STRING) + .addSingleValueDimension("high_cardinality_strings", FieldSpec.DataType.STRING) + .addDateTimeField("ts", FieldSpec.DataType.LONG, "1:MILLISECONDS:EPOCH", "1:MILLISECONDS").build(); + List stringColumns = + schema.getDimensionFieldSpecs().stream().filter(x -> x.getDataType() == FieldSpec.DataType.STRING).collect( + Collectors.toList()); + + List fieldConfigList = stringColumns.stream().map( + x -> new FieldConfig(x.getName(), FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(), null, null)) + .collect(Collectors.toList()); + + SegmentGeneratorConfig segmentGenSpec = + new SegmentGeneratorConfig(new TableConfigBuilder(TableType.OFFLINE).setTableName("tableName") + .setIngestionConfig(ingestionConfig).setFieldConfigList(fieldConfigList).build(), + schema); + + segmentGenSpec.setInputFilePath(_csvFile.getAbsolutePath()); + segmentGenSpec.setTimeColumnName("ts"); + segmentGenSpec.setSegmentTimeUnit(TimeUnit.SECONDS); + segmentGenSpec.setFormat(FileFormat.CSV); + segmentGenSpec.setSegmentVersion(SegmentVersion.v1); + segmentGenSpec.setTableName("tableName"); + segmentGenSpec.setOutDir(INDEX_DIR.getAbsolutePath()); + segmentGenSpec.setOptimizeDictionary(true); + segmentGenSpec.setNoDictionaryCardinalityRatioThreshold(0.1); // cardinality must be <10% of total docs to override + + SegmentIndexCreationDriver driver = SegmentCreationDriverFactory.get(null); + driver.init(segmentGenSpec); + driver.build(); + + _segmentDirectory = new File(INDEX_DIR, driver.getSegmentName()); + } + + // Generate a 3 columns csv file, sample format is: + // low_cardinality_strings,high_cardinality_strings,ts + // Red,kdeejdfnsd,1600000000 + private void generateCsv(File file, int numberOfRows) throws IOException { + String[] lowCardinalityOptions = {"Red", "Blue", "Green", "Yellow", "Purple"}; + String alphabet = "abcdefghijklmnopqrstuvwxyz"; + Random random = new Random(42); + + try (FileWriter writer = new FileWriter(file, false)) { + // Write the header + writer.append("low_cardinality_strings,high_cardinality_strings,ts\n"); + + long startTimestamp = System.currentTimeMillis() / 1000; + for (int i = 0; i < numberOfRows; i++) { + String lowCardinality = lowCardinalityOptions[random.nextInt(lowCardinalityOptions.length)]; + StringBuilder highCardinality = new StringBuilder(10); + for (int j = 0; j < 10; j++) { + highCardinality.append(alphabet.charAt(random.nextInt(alphabet.length()))); + } + long timestamp = startTimestamp + (i / 10); + writer.append(String.format("%s,%s,%d\n", lowCardinality, highCardinality, timestamp)); + } + } + } + + @AfterClass + public void cleanup() { + FileUtils.deleteQuietly(_csvFile); + FileUtils.deleteQuietly(_segmentDirectory); + FileUtils.deleteQuietly(INDEX_DIR); + } +} diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/MultiValueFixedByteRawIndexCreatorTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/MultiValueFixedByteRawIndexCreatorTest.java index 33c8525a42d2..9a2105726aa7 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/MultiValueFixedByteRawIndexCreatorTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/MultiValueFixedByteRawIndexCreatorTest.java @@ -49,10 +49,9 @@ public class MultiValueFixedByteRawIndexCreatorTest { - private static final String OUTPUT_DIR = - System.getProperty("java.io.tmpdir") + File.separator + "mvFixedRawTest"; + protected static String _outputDir; - private static final Random RANDOM = new Random(); + protected static final Random RANDOM = new Random(); @DataProvider(name = "compressionTypes") public Object[][] compressionTypes() { @@ -64,7 +63,8 @@ public Object[][] compressionTypes() { @BeforeClass public void setup() throws Exception { - FileUtils.forceMkdir(new File(OUTPUT_DIR)); + _outputDir = System.getProperty("java.io.tmpdir") + File.separator + "mvFixedRawTest"; + FileUtils.forceMkdir(new File(_outputDir)); } /** @@ -72,7 +72,7 @@ public void setup() */ @AfterClass public void cleanup() { - FileUtils.deleteQuietly(new File(OUTPUT_DIR)); + FileUtils.deleteQuietly(new File(_outputDir)); } @Test(dataProvider = "compressionTypes") @@ -147,25 +147,34 @@ public void testMVDouble(ChunkCompressionType compressionType, int writerVersion }, compressionType, writerVersion); } + public MultiValueFixedByteRawIndexCreator getMultiValueFixedByteRawIndexCreator(ChunkCompressionType compressionType, + String column, int numDocs, DataType dataType, int maxElements, int writerVersion) + throws IOException { + return new MultiValueFixedByteRawIndexCreator(new File(_outputDir), compressionType, column, numDocs, dataType, + maxElements, false, writerVersion, 1024 * 1024, 1000); + } + + public ForwardIndexReader getForwardIndexReader(PinotDataBuffer buffer, DataType dataType, int writerVersion) { + return writerVersion == VarByteChunkForwardIndexWriterV4.VERSION ? new VarByteChunkForwardIndexReaderV4(buffer, + dataType.getStoredType(), false) : new FixedByteChunkMVForwardIndexReader(buffer, dataType.getStoredType()); + } + public void testMV(DataType dataType, List inputs, ToIntFunction sizeof, IntFunction constructor, Injector injector, Extractor extractor, ChunkCompressionType compressionType, int writerVersion) throws IOException { String column = "testCol_" + dataType; int numDocs = inputs.size(); int maxElements = inputs.stream().mapToInt(sizeof).max().orElseThrow(RuntimeException::new); - File file = new File(OUTPUT_DIR, column + Indexes.RAW_MV_FORWARD_INDEX_FILE_EXTENSION); + File file = new File(_outputDir, column + Indexes.RAW_MV_FORWARD_INDEX_FILE_EXTENSION); file.delete(); MultiValueFixedByteRawIndexCreator creator = - new MultiValueFixedByteRawIndexCreator(new File(OUTPUT_DIR), compressionType, column, numDocs, dataType, - maxElements, false, writerVersion, 1024 * 1024, 1000); + getMultiValueFixedByteRawIndexCreator(compressionType, column, numDocs, dataType, maxElements, writerVersion); inputs.forEach(input -> injector.inject(creator, input)); creator.close(); //read final PinotDataBuffer buffer = PinotDataBuffer.mapFile(file, true, 0, file.length(), ByteOrder.BIG_ENDIAN, ""); - ForwardIndexReader reader = - writerVersion == VarByteChunkForwardIndexWriterV4.VERSION ? new VarByteChunkForwardIndexReaderV4(buffer, - dataType.getStoredType(), false) : new FixedByteChunkMVForwardIndexReader(buffer, dataType.getStoredType()); + ForwardIndexReader reader = getForwardIndexReader(buffer, dataType, writerVersion); final ForwardIndexReaderContext context = reader.createContext(); T valueBuffer = constructor.apply(maxElements); diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/VarByteChunkV4Test.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/VarByteChunkV4Test.java index 70313d91e701..8387c93dc838 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/VarByteChunkV4Test.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/VarByteChunkV4Test.java @@ -193,7 +193,7 @@ private void testWriteRead(File file, ChunkCompressionType compressionType, } } - private Stream randomStrings(int count, int lengthOfLongestEntry) { + protected Stream randomStrings(int count, int lengthOfLongestEntry) { return IntStream.range(0, count) .mapToObj(i -> { int length = ThreadLocalRandom.current().nextInt(lengthOfLongestEntry); diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/VarByteChunkV5Test.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/VarByteChunkV5Test.java new file mode 100644 index 000000000000..c5bbc75c2760 --- /dev/null +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/VarByteChunkV5Test.java @@ -0,0 +1,229 @@ +/** + * 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.pinot.segment.local.segment.index.creator; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import java.util.UUID; +import java.util.function.BiConsumer; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.commons.io.FileUtils; +import org.apache.pinot.segment.local.io.writer.impl.VarByteChunkForwardIndexWriterV5; +import org.apache.pinot.segment.local.segment.creator.impl.fwd.MultiValueFixedByteRawIndexCreator; +import org.apache.pinot.segment.local.segment.index.readers.forward.VarByteChunkForwardIndexReaderV5; +import org.apache.pinot.segment.spi.compression.ChunkCompressionType; +import org.apache.pinot.segment.spi.memory.PinotDataBuffer; +import org.apache.pinot.spi.data.FieldSpec; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +import static org.testng.Assert.assertEquals; + + +public class VarByteChunkV5Test extends VarByteChunkV4Test { + private static final Random RANDOM = new Random(); + private static File[] _dirs; + + @DataProvider(parallel = true) + public Object[][] params() { + Object[][] params = new Object[][]{ + {null, ChunkCompressionType.LZ4, 20, 1024}, + {null, ChunkCompressionType.LZ4_LENGTH_PREFIXED, 20, 1024}, + {null, ChunkCompressionType.PASS_THROUGH, 20, 1024}, + {null, ChunkCompressionType.SNAPPY, 20, 1024}, + {null, ChunkCompressionType.ZSTANDARD, 20, 1024}, + {null, ChunkCompressionType.LZ4, 2048, 1024}, + {null, ChunkCompressionType.LZ4_LENGTH_PREFIXED, 2048, 1024}, + {null, ChunkCompressionType.PASS_THROUGH, 2048, 1024}, + {null, ChunkCompressionType.SNAPPY, 2048, 1024}, + {null, ChunkCompressionType.ZSTANDARD, 2048, 1024} + }; + + for (int i = 0; i < _dirs.length; i++) { + params[i][0] = _dirs[i]; + } + + return params; + } + + @BeforeClass + public void forceMkDirs() + throws IOException { + _dirs = new File[10]; + for (int i = 0; i < _dirs.length; i++) { + _dirs[i] = new File(new File(FileUtils.getTempDirectory(), UUID.randomUUID().toString()), "VarByteChunkV5Test"); + FileUtils.forceMkdir(_dirs[i]); + } + } + + @AfterClass + public void deleteDirs() { + for (File dir : _dirs) { + FileUtils.deleteQuietly(dir); + } + } + + @Test(dataProvider = "params") + public void testStringSV(File file, ChunkCompressionType compressionType, int longestEntry, int chunkSize) + throws IOException { + File stringSVFile = new File(file, "testStringSV"); + testWriteRead(stringSVFile, compressionType, longestEntry, chunkSize, FieldSpec.DataType.STRING, x -> x, + VarByteChunkForwardIndexWriterV5::putString, (reader, context, docId) -> reader.getString(docId, context)); + FileUtils.deleteQuietly(stringSVFile); + } + + @Test(dataProvider = "params") + public void testBytesSV(File file, ChunkCompressionType compressionType, int longestEntry, int chunkSize) + throws IOException { + File bytesSVFile = new File(file, "testBytesSV"); + testWriteRead(bytesSVFile, compressionType, longestEntry, chunkSize, FieldSpec.DataType.BYTES, + x -> x.getBytes(StandardCharsets.UTF_8), VarByteChunkForwardIndexWriterV5::putBytes, + (reader, context, docId) -> reader.getBytes(docId, context)); + FileUtils.deleteQuietly(bytesSVFile); + } + + @Test(dataProvider = "params") + public void testStringMV(File file, ChunkCompressionType compressionType, int longestEntry, int chunkSize) + throws IOException { + File stringMVFile = new File(file, "testStringMV"); + testWriteRead(stringMVFile, compressionType, longestEntry, chunkSize, FieldSpec.DataType.STRING, + new StringSplitterMV(), VarByteChunkForwardIndexWriterV5::putStringMV, + (reader, context, docId) -> reader.getStringMV(docId, context)); + FileUtils.deleteQuietly(stringMVFile); + } + + @Test(dataProvider = "params") + public void testBytesMV(File file, ChunkCompressionType compressionType, int longestEntry, int chunkSize) + throws IOException { + File bytesMVFile = new File(file, "testBytesMV"); + testWriteRead(bytesMVFile, compressionType, longestEntry, chunkSize, FieldSpec.DataType.BYTES, new ByteSplitterMV(), + VarByteChunkForwardIndexWriterV5::putBytesMV, (reader, context, docId) -> reader.getBytesMV(docId, context)); + FileUtils.deleteQuietly(bytesMVFile); + } + + @Test + public void validateCompressionRatioIncrease() + throws IOException { + // Generate input data containing short MV docs with somewhat repetitive data + int numDocs = 1000000; + int numElements = 0; + int maxMVRowSize = 0; + List inputData = new ArrayList<>(numDocs); + for (int i = 0; i < numDocs; i++) { + long[] mvRow = new long[Math.abs((int) Math.floor(RANDOM.nextGaussian()))]; + maxMVRowSize = Math.max(maxMVRowSize, mvRow.length); + numElements += mvRow.length; + for (int j = 0; j < mvRow.length; j++, numElements++) { + mvRow[j] = numElements % 10; + } + inputData.add(mvRow); + } + + for (int i = 0; i < _dirs.length; i++) { + _dirs[i] = new File(new File(FileUtils.getTempDirectory(), UUID.randomUUID().toString()), "VarByteChunkV5Test"); + FileUtils.forceMkdir(_dirs[i]); + } + + // Generate MV fixed byte raw fwd index with explicit length + int rawIndexVersionV4 = 4; + File explicitLengthFwdIndexFile = new File(FileUtils.getTempDirectory(), Integer.toString(rawIndexVersionV4)); + FileUtils.deleteQuietly(explicitLengthFwdIndexFile); + try (MultiValueFixedByteRawIndexCreator creator = new MultiValueFixedByteRawIndexCreator(explicitLengthFwdIndexFile, + ChunkCompressionType.ZSTANDARD, numDocs, FieldSpec.DataType.LONG, numElements, true, rawIndexVersionV4)) { + for (long[] mvRow : inputData) { + creator.putLongMV(mvRow); + } + } + + // Generate MV fixed byte raw fwd index with implicit length + int rawIndexVersionV5 = 5; + File implicitLengthFwdIndexFile = new File(FileUtils.getTempDirectory(), Integer.toString(rawIndexVersionV5)); + FileUtils.deleteQuietly(implicitLengthFwdIndexFile); + try (MultiValueFixedByteRawIndexCreator creator = new MultiValueFixedByteRawIndexCreator(implicitLengthFwdIndexFile, + ChunkCompressionType.ZSTANDARD, numDocs, FieldSpec.DataType.LONG, numElements, true, rawIndexVersionV5)) { + for (long[] mvRow : inputData) { + creator.putLongMV(mvRow); + } + } + + // For the input data, the explicit length compressed MV fixed byte raw forward index is expected to be at least + // 2x larger size in explicit length variant in V4 compared to the new implicit length variant in V5 + long expectedImplicitLengthFwdIndexMaxSize = Math.round(implicitLengthFwdIndexFile.length() * 2.0d); + Assert.assertTrue(expectedImplicitLengthFwdIndexMaxSize < explicitLengthFwdIndexFile.length()); + + // Cleanup + FileUtils.deleteQuietly(explicitLengthFwdIndexFile); + FileUtils.deleteQuietly(implicitLengthFwdIndexFile); + } + + private void testWriteRead(File file, ChunkCompressionType compressionType, int longestEntry, int chunkSize, + FieldSpec.DataType dataType, Function forwardMapper, + BiConsumer write, Read read) + throws IOException { + List values = randomStrings(1000, longestEntry).map(forwardMapper).collect(Collectors.toList()); + try (VarByteChunkForwardIndexWriterV5 writer = new VarByteChunkForwardIndexWriterV5(file, compressionType, + chunkSize)) { + for (T value : values) { + write.accept(writer, value); + } + } + try (PinotDataBuffer buffer = PinotDataBuffer.mapReadOnlyBigEndianFile(file)) { + try (VarByteChunkForwardIndexReaderV5 reader = new VarByteChunkForwardIndexReaderV5(buffer, dataType, true); + VarByteChunkForwardIndexReaderV5.ReaderContext context = reader.createContext()) { + for (int i = 0; i < values.size(); i++) { + assertEquals(read.read(reader, context, i), values.get(i)); + } + for (int i = 0; i < values.size(); i += 2) { + assertEquals(read.read(reader, context, i), values.get(i)); + } + for (int i = 1; i < values.size(); i += 2) { + assertEquals(read.read(reader, context, i), values.get(i)); + } + for (int i = 1; i < values.size(); i += 100) { + assertEquals(read.read(reader, context, i), values.get(i)); + } + for (int i = values.size() - 1; i >= 0; i--) { + assertEquals(read.read(reader, context, i), values.get(i)); + } + for (int i = values.size() - 1; i >= 0; i -= 2) { + assertEquals(read.read(reader, context, i), values.get(i)); + } + for (int i = values.size() - 2; i >= 0; i -= 2) { + assertEquals(read.read(reader, context, i), values.get(i)); + } + for (int i = values.size() - 1; i >= 0; i -= 100) { + assertEquals(read.read(reader, context, i), values.get(i)); + } + } + } + } + + @FunctionalInterface + interface Read { + T read(VarByteChunkForwardIndexReaderV5 reader, VarByteChunkForwardIndexReaderV5.ReaderContext context, int docId); + } +} diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/dictionary/DictionaryIndexTypeTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/dictionary/DictionaryIndexTypeTest.java index ec82432ab14d..83af69e260d5 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/dictionary/DictionaryIndexTypeTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/dictionary/DictionaryIndexTypeTest.java @@ -27,15 +27,16 @@ import org.apache.pinot.segment.local.segment.index.AbstractSerdeIndexContract; import org.apache.pinot.segment.spi.index.DictionaryIndexConfig; import org.apache.pinot.segment.spi.index.FieldIndexConfigs; +import org.apache.pinot.segment.spi.index.IndexType; import org.apache.pinot.segment.spi.index.StandardIndexes; import org.apache.pinot.spi.config.table.FieldConfig; import org.apache.pinot.spi.config.table.IndexConfig; import org.apache.pinot.spi.config.table.Intern; -import org.apache.pinot.spi.config.table.JsonIndexConfig; import org.apache.pinot.spi.data.DimensionFieldSpec; import org.apache.pinot.spi.data.FieldSpec; import org.apache.pinot.spi.data.MetricFieldSpec; import org.apache.pinot.spi.utils.JsonUtils; +import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.Test; @@ -349,28 +350,60 @@ public void testStandardIndex() { + "the DictionaryIndexType static instance"); } + /** + * Tests to verify various combinations of inputs to test dictionary override optimization. + */ @Test public void testDictionaryOverride() { MetricFieldSpec metric = new MetricFieldSpec("testCol", FieldSpec.DataType.DOUBLE); - FieldIndexConfigs fieldIndexConfigs = new FieldIndexConfigs.Builder().build(); + IndexType index1 = Mockito.mock(IndexType.class); + Mockito.when(index1.getId()).thenReturn("index1"); + IndexConfig indexConf = new IndexConfig(true); + FieldIndexConfigs fieldIndexConfigs = new FieldIndexConfigs.Builder().add(index1, indexConf).build(); // No need to disable dictionary - assertTrue(DictionaryIndexType.ignoreDictionaryOverride(false, true, 2, metric, fieldIndexConfigs, 5, 20)); + boolean result = + DictionaryIndexType.ignoreDictionaryOverride(false, true, 2, null, metric, fieldIndexConfigs, 5, 20); + assertTrue(result); // Set a higher noDictionarySizeRatioThreshold - assertFalse(DictionaryIndexType.ignoreDictionaryOverride(false, true, 5, metric, fieldIndexConfigs, 5, 20)); + result = DictionaryIndexType.ignoreDictionaryOverride(false, true, 5, null, metric, fieldIndexConfigs, 5, 20); + assertFalse(result); // optimizeDictionary and optimizeDictionaryForMetrics both turned on - assertFalse(DictionaryIndexType.ignoreDictionaryOverride(true, true, 5, metric, fieldIndexConfigs, 5, 20)); + result = DictionaryIndexType.ignoreDictionaryOverride(true, true, 5, null, metric, fieldIndexConfigs, 5, 20); + assertFalse(result); + + // noDictionarySizeRatioThreshold and noDictionaryCardinalityThreshold are provided + result = DictionaryIndexType.ignoreDictionaryOverride(true, true, 5, 0.10, metric, fieldIndexConfigs, 5, 100); + assertTrue(result); + + // cardinality is much less than total docs, use dictionary + metric.setDataType(FieldSpec.DataType.STRING); + result = DictionaryIndexType.ignoreDictionaryOverride(true, true, 5, 0.10, metric, fieldIndexConfigs, 5, 100); + assertTrue(result); + + // cardinality is large % of total docs, do not use dictionary + result = DictionaryIndexType.ignoreDictionaryOverride(true, true, 5, 0.10, metric, fieldIndexConfigs, 5, 20); + assertFalse(result); + + // Test Dimension col + // Don't ignore for Json. We want to disable dictionary for json. + DimensionFieldSpec dimension = new DimensionFieldSpec("testCol", FieldSpec.DataType.JSON, true); + result = DictionaryIndexType.ignoreDictionaryOverride(true, true, 5, null, dimension, fieldIndexConfigs, 5, 20); + assertTrue(result); + + // cardinality is much less than total docs, use dictionary + dimension.setDataType(FieldSpec.DataType.STRING); + result = DictionaryIndexType.ignoreDictionaryOverride(true, false, 5, 0.10, dimension, fieldIndexConfigs, 5, 100); + assertTrue(result); + + // cardinality is large % of total docs, do not use dictionary + result = DictionaryIndexType.ignoreDictionaryOverride(true, false, 5, 0.10, dimension, fieldIndexConfigs, 5, 20); + assertFalse(result); // Ignore for inverted index IndexConfig indexConfig = new IndexConfig(false); fieldIndexConfigs = new FieldIndexConfigs.Builder().add(StandardIndexes.inverted(), indexConfig).build(); - assertTrue(DictionaryIndexType.ignoreDictionaryOverride(true, true, 5, metric, fieldIndexConfigs, 5, 20)); - - // Don't ignore for JSON index - DimensionFieldSpec dimension = new DimensionFieldSpec("testCol", FieldSpec.DataType.JSON, true); - JsonIndexConfig jsonIndexConfig = new JsonIndexConfig(); - fieldIndexConfigs = new FieldIndexConfigs.Builder().add(StandardIndexes.json(), jsonIndexConfig).build(); - assertFalse(DictionaryIndexType.ignoreDictionaryOverride(true, true, 5, dimension, fieldIndexConfigs, 5, 20)); + assertTrue(DictionaryIndexType.ignoreDictionaryOverride(true, true, 5, null, metric, fieldIndexConfigs, 5, 20)); } } diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/SegmentPreProcessorTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/SegmentPreProcessorTest.java index 5488b2cf2790..55b8227d41af 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/SegmentPreProcessorTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/SegmentPreProcessorTest.java @@ -783,7 +783,7 @@ private void validateIndex(IndexType indexType, String column, int card assertFalse(reader.hasIndexFor(column, StandardIndexes.inverted())); } else { // Updating dictionary or forward index for existing columns not supported for v1 segments yet - if (segmentMetadata.getVersion() == SegmentVersion.v3) { + if (segmentMetadata.getVersion() == SegmentVersion.v3 || isAutoGenerated) { assertFalse(reader.hasIndexFor(column, StandardIndexes.forward())); } else { assertTrue(reader.hasIndexFor(column, StandardIndexes.forward())); diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandlerTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandlerTest.java deleted file mode 100644 index 47ea357894c8..000000000000 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandlerTest.java +++ /dev/null @@ -1,234 +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.pinot.segment.local.segment.index.loader.defaultcolumn; - -import com.google.common.collect.ImmutableMap; -import java.io.File; -import java.nio.file.Files; -import java.util.Collections; -import java.util.concurrent.TimeUnit; -import org.apache.commons.io.FileUtils; -import org.apache.pinot.segment.local.segment.creator.SegmentTestUtils; -import org.apache.pinot.segment.local.segment.creator.impl.SegmentCreationDriverFactory; -import org.apache.pinot.segment.local.segment.index.SegmentMetadataImplTest; -import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; -import org.apache.pinot.segment.local.segment.store.SegmentLocalFSDirectory; -import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; -import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver; -import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl; -import org.apache.pinot.segment.spi.store.SegmentDirectory; -import org.apache.pinot.spi.data.FieldSpec; -import org.apache.pinot.spi.data.Schema; -import org.apache.pinot.spi.utils.ReadMode; -import org.apache.pinot.util.TestUtils; -import org.testng.Assert; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; - - -public class BaseDefaultColumnHandlerTest { - private static final String AVRO_DATA = "data/test_data-mv.avro"; - - private File _indexDir; - private File _segmentDirectory; - private SegmentMetadataImpl _committedSegmentMetadata; - private SegmentDirectory.Writer _writer; - - @BeforeMethod - public void setUp() - throws Exception { - _indexDir = Files.createTempDirectory(SegmentMetadataImplTest.class.getName() + "_segmentDir").toFile(); - - final String filePath = - TestUtils.getFileFromResourceUrl(SegmentMetadataImplTest.class.getClassLoader().getResource(AVRO_DATA)); - - // intentionally changed this to TimeUnit.Hours to make it non-default for testing - final SegmentGeneratorConfig config = SegmentTestUtils - .getSegmentGenSpecWithSchemAndProjectedColumns(new File(filePath), _indexDir, "daysSinceEpoch", TimeUnit.HOURS, - "testTable"); - config.setSegmentNamePostfix("1"); - config.setTimeColumnName("daysSinceEpoch"); - final SegmentIndexCreationDriver driver = SegmentCreationDriverFactory.get(null); - driver.init(config); - driver.build(); - _segmentDirectory = new File(_indexDir, driver.getSegmentName()); - _committedSegmentMetadata = new SegmentMetadataImpl(_segmentDirectory); - _writer = new SegmentLocalFSDirectory(_segmentDirectory, _committedSegmentMetadata, ReadMode.mmap).createWriter(); - } - - @AfterMethod - public void tearDown() { - FileUtils.deleteQuietly(_segmentDirectory); - } - - @Test - public void testComputeDefaultColumnActionMapForCommittedSegment() { - // Dummy IndexLoadingConfig - IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig(); - - // Same schema - Schema schema0 = - new Schema.SchemaBuilder().setSchemaName("testTable").addSingleValueDimension("column1", FieldSpec.DataType.INT) - .addSingleValueDimension("column2", FieldSpec.DataType.INT) - .addSingleValueDimension("column3", FieldSpec.DataType.STRING) - .addSingleValueDimension("column4", FieldSpec.DataType.STRING) - .addSingleValueDimension("column5", FieldSpec.DataType.STRING) - .addSingleValueDimension("column6", FieldSpec.DataType.INT) - .addSingleValueDimension("column7", FieldSpec.DataType.INT) - .addSingleValueDimension("column8", FieldSpec.DataType.INT) - .addSingleValueDimension("column9", FieldSpec.DataType.INT) - .addSingleValueDimension("column10", FieldSpec.DataType.INT) - .addSingleValueDimension("column13", FieldSpec.DataType.INT) - .addSingleValueDimension("count", FieldSpec.DataType.INT) - .addSingleValueDimension("daysSinceEpoch", FieldSpec.DataType.INT) - .addSingleValueDimension("weeksSinceEpochSunday", FieldSpec.DataType.INT).build(); - - BaseDefaultColumnHandler defaultColumnHandler = - new V3DefaultColumnHandler(_segmentDirectory, _committedSegmentMetadata, indexLoadingConfig, schema0, _writer); - Assert.assertEquals(defaultColumnHandler.computeDefaultColumnActionMap(), Collections.EMPTY_MAP); - - // Add single-value dimension in the schema - Schema schema1 = - new Schema.SchemaBuilder().setSchemaName("testTable").addSingleValueDimension("column1", FieldSpec.DataType.INT) - .addSingleValueDimension("column2", FieldSpec.DataType.INT) - .addSingleValueDimension("column3", FieldSpec.DataType.STRING) - .addSingleValueDimension("column4", FieldSpec.DataType.STRING) - .addSingleValueDimension("column5", FieldSpec.DataType.STRING) - .addSingleValueDimension("column6", FieldSpec.DataType.INT) - .addSingleValueDimension("column7", FieldSpec.DataType.INT) - .addSingleValueDimension("column8", FieldSpec.DataType.INT) - .addSingleValueDimension("column9", FieldSpec.DataType.INT) - .addSingleValueDimension("column10", FieldSpec.DataType.INT) - .addSingleValueDimension("column11", FieldSpec.DataType.INT) // add column11 - .addSingleValueDimension("column13", FieldSpec.DataType.INT) - .addSingleValueDimension("count", FieldSpec.DataType.INT) - .addSingleValueDimension("daysSinceEpoch", FieldSpec.DataType.INT) - .addSingleValueDimension("weeksSinceEpochSunday", FieldSpec.DataType.INT).build(); - defaultColumnHandler = - new V3DefaultColumnHandler(_segmentDirectory, _committedSegmentMetadata, indexLoadingConfig, schema1, _writer); - Assert.assertEquals(defaultColumnHandler.computeDefaultColumnActionMap(), - ImmutableMap.of("column11", BaseDefaultColumnHandler.DefaultColumnAction.ADD_DIMENSION)); - - // Add multi-value dimension in the schema - Schema schema2 = - new Schema.SchemaBuilder().setSchemaName("testTable").addSingleValueDimension("column1", FieldSpec.DataType.INT) - .addSingleValueDimension("column2", FieldSpec.DataType.INT) - .addSingleValueDimension("column3", FieldSpec.DataType.STRING) - .addSingleValueDimension("column4", FieldSpec.DataType.STRING) - .addSingleValueDimension("column5", FieldSpec.DataType.STRING) - .addSingleValueDimension("column6", FieldSpec.DataType.INT) - .addSingleValueDimension("column7", FieldSpec.DataType.INT) - .addSingleValueDimension("column8", FieldSpec.DataType.INT) - .addSingleValueDimension("column9", FieldSpec.DataType.INT) - .addSingleValueDimension("column10", FieldSpec.DataType.INT) - .addMultiValueDimension("column11", FieldSpec.DataType.INT) // add column11 - .addSingleValueDimension("column13", FieldSpec.DataType.INT) - .addSingleValueDimension("count", FieldSpec.DataType.INT) - .addSingleValueDimension("daysSinceEpoch", FieldSpec.DataType.INT) - .addSingleValueDimension("weeksSinceEpochSunday", FieldSpec.DataType.INT).build(); - defaultColumnHandler = - new V3DefaultColumnHandler(_segmentDirectory, _committedSegmentMetadata, indexLoadingConfig, schema2, _writer); - Assert.assertEquals(defaultColumnHandler.computeDefaultColumnActionMap(), - ImmutableMap.of("column11", BaseDefaultColumnHandler.DefaultColumnAction.ADD_DIMENSION)); - - // Add metric in the schema - Schema schema3 = - new Schema.SchemaBuilder().setSchemaName("testTable").addSingleValueDimension("column1", FieldSpec.DataType.INT) - .addSingleValueDimension("column2", FieldSpec.DataType.INT) - .addSingleValueDimension("column3", FieldSpec.DataType.STRING) - .addSingleValueDimension("column4", FieldSpec.DataType.STRING) - .addSingleValueDimension("column5", FieldSpec.DataType.STRING) - .addSingleValueDimension("column6", FieldSpec.DataType.INT) - .addSingleValueDimension("column7", FieldSpec.DataType.INT) - .addSingleValueDimension("column8", FieldSpec.DataType.INT) - .addSingleValueDimension("column9", FieldSpec.DataType.INT) - .addSingleValueDimension("column10", FieldSpec.DataType.INT) - .addSingleValueDimension("column13", FieldSpec.DataType.INT) - .addSingleValueDimension("count", FieldSpec.DataType.INT) - .addSingleValueDimension("daysSinceEpoch", FieldSpec.DataType.INT) - .addSingleValueDimension("weeksSinceEpochSunday", FieldSpec.DataType.INT) - .addMetric("column11", FieldSpec.DataType.INT).build(); // add column11 - defaultColumnHandler = - new V3DefaultColumnHandler(_segmentDirectory, _committedSegmentMetadata, indexLoadingConfig, schema3, _writer); - Assert.assertEquals(defaultColumnHandler.computeDefaultColumnActionMap(), - ImmutableMap.of("column11", BaseDefaultColumnHandler.DefaultColumnAction.ADD_METRIC)); - - // Add metric in the schema - Schema schema4 = - new Schema.SchemaBuilder().setSchemaName("testTable").addSingleValueDimension("column1", FieldSpec.DataType.INT) - .addSingleValueDimension("column2", FieldSpec.DataType.INT) - .addSingleValueDimension("column3", FieldSpec.DataType.STRING) - .addSingleValueDimension("column4", FieldSpec.DataType.STRING) - .addSingleValueDimension("column5", FieldSpec.DataType.STRING) - .addSingleValueDimension("column6", FieldSpec.DataType.INT) - .addSingleValueDimension("column7", FieldSpec.DataType.INT) - .addSingleValueDimension("column8", FieldSpec.DataType.INT) - .addSingleValueDimension("column9", FieldSpec.DataType.INT) - .addSingleValueDimension("column10", FieldSpec.DataType.INT) - .addSingleValueDimension("column13", FieldSpec.DataType.INT) - .addSingleValueDimension("count", FieldSpec.DataType.INT) - .addSingleValueDimension("daysSinceEpoch", FieldSpec.DataType.INT) - .addSingleValueDimension("weeksSinceEpochSunday", FieldSpec.DataType.INT) - .addDateTime("column11", FieldSpec.DataType.INT, "1:HOURS:EPOCH", "1:HOURS").build(); // add column11 - defaultColumnHandler = - new V3DefaultColumnHandler(_segmentDirectory, _committedSegmentMetadata, indexLoadingConfig, schema4, _writer); - Assert.assertEquals(defaultColumnHandler.computeDefaultColumnActionMap(), - ImmutableMap.of("column11", BaseDefaultColumnHandler.DefaultColumnAction.ADD_DATE_TIME)); - - // Do not remove non-autogenerated column in the segmentMetadata - Schema schema5 = - new Schema.SchemaBuilder().setSchemaName("testTable").addSingleValueDimension("column1", FieldSpec.DataType.INT) - .addSingleValueDimension("column3", FieldSpec.DataType.STRING) // remove column2 - .addSingleValueDimension("column4", FieldSpec.DataType.STRING) - .addSingleValueDimension("column5", FieldSpec.DataType.STRING) - .addSingleValueDimension("column6", FieldSpec.DataType.INT) - .addSingleValueDimension("column7", FieldSpec.DataType.INT) - .addSingleValueDimension("column8", FieldSpec.DataType.INT) - .addSingleValueDimension("column9", FieldSpec.DataType.INT) - .addSingleValueDimension("column10", FieldSpec.DataType.INT) - .addSingleValueDimension("column13", FieldSpec.DataType.INT) - .addSingleValueDimension("count", FieldSpec.DataType.INT) - .addSingleValueDimension("daysSinceEpoch", FieldSpec.DataType.INT) - .addSingleValueDimension("weeksSinceEpochSunday", FieldSpec.DataType.INT).build(); - defaultColumnHandler = - new V3DefaultColumnHandler(_segmentDirectory, _committedSegmentMetadata, indexLoadingConfig, schema5, _writer); - Assert.assertEquals(defaultColumnHandler.computeDefaultColumnActionMap(), Collections.EMPTY_MAP); - - // Do not update non-autogenerated column in the schema - Schema schema6 = - new Schema.SchemaBuilder().setSchemaName("testTable").addSingleValueDimension("column1", FieldSpec.DataType.INT) - .addSingleValueDimension("column2", FieldSpec.DataType.STRING) // update datatype - .addSingleValueDimension("column3", FieldSpec.DataType.STRING) - .addSingleValueDimension("column4", FieldSpec.DataType.STRING) - .addSingleValueDimension("column5", FieldSpec.DataType.STRING) - .addSingleValueDimension("column6", FieldSpec.DataType.INT) - .addSingleValueDimension("column7", FieldSpec.DataType.INT) - .addSingleValueDimension("column8", FieldSpec.DataType.INT) - .addSingleValueDimension("column9", FieldSpec.DataType.INT) - .addSingleValueDimension("column10", FieldSpec.DataType.INT) - .addSingleValueDimension("column13", FieldSpec.DataType.INT) - .addSingleValueDimension("count", FieldSpec.DataType.INT) - .addSingleValueDimension("daysSinceEpoch", FieldSpec.DataType.INT) - .addSingleValueDimension("weeksSinceEpochSunday", FieldSpec.DataType.INT).build(); - defaultColumnHandler = - new V3DefaultColumnHandler(_segmentDirectory, _committedSegmentMetadata, indexLoadingConfig, schema6, _writer); - Assert.assertEquals(defaultColumnHandler.computeDefaultColumnActionMap(), Collections.EMPTY_MAP); - } -} diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/DefaultColumnHandlerTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/DefaultColumnHandlerTest.java new file mode 100644 index 000000000000..dc008a487dfe --- /dev/null +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/DefaultColumnHandlerTest.java @@ -0,0 +1,134 @@ +/** + * 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.pinot.segment.local.segment.index.loader.defaultcolumn; + +import java.io.File; +import java.net.URL; +import java.util.Map; +import org.apache.commons.io.FileUtils; +import org.apache.pinot.segment.local.segment.creator.SegmentTestUtils; +import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl; +import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; +import org.apache.pinot.segment.local.segment.index.loader.defaultcolumn.BaseDefaultColumnHandler.DefaultColumnAction; +import org.apache.pinot.segment.local.segment.store.SegmentLocalFSDirectory; +import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; +import org.apache.pinot.segment.spi.store.SegmentDirectory; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.spi.data.DateTimeFieldSpec; +import org.apache.pinot.spi.data.DimensionFieldSpec; +import org.apache.pinot.spi.data.FieldSpec.DataType; +import org.apache.pinot.spi.data.MetricFieldSpec; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.utils.ReadMode; +import org.apache.pinot.spi.utils.builder.TableConfigBuilder; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; + + +public class DefaultColumnHandlerTest { + private static final String RAW_TABLE_NAME = "testTable"; + private static final String SEGMENT_NAME = "testSegment"; + private static final File TEMP_DIR = + new File(FileUtils.getTempDirectory(), DefaultColumnHandlerTest.class.getSimpleName()); + private static final File INDEX_DIR = new File(TEMP_DIR, SEGMENT_NAME); + private static final String AVRO_DATA = "data/test_data-mv.avro"; + + private static final TableConfig TABLE_CONFIG = + new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build(); + + private Schema _schema; + private SegmentDirectory _segmentDirectory; + private SegmentDirectory.Writer _writer; + + @BeforeClass + public void setUp() + throws Exception { + FileUtils.deleteQuietly(TEMP_DIR); + + URL resourceUrl = getClass().getClassLoader().getResource(AVRO_DATA); + assertNotNull(resourceUrl); + File avroFile = new File(resourceUrl.getFile()); + _schema = SegmentTestUtils.extractSchemaFromAvroWithoutTime(avroFile); + SegmentGeneratorConfig config = new SegmentGeneratorConfig(TABLE_CONFIG, _schema); + config.setInputFilePath(avroFile.getAbsolutePath()); + config.setOutDir(TEMP_DIR.getAbsolutePath()); + config.setSegmentName(SEGMENT_NAME); + SegmentIndexCreationDriverImpl driver = new SegmentIndexCreationDriverImpl(); + driver.init(config); + driver.build(); + } + + @AfterClass + public void tearDown() { + FileUtils.deleteQuietly(TEMP_DIR); + } + + @Test + public void testComputeDefaultColumnActionMap() + throws Exception { + try (SegmentDirectory segmentDirectory = new SegmentLocalFSDirectory(INDEX_DIR, ReadMode.mmap); + SegmentDirectory.Writer writer = segmentDirectory.createWriter()) { + _segmentDirectory = segmentDirectory; + _writer = writer; + + // Same schema + testComputeDefaultColumnActionMap(Map.of()); + + // Add single-value dimension in the schema + _schema.addField(new DimensionFieldSpec("column11", DataType.INT, true)); + testComputeDefaultColumnActionMap(Map.of("column11", DefaultColumnAction.ADD_DIMENSION)); + _schema.removeField("column11"); + + // Add multi-value dimension in the schema + _schema.addField(new DimensionFieldSpec("column11", DataType.INT, false)); + testComputeDefaultColumnActionMap(Map.of("column11", DefaultColumnAction.ADD_DIMENSION)); + _schema.removeField("column11"); + + // Add metric in the schema + _schema.addField(new MetricFieldSpec("column11", DataType.INT)); + testComputeDefaultColumnActionMap(Map.of("column11", DefaultColumnAction.ADD_METRIC)); + _schema.removeField("column11"); + + // Add date-time in the schema + _schema.addField(new DateTimeFieldSpec("column11", DataType.INT, "EPOCH|HOURS", "1:HOURS")); + testComputeDefaultColumnActionMap(Map.of("column11", DefaultColumnAction.ADD_DATE_TIME)); + _schema.removeField("column11"); + + // Do not remove non-autogenerated column in the segmentMetadata + _schema.removeField("column2"); + testComputeDefaultColumnActionMap(Map.of()); + + // Do not update non-autogenerated column in the schema + _schema.addField(new DimensionFieldSpec("column2", DataType.STRING, true)); + testComputeDefaultColumnActionMap(Map.of()); + } + } + + private void testComputeDefaultColumnActionMap(Map expected) { + BaseDefaultColumnHandler defaultColumnHandler = + new V3DefaultColumnHandler(INDEX_DIR, _segmentDirectory.getSegmentMetadata(), + new IndexLoadingConfig(TABLE_CONFIG, _schema), _schema, _writer); + assertEquals(defaultColumnHandler.computeDefaultColumnActionMap(), expected); + } +} diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/SegmentGeneratorConfig.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/SegmentGeneratorConfig.java index 4424be0883e6..f5065d417f4a 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/SegmentGeneratorConfig.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/SegmentGeneratorConfig.java @@ -127,6 +127,7 @@ public enum TimeColumnType { private boolean _optimizeDictionary = false; private boolean _optimizeDictionaryForMetrics = false; private double _noDictionarySizeRatioThreshold = IndexingConfig.DEFAULT_NO_DICTIONARY_SIZE_RATIO_THRESHOLD; + private Double _noDictionaryCardinalityRatioThreshold; private boolean _realtimeConversion = false; // consumerDir contains data from the consuming segment, and is used during _realtimeConversion optimization private File _consumerDir; @@ -208,6 +209,7 @@ public SegmentGeneratorConfig(TableConfig tableConfig, Schema schema) { _optimizeDictionary = indexingConfig.isOptimizeDictionary(); _optimizeDictionaryForMetrics = indexingConfig.isOptimizeDictionaryForMetrics(); _noDictionarySizeRatioThreshold = indexingConfig.getNoDictionarySizeRatioThreshold(); + _noDictionaryCardinalityRatioThreshold = indexingConfig.getNoDictionaryCardinalityRatioThreshold(); } IngestionConfig ingestionConfig = tableConfig.getIngestionConfig(); @@ -805,6 +807,16 @@ public void setNoDictionarySizeRatioThreshold(double noDictionarySizeRatioThresh _noDictionarySizeRatioThreshold = noDictionarySizeRatioThreshold; } + @Nullable + public Double getNoDictionaryCardinalityRatioThreshold() { + return _noDictionaryCardinalityRatioThreshold; + } + + public void setNoDictionaryCardinalityRatioThreshold(@Nullable Double noDictionaryCardinalityRatioThreshold) { + _noDictionaryCardinalityRatioThreshold = noDictionaryCardinalityRatioThreshold; + } + + public boolean isFailOnEmptySegment() { return _failOnEmptySegment; } diff --git a/pinot-spi/pom.xml b/pinot-spi/pom.xml index 5645b5de63ce..ec0016243112 100644 --- a/pinot-spi/pom.xml +++ b/pinot-spi/pom.xml @@ -238,5 +238,11 @@ package + + pinot-fastdev + + none + + diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java index 1cfd6107107c..369f27767507 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java @@ -27,7 +27,7 @@ public class IndexingConfig extends BaseJsonConfig { - // Default ratio for overriding dictionary + // Default ratio for overriding dictionary for fixed width columns public static final double DEFAULT_NO_DICTIONARY_SIZE_RATIO_THRESHOLD = 0.85d; /** @@ -78,6 +78,10 @@ public class IndexingConfig extends BaseJsonConfig { private double _noDictionarySizeRatioThreshold = DEFAULT_NO_DICTIONARY_SIZE_RATIO_THRESHOLD; + // Used in conjunction with `optimizeDictionary`, if cardinality / total docs is less than the threshold, + // then create a dictionary for the column. A value around 0.1 (10%) is a reasonable starting point + private Double _noDictionaryCardinalityRatioThreshold; + // TODO: Add a new configuration related to the segment generation private boolean _autoGeneratedInvertedIndex; private boolean _createInvertedIndexDuringSegmentGeneration; @@ -372,6 +376,15 @@ public void setNoDictionarySizeRatioThreshold(double noDictionarySizeRatioThresh _noDictionarySizeRatioThreshold = noDictionarySizeRatioThreshold; } + @Nullable + public Double getNoDictionaryCardinalityRatioThreshold() { + return _noDictionaryCardinalityRatioThreshold; + } + + public void setNoDictionaryCardinalityRatioThreshold(Double noDictionaryCardinalityRatioThreshold) { + _noDictionaryCardinalityRatioThreshold = noDictionaryCardinalityRatioThreshold; + } + public String getSegmentNameGeneratorType() { return _segmentNameGeneratorType; } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java index 56c03698f1a8..ff81f6bc4ea8 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java @@ -343,7 +343,11 @@ public static class Broker { public static final String CONFIG_OF_ENABLE_PARTITION_METADATA_MANAGER = "pinot.broker.enable.partition.metadata.manager"; - public static final boolean DEFAULT_ENABLE_PARTITION_METADATA_MANAGER = false; + public static final boolean DEFAULT_ENABLE_PARTITION_METADATA_MANAGER = true; + // Whether to infer partition hint by default or not. + // This value can always be overridden by INFER_PARTITION_HINT query option + public static final String CONFIG_OF_INFER_PARTITION_HINT = "pinot.broker.multistage.infer.partition.hint"; + public static final boolean DEFAULT_INFER_PARTITION_HINT = false; public static final String CONFIG_OF_USE_FIXED_REPLICA = "pinot.broker.use.fixed.replica"; public static final boolean DEFAULT_USE_FIXED_REPLICA = false; @@ -393,6 +397,7 @@ public static class QueryOptionKey { public static final String USE_FIXED_REPLICA = "useFixedReplica"; public static final String EXPLAIN_PLAN_VERBOSE = "explainPlanVerbose"; public static final String USE_MULTISTAGE_ENGINE = "useMultistageEngine"; + public static final String INFER_PARTITION_HINT = "inferPartitionHint"; public static final String ENABLE_NULL_HANDLING = "enableNullHandling"; /** * If set, changes the explain behavior in multi-stage engine. @@ -458,6 +463,15 @@ public static class QueryOptionKey { // executed in an Unbounded FCFS fashion. However, secondary workloads are executed in a constrainted FCFS // fashion with limited compute. public static final String IS_SECONDARY_WORKLOAD = "isSecondaryWorkload"; + + // For group by queries with only filtered aggregations (and no non-filtered aggregations), the default behavior + // is to compute all groups over the rows matching the main query filter. This ensures SQL compliant results, + // since empty groups are also expected to be returned in such queries. However, this could be quite inefficient + // if the main query does not have a filter (since a scan would be required to compute all groups). In case + // users are okay with skipping empty groups - i.e., only the groups matching at least one aggregation filter + // will be returned - this query option can be set. This is useful for performance, since indexes can be used + // for the aggregation filters and a full scan can be avoided. + public static final String FILTERED_AGGREGATIONS_SKIP_EMPTY_GROUPS = "filteredAggregationsSkipEmptyGroups"; } public static class QueryOptionValue { diff --git a/pinot-spi/src/test/java/org/apache/pinot/spi/config/table/IndexingConfigTest.java b/pinot-spi/src/test/java/org/apache/pinot/spi/config/table/IndexingConfigTest.java index f97e9b30b1a1..d3b9b8fc364a 100644 --- a/pinot-spi/src/test/java/org/apache/pinot/spi/config/table/IndexingConfigTest.java +++ b/pinot-spi/src/test/java/org/apache/pinot/spi/config/table/IndexingConfigTest.java @@ -18,6 +18,7 @@ */ package org.apache.pinot.spi.config.table; +import com.fasterxml.jackson.core.JsonProcessingException; import java.io.IOException; import java.util.Arrays; import java.util.HashMap; @@ -27,6 +28,7 @@ import org.testng.annotations.Test; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; @@ -97,4 +99,27 @@ public void testSegmentPartitionConfig() assertEquals(actualPartitionConfig.getNumPartitions(column), expectedPartitionConfig.getNumPartitions(column)); } } + + @Test + public void testOptimizeDictionaryConfigs() + throws JsonProcessingException { + String indexingConfigStr = "{" + + "\"optimizeDictionary\": true," + + "\"optimizeDictionaryForMetrics\": true," + + "\"noDictionarySizeRatioThreshold\": 0.50" + + "}"; + IndexingConfig indexingConfig = JsonUtils.stringToObject(indexingConfigStr, IndexingConfig.class); + assertTrue(indexingConfig.isOptimizeDictionary()); + assertTrue(indexingConfig.isOptimizeDictionaryForMetrics()); + assertEquals(indexingConfig.getNoDictionarySizeRatioThreshold(), 0.50d); + assertNull(indexingConfig.getNoDictionaryCardinalityRatioThreshold()); + + indexingConfigStr = "{" + + "\"optimizeDictionary\": true," + + "\"noDictionaryCardinalityRatioThreshold\": 0.07" + + "}"; + indexingConfig = JsonUtils.stringToObject(indexingConfigStr, IndexingConfig.class); + assertTrue(indexingConfig.isOptimizeDictionary()); + assertEquals(indexingConfig.getNoDictionaryCardinalityRatioThreshold(), 0.07d); + } } diff --git a/pinot-timeseries/pinot-timeseries-spi/src/main/java/org/apache/pinot/tsdb/spi/plan/LeafTimeSeriesPlanNode.java b/pinot-timeseries/pinot-timeseries-spi/src/main/java/org/apache/pinot/tsdb/spi/plan/LeafTimeSeriesPlanNode.java index e56350845011..856b81622a3a 100644 --- a/pinot-timeseries/pinot-timeseries-spi/src/main/java/org/apache/pinot/tsdb/spi/plan/LeafTimeSeriesPlanNode.java +++ b/pinot-timeseries/pinot-timeseries-spi/src/main/java/org/apache/pinot/tsdb/spi/plan/LeafTimeSeriesPlanNode.java @@ -119,7 +119,7 @@ public String getEffectiveFilter(TimeBuckets timeBuckets) { long endTime = _timeUnit.convert(Duration.ofSeconds( timeBuckets.getEndTime() + timeBuckets.getBucketSize().toSeconds() - _offsetSeconds)); - String addnFilter = String.format("%s >= %d AND %s <= %d", _timeColumn, startTime, _timeColumn, endTime); + String addnFilter = String.format("%s >= %d AND %s < %d", _timeColumn, startTime, _timeColumn, endTime); if (filter.strip().isEmpty()) { return addnFilter; } diff --git a/pinot-timeseries/pinot-timeseries-spi/src/test/java/org/apache/pinot/tsdb/spi/plan/LeafTimeSeriesPlanNodeTest.java b/pinot-timeseries/pinot-timeseries-spi/src/test/java/org/apache/pinot/tsdb/spi/plan/LeafTimeSeriesPlanNodeTest.java index ece46c332a9a..f439bfc0285a 100644 --- a/pinot-timeseries/pinot-timeseries-spi/src/test/java/org/apache/pinot/tsdb/spi/plan/LeafTimeSeriesPlanNodeTest.java +++ b/pinot-timeseries/pinot-timeseries-spi/src/test/java/org/apache/pinot/tsdb/spi/plan/LeafTimeSeriesPlanNodeTest.java @@ -46,7 +46,7 @@ public void testGetEffectiveFilter() { new LeafTimeSeriesPlanNode(ID, Collections.emptyList(), TABLE, TIME_COLUMN, TIME_UNIT, 0L, "", "value_col", new AggInfo("SUM", null), Collections.singletonList("cityName")); assertEquals(planNode.getEffectiveFilter(timeBuckets), - "orderTime >= " + expectedStartTimeInFilter + " AND orderTime <= " + expectedEndTimeInFilter); + "orderTime >= " + expectedStartTimeInFilter + " AND orderTime < " + expectedEndTimeInFilter); } // Case-2: Offset, but empty filter { @@ -54,7 +54,7 @@ public void testGetEffectiveFilter() { new LeafTimeSeriesPlanNode(ID, Collections.emptyList(), TABLE, TIME_COLUMN, TIME_UNIT, 123L, "", "value_col", new AggInfo("SUM", null), Collections.singletonList("cityName")); assertEquals(planNode.getEffectiveFilter(timeBuckets), - "orderTime >= " + (expectedStartTimeInFilter - 123) + " AND orderTime <= " + (expectedEndTimeInFilter - 123)); + "orderTime >= " + (expectedStartTimeInFilter - 123) + " AND orderTime < " + (expectedEndTimeInFilter - 123)); } // Case-3: Offset and non-empty filter { @@ -62,7 +62,7 @@ public void testGetEffectiveFilter() { new LeafTimeSeriesPlanNode(ID, Collections.emptyList(), TABLE, TIME_COLUMN, TIME_UNIT, 123L, nonEmptyFilter, "value_col", new AggInfo("SUM", null), Collections.singletonList("cityName")); assertEquals(planNode.getEffectiveFilter(timeBuckets), - String.format("(%s) AND (orderTime >= %s AND orderTime <= %s)", nonEmptyFilter, + String.format("(%s) AND (orderTime >= %s AND orderTime < %s)", nonEmptyFilter, (expectedStartTimeInFilter - 123), (expectedEndTimeInFilter - 123))); } // Case-4: Offset, and non-empty filter, and time-unit that is not seconds @@ -71,7 +71,7 @@ public void testGetEffectiveFilter() { new LeafTimeSeriesPlanNode(ID, Collections.emptyList(), TABLE, TIME_COLUMN, TimeUnit.MILLISECONDS, 123L, nonEmptyFilter, "value_col", new AggInfo("SUM", null), Collections.singletonList("cityName")); assertEquals(planNode.getEffectiveFilter(timeBuckets), - String.format("(%s) AND (orderTime >= %s AND orderTime <= %s)", nonEmptyFilter, + String.format("(%s) AND (orderTime >= %s AND orderTime < %s)", nonEmptyFilter, (expectedStartTimeInFilter * 1000 - 123 * 1000), (expectedEndTimeInFilter * 1000 - 123 * 1000))); } } diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/ColocatedJoinEngineQuickStart.java b/pinot-tools/src/main/java/org/apache/pinot/tools/ColocatedJoinEngineQuickStart.java index c9d8b42e4561..7ef5260034b3 100644 --- a/pinot-tools/src/main/java/org/apache/pinot/tools/ColocatedJoinEngineQuickStart.java +++ b/pinot-tools/src/main/java/org/apache/pinot/tools/ColocatedJoinEngineQuickStart.java @@ -42,6 +42,7 @@ public List types() { @Override public Map getConfigOverrides() { Map overrides = new HashMap<>(super.getConfigOverrides()); + // This is actually not required anymore, but we are keeping it as reference overrides.put(CommonConstants.Broker.CONFIG_OF_ENABLE_PARTITION_METADATA_MANAGER, "true"); return overrides; } diff --git a/pom.xml b/pom.xml index 50f11f4c6ec2..102522577310 100644 --- a/pom.xml +++ b/pom.xml @@ -145,7 +145,7 @@ 2.17.2 3.9.2 3.0.0 - 2.42 + 2.45 2.6.1 1.6.14 5.17.14 @@ -173,7 +173,7 @@ 4.1.1 1.37 0.15.0 - 0.4.5 + 0.4.7 4.2.2 2.28.12 1.2.28 @@ -232,7 +232,7 @@ 3.25.4 1.68.0 - 26.47.0 + 26.49.0 1.1.1 1.7 2.33.0 @@ -245,24 +245,24 @@ 2.1.0 - 3.27.0 + 3.27.1 2.0.1 1.5.4 9.41.2 3.6.2 9.4.56.v20240826 7.0.0 - 5.7.0 + 5.7.1 3.30.2-GA 1.78.1 0.27 5.15.0 - 2.2.16 + 2.2.17 0.10.4 9.7.1 2.8 - 2.0.20 - 26.0.0 + 2.0.21 + 26.0.1 3.9.1 2.24.0 3.4 @@ -274,11 +274,11 @@ 7.10.2 - 5.14.1 + 5.14.2 3.17.1 1.20.2 2.3.232 - 3.1.19 + 3.1.20 3.2.19 @@ -293,6 +293,7 @@ true true + none @@ -766,7 +767,7 @@ org.checkerframework checker-qual - 3.48.0 + 3.48.1 org.codehaus.groovy @@ -1390,7 +1391,7 @@ org.apache.datasketches datasketches-java - 6.0.0 + 6.1.1 com.dynatrace.hash4j @@ -2292,6 +2293,11 @@ auto-service ${google.auto-service.version} + + org.immutables + value-processor + ${immutables.version} +