From 7e88fddea5f1b22b8335369a695645815a810134 Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Sun, 19 Apr 2026 05:54:35 -0700 Subject: [PATCH 01/27] Add partition function expressions for chained partitioning Introduces expression-mode partition functions that allow composing scalar functions in a pipeline (e.g. fnv1a_32(md5(col))) for more flexible partition schemes, including full BYTES-column support. Key changes: - New PartitionFunctionExprCompiler that compiles a restricted expression syntax into a typed PartitionPipeline backed by deterministic scalar functions - PartitionPipelineFunction adapter implementing PartitionFunction / FunctionEvaluator for both ingestion and broker routing - ColumnPartitionConfig.functionExpr field (expression-mode) alongside existing functionName field (name-mode); getFunctionName() is now @Nullable - PartitionIntNormalizer enum (POSITIVE_MODULO / ABS / MASK) replaces ad-hoc modulo logic; normalizer name persisted in config and segment metadata - ColumnPartitionMetadata extended with functionExpr, partitionIdNormalizer, and inputType (BYTES only) fields; "FunctionExpr" sentinel written for functionName so pre-upgrade brokers degrade gracefully (no pruning) instead of NPE-ing - BYTES-column support: pipelines compiled with PartitionValueType.BYTES pass raw byte arrays through scalar functions; BytesColumnPreIndexStatsCollector and StatsCollectorConfig detect the column type and use the correct pipeline - Schema-aware factory helpers (PartitionFunctionFactory, PartitionerFactory, TableConfigPartitioner) so minion merge tasks and stale-segment detection use the correct input type for BYTES expression-mode columns - Broker routing extended to prune segments using expression-mode partition functions; SegmentPartitionMetadataManager and segment pruners handle both modes - Non-deterministic scalar functions (now/ago/sleep) blocked from partition expressions via isAllowedForPartitioning(); isDeterministic flag unchanged so compile-time constant-folding in query parsing continues to work - CommonPartitionScalarFunctionResolver wired via ServiceLoader; resolves scalar functions from FunctionRegistry for use in partition expression compilation - JSON round-trip test for ColumnPartitionMetadata expression-mode serialization Co-Authored-By: Claude Sonnet 4.6 --- .../manager/BaseBrokerRoutingManager.java | 2 +- .../SegmentPartitionMetadataManager.java | 39 +- .../SegmentPartitionUtils.java | 27 +- .../MultiPartitionColumnsSegmentPruner.java | 21 +- .../SinglePartitionColumnSegmentPruner.java | 20 +- .../SegmentPartitionMetadataManagerTest.java | 89 +++- ...artitionFunctionExprSegmentPrunerTest.java | 182 +++++++ .../segmentpruner/SegmentPrunerTest.java | 51 ++ .../evaluator/InbuiltFunctionEvaluator.java | 283 ++--------- ...CommonPartitionScalarFunctionResolver.java | 439 +++++++++++++++++ .../common/function/FunctionRegistry.java | 49 +- .../pinot/common/function/FunctionUtils.java | 11 + .../common/function/scalar/HashFunctions.java | 30 +- .../segment/SegmentZKMetadataUtils.java | 3 +- ...n.pipeline.PartitionScalarFunctionResolver | 19 + ...nFunctionExprCommonScalarFunctionTest.java | 62 +++ .../PinotLLCRealtimeSegmentManager.java | 7 +- .../data/manager/BaseTableDataManager.java | 19 +- .../realtime/RealtimeSegmentDataManager.java | 4 +- .../pruner/ColumnValueSegmentPruner.java | 26 +- .../processing/mapper/SegmentMapper.java | 2 +- .../partitioner/PartitionerFactory.java | 23 +- .../partitioner/TableConfigPartitioner.java | 26 +- .../pruner/ColumnValueSegmentPrunerTest.java | 101 ++++ ...tionLLCRealtimeClusterIntegrationTest.java | 3 + .../mutable/MutableSegmentImpl.java | 8 +- .../StatelessRealtimeSegmentWriter.java | 4 +- .../creator/impl/BaseSegmentCreator.java | 52 +- .../AbstractColumnStatisticsCollector.java | 8 + .../BytesColumnPreIndexStatsCollector.java | 2 +- .../segment/local/utils/TableConfigUtils.java | 2 + .../RealtimeSegmentConverterTest.java | 97 ++++ .../local/utils/TableConfigUtilsTest.java | 71 +++ .../apache/pinot/segment/spi/V1Constants.java | 2 + .../spi/creator/StatsCollectorConfig.java | 22 +- .../function/ExecutableFunctionEvaluator.java | 314 ++++++++++++ .../index/metadata/ColumnMetadataImpl.java | 23 +- .../partition/ByteArrayPartitionFunction.java | 6 + .../spi/partition/FnvPartitionFunction.java | 5 + .../partition/HashCodePartitionFunction.java | 6 + .../partition/ModuloPartitionFunction.java | 6 + .../partition/Murmur3PartitionFunction.java | 6 + .../partition/MurmurPartitionFunction.java | 6 + .../spi/partition/PartitionFunction.java | 37 +- .../partition/PartitionFunctionFactory.java | 179 +++++++ .../metadata/ColumnPartitionMetadata.java | 101 +++- .../PartitionFunctionExprCompiler.java | 459 ++++++++++++++++++ .../pipeline/PartitionIntNormalizer.java | 99 ++++ .../partition/pipeline/PartitionPipeline.java | 126 +++++ .../pipeline/PartitionPipelineFunction.java | 169 +++++++ .../PartitionScalarFunctionResolver.java | 86 ++++ .../spi/partition/pipeline/PartitionStep.java | 65 +++ .../partition/pipeline/PartitionValue.java | 163 +++++++ .../pipeline/PartitionValueConversions.java | 202 ++++++++ .../pipeline/PartitionValueType.java | 62 +++ ...artitionFunctionExprRacyTestFunctions.java | 62 +++ .../PartitionFunctionExprTestFunctions.java | 90 ++++ .../spi/partition/PartitionFunctionTest.java | 85 ++++ .../metadata/ColumnPartitionMetadataTest.java | 86 ++++ .../PartitionFunctionExprCompilerTest.java | 273 +++++++++++ .../TestPartitionScalarFunctionResolver.java | 351 ++++++++++++++ ...n.pipeline.PartitionScalarFunctionResolver | 19 + .../config/table/ColumnPartitionConfig.java | 85 +++- .../config/table/SegmentPartitionConfig.java | 13 +- .../pinot/spi/utils/ScalarFunctionUtils.java | 72 +++ .../spi/config/table/IndexingConfigTest.java | 53 ++ .../LaunchBackfillIngestionJobCommand.java | 5 +- 67 files changed, 4751 insertions(+), 369 deletions(-) create mode 100644 pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpruner/PartitionFunctionExprSegmentPrunerTest.java create mode 100644 pinot-common/src/main/java/org/apache/pinot/common/function/CommonPartitionScalarFunctionResolver.java create mode 100644 pinot-common/src/main/resources/META-INF/services/org.apache.pinot.segment.spi.partition.pipeline.PartitionScalarFunctionResolver create mode 100644 pinot-common/src/test/java/org/apache/pinot/common/function/scalar/PartitionFunctionExprCommonScalarFunctionTest.java create mode 100644 pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/function/ExecutableFunctionEvaluator.java create mode 100644 pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java create mode 100644 pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionIntNormalizer.java create mode 100644 pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipeline.java create mode 100644 pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java create mode 100644 pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionScalarFunctionResolver.java create mode 100644 pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionStep.java create mode 100644 pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionValue.java create mode 100644 pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionValueConversions.java create mode 100644 pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionValueType.java create mode 100644 pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprRacyTestFunctions.java create mode 100644 pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprTestFunctions.java create mode 100644 pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadataTest.java create mode 100644 pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompilerTest.java create mode 100644 pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/pipeline/TestPartitionScalarFunctionResolver.java create mode 100644 pinot-segment-spi/src/test/resources/META-INF/services/org.apache.pinot.segment.spi.partition.pipeline.PartitionScalarFunctionResolver create mode 100644 pinot-spi/src/main/java/org/apache/pinot/spi/utils/ScalarFunctionUtils.java diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/manager/BaseBrokerRoutingManager.java b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/manager/BaseBrokerRoutingManager.java index f984137546fe..9b65102029a3 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/manager/BaseBrokerRoutingManager.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/manager/BaseBrokerRoutingManager.java @@ -809,7 +809,7 @@ private void buildRoutingInternal(String tableNameWithType) { tableNameWithType, partitionConfig.getKey()); partitionMetadataManager = new SegmentPartitionMetadataManager(tableNameWithType, partitionConfig.getKey(), - partitionConfig.getValue().getFunctionName(), partitionConfig.getValue().getNumPartitions()); + partitionConfig.getValue()); } else { LOGGER.warn( "Cannot enable SegmentPartitionMetadataManager for table: {} with multiple partition columns: {}", diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManager.java b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManager.java index 6abc5864ae1c..9a3609f00a1d 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManager.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManager.java @@ -24,6 +24,7 @@ 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.commons.lang3.tuple.Pair; @@ -39,6 +40,8 @@ import org.apache.pinot.core.routing.TablePartitionReplicatedServersInfo; import org.apache.pinot.core.routing.TablePartitionReplicatedServersInfo.PartitionInfo; import org.apache.pinot.segment.spi.partition.PartitionFunction; +import org.apache.pinot.segment.spi.partition.PartitionFunctionFactory; +import org.apache.pinot.spi.config.table.ColumnPartitionConfig; import org.apache.pinot.spi.utils.CommonConstants.Helix.StateModel.SegmentStateModel; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -61,7 +64,7 @@ public class SegmentPartitionMetadataManager implements SegmentZkMetadataFetchLi // static content, if anything changes for the following. a rebuild of routing table is needed. private final String _partitionColumn; - private final String _partitionFunctionName; + private final PartitionFunction _partitionFunction; private final int _numPartitions; // cache-able content, only follow changes if onlineSegments list (of ideal-state) is changed. @@ -71,12 +74,12 @@ public class SegmentPartitionMetadataManager implements SegmentZkMetadataFetchLi private transient TablePartitionInfo _tablePartitionInfo; private transient TablePartitionReplicatedServersInfo _tablePartitionReplicatedServersInfo; - public SegmentPartitionMetadataManager(String tableNameWithType, String partitionColumn, String partitionFunctionName, - int numPartitions) { + public SegmentPartitionMetadataManager(String tableNameWithType, String partitionColumn, + ColumnPartitionConfig columnPartitionConfig) { _tableNameWithType = tableNameWithType; _partitionColumn = partitionColumn; - _partitionFunctionName = partitionFunctionName; - _numPartitions = numPartitions; + _partitionFunction = PartitionFunctionFactory.getPartitionFunction(partitionColumn, columnPartitionConfig); + _numPartitions = _partitionFunction.getNumPartitions(); } @Override @@ -103,7 +106,7 @@ private int getPartitionId(String segment, @Nullable ZNRecord znRecord) { return INVALID_PARTITION_ID; } PartitionFunction partitionFunction = segmentPartitionInfo.getPartitionFunction(); - if (!_partitionFunctionName.equalsIgnoreCase(partitionFunction.getName())) { + if (!isMatchingPartitionFunction(partitionFunction)) { return INVALID_PARTITION_ID; } if (_numPartitions != partitionFunction.getNumPartitions()) { @@ -116,6 +119,26 @@ private int getPartitionId(String segment, @Nullable ZNRecord znRecord) { return partitions.iterator().next(); } + private String getPartitionFunctionDescription() { + String functionExpr = _partitionFunction.getFunctionExpr(); + return functionExpr != null ? functionExpr : _partitionFunction.getName(); + } + + private boolean isMatchingPartitionFunction(PartitionFunction partitionFunction) { + return _partitionFunction.getName().equalsIgnoreCase(partitionFunction.getName()) + && Objects.equals(_partitionFunction.getFunctionConfig(), partitionFunction.getFunctionConfig()) + && Objects.equals(_partitionFunction.getFunctionExpr(), partitionFunction.getFunctionExpr()) + && equalsIgnoreCaseNullable(_partitionFunction.getPartitionIdNormalizer(), + partitionFunction.getPartitionIdNormalizer()); + } + + private static boolean equalsIgnoreCaseNullable(@Nullable String a, @Nullable String b) { + if (a == null) { + return b == null; + } + return a.equalsIgnoreCase(b); + } + private static long getCreationTimeMs(@Nullable ZNRecord znRecord) { if (znRecord == null) { return INVALID_CREATION_TIME_MS; @@ -306,7 +329,7 @@ private void computeTablePartitionReplicatedServersInfo() { } } _tablePartitionReplicatedServersInfo = - new TablePartitionReplicatedServersInfo(_tableNameWithType, _partitionColumn, _partitionFunctionName, + new TablePartitionReplicatedServersInfo(_tableNameWithType, _partitionColumn, getPartitionFunctionDescription(), _numPartitions, partitionInfoMap, segmentsWithInvalidPartition); } @@ -337,7 +360,7 @@ private void computeTablePartitionInfo() { _tableNameWithType); } _tablePartitionInfo = - new TablePartitionInfo(_tableNameWithType, _partitionColumn, _partitionFunctionName, _numPartitions, + new TablePartitionInfo(_tableNameWithType, _partitionColumn, getPartitionFunctionDescription(), _numPartitions, segmentsByPartition, segmentsWithInvalidPartition); } diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionUtils.java b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionUtils.java index 42b992475147..f17101a7c744 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionUtils.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionUtils.java @@ -25,6 +25,7 @@ import javax.annotation.Nullable; import org.apache.helix.zookeeper.datamodel.ZNRecord; import org.apache.pinot.common.metadata.segment.SegmentPartitionMetadata; +import org.apache.pinot.segment.spi.partition.PartitionFunction; import org.apache.pinot.segment.spi.partition.PartitionFunctionFactory; import org.apache.pinot.segment.spi.partition.metadata.ColumnPartitionMetadata; import org.apache.pinot.spi.utils.CommonConstants; @@ -79,9 +80,15 @@ public static SegmentPartitionInfo extractPartitionInfo(String tableNameWithType return INVALID_PARTITION_INFO; } - return new SegmentPartitionInfo(partitionColumn, - PartitionFunctionFactory.getPartitionFunction(columnPartitionMetadata), - columnPartitionMetadata.getPartitions()); + try { + PartitionFunction partitionFunction = + PartitionFunctionFactory.getPartitionFunction(partitionColumn, columnPartitionMetadata); + return new SegmentPartitionInfo(partitionColumn, partitionFunction, columnPartitionMetadata.getPartitions()); + } catch (Exception e) { + LOGGER.warn("Caught exception while constructing partition function for column: {}, segment: {}, table: {}", + partitionColumn, segment, tableNameWithType, e); + return INVALID_PARTITION_INFO; + } } /** @@ -123,10 +130,16 @@ public static Map extractPartitionInfoMap(String t segment, tableNameWithType); continue; } - SegmentPartitionInfo segmentPartitionInfo = new SegmentPartitionInfo(partitionColumn, - PartitionFunctionFactory.getPartitionFunction(columnPartitionMetadata), - columnPartitionMetadata.getPartitions()); - columnSegmentPartitionInfoMap.put(partitionColumn, segmentPartitionInfo); + try { + PartitionFunction partitionFunction = + PartitionFunctionFactory.getPartitionFunction(partitionColumn, columnPartitionMetadata); + SegmentPartitionInfo segmentPartitionInfo = + new SegmentPartitionInfo(partitionColumn, partitionFunction, columnPartitionMetadata.getPartitions()); + columnSegmentPartitionInfoMap.put(partitionColumn, segmentPartitionInfo); + } catch (Exception e) { + LOGGER.warn("Caught exception while constructing partition function for column: {}, segment: {}, table: {}", + partitionColumn, segment, tableNameWithType, e); + } } if (columnSegmentPartitionInfoMap.size() == 1) { String partitionColumn = columnSegmentPartitionInfoMap.keySet().iterator().next(); diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/MultiPartitionColumnsSegmentPruner.java b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/MultiPartitionColumnsSegmentPruner.java index 6970d3f54ea7..82a97b93d615 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/MultiPartitionColumnsSegmentPruner.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/MultiPartitionColumnsSegmentPruner.java @@ -36,6 +36,8 @@ import org.apache.pinot.common.request.Identifier; import org.apache.pinot.common.request.context.RequestContextUtils; import org.apache.pinot.sql.FilterKind; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** @@ -43,6 +45,8 @@ * pruner supports queries with filter (or nested filter) of EQUALITY and IN predicates. */ public class MultiPartitionColumnsSegmentPruner implements SegmentPruner { + private static final Logger LOGGER = LoggerFactory.getLogger(MultiPartitionColumnsSegmentPruner.class); + private final String _tableNameWithType; private final Set _partitionColumns; private final Map> _segmentColumnPartitionInfoMap = @@ -140,8 +144,8 @@ private boolean isPartitionMatch(Expression filterExpression, Identifier identifier = operands.get(0).getIdentifier(); if (identifier != null) { SegmentPartitionInfo partitionInfo = columnPartitionInfoMap.get(identifier.getName()); - return partitionInfo == null || partitionInfo.getPartitions().contains( - partitionInfo.getPartitionFunction().getPartition(RequestContextUtils.getStringValue(operands.get(1)))); + return partitionInfo == null || isPartitionMatch(partitionInfo, RequestContextUtils.getStringValue( + operands.get(1))); } else { return true; } @@ -155,8 +159,7 @@ private boolean isPartitionMatch(Expression filterExpression, } int numOperands = operands.size(); for (int i = 1; i < numOperands; i++) { - if (partitionInfo.getPartitions().contains(partitionInfo.getPartitionFunction() - .getPartition(RequestContextUtils.getStringValue(operands.get(i))))) { + if (isPartitionMatch(partitionInfo, RequestContextUtils.getStringValue(operands.get(i)))) { return true; } } @@ -169,4 +172,14 @@ private boolean isPartitionMatch(Expression filterExpression, return true; } } + + private boolean isPartitionMatch(SegmentPartitionInfo partitionInfo, String value) { + try { + return partitionInfo.getPartitions().contains(partitionInfo.getPartitionFunction().getPartition(value)); + } catch (RuntimeException e) { + LOGGER.warn("Failed to evaluate partition function for table: {}, partition column: {}; skipping partition " + + "pruning", _tableNameWithType, partitionInfo.getPartitionColumn(), e); + return true; + } + } } diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/SinglePartitionColumnSegmentPruner.java b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/SinglePartitionColumnSegmentPruner.java index 7016484e48c4..495d1df1b3c9 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/SinglePartitionColumnSegmentPruner.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/SinglePartitionColumnSegmentPruner.java @@ -35,6 +35,8 @@ import org.apache.pinot.common.request.Identifier; import org.apache.pinot.common.request.context.RequestContextUtils; import org.apache.pinot.sql.FilterKind; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** @@ -42,6 +44,8 @@ * pruner supports queries with filter (or nested filter) of EQUALITY and IN predicates. */ public class SinglePartitionColumnSegmentPruner implements SegmentPruner { + private static final Logger LOGGER = LoggerFactory.getLogger(SinglePartitionColumnSegmentPruner.class); + private final String _tableNameWithType; private final String _partitionColumn; private final Map _partitionInfoMap = new ConcurrentHashMap<>(); @@ -129,8 +133,7 @@ private boolean isPartitionMatch(Expression filterExpression, SegmentPartitionIn case EQUALS: { Identifier identifier = operands.get(0).getIdentifier(); if (identifier != null && identifier.getName().equals(_partitionColumn)) { - return partitionInfo.getPartitions().contains(partitionInfo.getPartitionFunction() - .getPartition(RequestContextUtils.getStringValue(operands.get(1)))); + return isPartitionMatch(partitionInfo, RequestContextUtils.getStringValue(operands.get(1))); } else { return true; } @@ -140,8 +143,7 @@ private boolean isPartitionMatch(Expression filterExpression, SegmentPartitionIn if (identifier != null && identifier.getName().equals(_partitionColumn)) { int numOperands = operands.size(); for (int i = 1; i < numOperands; i++) { - if (partitionInfo.getPartitions().contains(partitionInfo.getPartitionFunction() - .getPartition(RequestContextUtils.getStringValue(operands.get(i))))) { + if (isPartitionMatch(partitionInfo, RequestContextUtils.getStringValue(operands.get(i)))) { return true; } } @@ -154,4 +156,14 @@ private boolean isPartitionMatch(Expression filterExpression, SegmentPartitionIn return true; } } + + private boolean isPartitionMatch(SegmentPartitionInfo partitionInfo, String value) { + try { + return partitionInfo.getPartitions().contains(partitionInfo.getPartitionFunction().getPartition(value)); + } catch (RuntimeException e) { + LOGGER.warn("Failed to evaluate partition function for table: {}, partition column: {}; skipping partition " + + "pruning", _tableNameWithType, _partitionColumn, e); + return true; + } + } } diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManagerTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManagerTest.java index f80f32c62ef5..399ab2fef920 100644 --- a/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManagerTest.java +++ b/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManagerTest.java @@ -23,6 +23,7 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; +import javax.annotation.Nullable; import org.apache.helix.manager.zk.ZkBaseDataAccessor; import org.apache.helix.model.ExternalView; import org.apache.helix.model.IdealState; @@ -36,7 +37,11 @@ import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; import org.apache.pinot.controller.helix.ControllerTest; import org.apache.pinot.core.routing.TablePartitionReplicatedServersInfo; +import org.apache.pinot.segment.spi.partition.PartitionFunction; +import org.apache.pinot.segment.spi.partition.PartitionFunctionFactory; import org.apache.pinot.segment.spi.partition.metadata.ColumnPartitionMetadata; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionPipelineFunction; +import org.apache.pinot.spi.config.table.ColumnPartitionConfig; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; @@ -83,8 +88,8 @@ public void testPartitionMetadataManagerProcessingThroughSegmentChangesSinglePar IdealState idealState = new IdealState(OFFLINE_TABLE_NAME); SegmentPartitionMetadataManager partitionMetadataManager = - new SegmentPartitionMetadataManager(OFFLINE_TABLE_NAME, PARTITION_COLUMN, PARTITION_COLUMN_FUNC, - NUM_PARTITIONS); + new SegmentPartitionMetadataManager(OFFLINE_TABLE_NAME, PARTITION_COLUMN, + new ColumnPartitionConfig(PARTITION_COLUMN_FUNC, NUM_PARTITIONS)); SegmentZkMetadataFetcher segmentZkMetadataFetcher = new SegmentZkMetadataFetcher(OFFLINE_TABLE_NAME, _propertyStore); segmentZkMetadataFetcher.register(partitionMetadataManager); @@ -284,11 +289,89 @@ public void testPartitionMetadataManagerProcessingThroughSegmentChangesSinglePar assertEquals(tablePartitionReplicatedServersInfo.getSegmentsWithInvalidPartition().get(0), segmentInvalid); } + @Test + public void testPartitionMetadataManagerProcessingWithFunctionExpr() { + ExternalView externalView = new ExternalView(OFFLINE_TABLE_NAME); + Map> segmentAssignment = externalView.getRecord().getMapFields(); + Set onlineSegments = new HashSet<>(); + IdealState idealState = new IdealState(OFFLINE_TABLE_NAME); + String functionExpr = "fnv1a_32(md5(" + PARTITION_COLUMN + "))"; + String segment = "exprSegment"; + + SegmentPartitionMetadataManager partitionMetadataManager = + new SegmentPartitionMetadataManager(OFFLINE_TABLE_NAME, PARTITION_COLUMN, + ColumnPartitionConfig.forFunctionExpr(functionExpr, 8, "MASK")); + SegmentZkMetadataFetcher segmentZkMetadataFetcher = + new SegmentZkMetadataFetcher(OFFLINE_TABLE_NAME, _propertyStore); + segmentZkMetadataFetcher.register(partitionMetadataManager); + segmentZkMetadataFetcher.init(idealState, externalView, onlineSegments); + + onlineSegments.add(segment); + segmentAssignment.put(segment, Collections.singletonMap(SERVER_0, ONLINE)); + setSegmentZKMetadata(segment, PartitionPipelineFunction.NAME, 8, 1, functionExpr, "MASK", 0L); + segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); + + TablePartitionReplicatedServersInfo tablePartitionReplicatedServersInfo = + partitionMetadataManager.getTablePartitionReplicatedServersInfo(); + assertEquals(tablePartitionReplicatedServersInfo.getPartitionInfoMap()[1]._segments, + Collections.singleton(segment)); + assertTrue(tablePartitionReplicatedServersInfo.getSegmentsWithInvalidPartition().isEmpty()); + + setSegmentZKMetadata(segment, PartitionPipelineFunction.NAME, 8, 1, functionExpr, "POSITIVE_MODULO", 0L); + segmentZkMetadataFetcher.refreshSegment(segment); + + tablePartitionReplicatedServersInfo = partitionMetadataManager.getTablePartitionReplicatedServersInfo(); + assertEquals(tablePartitionReplicatedServersInfo.getSegmentsWithInvalidPartition(), + Collections.singletonList(segment)); + } + + @Test + public void testPartitionMetadataManagerProcessingWithFunctionExprAbsNormalizer() { + ExternalView externalView = new ExternalView(OFFLINE_TABLE_NAME); + Map> segmentAssignment = externalView.getRecord().getMapFields(); + Set onlineSegments = new HashSet<>(); + IdealState idealState = new IdealState(OFFLINE_TABLE_NAME); + String functionExpr = "murmur2(" + PARTITION_COLUMN + ")"; + String segment = "exprAbsSegment"; + + SegmentPartitionMetadataManager partitionMetadataManager = + new SegmentPartitionMetadataManager(OFFLINE_TABLE_NAME, PARTITION_COLUMN, + ColumnPartitionConfig.forFunctionExpr(functionExpr, 8, "ABS")); + SegmentZkMetadataFetcher segmentZkMetadataFetcher = + new SegmentZkMetadataFetcher(OFFLINE_TABLE_NAME, _propertyStore); + segmentZkMetadataFetcher.register(partitionMetadataManager); + segmentZkMetadataFetcher.init(idealState, externalView, onlineSegments); + + onlineSegments.add(segment); + segmentAssignment.put(segment, Collections.singletonMap(SERVER_0, ONLINE)); + setSegmentZKMetadata(segment, PartitionPipelineFunction.NAME, 8, 1, functionExpr, "ABS", 0L); + segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); + + TablePartitionReplicatedServersInfo tablePartitionReplicatedServersInfo = + partitionMetadataManager.getTablePartitionReplicatedServersInfo(); + assertEquals(tablePartitionReplicatedServersInfo.getPartitionInfoMap()[1]._segments, + Collections.singleton(segment)); + assertTrue(tablePartitionReplicatedServersInfo.getSegmentsWithInvalidPartition().isEmpty()); + } + private void setSegmentZKMetadata(String segment, String partitionFunction, int numPartitions, int partitionId, long creationTimeMs) { + setSegmentZKMetadata(segment, partitionFunction, numPartitions, partitionId, null, null, creationTimeMs); + } + + private void setSegmentZKMetadata(String segment, String partitionFunction, int numPartitions, int partitionId, + @Nullable String functionExpr, long creationTimeMs) { + setSegmentZKMetadata(segment, partitionFunction, numPartitions, partitionId, functionExpr, null, creationTimeMs); + } + + private void setSegmentZKMetadata(String segment, String partitionFunction, int numPartitions, int partitionId, + @Nullable String functionExpr, @Nullable String partitionIdNormalizer, long creationTimeMs) { SegmentZKMetadata segmentZKMetadata = new SegmentZKMetadata(segment); + PartitionFunction effectivePartitionFunction = + PartitionFunctionFactory.getPartitionFunction(PARTITION_COLUMN, partitionFunction, numPartitions, null, + functionExpr, partitionIdNormalizer); segmentZKMetadata.setPartitionMetadata(new SegmentPartitionMetadata(Collections.singletonMap(PARTITION_COLUMN, - new ColumnPartitionMetadata(partitionFunction, numPartitions, Collections.singleton(partitionId), null)))); + new ColumnPartitionMetadata(effectivePartitionFunction, Collections.singleton(partitionId))))); segmentZKMetadata.setCreationTime(creationTimeMs); ZKMetadataProvider.setSegmentZKMetadata(_propertyStore, OFFLINE_TABLE_NAME, segmentZKMetadata); } diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpruner/PartitionFunctionExprSegmentPrunerTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpruner/PartitionFunctionExprSegmentPrunerTest.java new file mode 100644 index 000000000000..35469c0ed58b --- /dev/null +++ b/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpruner/PartitionFunctionExprSegmentPrunerTest.java @@ -0,0 +1,182 @@ +/** + * 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.broker.routing.segmentpruner; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import org.apache.helix.zookeeper.datamodel.ZNRecord; +import org.apache.pinot.common.metadata.segment.SegmentPartitionMetadata; +import org.apache.pinot.common.request.BrokerRequest; +import org.apache.pinot.segment.spi.partition.PartitionFunction; +import org.apache.pinot.segment.spi.partition.PartitionFunctionFactory; +import org.apache.pinot.segment.spi.partition.metadata.ColumnPartitionMetadata; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionPipelineFunction; +import org.apache.pinot.spi.utils.CommonConstants; +import org.apache.pinot.spi.utils.JsonUtils; +import org.apache.pinot.sql.parsers.CalciteSqlCompiler; +import org.testng.annotations.Test; + +import static org.testng.Assert.assertEquals; + + +public class PartitionFunctionExprSegmentPrunerTest { + private static final String TABLE_NAME_WITH_TYPE = "testTable_OFFLINE"; + private static final String PARTITION_COLUMN = "memberId"; + private static final String UUID_PARTITION_COLUMN = "id"; + + @Test + public void testSinglePartitionColumnPrunerWithFunctionExpr() + throws Exception { + SinglePartitionColumnSegmentPruner pruner = + new SinglePartitionColumnSegmentPruner(TABLE_NAME_WITH_TYPE, PARTITION_COLUMN); + String functionExpr = "murmur2(lower(memberId))"; + PartitionFunction partitionFunction = + PartitionFunctionFactory.getPartitionFunction(PARTITION_COLUMN, null, 8, null, functionExpr); + int matchingPartition = partitionFunction.getPartition("Pinot"); + String firstNonMatchingValue = findValueWithDifferentPartition(partitionFunction, matchingPartition, "Kafka", + "Trino", "StarTree", "Presto", "Druid"); + String secondNonMatchingValue = findValueWithDifferentPartition(partitionFunction, matchingPartition, "Flink", + "Spark", "Hive", "Superset", "PinotDB"); + String segment = "segmentExpr"; + pruner.refreshSegment(segment, createPartitionMetadataRecord(segment, PARTITION_COLUMN, + new ColumnPartitionMetadata(partitionFunction, Set.of(matchingPartition)))); + + Set input = Set.of(segment); + BrokerRequest eqMatch = CalciteSqlCompiler.compileToBrokerRequest("SELECT * FROM testTable WHERE memberId = " + + "'Pinot'"); + BrokerRequest eqMiss = CalciteSqlCompiler.compileToBrokerRequest( + String.format("SELECT * FROM testTable WHERE memberId = '%s'", firstNonMatchingValue)); + BrokerRequest inMatch = CalciteSqlCompiler.compileToBrokerRequest( + String.format("SELECT * FROM testTable WHERE memberId IN ('%s', 'PiNoT')", firstNonMatchingValue)); + BrokerRequest inMiss = CalciteSqlCompiler.compileToBrokerRequest( + String.format("SELECT * FROM testTable WHERE memberId IN ('%s', '%s')", firstNonMatchingValue, + secondNonMatchingValue)); + + assertEquals(pruner.prune(eqMatch, input), input); + assertEquals(pruner.prune(eqMiss, input), Set.of()); + assertEquals(pruner.prune(inMatch, input), input); + assertEquals(pruner.prune(inMiss, input), Set.of()); + } + + @Test + public void testSinglePartitionColumnPrunerWithMd5FnvFunctionExpr() + throws Exception { + SinglePartitionColumnSegmentPruner pruner = + new SinglePartitionColumnSegmentPruner(TABLE_NAME_WITH_TYPE, UUID_PARTITION_COLUMN); + String functionExpr = "fnv1a_32(md5(id))"; + String matchingValue = "000016be-9d72-466c-9632-cfa680dc8fa3"; + PartitionFunction partitionFunction = + PartitionFunctionFactory.getPartitionFunction(UUID_PARTITION_COLUMN, null, 128, null, functionExpr, "MASK"); + int matchingPartition = partitionFunction.getPartition(matchingValue); + String nonMatchingValue = findValueWithDifferentPartition(partitionFunction, matchingPartition, + "00000000-0000-0000-0000-000000000001", "00000000-0000-0000-0000-000000000002", + "00000000-0000-0000-0000-000000000003"); + String segment = "segmentMd5FnvExpr"; + pruner.refreshSegment(segment, createPartitionMetadataRecord(segment, UUID_PARTITION_COLUMN, + new ColumnPartitionMetadata(partitionFunction, Set.of(matchingPartition)))); + + Set input = Set.of(segment); + BrokerRequest eqMatch = CalciteSqlCompiler.compileToBrokerRequest( + "SELECT * FROM testTable WHERE id = '000016be-9d72-466c-9632-cfa680dc8fa3'"); + BrokerRequest eqMiss = CalciteSqlCompiler.compileToBrokerRequest( + String.format("SELECT * FROM testTable WHERE id = '%s'", nonMatchingValue)); + + assertEquals(matchingPartition, 104); + assertEquals(pruner.prune(eqMatch, input), input); + assertEquals(pruner.prune(eqMiss, input), Set.of()); + } + + @Test + public void testSinglePartitionColumnPrunerFailsOpenOnInvalidLiteral() + throws Exception { + String partitionColumn = "eventTimeMillis"; + String segment = "segmentInvalidLiteralExpr"; + SinglePartitionColumnSegmentPruner pruner = + new SinglePartitionColumnSegmentPruner(TABLE_NAME_WITH_TYPE, partitionColumn); + PartitionFunction partitionFunction = + PartitionFunctionFactory.getPartitionFunction(partitionColumn, null, 128, null, + "intDiv(eventTimeMillis, 1000)"); + pruner.refreshSegment(segment, createPartitionMetadataRecord(segment, partitionColumn, + new ColumnPartitionMetadata(partitionFunction, Set.of(54)))); + + Set input = Set.of(segment); + BrokerRequest eqInvalid = CalciteSqlCompiler.compileToBrokerRequest( + "SELECT * FROM testTable WHERE eventTimeMillis = 'not_a_number'"); + BrokerRequest inInvalid = CalciteSqlCompiler.compileToBrokerRequest( + "SELECT * FROM testTable WHERE eventTimeMillis IN ('still_bad', 'not_a_number')"); + + assertEquals(pruner.prune(eqInvalid, input), input); + assertEquals(pruner.prune(inInvalid, input), input); + } + + @Test + public void testSinglePartitionColumnPrunerTreatsInvalidPartitionMetadataAsUnprunable() + throws Exception { + String segment = "segmentInvalidExprMetadata"; + SinglePartitionColumnSegmentPruner pruner = + new SinglePartitionColumnSegmentPruner(TABLE_NAME_WITH_TYPE, PARTITION_COLUMN); + pruner.refreshSegment(segment, + createRawPartitionMetadataRecord(segment, PARTITION_COLUMN, PartitionPipelineFunction.NAME, 8, Set.of(1), + "sha256(memberId)", null)); + + Set input = Set.of(segment); + BrokerRequest brokerRequest = + CalciteSqlCompiler.compileToBrokerRequest("SELECT * FROM testTable WHERE memberId = 'Pinot'"); + + assertEquals(pruner.prune(brokerRequest, input), input); + } + + private ZNRecord createPartitionMetadataRecord(String segmentName, String partitionColumn, + ColumnPartitionMetadata columnPartitionMetadata) + throws IOException { + ZNRecord znRecord = new ZNRecord(segmentName); + SegmentPartitionMetadata segmentPartitionMetadata = + new SegmentPartitionMetadata(Map.of(partitionColumn, columnPartitionMetadata)); + znRecord.setSimpleField(CommonConstants.Segment.PARTITION_METADATA, segmentPartitionMetadata.toJsonString()); + return znRecord; + } + + private ZNRecord createRawPartitionMetadataRecord(String segmentName, String partitionColumn, String functionName, + int numPartitions, Set partitions, String functionExpr, String partitionIdNormalizer) + throws IOException { + ZNRecord znRecord = new ZNRecord(segmentName); + Map columnPartitionMetadata = new HashMap<>(); + columnPartitionMetadata.put("functionName", functionName); + columnPartitionMetadata.put("numPartitions", numPartitions); + columnPartitionMetadata.put("partitions", partitions); + columnPartitionMetadata.put("functionExpr", functionExpr); + if (partitionIdNormalizer != null) { + columnPartitionMetadata.put("partitionIdNormalizer", partitionIdNormalizer); + } + znRecord.setSimpleField(CommonConstants.Segment.PARTITION_METADATA, JsonUtils.objectToString( + Map.of("columnPartitionMap", Map.of(partitionColumn, columnPartitionMetadata)))); + return znRecord; + } + + private String findValueWithDifferentPartition(PartitionFunction partitionFunction, int partition, String... values) { + for (String value : values) { + if (partitionFunction.getPartition(value) != partition) { + return value; + } + } + throw new IllegalStateException("Failed to find value on a different partition"); + } +} diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerTest.java index 5e48a981ccc4..4dbb1be5b544 100644 --- a/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerTest.java +++ b/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpruner/SegmentPrunerTest.java @@ -40,7 +40,10 @@ import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; import org.apache.pinot.common.request.BrokerRequest; import org.apache.pinot.controller.helix.ControllerTest; +import org.apache.pinot.segment.spi.partition.PartitionFunction; +import org.apache.pinot.segment.spi.partition.PartitionFunctionFactory; import org.apache.pinot.segment.spi.partition.metadata.ColumnPartitionMetadata; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionPipelineFunction; import org.apache.pinot.spi.config.table.ColumnPartitionConfig; import org.apache.pinot.spi.config.table.IndexingConfig; import org.apache.pinot.spi.config.table.RoutingConfig; @@ -382,6 +385,34 @@ public void testPartitionAwareSegmentPruner() { assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest2, input), Set.of(segment1, segment2)); assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest3, input), Set.of(segment0, segment1)); assertEquals(multiPartitionColumnsSegmentPruner.prune(brokerRequest4, input), Set.of(segment1, segment2)); + + String functionExpr = "murmur2(lower(memberId))"; + PartitionFunction partitionFunction = + PartitionFunctionFactory.getPartitionFunction(PARTITION_COLUMN_1, null, 8, null, functionExpr); + int matchingPartition = partitionFunction.getPartition("Pinot"); + String firstNonMatchingValue = findValueWithDifferentPartition(partitionFunction, matchingPartition, "Kafka", + "Trino", "StarTree", "Presto", "Druid"); + String secondNonMatchingValue = findValueWithDifferentPartition(partitionFunction, matchingPartition, "Flink", + "Spark", "Hive", "Superset", "PinotDB"); + String exprSegment = "exprSegment"; + setSegmentZKPartitionMetadata(OFFLINE_TABLE_NAME, exprSegment, PartitionPipelineFunction.NAME, 8, + matchingPartition, functionExpr); + onlineSegments.add(exprSegment); + segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); + input = Set.of(exprSegment); + BrokerRequest exprEqMatch = CalciteSqlCompiler.compileToBrokerRequest( + "SELECT * FROM testTable WHERE memberId = 'Pinot'"); + BrokerRequest exprEqMiss = CalciteSqlCompiler.compileToBrokerRequest( + String.format("SELECT * FROM testTable WHERE memberId = '%s'", firstNonMatchingValue)); + BrokerRequest exprInMatch = CalciteSqlCompiler.compileToBrokerRequest( + String.format("SELECT * FROM testTable WHERE memberId IN ('%s', 'PiNoT')", firstNonMatchingValue)); + BrokerRequest exprInMiss = CalciteSqlCompiler.compileToBrokerRequest( + String.format("SELECT * FROM testTable WHERE memberId IN ('%s', '%s')", firstNonMatchingValue, + secondNonMatchingValue)); + assertEquals(multiPartitionColumnsSegmentPruner.prune(exprEqMatch, input), input); + assertEquals(multiPartitionColumnsSegmentPruner.prune(exprEqMiss, input), Set.of()); + assertEquals(multiPartitionColumnsSegmentPruner.prune(exprInMatch, input), input); + assertEquals(multiPartitionColumnsSegmentPruner.prune(exprInMiss, input), Set.of()); } @Test @@ -680,6 +711,17 @@ private void setSegmentZKPartitionMetadata(String tableNameWithType, String segm ZKMetadataProvider.setSegmentZKMetadata(_propertyStore, tableNameWithType, segmentZKMetadata); } + private void setSegmentZKPartitionMetadata(String tableNameWithType, String segment, String partitionFunction, + int numPartitions, int partitionId, String functionExpr) { + SegmentZKMetadata segmentZKMetadata = new SegmentZKMetadata(segment); + PartitionFunction compiledPartitionFunction = + PartitionFunctionFactory.getPartitionFunction(PARTITION_COLUMN_1, partitionFunction, numPartitions, null, + functionExpr); + segmentZKMetadata.setPartitionMetadata(new SegmentPartitionMetadata(Map.of(PARTITION_COLUMN_1, + new ColumnPartitionMetadata(compiledPartitionFunction, Set.of(partitionId))))); + ZKMetadataProvider.setSegmentZKMetadata(_propertyStore, tableNameWithType, segmentZKMetadata); + } + private void setSegmentZKPartitionMetadata(String tableNameWithType, String segment, Map columnPartitionMap) { SegmentZKMetadata segmentZKMetadata = new SegmentZKMetadata(segment); @@ -701,4 +743,13 @@ private void setSegmentZKTotalDocsMetadata(String tableNameWithType, String segm segmentZKMetadata.setTotalDocs(totalDocs); ZKMetadataProvider.setSegmentZKMetadata(_propertyStore, tableNameWithType, segmentZKMetadata); } + + private String findValueWithDifferentPartition(PartitionFunction partitionFunction, int partition, String... values) { + for (String value : values) { + if (partitionFunction.getPartition(value) != partition) { + return value; + } + } + throw new IllegalStateException("Failed to find value on a different partition"); + } } diff --git a/pinot-common/src/main/java/org/apache/pinot/common/evaluator/InbuiltFunctionEvaluator.java b/pinot-common/src/main/java/org/apache/pinot/common/evaluator/InbuiltFunctionEvaluator.java index 5db855b7327a..9946e1372e44 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/evaluator/InbuiltFunctionEvaluator.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/evaluator/InbuiltFunctionEvaluator.java @@ -20,7 +20,6 @@ import com.google.common.base.Preconditions; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import org.apache.commons.lang3.StringUtils; import org.apache.pinot.common.function.FunctionInfo; @@ -29,66 +28,71 @@ import org.apache.pinot.common.request.context.ExpressionContext; import org.apache.pinot.common.request.context.FunctionContext; import org.apache.pinot.common.request.context.RequestContextUtils; +import org.apache.pinot.segment.spi.function.ExecutableFunctionEvaluator; import org.apache.pinot.spi.data.readers.GenericRow; -import org.apache.pinot.spi.function.FunctionEvaluator; /** - * Evaluates an expression. + * Evaluates an expression backed by the Pinot {@link FunctionRegistry}. + * *

This is optimized for evaluating an expression multiple times with different inputs. - *

Overall idea: parse the expression into an ExecutableNode, where an ExecutableNode can be: - *

    - *
  • FunctionNode - executes a function
  • - *
  • ColumnNode - fetches the value of the column from the input GenericRow
  • - *
  • ConstantNode - returns the literal value
  • - *
+ * The expression is compiled once into an {@link ExecutableFunctionEvaluator.ExecutableNode} tree + * whose nodes handle constants, column reads, logical operators, and function calls. + * + *

Thread-safety: Instances are thread-safe for all node types except + * {@link FunctionExecutionNode}, which uses a shared argument scratch array per node instance + * and therefore must not be invoked concurrently on the same node. */ -public class InbuiltFunctionEvaluator implements FunctionEvaluator { - // Root of the execution tree - private final ExecutableNode _rootNode; - private final List _arguments; - private final String _functionExpression; +public class InbuiltFunctionEvaluator extends ExecutableFunctionEvaluator { public InbuiltFunctionEvaluator(String functionExpression) { - _functionExpression = functionExpression; - _arguments = new ArrayList<>(); - _rootNode = planExecution(RequestContextUtils.getExpression(functionExpression)); + this(functionExpression, new ArrayList<>()); + } + + /** + * Two-phase constructor: {@code planExecution} is evaluated first (left-to-right argument + * evaluation in Java), populating {@code arguments} as a side effect, so the completed list is + * ready when passed to {@code super()}. + */ + private InbuiltFunctionEvaluator(String functionExpression, List arguments) { + super(planExecution(RequestContextUtils.getExpression(functionExpression), arguments), arguments, + functionExpression); } - private ExecutableNode planExecution(ExpressionContext expression) { + private static ExecutableNode planExecution(ExpressionContext expression, List arguments) { switch (expression.getType()) { case LITERAL: - return new ConstantExecutionNode(expression.getLiteral().getValue()); + return new ConstantNode(expression.getLiteral().getValue()); case IDENTIFIER: String columnName = expression.getIdentifier(); - ColumnExecutionNode columnExecutionNode = new ColumnExecutionNode(columnName, _arguments.size()); - _arguments.add(columnName); - return columnExecutionNode; + ColumnNode columnNode = new ColumnNode(columnName, arguments.size()); + arguments.add(columnName); + return columnNode; case FUNCTION: FunctionContext function = expression.getFunction(); - List arguments = function.getArguments(); - int numArguments = arguments.size(); + List args = function.getArguments(); + int numArguments = args.size(); ExecutableNode[] childNodes = new ExecutableNode[numArguments]; for (int i = 0; i < numArguments; i++) { - childNodes[i] = planExecution(arguments.get(i)); + childNodes[i] = planExecution(args.get(i), arguments); } String functionName = function.getFunctionName(); String canonicalName = FunctionRegistry.canonicalize(functionName); switch (canonicalName) { case "and": - return new AndExecutionNode(childNodes); + return new AndNode(childNodes); case "or": - return new OrExecutionNode(childNodes); + return new OrNode(childNodes); case "not": Preconditions.checkState(numArguments == 1, "NOT function expects 1 argument, got: %s", numArguments); - return new NotExecutionNode(childNodes[0]); + return new NotNode(childNodes[0]); case "arrayvalueconstructor": Object[] values = new Object[numArguments]; int i = 0; - for (ExpressionContext literal : arguments) { + for (ExpressionContext literal : args) { values[i++] = literal.getLiteral().getValue(); } - return new ArrayConstantExecutionNode(values); + return new ArrayConstantNode(values); default: FunctionInfo functionInfo = FunctionRegistry.lookupFunctionInfo(canonicalName, numArguments); if (functionInfo == null) { @@ -106,149 +110,11 @@ private ExecutableNode planExecution(ExpressionContext expression) { } } - @Override - public List getArguments() { - return _arguments; - } - - @Override - public Object evaluate(GenericRow row) { - return _rootNode.execute(row); - } - - @Override - public Object evaluate(Object[] values) { - return _rootNode.execute(values); - } - - @Override - public String toString() { - return _functionExpression; - } - - private interface ExecutableNode { - - Object execute(GenericRow row); - - Object execute(Object[] values); - } - - private static class NotExecutionNode implements ExecutableNode { - private final ExecutableNode _argumentNode; - - NotExecutionNode(ExecutableNode argumentNode) { - _argumentNode = argumentNode; - } - - @Override - public Object execute(GenericRow row) { - Boolean res = (Boolean) _argumentNode.execute(row); - if (res == null) { - return null; - } else { - return !res; - } - } - - @Override - public Object execute(Object[] values) { - Boolean res = (Boolean) _argumentNode.execute(values); - if (res == null) { - return null; - } else { - return !res; - } - } - } - - private static class OrExecutionNode implements ExecutableNode { - private final ExecutableNode[] _argumentNodes; - - OrExecutionNode(ExecutableNode[] argumentNodes) { - _argumentNodes = argumentNodes; - } - - @Override - public Object execute(GenericRow row) { - boolean hasNull = false; - - for (ExecutableNode executableNode : _argumentNodes) { - Boolean res = (Boolean) executableNode.execute(row); - if (res == null) { - hasNull = true; - continue; - } - if (res) { - return true; - } - } - - return hasNull ? null : false; - } - - @Override - public Object execute(Object[] values) { - boolean hasNull = false; - - for (ExecutableNode executableNode : _argumentNodes) { - Boolean res = (Boolean) executableNode.execute(values); - if (res == null) { - hasNull = true; - continue; - } - if (res) { - return true; - } - } - - return hasNull ? null : false; - } - } - - private static class AndExecutionNode implements ExecutableNode { - private final ExecutableNode[] _argumentNodes; - - AndExecutionNode(ExecutableNode[] argumentNodes) { - _argumentNodes = argumentNodes; - } - - @Override - public Object execute(GenericRow row) { - boolean hasNull = false; - - for (ExecutableNode executableNode : _argumentNodes) { - Boolean res = (Boolean) executableNode.execute(row); - if (res == null) { - hasNull = true; - continue; - } - if (!res) { - return false; - } - } - - return hasNull ? null : true; - } - - @Override - public Object execute(Object[] values) { - boolean hasNull = false; - - for (ExecutableNode executableNode : _argumentNodes) { - Boolean res = (Boolean) executableNode.execute(values); - if (res == null) { - hasNull = true; - continue; - } - if (!res) { - return false; - } - } - - return hasNull ? null : true; - } - } - + /** + * Executes a Pinot-registry function via {@link FunctionInvoker}, with null propagation and + * type conversion. Uses a shared argument scratch array (not thread-safe for concurrent + * invocations of the same node instance). + */ private static class FunctionExecutionNode implements ExecutableNode { final FunctionInvoker _functionInvoker; final FunctionInfo _functionInfo; @@ -319,75 +185,4 @@ public String toString() { return _functionInvoker.getMethod().getName() + '(' + StringUtils.join(_argumentNodes, ',') + ')'; } } - - private static class ConstantExecutionNode implements ExecutableNode { - final Object _value; - - ConstantExecutionNode(Object value) { - _value = value; - } - - @Override - public Object execute(GenericRow row) { - return _value; - } - - @Override - public Object execute(Object[] values) { - return _value; - } - - @Override - public String toString() { - return String.format("'%s'", _value); - } - } - - private static class ArrayConstantExecutionNode implements ExecutableNode { - final Object[] _value; - - ArrayConstantExecutionNode(Object[] value) { - _value = value; - } - - @Override - public Object[] execute(GenericRow row) { - return _value; - } - - @Override - public Object[] execute(Object[] values) { - return _value; - } - - @Override - public String toString() { - return String.format("'%s'", Arrays.toString(_value)); - } - } - - private static class ColumnExecutionNode implements ExecutableNode { - final String _column; - final int _id; - - ColumnExecutionNode(String column, int id) { - _column = column; - _id = id; - } - - @Override - public Object execute(GenericRow row) { - return row.getValue(_column); - } - - @Override - public Object execute(Object[] values) { - return values[_id]; - } - - @Override - public String toString() { - return _column; - } - } } diff --git a/pinot-common/src/main/java/org/apache/pinot/common/function/CommonPartitionScalarFunctionResolver.java b/pinot-common/src/main/java/org/apache/pinot/common/function/CommonPartitionScalarFunctionResolver.java new file mode 100644 index 000000000000..87c1511c20e3 --- /dev/null +++ b/pinot-common/src/main/java/org/apache/pinot/common/function/CommonPartitionScalarFunctionResolver.java @@ -0,0 +1,439 @@ +/** + * 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.common.function; + +import com.google.common.base.Preconditions; +import java.lang.reflect.Array; +import java.lang.reflect.Method; +import java.lang.reflect.Modifier; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Set; +import javax.annotation.Nullable; +import org.apache.pinot.common.function.scalar.DateTimeFunctions; +import org.apache.pinot.common.function.scalar.InternalFunctions; +import org.apache.pinot.common.utils.DataSchema.ColumnDataType; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionScalarFunctionResolver; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionValue; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionValueConversions; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionValueType; + + +/** + * Common-backed partition scalar-function resolver that reuses {@link FunctionRegistry} for lookup and + * {@link FunctionInvoker} for invocation. + */ +public class CommonPartitionScalarFunctionResolver implements PartitionScalarFunctionResolver { + private static final int MAX_CLASS_FUNCTION_TYPE_COMBINATIONS = 128; + private static final Object DYNAMIC_ARGUMENT = new Object(); + + public CommonPartitionScalarFunctionResolver() { + FunctionRegistry.init(); + } + + @Override + public ResolvedFunction resolve(String functionName, List arguments) { + String canonicalName = FunctionRegistry.canonicalize(functionName); + if (!FunctionRegistry.contains(canonicalName)) { + throw new IllegalArgumentException(String.format("Unsupported partition scalar function: %s", functionName)); + } + + List> candidateArgumentTypes = new ArrayList<>(arguments.size()); + int typeCombinations = 1; + for (Argument argument : arguments) { + List candidateTypes = getCandidateColumnDataTypes(argument); + candidateArgumentTypes.add(candidateTypes); + typeCombinations *= candidateTypes.size(); + if (typeCombinations > MAX_CLASS_FUNCTION_TYPE_COMBINATIONS) { + break; + } + } + + ResolutionState resolutionState = new ResolutionState(); + Set visitedMethods = new HashSet<>(); + LinkedHashSet supportedSignatures = new LinkedHashSet<>(); + if (typeCombinations <= MAX_CLASS_FUNCTION_TYPE_COMBINATIONS) { + bindByTypes(canonicalName, functionName, arguments, candidateArgumentTypes, new ColumnDataType[arguments.size()], + 0, visitedMethods, supportedSignatures, resolutionState); + } + bindFromFunctionInfo(FunctionRegistry.lookupFunctionInfo(canonicalName, arguments.size()), functionName, + arguments, visitedMethods, supportedSignatures, resolutionState); + + if (resolutionState._bestMatch != null) { + return resolutionState._bestMatch; + } + if (resolutionState._sawNonDeterministicCandidate && !resolutionState._sawDeterministicCandidate) { + throw new IllegalArgumentException(String.format( + "Partition scalar function '%s' is not allowed because it is non-deterministic", functionName)); + } + + String supportedSignatureList = + supportedSignatures.isEmpty() ? functionName + "(...)" : String.join(", ", supportedSignatures); + throw new IllegalArgumentException(String.format( + "Function '%s' does not accept argument types (%s). Supported signatures: %s", functionName, + formatArgumentTypes(arguments), supportedSignatureList)); + } + + private void bindByTypes(String canonicalName, String functionName, List arguments, + List> candidateArgumentTypes, ColumnDataType[] argumentTypes, int index, + Set visitedMethods, LinkedHashSet supportedSignatures, ResolutionState resolutionState) { + if (index == arguments.size()) { + bindFromFunctionInfo(FunctionRegistry.lookupFunctionInfo(canonicalName, argumentTypes), functionName, arguments, + visitedMethods, supportedSignatures, resolutionState); + return; + } + + for (ColumnDataType candidateType : candidateArgumentTypes.get(index)) { + argumentTypes[index] = candidateType; + bindByTypes(canonicalName, functionName, arguments, candidateArgumentTypes, argumentTypes, index + 1, + visitedMethods, supportedSignatures, resolutionState); + } + } + + private void bindFromFunctionInfo(@Nullable FunctionInfo functionInfo, String functionName, List arguments, + Set visitedMethods, LinkedHashSet supportedSignatures, ResolutionState resolutionState) { + if (functionInfo == null) { + return; + } + + Method method = functionInfo.getMethod(); + if (!visitedMethods.add(method) || !isSupportedMethod(method)) { + return; + } + + supportedSignatures.add(buildSignature(method)); + if (!functionInfo.isDeterministic() || !isAllowedForPartitioning(method)) { + resolutionState._sawNonDeterministicCandidate = true; + return; + } + resolutionState._sawDeterministicCandidate = true; + + BoundFunction candidate = new ScalarFunctionMethod(functionInfo).bind(arguments); + if (candidate == null) { + return; + } + if (resolutionState._bestMatch == null || candidate.getCost() < resolutionState._bestMatch.getCost()) { + resolutionState._bestMatch = candidate; + return; + } + if (candidate.getCost() == resolutionState._bestMatch.getCost()) { + throw new IllegalArgumentException(String.format( + "Ambiguous partition scalar function '%s' for argument types (%s). Matching signatures: %s", functionName, + formatArgumentTypes(arguments), String.join(", ", supportedSignatures))); + } + } + + private static List getCandidateColumnDataTypes(Argument argument) { + PartitionValueType sourceType = argument.getType(); + PartitionValue constantValue = argument.getConstantValue(); + List candidateTypes = new ArrayList<>(); + switch (sourceType) { + case STRING: + candidateTypes.add(ColumnDataType.STRING); + candidateTypes.add(ColumnDataType.BYTES); + candidateTypes.add(ColumnDataType.INT); + candidateTypes.add(ColumnDataType.LONG); + candidateTypes.add(ColumnDataType.FLOAT); + candidateTypes.add(ColumnDataType.DOUBLE); + break; + case BYTES: + candidateTypes.add(ColumnDataType.BYTES); + break; + case INT: + candidateTypes.add(ColumnDataType.INT); + candidateTypes.add(ColumnDataType.LONG); + candidateTypes.add(ColumnDataType.FLOAT); + candidateTypes.add(ColumnDataType.DOUBLE); + break; + case LONG: + candidateTypes.add(ColumnDataType.LONG); + if (constantValue != null) { + long longValue = constantValue.getLongValue(); + if (longValue >= Integer.MIN_VALUE && longValue <= Integer.MAX_VALUE) { + candidateTypes.add(ColumnDataType.INT); + } + } + candidateTypes.add(ColumnDataType.DOUBLE); + break; + case FLOAT: + candidateTypes.add(ColumnDataType.FLOAT); + candidateTypes.add(ColumnDataType.DOUBLE); + break; + case DOUBLE: + candidateTypes.add(ColumnDataType.DOUBLE); + if (constantValue != null) { + double doubleValue = constantValue.getDoubleValue(); + if (doubleValue == Math.rint(doubleValue)) { + if (doubleValue >= Long.MIN_VALUE && doubleValue <= Long.MAX_VALUE) { + candidateTypes.add(ColumnDataType.LONG); + } + if (doubleValue >= Integer.MIN_VALUE && doubleValue <= Integer.MAX_VALUE) { + candidateTypes.add(ColumnDataType.INT); + } + } + } + break; + default: + throw new IllegalStateException("Unsupported partition value type: " + sourceType); + } + return candidateTypes; + } + + private static boolean isSupportedMethod(Method method) { + try { + PartitionValueType.fromJavaType(method.getReturnType()); + } catch (IllegalArgumentException e) { + return false; + } + + Class[] parameterTypes = method.getParameterTypes(); + int lastIndex = parameterTypes.length - 1; + for (int i = 0; i < parameterTypes.length; i++) { + Class parameterType = parameterTypes[i]; + if (method.isVarArgs() && i == lastIndex) { + if (!parameterType.isArray() || parameterType == byte[].class) { + return false; + } + try { + PartitionValueType.fromJavaType(parameterType.getComponentType()); + } catch (IllegalArgumentException e) { + return false; + } + } else { + try { + PartitionValueType.fromJavaType(parameterType); + } catch (IllegalArgumentException e) { + return false; + } + } + } + return true; + } + + /** + * Partition expressions must stay stable for ingestion and query pruning. Functions that read query-thread context + * or intentionally block are not safe even if the broader SQL engine exposes them as regular scalar functions. + */ + private static boolean isAllowedForPartitioning(Method method) { + Class declaringClass = method.getDeclaringClass(); + if (declaringClass == InternalFunctions.class) { + return false; + } + if (declaringClass == DateTimeFunctions.class) { + // Block functions that read wall-clock time: the same functionExpr compiled on different nodes or at different + // times would produce different partition assignments, which would silently corrupt routing and pruning. + // sleep() is also blocked here (it has a side effect). Note: these functions are still deterministic in the SQL + // query engine sense (they are constant-folded once at query-parse time), so isDeterministic stays true on their + // @ScalarFunction annotation; we enforce partition-safety here instead. + String name = method.getName(); + return !name.equals("sleep") && !name.equals("now") && !name.equals("ago") && !name.equals("agoMV"); + } + return true; + } + + private static String formatArgumentTypes(List arguments) { + StringBuilder builder = new StringBuilder(); + for (int i = 0; i < arguments.size(); i++) { + if (i > 0) { + builder.append(", "); + } + builder.append(arguments.get(i).getType()); + } + return builder.toString(); + } + + private static String buildSignature(Method method) { + StringBuilder builder = new StringBuilder(method.getName()).append('('); + Class[] parameterTypes = method.getParameterTypes(); + for (int i = 0; i < parameterTypes.length; i++) { + if (i > 0) { + builder.append(", "); + } + Class parameterType = parameterTypes[i]; + if (method.isVarArgs() && i == parameterTypes.length - 1) { + builder.append(parameterType.getComponentType().getSimpleName()).append("..."); + } else { + builder.append(parameterType.getSimpleName()); + } + } + return builder.append(')').toString(); + } + + private static final class ResolutionState { + @Nullable + private BoundFunction _bestMatch; + private boolean _sawNonDeterministicCandidate; + private boolean _sawDeterministicCandidate; + } + + private static final class ScalarFunctionMethod { + private final Method _method; + private final Class[] _parameterTypes; + private final boolean _varArgs; + private final boolean _staticMethod; + private final PartitionValueType _outputType; + @Nullable + private final FunctionInvoker _sharedInvoker; + @Nullable + private final ThreadLocal _threadLocalInvoker; + + private ScalarFunctionMethod(FunctionInfo functionInfo) { + _method = functionInfo.getMethod(); + _parameterTypes = _method.getParameterTypes(); + _varArgs = _method.isVarArgs(); + _staticMethod = Modifier.isStatic(_method.getModifiers()); + _outputType = PartitionValueType.fromJavaType(_method.getReturnType()); + _sharedInvoker = _staticMethod ? new FunctionInvoker(functionInfo) : null; + _threadLocalInvoker = _staticMethod ? null : ThreadLocal.withInitial(() -> new FunctionInvoker(functionInfo)); + } + + @Nullable + public BoundFunction bind(List arguments) { + int parameterCount = _parameterTypes.length; + int fixedParameterCount = _varArgs ? parameterCount - 1 : parameterCount; + if ((!_varArgs && arguments.size() != parameterCount) || (_varArgs && arguments.size() < fixedParameterCount)) { + return null; + } + + Object[] constantArguments = new Object[arguments.size()]; + int totalCost = 0; + int dynamicIndex = -1; + PartitionValueType inputType = PartitionValueType.STRING; + Class dynamicParameterType = String.class; + for (int i = 0; i < arguments.size(); i++) { + Argument argument = arguments.get(i); + Class parameterType = getParameterType(i); + if (argument.isDynamic()) { + if (dynamicIndex >= 0) { + return null; + } + int cost = PartitionValueConversions.getDynamicConversionCost(argument.getType(), parameterType); + if (cost < 0) { + return null; + } + totalCost += cost; + dynamicIndex = i; + inputType = argument.getType(); + dynamicParameterType = parameterType; + constantArguments[i] = DYNAMIC_ARGUMENT; + } else { + PartitionValue constantValue = + Preconditions.checkNotNull(argument.getConstantValue(), "Constant argument must be configured"); + int cost = PartitionValueConversions.getConstantConversionCost(constantValue, parameterType); + if (cost < 0) { + return null; + } + totalCost += cost; + constantArguments[i] = PartitionValueConversions.convertValue(constantValue, parameterType, true); + } + } + return new BoundFunction(this, inputType, dynamicParameterType, dynamicIndex, constantArguments, totalCost); + } + + public PartitionValueType getOutputType() { + return _outputType; + } + + private Class getParameterType(int index) { + if (_varArgs && index >= _parameterTypes.length - 1) { + return _parameterTypes[_parameterTypes.length - 1].getComponentType(); + } + return _parameterTypes[index]; + } + + private Object invoke(Object[] expressionArguments) { + FunctionInvoker functionInvoker = + _staticMethod ? Preconditions.checkNotNull(_sharedInvoker) : Preconditions.checkNotNull(_threadLocalInvoker) + .get(); + return functionInvoker.invoke(adaptArguments(expressionArguments)); + } + + private Object[] adaptArguments(Object[] expressionArguments) { + if (!_varArgs) { + return expressionArguments; + } + + int fixedParameterCount = _parameterTypes.length - 1; + Object[] methodArguments = new Object[_parameterTypes.length]; + for (int i = 0; i < fixedParameterCount; i++) { + methodArguments[i] = expressionArguments[i]; + } + Class componentType = _parameterTypes[_parameterTypes.length - 1].getComponentType(); + int varArgCount = expressionArguments.length - fixedParameterCount; + Object varArgArray = Array.newInstance(componentType, varArgCount); + for (int i = 0; i < varArgCount; i++) { + Array.set(varArgArray, i, expressionArguments[fixedParameterCount + i]); + } + methodArguments[_parameterTypes.length - 1] = varArgArray; + return methodArguments; + } + } + + private static final class BoundFunction implements ResolvedFunction { + private final ScalarFunctionMethod _method; + private final PartitionValueType _inputType; + private final Class _dynamicParameterType; + private final int _dynamicIndex; + private final Object[] _constantArguments; + @Nullable + private final ThreadLocal _threadLocalArguments; + private final int _cost; + + private BoundFunction(ScalarFunctionMethod method, PartitionValueType inputType, Class dynamicParameterType, + int dynamicIndex, Object[] constantArguments, int cost) { + _method = method; + _inputType = inputType; + _dynamicParameterType = dynamicParameterType; + _dynamicIndex = dynamicIndex; + _constantArguments = constantArguments; + _threadLocalArguments = dynamicIndex >= 0 ? ThreadLocal.withInitial(_constantArguments::clone) : null; + _cost = cost; + } + + public int getCost() { + return _cost; + } + + @Override + public boolean isDynamic() { + return _dynamicIndex >= 0; + } + + @Override + public PartitionValueType getOutputType() { + return _method.getOutputType(); + } + + @Override + public PartitionValue invoke(@Nullable PartitionValue dynamicInput) { + Object[] expressionArguments = _dynamicIndex >= 0 ? Preconditions.checkNotNull(_threadLocalArguments).get() + : _constantArguments; + if (_dynamicIndex >= 0) { + Preconditions.checkNotNull(dynamicInput, "Dynamic partition step input must be configured"); + Preconditions.checkArgument(dynamicInput.getType() == _inputType, + "Expected %s dynamic input but got %s", _inputType, dynamicInput.getType()); + expressionArguments[_dynamicIndex] = PartitionValueConversions.convertValue(dynamicInput, _dynamicParameterType, + false); + } + return PartitionValue.fromObject(_method.invoke(expressionArguments)); + } + } +} diff --git a/pinot-common/src/main/java/org/apache/pinot/common/function/FunctionRegistry.java b/pinot-common/src/main/java/org/apache/pinot/common/function/FunctionRegistry.java index 395c49e8ca88..5c6a00b3cb2f 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/function/FunctionRegistry.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/function/FunctionRegistry.java @@ -24,7 +24,6 @@ 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; @@ -36,11 +35,10 @@ import org.apache.calcite.sql.type.SqlOperandTypeChecker; import org.apache.calcite.sql.type.SqlReturnTypeInference; import org.apache.calcite.sql.type.SqlTypeFamily; -import org.apache.commons.lang3.StringUtils; import org.apache.pinot.common.function.sql.PinotSqlFunction; import org.apache.pinot.common.utils.DataSchema.ColumnDataType; import org.apache.pinot.spi.annotations.ScalarFunction; -import org.apache.pinot.spi.utils.PinotReflectionUtils; +import org.apache.pinot.spi.utils.ScalarFunctionUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -88,8 +86,7 @@ private FunctionRegistry() { // Register ScalarFunction classes Map functionMap = new HashMap<>(); - Set> classes = - PinotReflectionUtils.getClassesThroughReflection(".*\\.function\\..*", ScalarFunction.class); + Set> classes = ScalarFunctionUtils.getScalarFunctionClasses(); for (Class clazz : classes) { if (!Modifier.isPublic(clazz.getModifiers())) { continue; @@ -102,26 +99,19 @@ private FunctionRegistry() { } catch (Exception e) { throw new IllegalStateException("Failed to instantiate PinotScalarFunction with class: " + clazz); } - String[] names = scalarFunction.names(); - if (names.length == 0) { - register(canonicalize(function.getName()), function, functionMap); - } else { - Set canonicalNames = new HashSet<>(); - for (String name : names) { - if (!canonicalNames.add(canonicalize(name))) { - LOGGER.warn("Duplicate names: {} in class: {}", Arrays.toString(names), clazz); - } - } - for (String canonicalName : canonicalNames) { - register(canonicalName, function, functionMap); - } + List canonicalNames = ScalarFunctionUtils.getScalarFunctionNames(scalarFunction, function.getName()); + if (scalarFunction.names().length > canonicalNames.size()) { + LOGGER.warn("Duplicate names: {} in class: {}", Arrays.toString(scalarFunction.names()), clazz); + } + for (String canonicalName : canonicalNames) { + register(canonicalName, function, functionMap); } } } // Register ScalarFunction methods Map> functionInfoMap = new HashMap<>(); - Set methods = PinotReflectionUtils.getMethodsThroughReflection(".*\\.function\\..*", ScalarFunction.class); + Set methods = ScalarFunctionUtils.getScalarFunctionMethods(); for (Method method : methods) { if (!Modifier.isPublic(method.getModifiers())) { continue; @@ -132,19 +122,12 @@ private FunctionRegistry() { new FunctionInfo(method, method.getDeclaringClass(), scalarFunction.nullableParameters(), scalarFunction.isDeterministic()); int numArguments = scalarFunction.isVarArg() ? VAR_ARG_KEY : method.getParameterCount(); - String[] names = scalarFunction.names(); - if (names.length == 0) { - register(canonicalize(method.getName()), functionInfo, numArguments, functionInfoMap); - } else { - Set canonicalNames = new HashSet<>(); - for (String name : names) { - if (!canonicalNames.add(canonicalize(name))) { - LOGGER.warn("Duplicate names: {} in method: {}", Arrays.toString(names), method); - } - } - for (String canonicalName : canonicalNames) { - register(canonicalName, functionInfo, numArguments, functionInfoMap); - } + List canonicalNames = ScalarFunctionUtils.getScalarFunctionNames(scalarFunction, method.getName()); + if (scalarFunction.names().length > canonicalNames.size()) { + LOGGER.warn("Duplicate names: {} in method: {}", Arrays.toString(scalarFunction.names()), method); + } + for (String canonicalName : canonicalNames) { + register(canonicalName, functionInfo, numArguments, functionInfoMap); } } } @@ -247,7 +230,7 @@ public static FunctionInfo getFunctionInfo(String name, int numArguments) { } public static String canonicalize(String name) { - return StringUtils.remove(name, '_').toLowerCase(); + return FunctionUtils.canonicalize(name); } public static class ArgumentCountBasedScalarFunction implements PinotScalarFunction { diff --git a/pinot-common/src/main/java/org/apache/pinot/common/function/FunctionUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/function/FunctionUtils.java index ab96dfd1b613..4348ae4528f8 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/function/FunctionUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/function/FunctionUtils.java @@ -22,11 +22,13 @@ import java.sql.Timestamp; import java.util.Collection; import java.util.HashMap; +import java.util.Locale; import java.util.Map; import javax.annotation.Nullable; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.commons.lang3.StringUtils; import org.apache.pinot.common.utils.DataSchema.ColumnDataType; import org.apache.pinot.common.utils.PinotDataType; @@ -216,4 +218,13 @@ public static boolean isAssertEnabled() { //CHECKSTYLE:ON return assertEnabled; } + + /** + * Returns the canonical form of a scalar function name: underscores stripped, lower-cased. + * This is the single authoritative implementation used by {@link FunctionRegistry} and other + * function-resolution paths. + */ + public static String canonicalize(String name) { + return StringUtils.remove(name, '_').toLowerCase(Locale.ROOT); + } } diff --git a/pinot-common/src/main/java/org/apache/pinot/common/function/scalar/HashFunctions.java b/pinot-common/src/main/java/org/apache/pinot/common/function/scalar/HashFunctions.java index ada184185d5c..8832f6f63196 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/function/scalar/HashFunctions.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/function/scalar/HashFunctions.java @@ -108,13 +108,21 @@ public static String md5(byte[] input) { return DigestUtils.md5Hex(input); } + /** + * Return raw MD5 digest bytes. + */ + @ScalarFunction(names = {"md5_raw"}) + public static byte[] md5Raw(byte[] input) { + return DigestUtils.md5(input); + } + /** * Computes 32-bit MurmurHash2 of the given byte array. * * @param input the byte array to hash * @return 32-bit hash */ - @ScalarFunction + @ScalarFunction(names = {"murmur2"}) public static int murmurHash2(byte[] input) { return MurmurHashFunctions.murmurHash2(input); } @@ -122,7 +130,7 @@ public static int murmurHash2(byte[] input) { /** * Computes 32-bit MurmurHash2 of the given string. * - * @param input the byte array to hash + * @param input the string to hash (converted to UTF-8 bytes) * @return 32-bit hash */ @ScalarFunction @@ -163,6 +171,14 @@ public static int murmurHash3Bit32(byte[] input, int seed) { return Hashing.murmur3_32_fixed(seed).hashBytes(input).asInt(); } + /** + * Computes 32-bit Murmur3 hash of the given byte array with seed 0 using the partition-expression friendly name. + */ + @ScalarFunction(names = {"murmur3_32"}) + public static int murmur3Bit32Default(byte[] input) { + return murmurHash3Bit32(input, 0); + } + /** * Computes 64-bit Murmur3 Hash of the given byte array and seed. * @@ -224,7 +240,7 @@ public static byte[] murmurHash3X64Bit128(byte[] input, int seed) { * @param input the byte array to hash * @return 32-bit hash */ - @ScalarFunction + @ScalarFunction(names = {"fnv1_32"}) public static int fnv1Hash32(byte[] input) { return FnvHashFunctions.fnv1Hash32(input); } @@ -232,7 +248,7 @@ public static int fnv1Hash32(byte[] input) { /** * Computes 32-bit FNV-1 hash of the given string using UTF-8 bytes. * - * @param input the string to hash + * @param input the string to hash (converted to UTF-8 bytes) * @return 32-bit hash */ @ScalarFunction @@ -246,7 +262,7 @@ public static int fnv1Hash32UTF8(String input) { * @param input the byte array to hash * @return 32-bit hash */ - @ScalarFunction + @ScalarFunction(names = {"fnv1a_32"}) public static int fnv1aHash32(byte[] input) { return FnvHashFunctions.fnv1aHash32(input); } @@ -268,7 +284,7 @@ public static int fnv1aHash32UTF8(String input) { * @param input the byte array to hash * @return 64-bit hash */ - @ScalarFunction + @ScalarFunction(names = {"fnv1_64"}) public static long fnv1Hash64(byte[] input) { return FnvHashFunctions.fnv1Hash64(input); } @@ -290,7 +306,7 @@ public static long fnv1Hash64UTF8(String input) { * @param input the byte array to hash * @return 64-bit hash */ - @ScalarFunction + @ScalarFunction(names = {"fnv1a_64"}) public static long fnv1aHash64(byte[] input) { return FnvHashFunctions.fnv1aHash64(input); } diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadataUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadataUtils.java index 9df510b3d542..d89b090fc43c 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadataUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/SegmentZKMetadataUtils.java @@ -178,8 +178,7 @@ private static void updateSegmentZKMetadata(String tableNameWithType, SegmentZKM PartitionFunction partitionFunction = columnMetadata.getPartitionFunction(); if (partitionFunction != null) { ColumnPartitionMetadata columnPartitionMetadata = - new ColumnPartitionMetadata(partitionFunction.getName(), partitionFunction.getNumPartitions(), - columnMetadata.getPartitions(), partitionFunction.getFunctionConfig()); + new ColumnPartitionMetadata(partitionFunction, columnMetadata.getPartitions()); columnPartitionMap.put(entry.getKey(), columnPartitionMetadata); } } diff --git a/pinot-common/src/main/resources/META-INF/services/org.apache.pinot.segment.spi.partition.pipeline.PartitionScalarFunctionResolver b/pinot-common/src/main/resources/META-INF/services/org.apache.pinot.segment.spi.partition.pipeline.PartitionScalarFunctionResolver new file mode 100644 index 000000000000..eb8f58c52117 --- /dev/null +++ b/pinot-common/src/main/resources/META-INF/services/org.apache.pinot.segment.spi.partition.pipeline.PartitionScalarFunctionResolver @@ -0,0 +1,19 @@ +# +# 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. +# +org.apache.pinot.common.function.CommonPartitionScalarFunctionResolver diff --git a/pinot-common/src/test/java/org/apache/pinot/common/function/scalar/PartitionFunctionExprCommonScalarFunctionTest.java b/pinot-common/src/test/java/org/apache/pinot/common/function/scalar/PartitionFunctionExprCommonScalarFunctionTest.java new file mode 100644 index 000000000000..dd98db1c545f --- /dev/null +++ b/pinot-common/src/test/java/org/apache/pinot/common/function/scalar/PartitionFunctionExprCommonScalarFunctionTest.java @@ -0,0 +1,62 @@ +/** + * 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.common.function.scalar; + +import org.apache.pinot.segment.spi.partition.pipeline.PartitionFunctionExprCompiler; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionPipelineFunction; +import org.testng.annotations.Test; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.expectThrows; + + +public class PartitionFunctionExprCommonScalarFunctionTest { + @Test + public void testClassBasedScalarFunctionSupportsRawStringNumericInput() { + PartitionPipelineFunction partitionFunction = + PartitionFunctionExprCompiler.compilePartitionFunction("timestampMillis", + "plus(intDiv(timestampMillis, 1000), 7)", 128); + + assertEquals(partitionFunction.getPartition("54321"), 61); + } + + @Test + public void testRejectsSideEffectScalarFunction() { + IllegalArgumentException error = expectThrows(IllegalArgumentException.class, + () -> PartitionFunctionExprCompiler.compile("raw_key", "sleep(raw_key)")); + assertEquals(error.getMessage(), + "Partition scalar function 'sleep' is not allowed because it is non-deterministic"); + } + + @Test + public void testRejectsNonDeterministicNowFunction() { + IllegalArgumentException error = expectThrows(IllegalArgumentException.class, + () -> PartitionFunctionExprCompiler.compile("raw_key", "now()")); + assertEquals(error.getMessage(), + "Partition scalar function 'now' is not allowed because it is non-deterministic"); + } + + @Test + public void testRejectsNonDeterministicAgoFunction() { + IllegalArgumentException error = expectThrows(IllegalArgumentException.class, + () -> PartitionFunctionExprCompiler.compile("raw_key", "ago('PT1H')")); + assertEquals(error.getMessage(), + "Partition scalar function 'ago' is not allowed because it is non-deterministic"); + } +} diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index a81a9e161124..c49bdefe71be 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -109,6 +109,8 @@ import org.apache.pinot.core.util.PeerServerSegmentFinder; import org.apache.pinot.segment.local.utils.TableConfigUtils; import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl; +import org.apache.pinot.segment.spi.partition.PartitionFunction; +import org.apache.pinot.segment.spi.partition.PartitionFunctionFactory; import org.apache.pinot.segment.spi.partition.metadata.ColumnPartitionMetadata; import org.apache.pinot.spi.config.provider.PinotClusterConfigChangeListener; import org.apache.pinot.spi.config.table.ColumnPartitionConfig; @@ -1168,9 +1170,10 @@ SegmentPartitionMetadata getPartitionMetadataFromTableConfig(TableConfig tableCo + "The stream partition count is used. Please update the table config accordingly.", perStreamNumPartitions, columnPartitionConfig.getNumPartitions()); } + PartitionFunction partitionFunction = + PartitionFunctionFactory.getPartitionFunction(entry.getKey(), columnPartitionConfig, perStreamNumPartitions); ColumnPartitionMetadata columnPartitionMetadata = - new ColumnPartitionMetadata(columnPartitionConfig.getFunctionName(), perStreamNumPartitions, - Collections.singleton(streamPartitionId), columnPartitionConfig.getFunctionConfig()); + new ColumnPartitionMetadata(partitionFunction, Collections.singleton(streamPartitionId)); return new SegmentPartitionMetadata(Collections.singletonMap(entry.getKey(), columnPartitionMetadata)); } else { LOGGER.warn( diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java index 2bd134a9e520..5e575141e8c5 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java @@ -104,6 +104,7 @@ import org.apache.pinot.segment.spi.loader.SegmentDirectoryLoaderContext; import org.apache.pinot.segment.spi.loader.SegmentDirectoryLoaderRegistry; import org.apache.pinot.segment.spi.partition.PartitionFunction; +import org.apache.pinot.segment.spi.partition.PartitionFunctionFactory; import org.apache.pinot.segment.spi.store.SegmentDirectory; import org.apache.pinot.spi.auth.AuthProvider; import org.apache.pinot.spi.config.instance.InstanceDataManagerConfig; @@ -1571,6 +1572,7 @@ StaleSegment isSegmentStale(IndexLoadingConfig indexLoadingConfig, SegmentDataMa String partitionColumn = null; ColumnPartitionConfig partitionConfig = null; + PartitionFunction expectedPartitionFunction = null; SegmentPartitionConfig segmentPartitionConfig = tableConfig.getIndexingConfig().getSegmentPartitionConfig(); // NOTE: Partition can only be enabled on a single column if (segmentPartitionConfig != null && segmentPartitionConfig.getColumnPartitionMap().size() == 1) { @@ -1578,6 +1580,8 @@ StaleSegment isSegmentStale(IndexLoadingConfig indexLoadingConfig, SegmentDataMa segmentPartitionConfig.getColumnPartitionMap().entrySet().iterator().next(); partitionColumn = entry.getKey(); partitionConfig = entry.getValue(); + expectedPartitionFunction = PartitionFunctionFactory.getPartitionFunction(partitionColumn, partitionConfig, + schema.getFieldSpecFor(partitionColumn)); } Set columnsInSegment = segmentMetadata.getAllColumns(); @@ -1800,12 +1804,23 @@ StaleSegment isSegmentStale(IndexLoadingConfig indexLoadingConfig, SegmentDataMa tableNameWithType, columnName, segmentName); return new StaleSegment(segmentName, true, "partition function added: " + columnName); } - if (!partitionFunction.getName().equalsIgnoreCase(partitionConfig.getFunctionName())) { + if (!partitionFunction.getName().equalsIgnoreCase(expectedPartitionFunction.getName())) { LOGGER.debug("tableNameWithType: {}, columnName: {}, segmentName: {}, change: partition function name", tableNameWithType, columnName, segmentName); return new StaleSegment(segmentName, true, "partition function name changed: " + columnName); } - if (partitionFunction.getNumPartitions() != partitionConfig.getNumPartitions()) { + if (!Objects.equals(partitionFunction.getFunctionExpr(), expectedPartitionFunction.getFunctionExpr())) { + LOGGER.debug("tableNameWithType: {}, columnName: {}, segmentName: {}, change: partition function expr", + tableNameWithType, columnName, segmentName); + return new StaleSegment(segmentName, true, "partition function expr changed: " + columnName); + } + if (!Objects.equals(partitionFunction.getPartitionIdNormalizer(), + expectedPartitionFunction.getPartitionIdNormalizer())) { + LOGGER.debug("tableNameWithType: {}, columnName: {}, segmentName: {}, change: partition id normalizer", + tableNameWithType, columnName, segmentName); + return new StaleSegment(segmentName, true, "partition id normalizer changed: " + columnName); + } + if (partitionFunction.getNumPartitions() != expectedPartitionFunction.getNumPartitions()) { LOGGER.debug("tableNameWithType: {}, columnName: {},, segmentName: {}, change: num partitions", tableNameWithType, columnName, segmentName); return new StaleSegment(segmentName, true, "num partitions changed: " + columnName); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java index fa423c7d081c..72fb44ac0de4 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java @@ -2015,7 +2015,6 @@ private void setPartitionParameters(RealtimeSegmentConfig.Builder realtimeSegmen Map.Entry entry = columnPartitionMap.entrySet().iterator().next(); String partitionColumn = entry.getKey(); ColumnPartitionConfig columnPartitionConfig = entry.getValue(); - String partitionFunctionName = columnPartitionConfig.getFunctionName(); // NOTE: Here we compare the number of partitions from the config and the stream, and log a warning and emit a // metric when they don't match, but use the one from the stream. The mismatch could happen when the @@ -2046,8 +2045,7 @@ private void setPartitionParameters(RealtimeSegmentConfig.Builder realtimeSegmen realtimeSegmentConfigBuilder.setPartitionColumn(partitionColumn); realtimeSegmentConfigBuilder.setPartitionFunction( - PartitionFunctionFactory.getPartitionFunction(partitionFunctionName, numPartitions, - columnPartitionConfig.getFunctionConfig())); + PartitionFunctionFactory.getPartitionFunction(partitionColumn, columnPartitionConfig, numPartitions)); realtimeSegmentConfigBuilder.setPartitionId(_partitionGroupId); } else { _segmentLogger.warn("Cannot partition on multiple columns: {}", columnPartitionMap.keySet()); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/pruner/ColumnValueSegmentPruner.java b/pinot-core/src/main/java/org/apache/pinot/core/query/pruner/ColumnValueSegmentPruner.java index 7b5a4b507bc4..0f377bea4f08 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/pruner/ColumnValueSegmentPruner.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/pruner/ColumnValueSegmentPruner.java @@ -33,6 +33,8 @@ import org.apache.pinot.segment.spi.datasource.DataSourceMetadata; import org.apache.pinot.segment.spi.partition.PartitionFunction; import org.apache.pinot.spi.data.FieldSpec.DataType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** @@ -57,6 +59,8 @@ */ @SuppressWarnings({"rawtypes", "unchecked", "RedundantIfStatement"}) public class ColumnValueSegmentPruner extends ValueBasedSegmentPruner { + private static final Logger LOGGER = LoggerFactory.getLogger(ColumnValueSegmentPruner.class); + @Override protected boolean isApplicableToPredicate(Predicate predicate) { // Only prune columns @@ -116,7 +120,7 @@ private boolean pruneEqPredicate(IndexSegment segment, EqPredicate eqPredicate, if (partitionFunction != null) { Set partitions = dataSourceMetadata.getPartitions(); assert partitions != null; - if (!partitions.contains(partitionFunction.getPartition(cachedValue.getValue()))) { + if (!isPartitionMatch(partitionFunction, partitions, cachedValue.getValue(), column)) { return true; } } @@ -143,10 +147,18 @@ private boolean pruneInPredicate(IndexSegment segment, InPredicate inPredicate, assert dataSource != null; DataSourceMetadata dataSourceMetadata = dataSource.getDataSourceMetadata(); List cachedValues = valueCache.get(inPredicate, dataSourceMetadata.getDataType()); + PartitionFunction partitionFunction = dataSourceMetadata.getPartitionFunction(); + Set partitions = partitionFunction != null ? dataSourceMetadata.getPartitions() : null; // Check min/max value for (ValueCache.CachedValue value : cachedValues) { if (checkMinMaxRange(dataSourceMetadata, value.getComparableValue())) { - return false; + if (partitionFunction == null) { + return false; + } + assert partitions != null; + if (isPartitionMatch(partitionFunction, partitions, value.getValue(), column)) { + return false; + } } } return true; @@ -245,4 +257,14 @@ private boolean checkMinMaxRange(DataSourceMetadata dataSourceMetadata, Comparab } return true; } + + private boolean isPartitionMatch(PartitionFunction partitionFunction, Set partitions, String value, + String column) { + try { + return partitions.contains(partitionFunction.getPartition(value)); + } catch (RuntimeException e) { + LOGGER.warn("Failed to evaluate partition function for column: {}; skipping partition pruning", column, e); + return true; + } + } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/mapper/SegmentMapper.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/mapper/SegmentMapper.java index 2eb73b86bd83..083282506fb1 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/mapper/SegmentMapper.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/mapper/SegmentMapper.java @@ -117,7 +117,7 @@ public SegmentMapper(List recordReaderFileConfigs, Trans schema.isEnableColumnBasedNullHandling() || tableConfig.getIndexingConfig().isNullHandlingEnabled(); _transformPipeline = transformPipeline; _timeHandler = TimeHandlerFactory.getTimeHandler(processorConfig); - _partitioners = PartitionerFactory.getPartitioners(processorConfig.getPartitionerConfigs()); + _partitioners = PartitionerFactory.getPartitioners(processorConfig.getPartitionerConfigs(), schema); // Time partition + partition from partitioners _partitionsBuffer = new String[_partitioners.length + 1]; _throttledLogger = new ThrottledLogger(LOGGER, tableConfig.getIngestionConfig()); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/partitioner/PartitionerFactory.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/partitioner/PartitionerFactory.java index 969aa4cd9662..9cb72dbfe526 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/partitioner/PartitionerFactory.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/partitioner/PartitionerFactory.java @@ -20,6 +20,8 @@ import com.google.common.base.Preconditions; import java.util.List; +import javax.annotation.Nullable; +import org.apache.pinot.spi.data.Schema; /** @@ -53,7 +55,14 @@ public enum PartitionerType { * Construct a Partitioner using the PartitioningConfig */ public static Partitioner getPartitioner(PartitionerConfig config) { + return getPartitioner(config, null); + } + /** + * Construct a Partitioner using the PartitioningConfig. The optional schema is used to determine the correct + * input type for expression-mode partition functions on BYTES-typed columns. + */ + public static Partitioner getPartitioner(PartitionerConfig config, @Nullable Schema schema) { Partitioner partitioner = null; switch (config.getPartitionerType()) { case NO_OP: @@ -79,7 +88,7 @@ public static Partitioner getPartitioner(PartitionerConfig config) { "Must provide columnName for TABLE_PARTITION_CONFIG Partitioner"); Preconditions.checkState(config.getColumnPartitionConfig() != null, "Must provide columnPartitionConfig for TABLE_PARTITION_CONFIG Partitioner"); - partitioner = new TableConfigPartitioner(config.getColumnName(), config.getColumnPartitionConfig()); + partitioner = new TableConfigPartitioner(config.getColumnName(), config.getColumnPartitionConfig(), schema); break; default: break; @@ -93,10 +102,20 @@ public static Partitioner getPartitioner(PartitionerConfig config) { * @return Array of partitioners */ public static Partitioner[] getPartitioners(List partitionerConfigs) { + return getPartitioners(partitionerConfigs, null); + } + + /** + * Create partitioner array from configuration. The optional schema is used to determine the correct + * input type for expression-mode partition functions on BYTES-typed columns. + * + * @return Array of partitioners + */ + public static Partitioner[] getPartitioners(List partitionerConfigs, @Nullable Schema schema) { int numPartitioners = partitionerConfigs.size(); Partitioner[] partitioners = new Partitioner[numPartitioners]; for (int i = 0; i < numPartitioners; i++) { - partitioners[i] = PartitionerFactory.getPartitioner(partitionerConfigs.get(i)); + partitioners[i] = PartitionerFactory.getPartitioner(partitionerConfigs.get(i), schema); } return partitioners; } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/partitioner/TableConfigPartitioner.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/partitioner/TableConfigPartitioner.java index ba0dbe7d1f63..d591512e9b82 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/partitioner/TableConfigPartitioner.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/partitioner/TableConfigPartitioner.java @@ -18,10 +18,12 @@ */ package org.apache.pinot.core.segment.processing.partitioner; +import javax.annotation.Nullable; import org.apache.pinot.segment.spi.partition.PartitionFunction; import org.apache.pinot.segment.spi.partition.PartitionFunctionFactory; import org.apache.pinot.spi.config.table.ColumnPartitionConfig; import org.apache.pinot.spi.data.FieldSpec; +import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.data.readers.GenericRow; @@ -31,15 +33,29 @@ public class TableConfigPartitioner implements Partitioner { private final String _column; private final PartitionFunction _partitionFunction; + /** True when the partition function was compiled with BYTES input; raw byte[] values are passed directly. */ + private final boolean _isBytesMode; public TableConfigPartitioner(String columnName, ColumnPartitionConfig columnPartitionConfig) { + this(columnName, columnPartitionConfig, null); + } + + public TableConfigPartitioner(String columnName, ColumnPartitionConfig columnPartitionConfig, + @Nullable Schema schema) { _column = columnName; - _partitionFunction = PartitionFunctionFactory.getPartitionFunction(columnPartitionConfig); + FieldSpec fieldSpec = schema != null ? schema.getFieldSpecFor(columnName) : null; + _isBytesMode = columnPartitionConfig.getFunctionExpr() != null && fieldSpec != null + && fieldSpec.getDataType().getStoredType() == FieldSpec.DataType.BYTES; + _partitionFunction = PartitionFunctionFactory.getPartitionFunction(columnName, columnPartitionConfig, fieldSpec); } @Override public String getPartition(GenericRow genericRow) { - return String.valueOf(_partitionFunction.getPartition(FieldSpec.getStringValue(genericRow.getValue(_column)))); + Object value = genericRow.getValue(_column); + if (_isBytesMode && value instanceof byte[]) { + return String.valueOf(_partitionFunction.getPartition((byte[]) value)); + } + return String.valueOf(_partitionFunction.getPartition(FieldSpec.getStringValue(value))); } @Override @@ -53,6 +69,10 @@ public String getPartitionFromColumns(Object[] columnValues) { throw new IllegalArgumentException( "TableConfigPartitioner expects exactly 1 column value, got " + columnValues.length); } - return String.valueOf(_partitionFunction.getPartition(FieldSpec.getStringValue(columnValues[0]))); + Object value = columnValues[0]; + if (_isBytesMode && value instanceof byte[]) { + return String.valueOf(_partitionFunction.getPartition((byte[]) value)); + } + return String.valueOf(_partitionFunction.getPartition(FieldSpec.getStringValue(value))); } } diff --git a/pinot-core/src/test/java/org/apache/pinot/core/query/pruner/ColumnValueSegmentPrunerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/query/pruner/ColumnValueSegmentPrunerTest.java index ffb74cec8c6f..2068b9beb00c 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/query/pruner/ColumnValueSegmentPrunerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/query/pruner/ColumnValueSegmentPrunerTest.java @@ -29,6 +29,7 @@ import org.apache.pinot.segment.spi.SegmentMetadata; import org.apache.pinot.segment.spi.datasource.DataSource; import org.apache.pinot.segment.spi.datasource.DataSourceMetadata; +import org.apache.pinot.segment.spi.partition.PartitionFunction; import org.apache.pinot.segment.spi.partition.PartitionFunctionFactory; import org.apache.pinot.spi.data.FieldSpec.DataType; import org.apache.pinot.spi.data.Schema; @@ -37,6 +38,7 @@ import org.testng.annotations.Test; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -142,6 +144,96 @@ public void testPartitionPruning() { assertTrue(runPruner(indexSegment, "SELECT COUNT(*) FROM testTable WHERE column = 0 OR column = 10")); } + @Test + public void testPartitionPruningWithFunctionExpr() { + IndexSegment indexSegment = mockIndexSegment(); + + DataSource dataSource = mock(DataSource.class); + when(indexSegment.getDataSource(eq("column"), any(Schema.class))).thenReturn(dataSource); + + DataSourceMetadata dataSourceMetadata = mock(DataSourceMetadata.class); + when(dataSourceMetadata.getDataType()).thenReturn(DataType.STRING); + PartitionFunction partitionFunction = + PartitionFunctionFactory.getPartitionFunction("column", null, 8, null, "murmur2(lower(column))"); + int matchingPartition = partitionFunction.getPartition("Pinot"); + String firstNonMatchingValue = findValueWithDifferentPartition(partitionFunction, matchingPartition, "Kafka", + "Trino", "StarTree", "Presto", "Druid"); + String secondNonMatchingValue = findValueWithDifferentPartition(partitionFunction, matchingPartition, "Flink", + "Spark", "Hive", "Superset", "PinotDB"); + when(dataSourceMetadata.getPartitionFunction()).thenReturn(partitionFunction); + when(dataSourceMetadata.getPartitions()).thenReturn(Collections.singleton(matchingPartition)); + when(dataSource.getDataSourceMetadata()).thenReturn(dataSourceMetadata); + + assertFalse(runPruner(indexSegment, "SELECT COUNT(*) FROM testTable WHERE column = 'Pinot'")); + assertFalse(runPruner(indexSegment, "SELECT COUNT(*) FROM testTable WHERE column = 'pinot'")); + assertTrue(runPruner(indexSegment, + String.format("SELECT COUNT(*) FROM testTable WHERE column = '%s'", firstNonMatchingValue))); + assertFalse(runPruner(indexSegment, + String.format("SELECT COUNT(*) FROM testTable WHERE column IN ('%s', 'Pinot')", firstNonMatchingValue))); + assertTrue(runPruner(indexSegment, String.format("SELECT COUNT(*) FROM testTable WHERE column IN ('%s', '%s')", + firstNonMatchingValue, secondNonMatchingValue))); + } + + /** + * Verifies that IN-predicate pruning combining min/max and partition checks is correct: + *

    + *
  • A value that is within the min/max range AND maps to the segment's partition → keep (false)
  • + *
  • All values within the min/max range but NONE maps to the segment's partition → prune (true)
  • + *
  • All values outside the min/max range → prune (true) regardless of partition
  • + *
+ */ + @Test + public void testInPredicateWithMinMaxAndPartitionPruning() { + IndexSegment indexSegment = mockIndexSegment(); + + DataSource dataSource = mock(DataSource.class); + when(indexSegment.getDataSource(eq("column"), any(Schema.class))).thenReturn(dataSource); + + DataSourceMetadata dataSourceMetadata = mock(DataSourceMetadata.class); + when(dataSourceMetadata.getDataType()).thenReturn(DataType.INT); + // Segment contains values 10..20 and belongs to partition 2 of a 5-partition Modulo function + when(dataSourceMetadata.getMinValue()).thenReturn(10); + when(dataSourceMetadata.getMaxValue()).thenReturn(20); + PartitionFunction modulo5 = PartitionFunctionFactory.getPartitionFunction("Modulo", 5, null); + when(dataSourceMetadata.getPartitionFunction()).thenReturn(modulo5); + // partition 2 → values 2, 7, 12, 17 + when(dataSourceMetadata.getPartitions()).thenReturn(Collections.singleton(2)); + when(dataSource.getDataSourceMetadata()).thenReturn(dataSourceMetadata); + + // value 12 is in range [10,20] AND maps to partition 2 → keep + assertFalse(runPruner(indexSegment, "SELECT COUNT(*) FROM testTable WHERE column IN (12)")); + // value 17 is in range [10,20] AND maps to partition 2 → keep + assertFalse(runPruner(indexSegment, "SELECT COUNT(*) FROM testTable WHERE column IN (17)")); + // value 11 is in range [10,20] but maps to partition 1 (11 % 5 = 1); value 13 maps to partition 3 → both wrong + // partition → prune + assertTrue(runPruner(indexSegment, "SELECT COUNT(*) FROM testTable WHERE column IN (11, 13)")); + // value 5 is out of range; value 11 is in range but wrong partition → prune + assertTrue(runPruner(indexSegment, "SELECT COUNT(*) FROM testTable WHERE column IN (5, 11)")); + // value 5 is out of range; value 12 is in range AND right partition → keep + assertFalse(runPruner(indexSegment, "SELECT COUNT(*) FROM testTable WHERE column IN (5, 12)")); + // all values out of range → prune + assertTrue(runPruner(indexSegment, "SELECT COUNT(*) FROM testTable WHERE column IN (1, 2, 3)")); + } + + @Test + public void testPartitionPruningFailsOpenWhenPartitionFunctionThrows() { + IndexSegment indexSegment = mockIndexSegment(); + + DataSource dataSource = mock(DataSource.class); + when(indexSegment.getDataSource(eq("column"), any(Schema.class))).thenReturn(dataSource); + + DataSourceMetadata dataSourceMetadata = mock(DataSourceMetadata.class); + when(dataSourceMetadata.getDataType()).thenReturn(DataType.STRING); + PartitionFunction partitionFunction = mock(PartitionFunction.class); + when(partitionFunction.getPartition(anyString())).thenThrow(new IllegalArgumentException("boom")); + when(dataSourceMetadata.getPartitionFunction()).thenReturn(partitionFunction); + when(dataSourceMetadata.getPartitions()).thenReturn(Collections.singleton(2)); + when(dataSource.getDataSourceMetadata()).thenReturn(dataSourceMetadata); + + assertFalse(runPruner(indexSegment, "SELECT COUNT(*) FROM testTable WHERE column = 'boom'")); + assertFalse(runPruner(indexSegment, "SELECT COUNT(*) FROM testTable WHERE column IN ('boom', 'still_boom')")); + } + @Test public void testIsApplicableTo() { // EQ, RANGE and IN (with small number of values) are applicable for min/max/partitionId based pruning. @@ -195,4 +287,13 @@ private boolean runPruner(IndexSegment indexSegment, String query) { queryContext.setSchema(mock(Schema.class)); return PRUNER.prune(Arrays.asList(indexSegment), queryContext).isEmpty(); } + + private String findValueWithDifferentPartition(PartitionFunction partitionFunction, int partition, String... values) { + for (String value : values) { + if (partitionFunction.getPartition(value) != partition) { + return value; + } + } + throw new IllegalStateException("Failed to find value on a different partition"); + } } diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/SegmentPartitionLLCRealtimeClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/SegmentPartitionLLCRealtimeClusterIntegrationTest.java index d2ed66a7796e..9426f3086f59 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/SegmentPartitionLLCRealtimeClusterIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/SegmentPartitionLLCRealtimeClusterIntegrationTest.java @@ -160,6 +160,7 @@ public void testPartitionMetadata() { assertEquals(columnPartitionMetadataMap.size(), 1); ColumnPartitionMetadata columnPartitionMetadata = columnPartitionMetadataMap.get(PARTITION_COLUMN); assertNotNull(columnPartitionMetadata); + assertNotNull(columnPartitionMetadata.getFunctionName()); assertTrue(columnPartitionMetadata.getFunctionName().equalsIgnoreCase("murmur")); assertEquals(columnPartitionMetadata.getNumPartitions(), 2); int partitionGroupId = new LLCSegmentName(segmentZKMetadata.getSegmentName()).getPartitionGroupId(); @@ -267,6 +268,7 @@ public void testNonPartitionedStream() assertEquals(columnPartitionMetadataMap.size(), 1); ColumnPartitionMetadata columnPartitionMetadata = columnPartitionMetadataMap.get(PARTITION_COLUMN); assertNotNull(columnPartitionMetadata); + assertNotNull(columnPartitionMetadata.getFunctionName()); assertTrue(columnPartitionMetadata.getFunctionName().equalsIgnoreCase("murmur")); assertEquals(columnPartitionMetadata.getNumPartitions(), 2); int partitionGroupId = new LLCSegmentName(segmentZKMetadata.getSegmentName()).getPartitionGroupId(); @@ -344,6 +346,7 @@ public void testNonPartitionedStream() assertEquals(columnPartitionMetadataMap.size(), 1); ColumnPartitionMetadata columnPartitionMetadata = columnPartitionMetadataMap.get(PARTITION_COLUMN); assertNotNull(columnPartitionMetadata); + assertNotNull(columnPartitionMetadata.getFunctionName()); assertTrue(columnPartitionMetadata.getFunctionName().equalsIgnoreCase("murmur")); assertEquals(columnPartitionMetadata.getNumPartitions(), 2); int partitionGroupId = new LLCSegmentName(segmentZKMetadata.getSegmentName()).getPartitionGroupId(); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java index 118d3b9f2bfe..2ccfb8b24974 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java @@ -601,9 +601,13 @@ private boolean isNoDictionaryColumn(FieldIndexConfigs indexConfigs, FieldSpec f public SegmentPartitionConfig getSegmentPartitionConfig() { if (_partitionColumn != null) { + ColumnPartitionConfig columnPartitionConfig = _partitionFunction.getFunctionExpr() != null + ? ColumnPartitionConfig.forFunctionExpr(_partitionFunction.getFunctionExpr(), + _partitionFunction.getNumPartitions(), _partitionFunction.getPartitionIdNormalizer()) + : new ColumnPartitionConfig(_partitionFunction.getName(), _partitionFunction.getNumPartitions(), + _partitionFunction.getFunctionConfig()); return new SegmentPartitionConfig(Collections.singletonMap(_partitionColumn, - new ColumnPartitionConfig(_partitionFunction.getName(), _partitionFunction.getNumPartitions(), - _partitionFunction.getFunctionConfig()))); + columnPartitionConfig)); } else { return null; } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java index c7f623b6c90f..593927d3f577 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java @@ -407,7 +407,6 @@ private void setPartitionParameters(RealtimeSegmentConfig.Builder realtimeSegmen Map.Entry entry = columnPartitionMap.entrySet().iterator().next(); String partitionColumn = entry.getKey(); ColumnPartitionConfig columnPartitionConfig = entry.getValue(); - String partitionFunctionName = columnPartitionConfig.getFunctionName(); // NOTE: Here we compare the number of partitions from the config and the stream, and log a warning and emit a // metric when they don't match, but use the one from the stream. The mismatch could happen when the @@ -439,8 +438,7 @@ private void setPartitionParameters(RealtimeSegmentConfig.Builder realtimeSegmen realtimeSegmentConfigBuilder.setPartitionColumn(partitionColumn); realtimeSegmentConfigBuilder.setPartitionFunction( - PartitionFunctionFactory.getPartitionFunction(partitionFunctionName, numPartitions, - columnPartitionConfig.getFunctionConfig())); + PartitionFunctionFactory.getPartitionFunction(partitionColumn, columnPartitionConfig, numPartitions)); realtimeSegmentConfigBuilder.setPartitionId(_partitionGroupId); } else { _logger.warn("Cannot partition on multiple columns: {}", columnPartitionMap.keySet()); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/BaseSegmentCreator.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/BaseSegmentCreator.java index 006e85612c89..39437cdab4cc 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/BaseSegmentCreator.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/BaseSegmentCreator.java @@ -617,8 +617,41 @@ public static void addColumnMetadataInfo(PropertiesConfiguration properties, Str properties.setProperty(getKeyFor(column, IS_AUTO_GENERATED), "true"); } - // Min/max value - if (fieldSpec.getFieldType() != FieldType.COMPLEX) { + PartitionFunction partitionFunction = columnStatistics.getPartitionFunction(); + if (partitionFunction != null) { + // Expression-mode functions are identified by the presence of PARTITION_FUNCTION_EXPR in segment metadata. + // The legacy PARTITION_FUNCTION name key is omitted for expression-mode to avoid writing a sentinel value + // ("FunctionExpr") that pre-feature readers would reject as an unknown function name. Note: segments written + // with expression-mode metadata use v7 format and are not readable by older servers regardless of this key. + if (partitionFunction.getFunctionExpr() == null) { + properties.setProperty(getKeyFor(column, PARTITION_FUNCTION), partitionFunction.getName()); + } + properties.setProperty(getKeyFor(column, NUM_PARTITIONS), partitionFunction.getNumPartitions()); + if (partitionFunction.getFunctionExpr() != null) { + properties.setProperty(getKeyFor(column, PARTITION_FUNCTION_EXPR), partitionFunction.getFunctionExpr()); + } + if (partitionFunction.getPartitionIdNormalizer() != null) { + properties.setProperty(getKeyFor(column, PARTITION_ID_NORMALIZER), + partitionFunction.getPartitionIdNormalizer()); + } + properties.setProperty(getKeyFor(column, PARTITION_VALUES), columnStatistics.getPartitions()); + Map partitionFunctionConfig = partitionFunction.getFunctionConfig(); + if (partitionFunctionConfig != null) { + for (Map.Entry entry : partitionFunctionConfig.entrySet()) { + properties.setProperty(getKeyFor(column, String.format("%s.%s", PARTITION_FUNCTION_CONFIG, entry.getKey())), + entry.getValue()); + } + } + } + + // Datetime field + if (fieldType == FieldType.DATE_TIME) { + DateTimeFieldSpec dateTimeFieldSpec = (DateTimeFieldSpec) fieldSpec; + properties.setProperty(getKeyFor(column, DATETIME_FORMAT), dateTimeFieldSpec.getFormat()); + properties.setProperty(getKeyFor(column, DATETIME_GRANULARITY), dateTimeFieldSpec.getGranularity()); + } + + if (fieldType != FieldType.COMPLEX) { // Regular (non-complex) field if (totalDocs > 0) { Object min = columnStatistics.getMinValue(); @@ -632,21 +665,6 @@ public static void addColumnMetadataInfo(PropertiesConfiguration properties, Str } } } - - // Partition function - PartitionFunction partitionFunction = columnStatistics.getPartitionFunction(); - if (partitionFunction != null) { - properties.setProperty(getKeyFor(column, PARTITION_FUNCTION), partitionFunction.getName()); - properties.setProperty(getKeyFor(column, NUM_PARTITIONS), partitionFunction.getNumPartitions()); - properties.setProperty(getKeyFor(column, PARTITION_VALUES), columnStatistics.getPartitions()); - Map partitionFunctionConfig = partitionFunction.getFunctionConfig(); - if (partitionFunctionConfig != null) { - for (Map.Entry entry : partitionFunctionConfig.entrySet()) { - properties.setProperty(getKeyFor(column, String.format("%s.%s", PARTITION_FUNCTION_CONFIG, entry.getKey())), - entry.getValue()); - } - } - } } /** diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/AbstractColumnStatisticsCollector.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/AbstractColumnStatisticsCollector.java index 14db5917e079..6b18675a2e52 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/AbstractColumnStatisticsCollector.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/AbstractColumnStatisticsCollector.java @@ -164,4 +164,12 @@ protected boolean isPartitionEnabled() { protected void updatePartition(String value) { _partitions.add(_partitionFunction.getPartition(value)); } + + /** + * Updates the partition set using raw bytes. Expression-mode pipelines compiled with BYTES input type will hash + * the raw bytes directly; legacy partition functions fall back to hex-encoding via the default interface method. + */ + protected void updatePartition(byte[] bytes) { + _partitions.add(_partitionFunction.getPartition(bytes)); + } } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/BytesColumnPreIndexStatsCollector.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/BytesColumnPreIndexStatsCollector.java index 5b2f41cdbc29..2cdcf47bd929 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/BytesColumnPreIndexStatsCollector.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/BytesColumnPreIndexStatsCollector.java @@ -64,7 +64,7 @@ public void collect(Object entry) { addressSorted(value); if (_values.add(value)) { if (isPartitionEnabled()) { - updatePartition(value.toString()); + updatePartition(value.getBytes()); } int length = value.length(); _minLength = Math.min(_minLength, length); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java index 31bea7e4a708..dfae571e4f52 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java @@ -58,6 +58,7 @@ import org.apache.pinot.segment.spi.index.StandardIndexes; import org.apache.pinot.segment.spi.index.multicolumntext.MultiColumnTextMetadata; import org.apache.pinot.segment.spi.index.startree.AggregationFunctionColumnPair; +import org.apache.pinot.segment.spi.partition.PartitionFunctionFactory; import org.apache.pinot.spi.config.table.ColumnPartitionConfig; import org.apache.pinot.spi.config.table.DedupConfig; import org.apache.pinot.spi.config.table.FieldConfig; @@ -1568,6 +1569,7 @@ private static void validateIndexingConfigAndFieldConfigList(TableConfig tableCo Preconditions.checkState(fieldSpec != null, "Failed to find partition column: %s in schema", column); Preconditions.checkState(fieldSpec.isSingleValueField(), "Cannot partition on multi-value column: %s", column); + PartitionFunctionFactory.getPartitionFunction(column, columnPartitionMap.get(column), fieldSpec); } } } diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/realtime/converter/RealtimeSegmentConverterTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/realtime/converter/RealtimeSegmentConverterTest.java index b5b2d400a282..e77e24d08480 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/realtime/converter/RealtimeSegmentConverterTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/realtime/converter/RealtimeSegmentConverterTest.java @@ -58,6 +58,7 @@ import org.apache.pinot.segment.spi.index.reader.TextIndexReader; import org.apache.pinot.segment.spi.partition.PartitionFunction; import org.apache.pinot.segment.spi.partition.PartitionFunctionFactory; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionPipelineFunction; import org.apache.pinot.segment.spi.store.SegmentDirectory; import org.apache.pinot.spi.config.table.ColumnPartitionConfig; import org.apache.pinot.spi.config.table.FieldConfig; @@ -958,6 +959,102 @@ public void testMurmur3PartitionFunctionConfigPreservedInConvertedSegment() } } + @Test + public void testPartitionFunctionExprPreservedInConvertedSegment() + throws Exception { + File tmpDir = new File(TMP_DIR, "tmp_" + System.currentTimeMillis()); + String functionExpr = "fnv1a_32(md5(" + STRING_COLUMN1 + "))"; + String partitionIdNormalizer = "MASK"; + + SegmentPartitionConfig segmentPartitionConfig = new SegmentPartitionConfig( + Map.of(STRING_COLUMN1, ColumnPartitionConfig.forFunctionExpr(functionExpr, 8, partitionIdNormalizer))); + + TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME) + .setTableName("testTable") + .setTimeColumnName(DATE_TIME_COLUMN) + .setSegmentPartitionConfig(segmentPartitionConfig) + .setColumnMajorSegmentBuilderEnabled(false) + .build(); + Schema schema = new Schema.SchemaBuilder() + .setSchemaName("testTable") + .addSingleValueDimension(STRING_COLUMN1, DataType.STRING) + .addSingleValueDimension(STRING_COLUMN2, DataType.STRING) + .addSingleValueDimension(STRING_COLUMN3, DataType.STRING) + .addSingleValueDimension(STRING_COLUMN4, DataType.STRING) + .addSingleValueDimension(LONG_COLUMN1, DataType.LONG) + .addSingleValueDimension(LONG_COLUMN2, DataType.LONG) + .addSingleValueDimension(LONG_COLUMN3, DataType.LONG) + .addMultiValueDimension(MV_INT_COLUMN, DataType.INT) + .addMetric(LONG_COLUMN4, DataType.LONG) + .addDateTime(DATE_TIME_COLUMN, DataType.LONG, "1:MILLISECONDS:EPOCH", "1:MILLISECONDS") + .build(); + + String tableNameWithType = tableConfig.getTableName(); + String segmentName = "testTable__0__0__123456"; + + PartitionFunction partitionFunction = + PartitionFunctionFactory.getPartitionFunction(STRING_COLUMN1, + segmentPartitionConfig.getColumnPartitionMap().get(STRING_COLUMN1)); + RealtimeSegmentConfig realtimeSegmentConfig = new RealtimeSegmentConfig.Builder() + .setTableNameWithType(tableNameWithType) + .setSegmentName(segmentName) + .setStreamName(tableNameWithType) + .setSchema(schema) + .setTimeColumnName(DATE_TIME_COLUMN) + .setCapacity(1000) + .setAvgNumMultiValues(3) + .setPartitionColumn(STRING_COLUMN1) + .setPartitionFunction(partitionFunction) + .setPartitionId(0) + .setSegmentZKMetadata(getSegmentZKMetadata(segmentName)) + .setOffHeap(true) + .setMemoryManager(new DirectMemoryManager(segmentName)) + .setStatsHistory(RealtimeSegmentStatsHistory.deserializeFrom(new File(tmpDir, "stats"))) + .setConsumerDir(new File(tmpDir, "consumerDir").getAbsolutePath()) + .build(); + + MutableSegmentImpl mutableSegmentImpl = new MutableSegmentImpl(realtimeSegmentConfig, null); + try { + List rows = generateTestData(); + for (GenericRow row : rows) { + mutableSegmentImpl.index(row, null); + } + + SegmentPartitionConfig mutablePartitionConfig = mutableSegmentImpl.getSegmentPartitionConfig(); + assertNotNull(mutablePartitionConfig); + ColumnPartitionConfig columnPartitionConfig = + mutablePartitionConfig.getColumnPartitionMap().get(STRING_COLUMN1); + assertNotNull(columnPartitionConfig); + assertNull(columnPartitionConfig.getFunctionName()); + assertEquals(columnPartitionConfig.getFunctionExpr(), functionExpr); + assertEquals(columnPartitionConfig.getPartitionIdNormalizer(), partitionIdNormalizer); + assertEquals(columnPartitionConfig.getNumPartitions(), 8); + assertNull(columnPartitionConfig.getFunctionConfig()); + + File outputDir = new File(tmpDir, "outputDir"); + SegmentZKPropsConfig segmentZKPropsConfig = new SegmentZKPropsConfig(); + segmentZKPropsConfig.setStartOffset("1"); + segmentZKPropsConfig.setEndOffset("100"); + RealtimeSegmentConverter converter = + new RealtimeSegmentConverter(mutableSegmentImpl, segmentZKPropsConfig, outputDir.getAbsolutePath(), schema, + tableNameWithType, tableConfig, segmentName, false); + converter.build(SegmentVersion.v3); + + File indexDir = new File(outputDir, segmentName); + SegmentMetadataImpl segmentMetadata = new SegmentMetadataImpl(indexDir); + + ColumnMetadata col1Meta = segmentMetadata.getColumnMetadataFor(STRING_COLUMN1); + assertNotNull(col1Meta.getPartitionFunction(), "Partition function should be present in converted segment"); + assertEquals(col1Meta.getPartitionFunction().getName(), PartitionPipelineFunction.NAME); + assertEquals(col1Meta.getPartitionFunction().getNumPartitions(), 8); + assertEquals(col1Meta.getPartitionFunction().getFunctionExpr(), functionExpr); + assertEquals(col1Meta.getPartitionFunction().getPartitionIdNormalizer(), partitionIdNormalizer); + assertNull(col1Meta.getPartitionFunction().getFunctionConfig()); + } finally { + mutableSegmentImpl.destroy(); + } + } + @Test public void testFnvPartitionFunctionConfigPreservedInConvertedSegment() throws Exception { diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/utils/TableConfigUtilsTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/utils/TableConfigUtilsTest.java index b6eac19916d3..698ed0df82d2 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/utils/TableConfigUtilsTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/utils/TableConfigUtilsTest.java @@ -3435,6 +3435,77 @@ public void testValidateImplicitRealtimeTablePartitionSelectorConfigs() { TableConfigUtils.validateInstanceAssignmentConfigs(tableConfig6); } + @Test + public void testValidateSegmentPartitionFunctionExpr() { + Schema schema = new Schema.SchemaBuilder().setSchemaName(TABLE_NAME) + .addSingleValueDimension(PARTITION_COLUMN, FieldSpec.DataType.STRING) + .addDateTime(TIME_COLUMN, FieldSpec.DataType.LONG, "1:MILLISECONDS:EPOCH", "1:MILLISECONDS") + .build(); + SegmentPartitionConfig segmentPartitionConfig = new SegmentPartitionConfig(Collections.singletonMap( + PARTITION_COLUMN, ColumnPartitionConfig.forFunctionExpr("murmur2(lower(" + PARTITION_COLUMN + "))", 8))); + TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(TABLE_NAME) + .setTimeColumnName(TIME_COLUMN) + .setStreamConfigs(getStreamConfigs()) + .setSegmentPartitionConfig(segmentPartitionConfig) + .build(); + + TableConfigUtils.validate(tableConfig, schema); + } + + @Test + public void testValidateSegmentPartitionFunctionExprWithLiteralArgument() { + Schema schema = new Schema.SchemaBuilder().setSchemaName(TABLE_NAME) + .addSingleValueDimension(PARTITION_COLUMN, FieldSpec.DataType.STRING) + .addDateTime(TIME_COLUMN, FieldSpec.DataType.LONG, "1:MILLISECONDS:EPOCH", "1:MILLISECONDS") + .build(); + SegmentPartitionConfig segmentPartitionConfig = new SegmentPartitionConfig(Collections.singletonMap( + TIME_COLUMN, ColumnPartitionConfig.forFunctionExpr("toEpochSecondsRounded(" + TIME_COLUMN + ", 10)", 8))); + TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(TABLE_NAME) + .setTimeColumnName(TIME_COLUMN) + .setStreamConfigs(getStreamConfigs()) + .setSegmentPartitionConfig(segmentPartitionConfig) + .build(); + + TableConfigUtils.validate(tableConfig, schema); + } + + @Test + public void testValidateSegmentPartitionFunctionExprWithExplicitMaskNormalizer() { + Schema schema = new Schema.SchemaBuilder().setSchemaName(TABLE_NAME) + .addSingleValueDimension(PARTITION_COLUMN, FieldSpec.DataType.STRING) + .addDateTime(TIME_COLUMN, FieldSpec.DataType.LONG, "1:MILLISECONDS:EPOCH", "1:MILLISECONDS") + .build(); + SegmentPartitionConfig segmentPartitionConfig = new SegmentPartitionConfig(Collections.singletonMap( + PARTITION_COLUMN, ColumnPartitionConfig.forFunctionExpr("fnv1a_32(md5(" + PARTITION_COLUMN + "))", 128, + "MASK"))); + TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(TABLE_NAME) + .setTimeColumnName(TIME_COLUMN) + .setStreamConfigs(getStreamConfigs()) + .setSegmentPartitionConfig(segmentPartitionConfig) + .build(); + + TableConfigUtils.validate(tableConfig, schema); + } + + @Test + public void testRejectInvalidSegmentPartitionFunctionExpr() { + Schema schema = new Schema.SchemaBuilder().setSchemaName(TABLE_NAME) + .addSingleValueDimension(PARTITION_COLUMN, FieldSpec.DataType.STRING) + .addDateTime(TIME_COLUMN, FieldSpec.DataType.LONG, "1:MILLISECONDS:EPOCH", "1:MILLISECONDS") + .build(); + SegmentPartitionConfig segmentPartitionConfig = new SegmentPartitionConfig(Collections.singletonMap( + PARTITION_COLUMN, ColumnPartitionConfig.forFunctionExpr("md5(" + PARTITION_COLUMN + ")", 8))); + TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(TABLE_NAME) + .setTimeColumnName(TIME_COLUMN) + .setStreamConfigs(getStreamConfigs()) + .setSegmentPartitionConfig(segmentPartitionConfig) + .build(); + + IllegalArgumentException e = + expectThrows(IllegalArgumentException.class, () -> TableConfigUtils.validate(tableConfig, schema)); + assertEquals(e.getMessage(), "Partition pipeline must produce INT or LONG output, got: STRING"); + } + private Map getStreamConfigs() { Map streamConfigs = new HashMap<>(); streamConfigs.put("streamType", "kafka"); diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/V1Constants.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/V1Constants.java index 7605c98c554c..52e9885b5d69 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/V1Constants.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/V1Constants.java @@ -173,6 +173,8 @@ public static class Column { /// Partition function, all optional public static final String PARTITION_FUNCTION = "partitionFunction"; + public static final String PARTITION_FUNCTION_EXPR = "partitionFunctionExpr"; + public static final String PARTITION_ID_NORMALIZER = "partitionIdNormalizer"; public static final String NUM_PARTITIONS = "numPartitions"; public static final String PARTITION_VALUES = "partitionValues"; public static final String PARTITION_FUNCTION_CONFIG = "partitionFunctionConfig"; diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/StatsCollectorConfig.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/StatsCollectorConfig.java index f4f216b0505e..6e2c6d61fd9b 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/StatsCollectorConfig.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/StatsCollectorConfig.java @@ -24,6 +24,7 @@ import javax.annotation.Nullable; import org.apache.pinot.segment.spi.partition.PartitionFunction; import org.apache.pinot.segment.spi.partition.PartitionFunctionFactory; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionValueType; import org.apache.pinot.spi.config.table.ColumnPartitionConfig; import org.apache.pinot.spi.config.table.FieldConfig; import org.apache.pinot.spi.config.table.SegmentPartitionConfig; @@ -85,7 +86,16 @@ public PartitionFunction getPartitionFunction(String column) { if (_segmentPartitionConfig != null) { ColumnPartitionConfig columnPartitionConfig = _segmentPartitionConfig.getColumnPartitionConfig(column); if (columnPartitionConfig != null) { - return PartitionFunctionFactory.getPartitionFunction(columnPartitionConfig); + if (columnPartitionConfig.getFunctionExpr() != null) { + // For expression-mode, determine input type from the column schema so BYTES columns are hashed as raw + // bytes rather than hex-encoded strings. + FieldSpec fieldSpec = _schema.getFieldSpecFor(column); + PartitionValueType inputType = + fieldSpec != null && fieldSpec.getDataType().getStoredType() == FieldSpec.DataType.BYTES + ? PartitionValueType.BYTES : PartitionValueType.STRING; + return PartitionFunctionFactory.getPartitionFunction(column, columnPartitionConfig, inputType); + } + return PartitionFunctionFactory.getPartitionFunction(column, columnPartitionConfig); } } return null; @@ -108,8 +118,18 @@ public int getNumPartitions(String column) { } @Deprecated + @Nullable + public String getPartitionFunctionExpr(String column) { + return (_segmentPartitionConfig != null) ? _segmentPartitionConfig.getFunctionExpr(column) : null; + } + @Nullable public Map getPartitionFunctionConfig(String column) { return (_segmentPartitionConfig != null) ? _segmentPartitionConfig.getFunctionConfig(column) : null; } + + @Nullable + public String getPartitionIdNormalizer(String column) { + return (_segmentPartitionConfig != null) ? _segmentPartitionConfig.getPartitionIdNormalizer(column) : null; + } } diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/function/ExecutableFunctionEvaluator.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/function/ExecutableFunctionEvaluator.java new file mode 100644 index 000000000000..d692ef807dac --- /dev/null +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/function/ExecutableFunctionEvaluator.java @@ -0,0 +1,314 @@ +/** + * 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.spi.function; + +import com.google.common.base.Preconditions; +import java.util.Arrays; +import java.util.List; +import org.apache.pinot.spi.data.readers.GenericRow; + + +/** + * Shared runtime implementation for expression evaluators backed by an executable node tree. + * + *

The planner for a concrete evaluator is responsible for binding scalar functions and constructing the root + * {@link ExecutableNode}. This class owns the repeated row/value execution logic so ingestion and partition paths do + * not maintain separate node evaluators. + * + *

Thread-safety: Once constructed, an {@code ExecutableFunctionEvaluator} instance is safe to use + * concurrently from multiple threads. {@link FunctionNode} maintains per-thread argument scratch arrays via + * {@link ThreadLocal}, so concurrent invocations do not share mutable state. A single call chain, however, is + * not reentrant: do not invoke {@code evaluate} recursively on the same thread with the same node instance. + * Subclasses that extend or wrap this evaluator must preserve these guarantees. + */ +public class ExecutableFunctionEvaluator implements org.apache.pinot.spi.function.FunctionEvaluator { + /** + * Invokes a bound function with already-evaluated child arguments. + */ + public interface Invoker { + Object invoke(Object[] arguments); + } + + /** + * Executes one node within the evaluator tree. + */ + public interface ExecutableNode { + Object execute(GenericRow row); + + Object execute(Object[] values); + } + + private final ExecutableNode _rootNode; + private final List _arguments; + private final String _expression; + + public ExecutableFunctionEvaluator(ExecutableNode rootNode, List arguments, String expression) { + _rootNode = Preconditions.checkNotNull(rootNode, "Executable root node must be configured"); + _arguments = List.copyOf(Preconditions.checkNotNull(arguments, "Evaluator arguments must be configured")); + _expression = Preconditions.checkNotNull(expression, "Evaluator expression must be configured"); + } + + @Override + public List getArguments() { + return _arguments; + } + + @Override + public Object evaluate(GenericRow genericRow) { + return _rootNode.execute(genericRow); + } + + @Override + public Object evaluate(Object[] values) { + return _rootNode.execute(values); + } + + @Override + public String toString() { + return _expression; + } + + /** + * Returns a constant value. + */ + public static class ConstantNode implements ExecutableNode { + private final Object _value; + + public ConstantNode(Object value) { + _value = value; + } + + @Override + public Object execute(GenericRow row) { + return _value; + } + + @Override + public Object execute(Object[] values) { + return _value; + } + + @Override + public String toString() { + return String.format("'%s'", _value); + } + } + + /** + * Returns a constant array value. + */ + public static class ArrayConstantNode implements ExecutableNode { + private final Object[] _value; + + public ArrayConstantNode(Object[] value) { + _value = value; + } + + @Override + public Object[] execute(GenericRow row) { + return _value; + } + + @Override + public Object[] execute(Object[] values) { + return _value; + } + + @Override + public String toString() { + return String.format("'%s'", Arrays.toString(_value)); + } + } + + /** + * Reads one referenced argument from a row or positional argument array. + */ + public static class ColumnNode implements ExecutableNode { + private final String _column; + private final int _id; + + public ColumnNode(String column, int id) { + _column = column; + _id = id; + } + + @Override + public Object execute(GenericRow row) { + return row.getValue(_column); + } + + @Override + public Object execute(Object[] values) { + return values[_id]; + } + + @Override + public String toString() { + return _column; + } + } + + /** + * Executes a bound function after evaluating all child arguments. + */ + public static class FunctionNode implements ExecutableNode { + private final String _functionName; + private final Invoker _invoker; + private final ExecutableNode[] _argumentNodes; + private final ThreadLocal _arguments; + + public FunctionNode(String functionName, Invoker invoker, ExecutableNode[] argumentNodes) { + _functionName = functionName; + _invoker = invoker; + _argumentNodes = argumentNodes; + _arguments = ThreadLocal.withInitial(() -> new Object[argumentNodes.length]); + } + + @Override + public Object execute(GenericRow row) { + Object[] arguments = _arguments.get(); + for (int i = 0; i < _argumentNodes.length; i++) { + arguments[i] = _argumentNodes[i].execute(row); + } + return _invoker.invoke(arguments); + } + + @Override + public Object execute(Object[] values) { + Object[] arguments = _arguments.get(); + for (int i = 0; i < _argumentNodes.length; i++) { + arguments[i] = _argumentNodes[i].execute(values); + } + return _invoker.invoke(arguments); + } + + @Override + public String toString() { + StringBuilder builder = new StringBuilder(_functionName).append('('); + for (int i = 0; i < _argumentNodes.length; i++) { + if (i > 0) { + builder.append(','); + } + builder.append(_argumentNodes[i]); + } + return builder.append(')').toString(); + } + } + + /** + * Three-valued logical NOT. + */ + public static class NotNode implements ExecutableNode { + private final ExecutableNode _argumentNode; + + public NotNode(ExecutableNode argumentNode) { + _argumentNode = argumentNode; + } + + @Override + public Object execute(GenericRow row) { + Boolean result = (Boolean) _argumentNode.execute(row); + return result != null ? !result : null; + } + + @Override + public Object execute(Object[] values) { + Boolean result = (Boolean) _argumentNode.execute(values); + return result != null ? !result : null; + } + } + + /** + * Three-valued logical OR with short-circuit evaluation. + */ + public static class OrNode implements ExecutableNode { + private final ExecutableNode[] _argumentNodes; + + public OrNode(ExecutableNode[] argumentNodes) { + _argumentNodes = argumentNodes; + } + + @Override + public Object execute(GenericRow row) { + boolean hasNull = false; + for (ExecutableNode argumentNode : _argumentNodes) { + Boolean result = (Boolean) argumentNode.execute(row); + if (result == null) { + hasNull = true; + } else if (result) { + return true; + } + } + return hasNull ? null : false; + } + + @Override + public Object execute(Object[] values) { + boolean hasNull = false; + for (ExecutableNode argumentNode : _argumentNodes) { + Boolean result = (Boolean) argumentNode.execute(values); + if (result == null) { + hasNull = true; + } else if (result) { + return true; + } + } + return hasNull ? null : false; + } + } + + /** + * Three-valued logical AND with short-circuit evaluation. + */ + public static class AndNode implements ExecutableNode { + private final ExecutableNode[] _argumentNodes; + + public AndNode(ExecutableNode[] argumentNodes) { + _argumentNodes = argumentNodes; + } + + @Override + public Object execute(GenericRow row) { + boolean hasNull = false; + for (ExecutableNode argumentNode : _argumentNodes) { + Boolean result = (Boolean) argumentNode.execute(row); + if (result == null) { + hasNull = true; + } else if (!result) { + return false; + } + } + return hasNull ? null : true; + } + + @Override + public Object execute(Object[] values) { + boolean hasNull = false; + for (ExecutableNode argumentNode : _argumentNodes) { + Boolean result = (Boolean) argumentNode.execute(values); + if (result == null) { + hasNull = true; + } else if (!result) { + return false; + } + } + return hasNull ? null : true; + } + } +} diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/metadata/ColumnMetadataImpl.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/metadata/ColumnMetadataImpl.java index 8ce7d53f8209..591e68c96bcb 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/metadata/ColumnMetadataImpl.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/metadata/ColumnMetadataImpl.java @@ -38,6 +38,8 @@ import org.apache.pinot.segment.spi.partition.PartitionFunction; import org.apache.pinot.segment.spi.partition.PartitionFunctionFactory; import org.apache.pinot.segment.spi.partition.metadata.ColumnPartitionMetadata; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionFunctionExprCompiler; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionValueType; import org.apache.pinot.spi.config.table.FieldConfig.EncodingType; import org.apache.pinot.spi.data.ComplexFieldSpec; import org.apache.pinot.spi.data.DateTimeFieldSpec; @@ -338,7 +340,9 @@ public static ColumnMetadataImpl fromPropertiesConfiguration(PropertiesConfigura // Set partition function String partitionFunctionName = config.getString(Column.getKeyFor(column, Column.PARTITION_FUNCTION), null); - if (partitionFunctionName != null) { + String partitionFunctionExpr = config.getString(Column.getKeyFor(column, Column.PARTITION_FUNCTION_EXPR), null); + String partitionIdNormalizer = config.getString(Column.getKeyFor(column, Column.PARTITION_ID_NORMALIZER), null); + if (partitionFunctionName != null || partitionFunctionExpr != null) { int numPartitions = config.getInt(Column.getKeyFor(column, Column.NUM_PARTITIONS)); Configuration partitionFunctionConfig = config.subset(Column.getKeyFor(column, Column.PARTITION_FUNCTION_CONFIG)); Map partitionFunctionConfigMap; @@ -356,9 +360,20 @@ public static ColumnMetadataImpl fromPropertiesConfiguration(PropertiesConfigura } else { partitionFunctionConfigMap = null; } - PartitionFunction partitionFunction = - PartitionFunctionFactory.getPartitionFunction(partitionFunctionName, numPartitions, - partitionFunctionConfigMap); + // For segment metadata, use the raw (non-column-bound) function: FunctionEvaluator interface is not + // needed here (only required by the ingestion path). Expression-mode needs the column for the pipeline. + PartitionFunction partitionFunction; + if (partitionFunctionExpr != null) { + // Pass BYTES input type for BYTES columns so functions receive raw bytes directly. + PartitionValueType inputType = + storedType == DataType.BYTES ? PartitionValueType.BYTES : PartitionValueType.STRING; + partitionFunction = PartitionFunctionExprCompiler.compilePartitionFunction(column, inputType, + partitionFunctionExpr, numPartitions, partitionIdNormalizer); + } else { + partitionFunction = + PartitionFunctionFactory.getPartitionFunction(partitionFunctionName, numPartitions, + partitionFunctionConfigMap); + } builder.setPartitionFunction(partitionFunction); builder.setPartitions( ColumnPartitionMetadata.extractPartitions(config.getList(Column.getKeyFor(column, Column.PARTITION_VALUES)))); diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/ByteArrayPartitionFunction.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/ByteArrayPartitionFunction.java index e8ff8edc2457..e1b8e98d1c33 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/ByteArrayPartitionFunction.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/ByteArrayPartitionFunction.java @@ -20,6 +20,7 @@ import com.google.common.base.Preconditions; import java.util.Arrays; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionIntNormalizer; import static java.nio.charset.StandardCharsets.UTF_8; @@ -56,6 +57,11 @@ public int getNumPartitions() { return _numPartitions; } + @Override + public String getPartitionIdNormalizer() { + return PartitionIntNormalizer.ABS.name(); + } + // Keep it for backward-compatibility, use getName() instead @Override public String toString() { diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/FnvPartitionFunction.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/FnvPartitionFunction.java index 1f96a2ba6451..0ac35091cb35 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/FnvPartitionFunction.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/FnvPartitionFunction.java @@ -140,6 +140,11 @@ public Map getFunctionConfig() { return _functionConfig; } + @Override + public String getPartitionIdNormalizer() { + return _negativePartitionHandling.name(); + } + // Keep it for backward-compatibility, use getName() instead @Override public String toString() { diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/HashCodePartitionFunction.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/HashCodePartitionFunction.java index 182760cf44c9..f8172359dc65 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/HashCodePartitionFunction.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/HashCodePartitionFunction.java @@ -19,6 +19,7 @@ package org.apache.pinot.segment.spi.partition; import com.google.common.base.Preconditions; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionIntNormalizer; /** @@ -51,6 +52,11 @@ public int getNumPartitions() { return _numPartitions; } + @Override + public String getPartitionIdNormalizer() { + return PartitionIntNormalizer.ABS.name(); + } + // Keep it for backward-compatibility, use getName() instead @Override public String toString() { diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/ModuloPartitionFunction.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/ModuloPartitionFunction.java index a4b8eb49abc0..c9ac8beb62fb 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/ModuloPartitionFunction.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/ModuloPartitionFunction.java @@ -19,6 +19,7 @@ package org.apache.pinot.segment.spi.partition; import com.google.common.base.Preconditions; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionIntNormalizer; /** @@ -63,6 +64,11 @@ public int getNumPartitions() { return _numPartitions; } + @Override + public String getPartitionIdNormalizer() { + return PartitionIntNormalizer.POSITIVE_MODULO.name(); + } + // Keep it for backward-compatibility, use getName() instead @Override public String toString() { diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/Murmur3PartitionFunction.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/Murmur3PartitionFunction.java index 9a0110f810ec..1106967137eb 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/Murmur3PartitionFunction.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/Murmur3PartitionFunction.java @@ -23,6 +23,7 @@ import java.util.Map; import javax.annotation.Nullable; import org.apache.commons.lang3.StringUtils; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionIntNormalizer; import org.apache.pinot.spi.utils.BytesUtils; import org.apache.pinot.spi.utils.hash.MurmurHashFunctions; @@ -107,6 +108,11 @@ public Map getFunctionConfig() { return _functionConfig; } + @Override + public String getPartitionIdNormalizer() { + return PartitionIntNormalizer.MASK.name(); + } + // Keep it for backward-compatibility, use getName() instead @Override public String toString() { diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/MurmurPartitionFunction.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/MurmurPartitionFunction.java index 6c64dee763a4..c1d0e35a6a79 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/MurmurPartitionFunction.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/MurmurPartitionFunction.java @@ -22,6 +22,7 @@ import java.util.Collections; import java.util.Map; import javax.annotation.Nullable; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionIntNormalizer; import org.apache.pinot.spi.utils.BytesUtils; import org.apache.pinot.spi.utils.hash.MurmurHashFunctions; @@ -81,6 +82,11 @@ public Map getFunctionConfig() { return _functionConfig; } + @Override + public String getPartitionIdNormalizer() { + return PartitionIntNormalizer.MASK.name(); + } + // Keep it for backward-compatibility, use getName() instead @Override public String toString() { diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunction.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunction.java index 3e6d26146d64..fafc5eaf46bd 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunction.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunction.java @@ -18,15 +18,17 @@ */ package org.apache.pinot.segment.spi.partition; +import com.fasterxml.jackson.annotation.JsonIgnore; import java.io.Serializable; import java.util.Map; import javax.annotation.Nullable; +import org.apache.pinot.spi.utils.BytesUtils; /** * Interface for partition function. * - * Implementations of this interface are assumed not to be stateful. + *

Implementations of this interface are assumed not to be stateful. * That is, two invocations of {@code PartitionFunction.getPartition(value)} * with the same value are expected to produce the same result. */ @@ -41,6 +43,21 @@ public interface PartitionFunction extends Serializable { */ int getPartition(String value); + /** + * Returns the partition id for a raw byte array value. + * + *

The default implementation converts the bytes to a hex string and delegates to {@link #getPartition(String)}, + * matching the historical behaviour for BYTES columns. Expression-mode pipelines that were compiled with + * {@link org.apache.pinot.segment.spi.partition.pipeline.PartitionValueType#BYTES} input type override this method + * to hash the raw bytes directly without the hex-encoding round-trip. + * + * @param bytes Raw byte array value. + * @return partition id for the value. + */ + default int getPartition(byte[] bytes) { + return getPartition(BytesUtils.toHexString(bytes)); + } + /** * Returns the name of the partition function. * @return Name of the partition function. @@ -57,4 +74,22 @@ public interface PartitionFunction extends Serializable { default Map getFunctionConfig() { return null; } + + @JsonIgnore + @Nullable + default String getPartitionColumn() { + return null; + } + + @JsonIgnore + @Nullable + default String getFunctionExpr() { + return null; + } + + @JsonIgnore + @Nullable + default String getPartitionIdNormalizer() { + return null; + } } diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java index 43890a614c16..5a83db427ac1 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java @@ -18,11 +18,19 @@ */ package org.apache.pinot.segment.spi.partition; +import com.google.common.base.Preconditions; +import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import javax.annotation.Nullable; import org.apache.pinot.segment.spi.partition.metadata.ColumnPartitionMetadata; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionFunctionExprCompiler; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionValueType; import org.apache.pinot.spi.config.table.ColumnPartitionConfig; +import org.apache.pinot.spi.data.FieldSpec; +import org.apache.pinot.spi.data.readers.GenericRow; +import org.apache.pinot.spi.function.FunctionEvaluator; /** @@ -101,11 +109,182 @@ public static PartitionFunction getPartitionFunction(String functionName, int nu } } + /** + * Returns the legacy (name-mode) partition function for the given config. + * + * @throws IllegalArgumentException if {@code config.getFunctionExpr()} is non-null; use + * {@link #getPartitionFunction(String, ColumnPartitionConfig)} instead for + * expression-mode configs. + */ public static PartitionFunction getPartitionFunction(ColumnPartitionConfig config) { + Preconditions.checkNotNull(config, "Column partition config must be configured"); + Preconditions.checkArgument(config.getFunctionExpr() == null, + "Expression-mode config requires a column name; use getPartitionFunction(String, ColumnPartitionConfig)"); return getPartitionFunction(config.getFunctionName(), config.getNumPartitions(), config.getFunctionConfig()); } + /** + * Returns the legacy (name-mode) partition function for the given segment metadata. + * + * @throws IllegalArgumentException if {@code metadata.getFunctionExpr()} is non-null; use + * {@link #getPartitionFunction(String, ColumnPartitionMetadata)} instead for + * expression-mode metadata. + */ public static PartitionFunction getPartitionFunction(ColumnPartitionMetadata metadata) { + Preconditions.checkNotNull(metadata, "Column partition metadata must be configured"); + Preconditions.checkArgument(metadata.getFunctionExpr() == null, + "Expression-mode metadata requires a column name; use getPartitionFunction(String, ColumnPartitionMetadata)"); return getPartitionFunction(metadata.getFunctionName(), metadata.getNumPartitions(), metadata.getFunctionConfig()); } + + public static PartitionFunction getPartitionFunction(String columnName, ColumnPartitionMetadata metadata) { + Preconditions.checkNotNull(metadata, "Column partition metadata must be configured"); + if (metadata.getFunctionExpr() != null && metadata.getInputType() != null) { + PartitionValueType inputType = PartitionValueType.valueOf(metadata.getInputType()); + return PartitionFunctionExprCompiler.compilePartitionFunction(columnName, inputType, metadata.getFunctionExpr(), + metadata.getNumPartitions(), metadata.getPartitionIdNormalizer()); + } + return getPartitionFunction(columnName, metadata.getFunctionName(), metadata.getNumPartitions(), + metadata.getFunctionConfig(), metadata.getFunctionExpr(), metadata.getPartitionIdNormalizer()); + } + + public static PartitionFunction getPartitionFunction(String columnName, ColumnPartitionConfig columnPartitionConfig) { + return getPartitionFunction(columnName, columnPartitionConfig, columnPartitionConfig.getNumPartitions()); + } + + public static PartitionFunction getPartitionFunction(String columnName, ColumnPartitionConfig columnPartitionConfig, + int numPartitions) { + Preconditions.checkNotNull(columnPartitionConfig, "Column partition config must be configured"); + return getPartitionFunction(columnName, columnPartitionConfig.getFunctionName(), numPartitions, + columnPartitionConfig.getFunctionConfig(), columnPartitionConfig.getFunctionExpr(), + columnPartitionConfig.getPartitionIdNormalizer()); + } + + public static PartitionFunction getPartitionFunction(String columnName, @Nullable String functionName, + int numPartitions, @Nullable Map functionConfig, @Nullable String functionExpr) { + return getPartitionFunction(columnName, functionName, numPartitions, functionConfig, functionExpr, null); + } + + /** + * Builds a partition function for expression mode using an explicit input type. + * + *

Use {@link PartitionValueType#BYTES} when the partition column stores raw byte arrays so that functions in the + * expression receive the original bytes directly rather than a hex-encoded string representation. + */ + public static PartitionFunction getPartitionFunction(String columnName, ColumnPartitionConfig config, + PartitionValueType inputType) { + Preconditions.checkNotNull(config, "Column partition config must be configured"); + Preconditions.checkArgument(config.getFunctionExpr() != null, + "inputType overload is only valid for expression-mode configs (functionExpr must be set)"); + return PartitionFunctionExprCompiler.compilePartitionFunction(columnName, inputType, config.getFunctionExpr(), + config.getNumPartitions(), config.getPartitionIdNormalizer()); + } + + /** + * Builds a partition function using the schema field spec to determine the correct input type for expression-mode + * partition functions on BYTES-typed columns. + * + *

When {@code fieldSpec} is non-null and the stored type is {@link FieldSpec.DataType#BYTES}, the expression is + * compiled with {@link PartitionValueType#BYTES} input so that scalar functions receive raw byte arrays rather than + * hex-encoded strings. For all other cases the default STRING input type is used. + */ + public static PartitionFunction getPartitionFunction(String columnName, ColumnPartitionConfig config, + @Nullable FieldSpec fieldSpec) { + if (config.getFunctionExpr() != null && fieldSpec != null + && fieldSpec.getDataType().getStoredType() == FieldSpec.DataType.BYTES) { + return getPartitionFunction(columnName, config, PartitionValueType.BYTES); + } + return getPartitionFunction(columnName, config); + } + + public static PartitionFunction getPartitionFunction(String columnName, @Nullable String functionName, + int numPartitions, @Nullable Map functionConfig, @Nullable String functionExpr, + @Nullable String partitionIdNormalizer) { + if (functionExpr != null) { + return PartitionFunctionExprCompiler + .compilePartitionFunction(columnName, functionExpr, numPartitions, partitionIdNormalizer); + } + Preconditions.checkArgument(functionName != null, "Partition function name must be configured"); + PartitionFunction partitionFunction = getPartitionFunction(functionName, numPartitions, functionConfig); + if (partitionIdNormalizer != null) { + String effectivePartitionIdNormalizer = partitionFunction.getPartitionIdNormalizer(); + Preconditions.checkArgument(effectivePartitionIdNormalizer != null + && effectivePartitionIdNormalizer.equalsIgnoreCase(partitionIdNormalizer), + "'partitionIdNormalizer'=%s is incompatible with legacy partition function '%s'; expected '%s'", + partitionIdNormalizer, functionName, effectivePartitionIdNormalizer); + } + return new ColumnBoundPartitionFunction(columnName, partitionFunction); + } + + // PartitionFunction extends Serializable for historical reasons; partition functions are never + // Java-serialized in Pinot's runtime. Suppress the warning to avoid noise. + @SuppressWarnings("serial") + private static final class ColumnBoundPartitionFunction implements PartitionFunction, FunctionEvaluator { + private final String _columnName; + private final PartitionFunction _delegate; + + private ColumnBoundPartitionFunction(String columnName, PartitionFunction delegate) { + _columnName = Preconditions.checkNotNull(columnName, "Partition column must be configured"); + _delegate = Preconditions.checkNotNull(delegate, "Delegate partition function must be configured"); + } + + @Override + public int getPartition(String value) { + return _delegate.getPartition(value); + } + + @Override + public String getName() { + return _delegate.getName(); + } + + @Override + public int getNumPartitions() { + return _delegate.getNumPartitions(); + } + + @Override + public Map getFunctionConfig() { + return _delegate.getFunctionConfig(); + } + + @Override + public String getPartitionColumn() { + return _columnName; + } + + @Override + public String getFunctionExpr() { + return _delegate.getFunctionExpr(); + } + + @Override + public String getPartitionIdNormalizer() { + return _delegate.getPartitionIdNormalizer(); + } + + @Override + public List getArguments() { + return Collections.singletonList(_columnName); + } + + @Override + public Object evaluate(GenericRow genericRow) { + Object value = genericRow.getValue(_columnName); + return value != null ? getPartition(FieldSpec.getStringValue(value)) : null; + } + + @Override + public Object evaluate(Object[] values) { + Preconditions.checkArgument(values.length == 1, + "Partition function for column '%s' expects exactly 1 positional argument, got: %s", _columnName, + values.length); + return values[0] != null ? getPartition(FieldSpec.getStringValue(values[0])) : null; + } + + @Override + public String toString() { + return _delegate.toString(); + } + } } diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java index 2110a5e93bd9..e8504e67816d 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java @@ -34,6 +34,9 @@ import java.util.Set; import javax.annotation.Nullable; import org.apache.commons.lang3.StringUtils; +import org.apache.pinot.segment.spi.partition.PartitionFunction; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionPipelineFunction; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionValueType; import org.apache.pinot.spi.utils.JsonUtils; @@ -49,6 +52,15 @@ @JsonDeserialize(using = ColumnPartitionMetadata.ColumnPartitionMetadataDeserializer.class) public class ColumnPartitionMetadata { private final String _functionName; + private final String _functionExpr; + private final String _partitionIdNormalizer; + /** + * Pipeline input type for expression-mode partition functions. {@code null} means STRING (the default and legacy + * behavior). Currently only {@code "BYTES"} is stored for BYTES-typed partition columns so that the broker can + * compile the same BYTES-input pipeline and produce consistent partition assignments with ingestion. + */ + @Nullable + private final String _inputType; private final int _numPartitions; private final Map _functionConfig; private final Set _partitions; @@ -60,23 +72,78 @@ public class ColumnPartitionMetadata { * @param numPartitions Number of total partitions for this column * @param partitions Set of partitions the column contains * @param functionConfig Configuration required by partition function. + * @deprecated Use {@link #ColumnPartitionMetadata(PartitionFunction, Set)} instead, which derives all fields + * directly from the {@link PartitionFunction} contract and keeps them consistent. */ + @Deprecated public ColumnPartitionMetadata(String functionName, int numPartitions, Set partitions, @Nullable Map functionConfig) { _functionName = functionName; + _functionExpr = null; + _partitionIdNormalizer = null; + _inputType = null; + _numPartitions = numPartitions; + _partitions = partitions; + _functionConfig = functionConfig; + } + + private ColumnPartitionMetadata(@Nullable String functionName, int numPartitions, Set partitions, + @Nullable Map functionConfig, @Nullable String functionExpr, + @Nullable String partitionIdNormalizer, @Nullable String inputType) { + _functionName = functionName; + _functionExpr = normalizeOptionalText(functionExpr); + _partitionIdNormalizer = normalizeOptionalText(partitionIdNormalizer); + _inputType = normalizeOptionalText(inputType); _numPartitions = numPartitions; _partitions = partitions; _functionConfig = functionConfig; } + public ColumnPartitionMetadata(PartitionFunction partitionFunction, Set partitions) { + this( + // Expression-mode uses "FunctionExpr" as a stable sentinel for the function name field. + // This ensures old brokers (which call jsonMetadata.get("functionName").asText() without + // a null guard) receive a non-null string rather than NPE-ing on a missing JSON key. + // Old brokers fail gracefully with IllegalArgumentException ("No enum constant for: FunctionExpr") + // and fall back to querying all segments rather than corrupting routing state. + partitionFunction.getFunctionExpr() != null ? PartitionPipelineFunction.NAME + : partitionFunction.getName(), + partitionFunction.getNumPartitions(), partitions, partitionFunction.getFunctionConfig(), + partitionFunction.getFunctionExpr(), partitionFunction.getPartitionIdNormalizer(), + // Store the pipeline input type only when it is non-default (BYTES) to avoid bloating the metadata. + partitionFunction instanceof PartitionPipelineFunction + && ((PartitionPipelineFunction) partitionFunction).getPartitionPipeline().getInputType() + == PartitionValueType.BYTES ? PartitionValueType.BYTES.name() : null); + } + + @Nullable public String getFunctionName() { return _functionName; } + /** + * Returns the pipeline input type for expression-mode partition functions, or {@code null} if the default + * ({@link PartitionValueType#STRING}) input type applies. + */ + @Nullable + public String getInputType() { + return _inputType; + } + public int getNumPartitions() { return _numPartitions; } + @Nullable + public String getFunctionExpr() { + return _functionExpr; + } + + @Nullable + public String getPartitionIdNormalizer() { + return _partitionIdNormalizer; + } + public Set getPartitions() { return _partitions; } @@ -92,16 +159,18 @@ public boolean equals(Object obj) { } if (obj instanceof ColumnPartitionMetadata) { ColumnPartitionMetadata that = (ColumnPartitionMetadata) obj; - return _functionName.equals(that._functionName) && _numPartitions == that._numPartitions && _partitions.equals( - that._partitions) && Objects.equals(_functionConfig, that._functionConfig); + return Objects.equals(_functionName, that._functionName) && _numPartitions == that._numPartitions + && _partitions.equals(that._partitions) && Objects.equals(_functionConfig, that._functionConfig) + && Objects.equals(_functionExpr, that._functionExpr) && Objects.equals(_partitionIdNormalizer, + that._partitionIdNormalizer) && Objects.equals(_inputType, that._inputType); } return false; } @Override public int hashCode() { - return 37 * 37 * _functionName.hashCode() + 37 * _numPartitions + _partitions.hashCode() - + Objects.hashCode(_functionConfig); + return Objects.hash(_functionName, _numPartitions, _functionConfig, _functionExpr, _partitionIdNormalizer, + _inputType) + 31 * _partitions.hashCode(); } /** @@ -140,6 +209,11 @@ private static void addRangeToPartitions(String rangeString, IntSet partitions) } } + @Nullable + private static String normalizeOptionalText(@Nullable String value) { + return StringUtils.isBlank(value) || "null".equals(value) ? null : value; + } + /** * Custom deserializer for {@link ColumnPartitionMetadata}. *

@@ -150,6 +224,9 @@ public static class ColumnPartitionMetadataDeserializer extends JsonDeserializer private static final String FUNCTION_NAME_KEY = "functionName"; private static final String NUM_PARTITIONS_KEY = "numPartitions"; private static final String FUNCTION_CONFIG_KEY = "functionConfig"; + private static final String FUNCTION_EXPR_KEY = "functionExpr"; + private static final String PARTITION_ID_NORMALIZER_KEY = "partitionIdNormalizer"; + private static final String INPUT_TYPE_KEY = "inputType"; private static final String PARTITIONS_KEY = "partitions"; // DO NOT CHANGE: for backward-compatibility @@ -181,8 +258,20 @@ public ColumnPartitionMetadata deserialize(JsonParser p, DeserializationContext functionConfig = JsonUtils.jsonNodeToObject(jsonMetadata.get(FUNCTION_CONFIG_KEY), new TypeReference<>() { }); } - return new ColumnPartitionMetadata(jsonMetadata.get(FUNCTION_NAME_KEY).asText(), - jsonMetadata.get(NUM_PARTITIONS_KEY).asInt(), partitions, functionConfig); + + JsonNode functionNameNode = jsonMetadata.get(FUNCTION_NAME_KEY); + JsonNode functionExprNode = jsonMetadata.get(FUNCTION_EXPR_KEY); + JsonNode partitionIdNormalizerNode = jsonMetadata.get(PARTITION_ID_NORMALIZER_KEY); + JsonNode inputTypeNode = jsonMetadata.get(INPUT_TYPE_KEY); + return new ColumnPartitionMetadata(readOptionalText(functionNameNode), + jsonMetadata.get(NUM_PARTITIONS_KEY).asInt(), partitions, functionConfig, + readOptionalText(functionExprNode), readOptionalText(partitionIdNormalizerNode), + readOptionalText(inputTypeNode)); + } + + @Nullable + private static String readOptionalText(@Nullable JsonNode node) { + return node != null && !node.isNull() ? normalizeOptionalText(node.asText()) : null; } } } diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java new file mode 100644 index 000000000000..667a4da011b1 --- /dev/null +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java @@ -0,0 +1,459 @@ +/** + * 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.spi.partition.pipeline; + +import com.google.common.base.Preconditions; +import java.util.ArrayList; +import java.util.List; +import java.util.Locale; +import java.util.ServiceLoader; +import javax.annotation.Nullable; +import org.apache.pinot.segment.spi.function.ExecutableFunctionEvaluator; +import org.apache.pinot.spi.config.table.ColumnPartitionConfig; + + +/** + * Compiles a restricted partition-function expression into a typed {@link PartitionPipeline} backed by deterministic + * scalar functions. + */ +public final class PartitionFunctionExprCompiler { + private static final int MAX_EXPRESSION_LENGTH = 256; + private static final int MAX_DEPTH = 8; + private static final int MAX_ARGUMENTS = 32; + + private PartitionFunctionExprCompiler() { + } + + // Lazy holder: initialized on first use so that class-load of this utility does not fail when + // pinot-common (which provides CommonPartitionScalarFunctionResolver) is not yet on the classpath. + // The JVM class-initialization guarantee makes this inherently thread-safe and single-init. + private static final class FunctionResolverHolder { + static final PartitionScalarFunctionResolver INSTANCE = loadFunctionResolver(); + } + + public static PartitionPipeline compile(String rawColumn, String functionExpr) { + return compile(rawColumn, PartitionValueType.STRING, functionExpr, PartitionIntNormalizer.POSITIVE_MODULO); + } + + public static PartitionPipeline compile(String rawColumn, String functionExpr, + @Nullable PartitionIntNormalizer partitionIdNormalizer) { + return compile(rawColumn, PartitionValueType.STRING, functionExpr, partitionIdNormalizer); + } + + /** + * Compiles a partition pipeline with an explicit input type. + * + *

Use {@link PartitionValueType#BYTES} when the partition column stores raw byte arrays so that functions in the + * expression receive the original bytes directly rather than a hex-encoded string representation. + */ + public static PartitionPipeline compile(String rawColumn, PartitionValueType inputType, String functionExpr, + @Nullable PartitionIntNormalizer partitionIdNormalizer) { + Preconditions.checkArgument(hasText(rawColumn), "Raw column must be configured"); + Preconditions.checkArgument(inputType != null, "Input type must be configured"); + Preconditions.checkArgument(hasText(functionExpr), "'functionExpr' must be configured"); + Preconditions.checkArgument(functionExpr.length() <= MAX_EXPRESSION_LENGTH, + "'functionExpr' must be <= %s characters", MAX_EXPRESSION_LENGTH); + + Parser parser = new Parser(functionExpr); + Node root = parser.parse(); + List steps = new ArrayList<>(); + CompileResult result = compile(rawColumn, inputType, root, steps, 0); + Preconditions.checkArgument(result._dynamic, + "Partition function expression must reference partition column '%s'", rawColumn); + PartitionIntNormalizer effectivePartitionIdNormalizer = + result._outputType.isIntegral() ? partitionIdNormalizer : null; + return new PartitionPipeline(rawColumn, inputType, result._outputType, result._canonicalExpr, + effectivePartitionIdNormalizer, steps, result._rootNode); + } + + public static PartitionPipelineFunction compilePartitionFunction(String rawColumn, String functionExpr, + int numPartitions) { + return compilePartitionFunction(rawColumn, PartitionValueType.STRING, functionExpr, numPartitions, null); + } + + public static PartitionPipelineFunction compilePartitionFunction(String rawColumn, String functionExpr, + int numPartitions, @Nullable String partitionIdNormalizer) { + return compilePartitionFunction(rawColumn, PartitionValueType.STRING, functionExpr, numPartitions, + partitionIdNormalizer); + } + + /** + * Compiles a partition pipeline function with an explicit input type. + * + *

Use {@link PartitionValueType#BYTES} when the partition column stores raw byte arrays so that functions in the + * expression receive the original bytes directly rather than a hex-encoded string representation. + */ + public static PartitionPipelineFunction compilePartitionFunction(String rawColumn, PartitionValueType inputType, + String functionExpr, int numPartitions, @Nullable String partitionIdNormalizer) { + PartitionIntNormalizer configuredPartitionIdNormalizer = partitionIdNormalizer != null + ? PartitionIntNormalizer.fromConfigString(partitionIdNormalizer) + : PartitionIntNormalizer.fromConfigString(ColumnPartitionConfig.PARTITION_ID_NORMALIZER_POSITIVE_MODULO); + return new PartitionPipelineFunction(compile(rawColumn, inputType, functionExpr, configuredPartitionIdNormalizer), + numPartitions); + } + + private static CompileResult compile(String rawColumn, PartitionValueType inputType, Node node, + List steps, int depth) { + Preconditions.checkArgument(depth <= MAX_DEPTH, + "Partition function expression depth exceeds the maximum of %s", MAX_DEPTH); + if (node instanceof IdentifierNode) { + IdentifierNode identifierNode = (IdentifierNode) node; + Preconditions.checkArgument(identifierNode._name.equals(rawColumn), + "Partition function expression must reference only partition column '%s', got '%s'", + rawColumn, identifierNode._name); + return CompileResult.dynamic(inputType, rawColumn, new ExecutableFunctionEvaluator.ColumnNode(rawColumn, 0)); + } + if (node instanceof LiteralNode) { + LiteralNode literalNode = (LiteralNode) node; + return CompileResult.literal(literalNode._value.getType(), literalNode._canonicalForm, literalNode._value, + new ExecutableFunctionEvaluator.ConstantNode(literalNode._value.toObject())); + } + + FunctionNode functionNode = (FunctionNode) node; + List arguments = new ArrayList<>(functionNode._arguments.size()); + int dynamicCount = 0; + int dynamicIndex = -1; + for (int i = 0; i < functionNode._arguments.size(); i++) { + CompileResult argument = compile(rawColumn, inputType, functionNode._arguments.get(i), steps, depth + 1); + Preconditions.checkArgument(argument._dynamic || argument._literalConstant, + "Partition function expression only supports literal constants, got function subexpression: %s", + argument._canonicalExpr); + arguments.add(argument); + if (argument._dynamic) { + dynamicCount++; + dynamicIndex = i; + } + } + Preconditions.checkArgument(dynamicCount <= 1, + "Partition function expression must reference partition column '%s' through a single argument chain", + rawColumn); + + String displayName = functionNode._name.toLowerCase(Locale.ROOT); + PartitionScalarFunctionResolver.ResolvedFunction resolvedFunction = + getFunctionResolver().resolve(displayName, toResolverArguments(arguments)); + String canonicalExpr = toCanonicalExpr(displayName, arguments); + if (!resolvedFunction.isDynamic()) { + PartitionValue constantValue = resolvedFunction.invoke(null); + return CompileResult.constant(resolvedFunction.getOutputType(), canonicalExpr, constantValue, + new ExecutableFunctionEvaluator.ConstantNode(constantValue.toObject())); + } + + steps.add(new PartitionStep(displayName, arguments.get(dynamicIndex)._outputType, resolvedFunction.getOutputType(), + resolvedFunction::invoke)); + ExecutableFunctionEvaluator.ExecutableNode[] dynamicArgumentNodes = dynamicCount == 1 + ? new ExecutableFunctionEvaluator.ExecutableNode[]{arguments.get(dynamicIndex)._rootNode} + : new ExecutableFunctionEvaluator.ExecutableNode[0]; + return CompileResult.dynamic(resolvedFunction.getOutputType(), canonicalExpr, + new ExecutableFunctionEvaluator.FunctionNode(displayName, new PartitionBoundFunctionInvoker(resolvedFunction), + dynamicArgumentNodes)); + } + + private static List toResolverArguments(List arguments) { + List resolverArguments = new ArrayList<>(arguments.size()); + for (CompileResult argument : arguments) { + resolverArguments.add(argument._dynamic ? PartitionScalarFunctionResolver.Argument.dynamic(argument._outputType) + : PartitionScalarFunctionResolver.Argument.constant( + Preconditions.checkNotNull(argument._constantValue, "Constant argument must be configured"))); + } + return resolverArguments; + } + + private static PartitionScalarFunctionResolver getFunctionResolver() { + return FunctionResolverHolder.INSTANCE; + } + + private static PartitionScalarFunctionResolver loadFunctionResolver() { + List resolvers = new ArrayList<>(); + for (PartitionScalarFunctionResolver r : ServiceLoader.load(PartitionScalarFunctionResolver.class)) { + resolvers.add(r); + } + Preconditions.checkState(!resolvers.isEmpty(), + "No PartitionScalarFunctionResolver implementation found on the classpath"); + Preconditions.checkState(resolvers.size() == 1, + "Expected exactly 1 PartitionScalarFunctionResolver implementation but found %s: %s", resolvers.size(), + resolvers); + return resolvers.get(0); + } + + private static String toCanonicalExpr(String functionName, List arguments) { + StringBuilder builder = new StringBuilder(functionName).append('('); + for (int i = 0; i < arguments.size(); i++) { + if (i > 0) { + builder.append(", "); + } + builder.append(arguments.get(i)._canonicalExpr); + } + return builder.append(')').toString(); + } + + private static boolean hasText(@Nullable String value) { + return value != null && !value.trim().isEmpty(); + } + + private abstract static class Node { + } + + private static final class IdentifierNode extends Node { + private final String _name; + + private IdentifierNode(String name) { + _name = name; + } + } + + private static final class LiteralNode extends Node { + private final PartitionValue _value; + private final String _canonicalForm; + + private LiteralNode(PartitionValue value, String canonicalForm) { + _value = value; + _canonicalForm = canonicalForm; + } + } + + private static final class FunctionNode extends Node { + private final String _name; + private final List _arguments; + + private FunctionNode(String name, List arguments) { + _name = name; + _arguments = arguments; + } + } + + private static final class CompileResult { + private final PartitionValueType _outputType; + private final String _canonicalExpr; + private final boolean _dynamic; + private final boolean _literalConstant; + private final ExecutableFunctionEvaluator.ExecutableNode _rootNode; + @Nullable + private final PartitionValue _constantValue; + + private CompileResult(PartitionValueType outputType, String canonicalExpr, boolean dynamic, boolean literalConstant, + @Nullable PartitionValue constantValue, ExecutableFunctionEvaluator.ExecutableNode rootNode) { + _outputType = outputType; + _canonicalExpr = canonicalExpr; + _dynamic = dynamic; + _literalConstant = literalConstant; + _constantValue = constantValue; + _rootNode = rootNode; + } + + private static CompileResult dynamic(PartitionValueType outputType, String canonicalExpr, + ExecutableFunctionEvaluator.ExecutableNode rootNode) { + return new CompileResult(outputType, canonicalExpr, true, false, null, rootNode); + } + + private static CompileResult literal(PartitionValueType outputType, String canonicalExpr, PartitionValue value, + ExecutableFunctionEvaluator.ExecutableNode rootNode) { + return new CompileResult(outputType, canonicalExpr, false, true, value, rootNode); + } + + private static CompileResult constant(PartitionValueType outputType, String canonicalExpr, PartitionValue value, + ExecutableFunctionEvaluator.ExecutableNode rootNode) { + return new CompileResult(outputType, canonicalExpr, false, false, value, rootNode); + } + } + + /** + * Bridges a {@link PartitionScalarFunctionResolver.ResolvedFunction} to the + * {@link ExecutableFunctionEvaluator.Invoker} contract required by {@link ExecutableFunctionEvaluator.FunctionNode}. + */ + private static final class PartitionBoundFunctionInvoker implements ExecutableFunctionEvaluator.Invoker { + private final PartitionScalarFunctionResolver.ResolvedFunction _resolvedFunction; + + private PartitionBoundFunctionInvoker(PartitionScalarFunctionResolver.ResolvedFunction resolvedFunction) { + _resolvedFunction = resolvedFunction; + } + + @Override + public Object invoke(Object[] arguments) { + Preconditions.checkState(arguments.length <= 1, + "Partition expression runtime expects at most one dynamic argument, got: %s", arguments.length); + PartitionValue dynamicInput = arguments.length == 0 ? null : PartitionValue.fromObject(arguments[0]); + return _resolvedFunction.invoke(dynamicInput).toObject(); + } + } + + private static final class Parser { + private final String _expression; + private int _index; + + private Parser(String expression) { + _expression = expression; + } + + private Node parse() { + skipWhitespace(); + Node node = parseTerm(); + skipWhitespace(); + if (_index != _expression.length()) { + throw error("Unexpected trailing content"); + } + return node; + } + + private Node parseTerm() { + skipWhitespace(); + if (_index >= _expression.length()) { + throw error("Unexpected end of expression"); + } + char character = _expression.charAt(_index); + if (character == '\'') { + return parseStringLiteral(); + } + if (isNumericStart(character)) { + return parseNumericLiteral(); + } + String identifier = parseIdentifier(); + skipWhitespace(); + if (_index < _expression.length() && _expression.charAt(_index) == '(') { + _index++; + skipWhitespace(); + List arguments = new ArrayList<>(); + if (_index < _expression.length() && _expression.charAt(_index) == ')') { + _index++; + return new FunctionNode(identifier, arguments); + } + while (true) { + arguments.add(parseTerm()); + Preconditions.checkArgument(arguments.size() <= MAX_ARGUMENTS, + "Partition function expression cannot have more than %s arguments", MAX_ARGUMENTS); + skipWhitespace(); + if (_index >= _expression.length()) { + throw error("Expected ')'"); + } + char delimiter = _expression.charAt(_index); + if (delimiter == ',') { + _index++; + skipWhitespace(); + continue; + } + if (delimiter == ')') { + _index++; + return new FunctionNode(identifier, arguments); + } + throw error("Expected ',' or ')'"); + } + } + return new IdentifierNode(identifier); + } + + private Node parseStringLiteral() { + _index++; + StringBuilder builder = new StringBuilder(); + while (_index < _expression.length()) { + char character = _expression.charAt(_index++); + if (character == '\'') { + if (_index < _expression.length() && _expression.charAt(_index) == '\'') { + builder.append('\''); + _index++; + continue; + } + return new LiteralNode(PartitionValue.stringValue(builder.toString()), quote(builder.toString())); + } + builder.append(character); + } + throw error("Unterminated string literal"); + } + + private Node parseNumericLiteral() { + int start = _index; + if (_expression.charAt(_index) == '+' || _expression.charAt(_index) == '-') { + _index++; + } + boolean sawDecimal = false; + while (_index < _expression.length()) { + char character = _expression.charAt(_index); + if (Character.isDigit(character)) { + _index++; + continue; + } + if (character == '.') { + sawDecimal = true; + _index++; + continue; + } + if (character == 'e' || character == 'E') { + sawDecimal = true; + _index++; + if (_index < _expression.length() + && (_expression.charAt(_index) == '+' || _expression.charAt(_index) == '-')) { + _index++; + } + continue; + } + break; + } + String token = _expression.substring(start, _index); + try { + if (sawDecimal) { + return new LiteralNode(PartitionValue.doubleValue(Double.parseDouble(token)), token); + } + return new LiteralNode(PartitionValue.longValue(Long.parseLong(token)), token); + } catch (NumberFormatException e) { + throw error("Invalid numeric literal: " + token); + } + } + + private String parseIdentifier() { + Preconditions.checkArgument(_index < _expression.length() && isIdentifierStart(_expression.charAt(_index)), + "Expected identifier at position %s", _index); + int start = _index++; + while (_index < _expression.length() && isIdentifierPart(_expression.charAt(_index))) { + _index++; + } + return _expression.substring(start, _index); + } + + private void skipWhitespace() { + while (_index < _expression.length() && Character.isWhitespace(_expression.charAt(_index))) { + _index++; + } + } + + private boolean isNumericStart(char character) { + if (Character.isDigit(character)) { + return true; + } + if ((character == '+' || character == '-') && _index + 1 < _expression.length()) { + return Character.isDigit(_expression.charAt(_index + 1)); + } + return false; + } + + private boolean isIdentifierStart(char character) { + return Character.isLetter(character) || character == '_'; + } + + private boolean isIdentifierPart(char character) { + return Character.isLetterOrDigit(character) || character == '_'; + } + + private IllegalArgumentException error(String message) { + return new IllegalArgumentException( + String.format("%s at position %s in partition function expression '%s'", message, _index, _expression)); + } + } + + private static String quote(String value) { + return '\'' + value.replace("'", "''") + '\''; + } +} diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionIntNormalizer.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionIntNormalizer.java new file mode 100644 index 000000000000..6840343ae0c1 --- /dev/null +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionIntNormalizer.java @@ -0,0 +1,99 @@ +/** + * 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.spi.partition.pipeline; + +import com.google.common.base.Preconditions; +import java.util.Locale; + + +/** + * Normalizes the final INT output from a compiled partition pipeline into a partition id. + */ +public enum PartitionIntNormalizer { + /** + * Computes the remainder first, then shifts negative partition ids into the valid range with {@code + numPartitions}. + */ + POSITIVE_MODULO { + @Override + int toPartitionId(int value, int numPartitions) { + int partition = value % numPartitions; + return partition < 0 ? partition + numPartitions : partition; + } + + @Override + int toPartitionId(long value, int numPartitions) { + long partition = value % numPartitions; + return (int) (partition < 0 ? partition + numPartitions : partition); + } + }, + /** + * Computes the remainder first, then takes the absolute value of the remainder. + */ + ABS { + @Override + int toPartitionId(int value, int numPartitions) { + int partition = value % numPartitions; + return partition < 0 ? -partition : partition; + } + + @Override + int toPartitionId(long value, int numPartitions) { + long partition = value % numPartitions; + return (int) (partition < 0 ? -partition : partition); + } + }, + /** + * Makes the raw INT/LONG output non-negative first by masking the sign bit, then applies modulo. + */ + MASK { + @Override + int toPartitionId(int value, int numPartitions) { + return (value & Integer.MAX_VALUE) % numPartitions; + } + + @Override + int toPartitionId(long value, int numPartitions) { + return (int) ((value & Long.MAX_VALUE) % numPartitions); + } + }; + + public final int getPartitionId(int value, int numPartitions) { + Preconditions.checkArgument(numPartitions > 0, "Number of partitions must be > 0"); + return toPartitionId(value, numPartitions); + } + + public final int getPartitionId(long value, int numPartitions) { + Preconditions.checkArgument(numPartitions > 0, "Number of partitions must be > 0"); + return toPartitionId(value, numPartitions); + } + + public static PartitionIntNormalizer fromConfigString(String partitionIdNormalizer) { + Preconditions.checkArgument(partitionIdNormalizer != null && !partitionIdNormalizer.trim().isEmpty(), + "'partitionIdNormalizer' must not be blank"); + try { + return valueOf(partitionIdNormalizer.trim().toUpperCase(Locale.ROOT)); + } catch (IllegalArgumentException e) { + throw new IllegalArgumentException("Unsupported partitionIdNormalizer: " + partitionIdNormalizer, e); + } + } + + abstract int toPartitionId(int value, int numPartitions); + + abstract int toPartitionId(long value, int numPartitions); +} diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipeline.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipeline.java new file mode 100644 index 000000000000..3d7d8f1f27c3 --- /dev/null +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipeline.java @@ -0,0 +1,126 @@ +/** + * 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.spi.partition.pipeline; + +import com.google.common.base.Preconditions; +import java.util.Collections; +import java.util.List; +import javax.annotation.Nullable; +import org.apache.pinot.segment.spi.function.ExecutableFunctionEvaluator; +import org.apache.pinot.spi.data.FieldSpec; +import org.apache.pinot.spi.data.readers.GenericRow; + + +/** + * Immutable compiled pipeline for one raw partition column. + */ +public final class PartitionPipeline extends ExecutableFunctionEvaluator { + private final String _rawColumn; + private final PartitionValueType _inputType; + private final PartitionValueType _outputType; + private final String _canonicalFunctionExpr; + @Nullable + private final PartitionIntNormalizer _intNormalizer; + private final List _steps; + + public PartitionPipeline(String rawColumn, PartitionValueType inputType, PartitionValueType outputType, + String canonicalFunctionExpr, @Nullable PartitionIntNormalizer intNormalizer, List steps, + ExecutableNode rootNode) { + // super() guards rootNode (non-null) and canonicalFunctionExpr (non-null); rawColumn null is caught below + super(rootNode, Collections.singletonList(rawColumn), canonicalFunctionExpr); + Preconditions.checkNotNull(rawColumn, "Raw column must be configured"); + Preconditions.checkNotNull(inputType, "Input type must be configured"); + Preconditions.checkNotNull(outputType, "Output type must be configured"); + Preconditions.checkNotNull(steps, "Pipeline steps must be configured"); + Preconditions.checkArgument(!outputType.isIntegral() || intNormalizer != null, + "Integral-output pipelines must configure an INT normalizer"); + _rawColumn = rawColumn; + _inputType = inputType; + _outputType = outputType; + _canonicalFunctionExpr = canonicalFunctionExpr; + _intNormalizer = intNormalizer; + _steps = Collections.unmodifiableList(steps); + } + + public String getRawColumn() { + return _rawColumn; + } + + public PartitionValueType getInputType() { + return _inputType; + } + + public PartitionValueType getOutputType() { + return _outputType; + } + + public String getCanonicalFunctionExpr() { + return _canonicalFunctionExpr; + } + + @Nullable + public PartitionIntNormalizer getIntNormalizer() { + return _intNormalizer; + } + + public List getSteps() { + return _steps; + } + + @Override + public Object evaluate(GenericRow genericRow) { + Object inputValue = genericRow.getValue(_rawColumn); + if (inputValue == null) { + return null; + } + // Pass raw bytes directly for BYTES-input pipelines to avoid hex-encoding the payload. + if (inputValue instanceof byte[] && _inputType == PartitionValueType.BYTES) { + return super.evaluate(new Object[]{inputValue}); + } + return super.evaluate(new Object[]{FieldSpec.getStringValue(inputValue)}); + } + + @Override + public Object evaluate(Object[] values) { + Preconditions.checkArgument(values.length == 1, + "Partition pipeline for column '%s' expects exactly 1 positional argument, got: %s", _rawColumn, + values.length); + Object inputValue = values[0]; + if (inputValue == null) { + return null; + } + // Pass raw bytes directly for BYTES-input pipelines to avoid hex-encoding the payload. + if (inputValue instanceof byte[] && _inputType == PartitionValueType.BYTES) { + return super.evaluate(new Object[]{inputValue}); + } + return super.evaluate(new Object[]{FieldSpec.getStringValue(inputValue)}); + } + + public PartitionValue evaluate(String rawValue) { + Preconditions.checkState(_inputType == PartitionValueType.STRING, + "evaluate(String) is only supported for STRING-input pipelines"); + return PartitionValue.fromObject(super.evaluate(new Object[]{rawValue})); + } + + public PartitionValue evaluate(PartitionValue input) { + Preconditions.checkArgument(input.getType() == _inputType, + "Pipeline for column '%s' expects %s input but got %s", _rawColumn, _inputType, input.getType()); + return PartitionValue.fromObject(super.evaluate(new Object[]{input.toObject()})); + } +} diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java new file mode 100644 index 000000000000..db9740a5af58 --- /dev/null +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java @@ -0,0 +1,169 @@ +/** + * 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.spi.partition.pipeline; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import java.util.Collections; +import java.util.List; +import org.apache.pinot.segment.spi.partition.PartitionFunction; +import org.apache.pinot.spi.data.FieldSpec; +import org.apache.pinot.spi.data.readers.GenericRow; +import org.apache.pinot.spi.function.FunctionEvaluator; +import org.apache.pinot.spi.utils.BytesUtils; + + +/** + * {@link PartitionFunction} adapter for expression-mode partition pipelines. + * + *

Note on {@code Serializable}: {@link PartitionFunction} extends {@link java.io.Serializable} for + * historical reasons, but partition functions are never Java-serialized in Pinot's runtime. This class holds a + * {@link PartitionPipeline} that extends {@link org.apache.pinot.segment.spi.function.ExecutableFunctionEvaluator}, + * which contains {@link ThreadLocal} fields that are not serializable. Do not rely on Java serialization for this + * class. + */ +@SuppressWarnings("serial") +public class PartitionPipelineFunction implements PartitionFunction, FunctionEvaluator { + public static final String NAME = "FunctionExpr"; + + private final PartitionPipeline _pipeline; + private final int _numPartitions; + + public PartitionPipelineFunction(PartitionPipeline pipeline, int numPartitions) { + Preconditions.checkNotNull(pipeline, "Partition pipeline must be configured"); + Preconditions.checkArgument(numPartitions > 0, "Number of partitions must be > 0"); + Preconditions.checkArgument(pipeline.getOutputType().isIntegral(), + "Partition pipeline must produce INT or LONG output, got: %s", pipeline.getOutputType()); + _pipeline = pipeline; + _numPartitions = numPartitions; + } + + public PartitionPipeline getPartitionPipeline() { + return _pipeline; + } + + @Override + public int getPartition(String value) { + // BYTES-input pipelines expect raw bytes. When the caller provides a string (e.g. a hex-encoded predicate value + // from broker routing), convert the hex string back to raw bytes so the partition computation matches ingestion. + if (_pipeline.getInputType() == PartitionValueType.BYTES) { + return getPartition(BytesUtils.toBytes(value)); + } + PartitionValue partitionValue = _pipeline.evaluate(value); + PartitionIntNormalizer intNormalizer = _pipeline.getIntNormalizer(); + Preconditions.checkState(intNormalizer != null, "Integral-output partition pipeline must have an INT normalizer"); + if (partitionValue.getType() == PartitionValueType.INT) { + return intNormalizer.getPartitionId(partitionValue.getIntValue(), _numPartitions); + } + Preconditions.checkState(partitionValue.getType() == PartitionValueType.LONG, + "Expected INT or LONG partition value but got: %s", partitionValue.getType()); + return intNormalizer.getPartitionId(partitionValue.getLongValue(), _numPartitions); + } + + /** + * Overrides the default bytes partition to pass raw bytes directly through the pipeline when this pipeline was + * compiled with {@link PartitionValueType#BYTES} input type, avoiding the hex-encoding round-trip. + */ + @Override + public int getPartition(byte[] bytes) { + if (_pipeline.getInputType() != PartitionValueType.BYTES) { + return getPartition(BytesUtils.toHexString(bytes)); + } + PartitionValue partitionValue = _pipeline.evaluate(PartitionValue.bytesValue(bytes)); + PartitionIntNormalizer intNormalizer = _pipeline.getIntNormalizer(); + Preconditions.checkState(intNormalizer != null, "Integral-output partition pipeline must have an INT normalizer"); + if (partitionValue.getType() == PartitionValueType.INT) { + return intNormalizer.getPartitionId(partitionValue.getIntValue(), _numPartitions); + } + Preconditions.checkState(partitionValue.getType() == PartitionValueType.LONG, + "Expected INT or LONG partition value but got: %s", partitionValue.getType()); + return intNormalizer.getPartitionId(partitionValue.getLongValue(), _numPartitions); + } + + @Override + public String getName() { + return NAME; + } + + @Override + public int getNumPartitions() { + return _numPartitions; + } + + @Override + @JsonIgnore + public String getPartitionColumn() { + return _pipeline.getRawColumn(); + } + + @Override + @JsonIgnore(false) + @JsonProperty("functionExpr") + public String getFunctionExpr() { + return _pipeline.getCanonicalFunctionExpr(); + } + + @Override + @JsonIgnore(false) + @JsonProperty("partitionIdNormalizer") + public String getPartitionIdNormalizer() { + PartitionIntNormalizer intNormalizer = _pipeline.getIntNormalizer(); + return intNormalizer != null ? intNormalizer.name() : null; + } + + // FunctionEvaluator implementation + + @Override + public List getArguments() { + return Collections.singletonList(_pipeline.getRawColumn()); + } + + @Override + public Object evaluate(GenericRow genericRow) { + Object inputValue = genericRow.getValue(_pipeline.getRawColumn()); + if (inputValue == null) { + return null; + } + if (inputValue instanceof byte[] && _pipeline.getInputType() == PartitionValueType.BYTES) { + return getPartition((byte[]) inputValue); + } + return getPartition(FieldSpec.getStringValue(inputValue)); + } + + @Override + public Object evaluate(Object[] values) { + Preconditions.checkArgument(values.length == 1, + "Partition pipeline function for column '%s' expects exactly 1 positional argument, got: %s", + _pipeline.getRawColumn(), values.length); + Object inputValue = values[0]; + if (inputValue == null) { + return null; + } + if (inputValue instanceof byte[] && _pipeline.getInputType() == PartitionValueType.BYTES) { + return getPartition((byte[]) inputValue); + } + return getPartition(FieldSpec.getStringValue(inputValue)); + } + + @Override + public String toString() { + return NAME; + } +} diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionScalarFunctionResolver.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionScalarFunctionResolver.java new file mode 100644 index 000000000000..af108e062e4b --- /dev/null +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionScalarFunctionResolver.java @@ -0,0 +1,86 @@ +/** + * 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.spi.partition.pipeline; + +import com.google.common.base.Preconditions; +import java.util.List; +import javax.annotation.Nullable; + + +/** + * Resolves one partition-expression scalar function call into an executable binding. + * + *

The compiler owns DSL parsing and raw-column validation. Implementations of this interface own scalar-function + * discovery, overload resolution, determinism validation and invocation. + */ +public interface PartitionScalarFunctionResolver { + /** + * Resolve the scalar function call or throw an {@link IllegalArgumentException} when the call is invalid. + */ + ResolvedFunction resolve(String functionName, List arguments); + + /** + * One validated partition-expression argument. + */ + final class Argument { + private final PartitionValueType _type; + private final boolean _dynamic; + @Nullable + private final PartitionValue _constantValue; + + private Argument(PartitionValueType type, boolean dynamic, @Nullable PartitionValue constantValue) { + _type = Preconditions.checkNotNull(type, "Partition scalar function argument type must be configured"); + _dynamic = dynamic; + _constantValue = constantValue; + } + + public static Argument dynamic(PartitionValueType type) { + return new Argument(type, true, null); + } + + public static Argument constant(PartitionValue value) { + Preconditions.checkNotNull(value, "Partition scalar function constant must be configured"); + return new Argument(value.getType(), false, value); + } + + public PartitionValueType getType() { + return _type; + } + + public boolean isDynamic() { + return _dynamic; + } + + @Nullable + public PartitionValue getConstantValue() { + return _constantValue; + } + } + + /** + * One resolved scalar-function binding. + */ + interface ResolvedFunction { + boolean isDynamic(); + + PartitionValueType getOutputType(); + + PartitionValue invoke(@Nullable PartitionValue dynamicInput); + } +} diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionStep.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionStep.java new file mode 100644 index 000000000000..0e95954329db --- /dev/null +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionStep.java @@ -0,0 +1,65 @@ +/** + * 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.spi.partition.pipeline; + +import com.google.common.base.Preconditions; + + +/** + * One compiled pipeline step with a fixed input/output type contract. + */ +public final class PartitionStep { + @FunctionalInterface + interface Evaluator { + PartitionValue apply(PartitionValue input); + } + + private final String _name; + private final PartitionValueType _inputType; + private final PartitionValueType _outputType; + private final Evaluator _evaluator; + + PartitionStep(String name, PartitionValueType inputType, PartitionValueType outputType, Evaluator evaluator) { + _name = name; + _inputType = inputType; + _outputType = outputType; + _evaluator = evaluator; + } + + public String getName() { + return _name; + } + + public PartitionValueType getInputType() { + return _inputType; + } + + public PartitionValueType getOutputType() { + return _outputType; + } + + public PartitionValue evaluate(PartitionValue input) { + Preconditions.checkArgument(input.getType() == _inputType, + "Step '%s' expects %s input but got %s", _name, _inputType, input.getType()); + PartitionValue output = _evaluator.apply(input); + Preconditions.checkState(output.getType() == _outputType, + "Step '%s' produced %s output but expected %s", _name, output.getType(), _outputType); + return output; + } +} diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionValue.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionValue.java new file mode 100644 index 000000000000..f6b8c55ee51c --- /dev/null +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionValue.java @@ -0,0 +1,163 @@ +/** + * 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.spi.partition.pipeline; + +import com.google.common.base.Preconditions; +import java.util.Arrays; +import java.util.Objects; + + +/** + * Typed runtime value used within the compiled partition pipeline. + */ +public final class PartitionValue { + private final PartitionValueType _type; + private final Object _value; + + private PartitionValue(PartitionValueType type, Object value) { + _type = type; + _value = value; + } + + public static PartitionValue stringValue(String value) { + Preconditions.checkNotNull(value, "Partition pipeline does not accept null STRING values"); + return new PartitionValue(PartitionValueType.STRING, value); + } + + public static PartitionValue bytesValue(byte[] value) { + Preconditions.checkNotNull(value, "Partition pipeline does not accept null BYTES values"); + return new PartitionValue(PartitionValueType.BYTES, value.clone()); + } + + public static PartitionValue intValue(int value) { + return new PartitionValue(PartitionValueType.INT, value); + } + + public static PartitionValue longValue(long value) { + return new PartitionValue(PartitionValueType.LONG, value); + } + + public static PartitionValue floatValue(float value) { + return new PartitionValue(PartitionValueType.FLOAT, value); + } + + public static PartitionValue doubleValue(double value) { + return new PartitionValue(PartitionValueType.DOUBLE, value); + } + + public PartitionValueType getType() { + return _type; + } + + public String getStringValue() { + Preconditions.checkState(_type == PartitionValueType.STRING, "Expected STRING value but got: %s", _type); + return (String) _value; + } + + public byte[] getBytesValue() { + Preconditions.checkState(_type == PartitionValueType.BYTES, "Expected BYTES value but got: %s", _type); + return (byte[]) _value; + } + + public int getIntValue() { + Preconditions.checkState(_type == PartitionValueType.INT, "Expected INT value but got: %s", _type); + return (Integer) _value; + } + + public long getLongValue() { + Preconditions.checkState(_type == PartitionValueType.LONG, "Expected LONG value but got: %s", _type); + return (Long) _value; + } + + public float getFloatValue() { + Preconditions.checkState(_type == PartitionValueType.FLOAT, "Expected FLOAT value but got: %s", _type); + return (Float) _value; + } + + public double getDoubleValue() { + Preconditions.checkState(_type == PartitionValueType.DOUBLE, "Expected DOUBLE value but got: %s", _type); + return (Double) _value; + } + + public Object toObject() { + switch (_type) { + case STRING: + return getStringValue(); + case BYTES: + return getBytesValue(); + case INT: + return getIntValue(); + case LONG: + return getLongValue(); + case FLOAT: + return getFloatValue(); + case DOUBLE: + return getDoubleValue(); + default: + throw new IllegalStateException("Unsupported partition pipeline runtime value type: " + _type); + } + } + + public static PartitionValue fromObject(Object value) { + Preconditions.checkNotNull(value, "Partition pipeline function returned null"); + if (value instanceof String) { + return stringValue((String) value); + } + if (value instanceof byte[]) { + return bytesValue((byte[]) value); + } + if (value instanceof Integer) { + return intValue((Integer) value); + } + if (value instanceof Long) { + return longValue((Long) value); + } + if (value instanceof Float) { + return floatValue((Float) value); + } + if (value instanceof Double) { + return doubleValue((Double) value); + } + throw new IllegalArgumentException("Unsupported partition pipeline runtime value type: " + value.getClass()); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (!(obj instanceof PartitionValue)) { + return false; + } + PartitionValue that = (PartitionValue) obj; + if (_type != that._type) { + return false; + } + if (_type == PartitionValueType.BYTES) { + return Arrays.equals((byte[]) _value, (byte[]) that._value); + } + return Objects.equals(_value, that._value); + } + + @Override + public int hashCode() { + return _type == PartitionValueType.BYTES ? 31 * _type.hashCode() + Arrays.hashCode((byte[]) _value) + : 31 * _type.hashCode() + _value.hashCode(); + } +} diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionValueConversions.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionValueConversions.java new file mode 100644 index 000000000000..6807b28062d1 --- /dev/null +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionValueConversions.java @@ -0,0 +1,202 @@ +/** + * 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.spi.partition.pipeline; + +import com.google.common.base.Preconditions; +import java.util.EnumSet; +import java.util.Set; + +import static java.nio.charset.StandardCharsets.UTF_8; + + +/** + * Static type-conversion utilities shared by all implementations of {@link PartitionScalarFunctionResolver}. + * + *

These methods encode which source/target type pairs are allowed during partition-expression binding, what the + * conversion costs are (lower = preferred during overload resolution), and how to perform the actual Java-type + * conversion at binding time. + */ +public final class PartitionValueConversions { + // Pre-computed widening sets used during overload resolution. Kept as constants to avoid per-call allocation. + private static final Set INT_WIDENING_TARGETS = + EnumSet.of(PartitionValueType.LONG, PartitionValueType.FLOAT, PartitionValueType.DOUBLE); + private static final Set LONG_WIDENING_TARGETS = EnumSet.of(PartitionValueType.DOUBLE); + private static final Set FLOAT_WIDENING_TARGETS = EnumSet.of(PartitionValueType.DOUBLE); + + private PartitionValueConversions() { + } + + /** + * Returns the overload-resolution cost for converting a dynamic pipeline value of {@code sourceType} to + * {@code targetType}, or {@code -1} if the conversion is not allowed. + */ + public static int getDynamicConversionCost(PartitionValueType sourceType, Class targetType) { + PartitionValueType targetValueType = PartitionValueType.fromJavaType(targetType); + if (sourceType == targetValueType) { + return 0; + } + switch (sourceType) { + case STRING: + if (targetValueType == PartitionValueType.BYTES || targetValueType.isNumeric()) { + return 1; + } + return -1; + case BYTES: + return -1; + case INT: + return wideningNumericCost(targetValueType, INT_WIDENING_TARGETS); + case LONG: + return wideningNumericCost(targetValueType, LONG_WIDENING_TARGETS); + case FLOAT: + return wideningNumericCost(targetValueType, FLOAT_WIDENING_TARGETS); + case DOUBLE: + default: + return -1; + } + } + + /** + * Returns the overload-resolution cost for passing a compile-time constant {@code value} as {@code targetType}, or + * {@code -1} if the conversion is not allowed. + */ + public static int getConstantConversionCost(PartitionValue value, Class targetType) { + PartitionValueType targetValueType = PartitionValueType.fromJavaType(targetType); + PartitionValueType sourceType = value.getType(); + if (sourceType == targetValueType) { + return 0; + } + switch (sourceType) { + case STRING: + if (targetValueType == PartitionValueType.BYTES || targetValueType.isNumeric()) { + return 1; + } + return -1; + case LONG: + if (targetValueType == PartitionValueType.INT) { + long longValue = value.getLongValue(); + return longValue >= Integer.MIN_VALUE && longValue <= Integer.MAX_VALUE ? 2 : -1; + } + return getDynamicConversionCost(sourceType, targetType); + case DOUBLE: + if (targetValueType == PartitionValueType.INT) { + double doubleValue = value.getDoubleValue(); + return doubleValue >= Integer.MIN_VALUE && doubleValue <= Integer.MAX_VALUE + && doubleValue == Math.rint(doubleValue) ? 2 : -1; + } + if (targetValueType == PartitionValueType.LONG) { + double doubleValue = value.getDoubleValue(); + return doubleValue >= Long.MIN_VALUE && doubleValue <= Long.MAX_VALUE + && doubleValue == Math.rint(doubleValue) ? 2 : -1; + } + return getDynamicConversionCost(sourceType, targetType); + default: + return getDynamicConversionCost(sourceType, targetType); + } + } + + /** + * Converts {@code value} to the Java type required by {@code targetType}. + * + * @param constant {@code true} when converting a compile-time literal; allows narrowing conversions (e.g. LONG → + * INT) that are rejected for dynamic runtime values. + */ + public static Object convertValue(PartitionValue value, Class targetType, boolean constant) { + PartitionValueType targetValueType = PartitionValueType.fromJavaType(targetType); + switch (targetValueType) { + case STRING: + Preconditions.checkArgument(value.getType() == PartitionValueType.STRING, + "Expected STRING argument but got %s", value.getType()); + return value.getStringValue(); + case BYTES: + if (value.getType() == PartitionValueType.BYTES) { + return value.getBytesValue(); + } + Preconditions.checkArgument(value.getType() == PartitionValueType.STRING, + "Expected STRING or BYTES argument for BYTES parameter but got %s", value.getType()); + return value.getStringValue().getBytes(UTF_8); + case INT: + switch (value.getType()) { + case INT: + return value.getIntValue(); + case LONG: + Preconditions.checkArgument(constant, "Cannot narrow LONG pipeline value to INT dynamically"); + return Math.toIntExact(value.getLongValue()); + case DOUBLE: + Preconditions.checkArgument(constant, "Cannot narrow DOUBLE pipeline value to INT dynamically"); + return (int) value.getDoubleValue(); + case STRING: + return Integer.parseInt(value.getStringValue()); + default: + throw new IllegalArgumentException( + String.format("Cannot convert %s partition value to INT", value.getType())); + } + case LONG: + switch (value.getType()) { + case INT: + return (long) value.getIntValue(); + case LONG: + return value.getLongValue(); + case DOUBLE: + Preconditions.checkArgument(constant, "Cannot narrow DOUBLE pipeline value to LONG dynamically"); + return (long) value.getDoubleValue(); + case STRING: + return Long.parseLong(value.getStringValue()); + default: + throw new IllegalArgumentException( + String.format("Cannot convert %s partition value to LONG", value.getType())); + } + case FLOAT: + switch (value.getType()) { + case INT: + return (float) value.getIntValue(); + case LONG: + return (float) value.getLongValue(); + case FLOAT: + return value.getFloatValue(); + case STRING: + return Float.parseFloat(value.getStringValue()); + default: + throw new IllegalArgumentException( + String.format("Cannot convert %s partition value to FLOAT", value.getType())); + } + case DOUBLE: + switch (value.getType()) { + case INT: + return (double) value.getIntValue(); + case LONG: + return (double) value.getLongValue(); + case FLOAT: + return (double) value.getFloatValue(); + case DOUBLE: + return value.getDoubleValue(); + case STRING: + return Double.parseDouble(value.getStringValue()); + default: + throw new IllegalArgumentException( + String.format("Cannot convert %s partition value to DOUBLE", value.getType())); + } + default: + throw new IllegalStateException("Unsupported partition parameter type: " + targetType); + } + } + + private static int wideningNumericCost(PartitionValueType targetType, Set supportedTypes) { + return supportedTypes.contains(targetType) ? 1 : -1; + } +} diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionValueType.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionValueType.java new file mode 100644 index 000000000000..4cedb067302a --- /dev/null +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionValueType.java @@ -0,0 +1,62 @@ +/** + * 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.spi.partition.pipeline; + + +/** + * Value types supported by the partition-pipeline compiler. + */ +public enum PartitionValueType { + STRING, + BYTES, + INT, + LONG, + FLOAT, + DOUBLE; + + public boolean isIntegral() { + return this == INT || this == LONG; + } + + public boolean isNumeric() { + return isIntegral() || this == FLOAT || this == DOUBLE; + } + + public static PartitionValueType fromJavaType(Class clazz) { + if (clazz == String.class) { + return STRING; + } + if (clazz == byte[].class) { + return BYTES; + } + if (clazz == int.class || clazz == Integer.class) { + return INT; + } + if (clazz == long.class || clazz == Long.class) { + return LONG; + } + if (clazz == float.class || clazz == Float.class) { + return FLOAT; + } + if (clazz == double.class || clazz == Double.class) { + return DOUBLE; + } + throw new IllegalArgumentException("Unsupported partition pipeline java type: " + clazz.getName()); + } +} diff --git a/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprRacyTestFunctions.java b/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprRacyTestFunctions.java new file mode 100644 index 000000000000..f55fe626fcd8 --- /dev/null +++ b/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprRacyTestFunctions.java @@ -0,0 +1,62 @@ +/** + * 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.spi.function.scalar; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import org.apache.pinot.spi.annotations.ScalarFunction; + + +/** + * Test-only non-static scalar function with mutable instance state used to validate thread-safe target handling for + * partition-expression scalar methods. + */ +public class PartitionFunctionExprRacyTestFunctions { + private static volatile CountDownLatch _firstEntered = new CountDownLatch(1); + private static volatile CountDownLatch _secondCompleted = new CountDownLatch(1); + + private String _value; + + public static void reset() { + _firstEntered = new CountDownLatch(1); + _secondCompleted = new CountDownLatch(1); + } + + @ScalarFunction(names = {"racy_echo"}) + public String racyEcho(String input) { + _value = input; + try { + if ("first".equals(input)) { + _firstEntered.countDown(); + if (!_secondCompleted.await(5, TimeUnit.SECONDS)) { + throw new IllegalStateException("Timed out waiting for second invocation"); + } + } else if ("second".equals(input)) { + if (!_firstEntered.await(5, TimeUnit.SECONDS)) { + throw new IllegalStateException("Timed out waiting for first invocation"); + } + _secondCompleted.countDown(); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IllegalStateException("Interrupted while coordinating racy scalar function", e); + } + return _value; + } +} diff --git a/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprTestFunctions.java b/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprTestFunctions.java new file mode 100644 index 000000000000..af6eba323e93 --- /dev/null +++ b/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprTestFunctions.java @@ -0,0 +1,90 @@ +/** + * 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.spi.function.scalar; + +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.Locale; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.pinot.spi.annotations.ScalarFunction; +import org.apache.pinot.spi.utils.BytesUtils; +import org.apache.pinot.spi.utils.hash.FnvHashFunctions; +import org.apache.pinot.spi.utils.hash.MurmurHashFunctions; + + +/** + * Test-only scalar functions used to exercise the partition-expression compiler inside the pinot-segment-spi module. + */ +public final class PartitionFunctionExprTestFunctions { + private PartitionFunctionExprTestFunctions() { + } + + @ScalarFunction + public static String lower(String input) { + return input.toLowerCase(Locale.ROOT); + } + + @ScalarFunction + public static String md5(byte[] input) { + return BytesUtils.toHexString(md5Digest(input)); + } + + @ScalarFunction(names = {"md5_raw"}) + public static byte[] md5Raw(byte[] input) { + return md5Digest(input); + } + + @ScalarFunction(names = {"fnv1a_32"}) + public static int fnv1a32(byte[] input) { + return FnvHashFunctions.fnv1aHash32(input); + } + + @ScalarFunction + public static int murmur2(byte[] input) { + return MurmurHashFunctions.murmurHash2(input); + } + + @ScalarFunction + public static int identity(int value) { + return value; + } + + @ScalarFunction + public static long bucket(long value, long divisor) { + return value / divisor; + } + + @ScalarFunction(names = {"cid"}, isDeterministic = false) + public static String nonDeterministicCid(String input) { + return input; + } + + @ScalarFunction(isDeterministic = false) + public static long randomBucket(long value) { + return value + ThreadLocalRandom.current().nextLong(); + } + + private static byte[] md5Digest(byte[] input) { + try { + return MessageDigest.getInstance("MD5").digest(input); + } catch (NoSuchAlgorithmException e) { + throw new IllegalStateException("MD5 digest is not available", e); + } + } +} diff --git a/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionTest.java b/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionTest.java index aa4537a23ac4..4505236f2a0e 100644 --- a/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionTest.java +++ b/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionTest.java @@ -22,6 +22,9 @@ import java.util.HashMap; import java.util.Map; import java.util.Random; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionPipelineFunction; +import org.apache.pinot.spi.data.readers.GenericRow; +import org.apache.pinot.spi.function.FunctionEvaluator; import org.apache.pinot.spi.utils.BytesUtils; import org.apache.pinot.spi.utils.JsonUtils; import org.apache.pinot.spi.utils.hash.FnvHashFunctions; @@ -30,6 +33,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertTrue; import static org.testng.Assert.expectThrows; @@ -433,6 +437,87 @@ public void testBoundedColumnValuePartitioner() { assertEquals(partitionFunction.getPartition("Physics"), 0); } + @Test + public void testLegacyPartitionFunctionsExposeDefaultPartitionIdNormalizer() { + assertEquals(PartitionFunctionFactory.getPartitionFunction("Modulo", 8, null).getPartitionIdNormalizer(), + "POSITIVE_MODULO"); + assertEquals(PartitionFunctionFactory.getPartitionFunction("HashCode", 8, null).getPartitionIdNormalizer(), + "ABS"); + assertEquals(PartitionFunctionFactory.getPartitionFunction("ByteArray", 8, null).getPartitionIdNormalizer(), + "ABS"); + assertEquals(PartitionFunctionFactory.getPartitionFunction("Murmur2", 8, null).getPartitionIdNormalizer(), + "MASK"); + assertEquals(PartitionFunctionFactory.getPartitionFunction("Murmur3", 8, null).getPartitionIdNormalizer(), + "MASK"); + assertEquals(PartitionFunctionFactory.getPartitionFunction("Fnv", 8, null).getPartitionIdNormalizer(), "MASK"); + assertEquals(PartitionFunctionFactory.getPartitionFunction("Fnv", 8, + Map.of("negativePartitionHandling", "abs")).getPartitionIdNormalizer(), "ABS"); + } + + @Test + public void testLegacyPartitionFunctionFactoryAcceptsMatchingPartitionIdNormalizer() { + PartitionFunction partitionFunction = + PartitionFunctionFactory.getPartitionFunction("id", "Murmur2", 8, null, null, "MASK"); + + assertEquals(partitionFunction.getName(), "Murmur"); + assertEquals(partitionFunction.getPartitionIdNormalizer(), "MASK"); + } + + @Test + public void testLegacyPartitionFunctionFactoryRejectsMismatchedPartitionIdNormalizer() { + IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, + () -> PartitionFunctionFactory.getPartitionFunction("id", "Murmur2", 8, null, null, "ABS")); + + assertTrue(exception.getMessage().contains("incompatible with legacy partition function 'Murmur2'")); + } + + @Test + public void testLegacyPartitionFunctionImplementsFunctionEvaluatorWhenBoundToColumn() { + PartitionFunction partitionFunction = + PartitionFunctionFactory.getPartitionFunction("id", "Murmur2", 8, null, null, "MASK"); + assertTrue(partitionFunction instanceof FunctionEvaluator); + FunctionEvaluator evaluator = (FunctionEvaluator) partitionFunction; + GenericRow row = new GenericRow(); + row.putValue("id", "Pinot"); + + assertEquals(evaluator.getArguments(), java.util.List.of("id")); + assertEquals(evaluator.evaluate(row), partitionFunction.getPartition("Pinot")); + assertEquals(evaluator.evaluate(new Object[]{"Pinot"}), partitionFunction.getPartition("Pinot")); + } + + @Test + public void testUnboundLegacyPartitionFunctionDoesNotImplementFunctionEvaluator() { + PartitionFunction partitionFunction = PartitionFunctionFactory.getPartitionFunction("Murmur2", 8, null); + assertFalse(partitionFunction instanceof FunctionEvaluator); + } + + @Test + public void testFunctionExprPartitionFunctionImplementsFunctionEvaluator() { + PartitionFunction partitionFunction = + PartitionFunctionFactory.getPartitionFunction("id", null, 128, null, "fnv1a_32(md5(id))", "MASK"); + assertTrue(partitionFunction instanceof FunctionEvaluator); + FunctionEvaluator evaluator = (FunctionEvaluator) partitionFunction; + GenericRow row = new GenericRow(); + row.putValue("id", "000016be-9d72-466c-9632-cfa680dc8fa3"); + + assertEquals(evaluator.getArguments(), java.util.List.of("id")); + assertEquals(evaluator.evaluate(row), 104); + assertEquals(evaluator.evaluate(new Object[]{"000016be-9d72-466c-9632-cfa680dc8fa3"}), 104); + } + + @Test + public void testFunctionExprPartitionFunctionSerialization() { + PartitionFunction partitionFunction = + PartitionFunctionFactory.getPartitionFunction("id", null, 128, null, "fnv1a_32(md5(id))", "MASK"); + + JsonNode jsonNode = JsonUtils.objectToJsonNode(partitionFunction); + assertEquals(partitionFunction.getName(), PartitionPipelineFunction.NAME); + assertEquals(jsonNode.get("name").asText(), PartitionPipelineFunction.NAME); + assertEquals(jsonNode.get("numPartitions").asInt(), 128); + assertEquals(jsonNode.get("functionExpr").asText(), "fnv1a_32(md5(id))"); + assertEquals(jsonNode.get("partitionIdNormalizer").asText(), "MASK"); + } + private void testBasicProperties(PartitionFunction partitionFunction, String functionName, int numPartitions) { testBasicProperties(partitionFunction, functionName, numPartitions, null); } diff --git a/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadataTest.java b/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadataTest.java new file mode 100644 index 000000000000..4d18415036f6 --- /dev/null +++ b/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadataTest.java @@ -0,0 +1,86 @@ +/** + * 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.spi.partition.metadata; + +import java.util.Set; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionFunctionExprCompiler; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionPipelineFunction; +import org.apache.pinot.spi.utils.JsonUtils; +import org.testng.annotations.Test; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; + + +public class ColumnPartitionMetadataTest { + @Test + public void testRoundTripTreatsNullExpressionFieldsAsAbsent() + throws Exception { + ColumnPartitionMetadata metadata = new ColumnPartitionMetadata("Modulo", 8, Set.of(3), null); + + ColumnPartitionMetadata roundTripped = + JsonUtils.stringToObject(JsonUtils.objectToString(metadata), ColumnPartitionMetadata.class); + + assertEquals(roundTripped, metadata); + assertNull(roundTripped.getFunctionExpr()); + assertNull(roundTripped.getPartitionIdNormalizer()); + } + + @Test + public void testConstructorFromPartitionFunctionPreservesExpressionFields() { + ColumnPartitionMetadata metadata = new ColumnPartitionMetadata( + PartitionFunctionExprCompiler.compilePartitionFunction("id", "fnv1a_32(md5(id))", 128, "MASK"), Set.of(104)); + + // Expression-mode: functionName is the stable "FunctionExpr" sentinel for backward compatibility. + // Old brokers that call jsonMetadata.get("functionName").asText() without a null guard will receive + // this non-null value and fail with IllegalArgumentException (graceful degradation: no pruning). + assertEquals(metadata.getFunctionName(), PartitionPipelineFunction.NAME); + assertEquals(metadata.getNumPartitions(), 128); + assertEquals(metadata.getPartitions(), Set.of(104)); + assertEquals(metadata.getFunctionExpr(), "fnv1a_32(md5(id))"); + assertEquals(metadata.getPartitionIdNormalizer(), "MASK"); + assertNull(metadata.getInputType()); // STRING is the default, not stored + } + + @Test + public void testExpressionModeRoundTripPreservesAllFields() + throws Exception { + ColumnPartitionMetadata metadata = new ColumnPartitionMetadata( + PartitionFunctionExprCompiler.compilePartitionFunction("id", "fnv1a_32(md5(id))", 64, "POSITIVE_MODULO"), + Set.of(7, 15)); + + String json = JsonUtils.objectToString(metadata); + + // The serialized JSON must contain "functionName" so old brokers do not NPE when they call + // jsonMetadata.get("functionName").asText() without a null check. + assertTrue(json.contains("\"functionName\""), "JSON must include functionName for backward compatibility"); + assertTrue(json.contains("\"functionExpr\""), "JSON must include functionExpr"); + + ColumnPartitionMetadata roundTripped = JsonUtils.stringToObject(json, ColumnPartitionMetadata.class); + + assertEquals(roundTripped, metadata); + assertEquals(roundTripped.getFunctionName(), PartitionPipelineFunction.NAME); + assertEquals(roundTripped.getFunctionExpr(), "fnv1a_32(md5(id))"); + assertEquals(roundTripped.getPartitionIdNormalizer(), "POSITIVE_MODULO"); + assertEquals(roundTripped.getNumPartitions(), 64); + assertEquals(roundTripped.getPartitions(), Set.of(7, 15)); + assertNull(roundTripped.getInputType()); + } +} diff --git a/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompilerTest.java b/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompilerTest.java new file mode 100644 index 000000000000..b0fce9cdbcdb --- /dev/null +++ b/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompilerTest.java @@ -0,0 +1,273 @@ +/** + * 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.spi.partition.pipeline; + +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.function.ToIntFunction; +import org.apache.pinot.segment.spi.function.scalar.PartitionFunctionExprRacyTestFunctions; +import org.apache.pinot.segment.spi.partition.PartitionFunction; +import org.apache.pinot.segment.spi.partition.PartitionFunctionFactory; +import org.apache.pinot.spi.data.readers.GenericRow; +import org.apache.pinot.spi.function.FunctionEvaluator; +import org.apache.pinot.spi.utils.BytesUtils; +import org.apache.pinot.spi.utils.hash.FnvHashFunctions; +import org.apache.pinot.spi.utils.hash.MurmurHashFunctions; +import org.testng.annotations.Test; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotEquals; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.expectThrows; + + +public class PartitionFunctionExprCompilerTest { + @Test + public void testCompileHexStringPipeline() { + PartitionPipeline pipeline = PartitionFunctionExprCompiler.compile("raw_key", " MD5 ( raw_key ) "); + + assertEquals(pipeline.getCanonicalFunctionExpr(), "md5(raw_key)"); + assertEquals(pipeline.getOutputType(), PartitionValueType.STRING); + assertEquals(pipeline.evaluate("hello").getStringValue(), "5d41402abc4b2a76b9719d911017c592"); + } + + @Test + public void testCompileRawBytesPipeline() { + PartitionPipeline pipeline = PartitionFunctionExprCompiler.compile("raw_key", " MD5_RAW ( raw_key ) "); + + assertEquals(pipeline.getCanonicalFunctionExpr(), "md5_raw(raw_key)"); + assertEquals(pipeline.getOutputType(), PartitionValueType.BYTES); + assertEquals(BytesUtils.toHexString(pipeline.evaluate("hello").getBytesValue()), + "5d41402abc4b2a76b9719d911017c592"); + } + + @Test + public void testCompilePartitionFunctionForMd5Fnv() { + PartitionPipelineFunction partitionFunction = + PartitionFunctionExprCompiler.compilePartitionFunction("raw_key", "fnv1a_32(md5(raw_key))", 64); + + String digestHex = BytesUtils.toHexString(md5("Pinot".getBytes(UTF_8))); + int expectedHash = FnvHashFunctions.fnv1aHash32(digestHex.getBytes(UTF_8)); + int expectedPartition = positiveModulo(expectedHash, 64); + assertEquals(partitionFunction.getPartition("Pinot"), expectedPartition); + assertEquals(partitionFunction.getFunctionExpr(), "fnv1a_32(md5(raw_key))"); + assertEquals(partitionFunction.getPartitionIdNormalizer(), "POSITIVE_MODULO"); + } + + @Test + public void testCompilePartitionFunctionForMd5FnvWithExactUuidUsingMaskNormalizer() { + PartitionPipelineFunction partitionFunction = + PartitionFunctionExprCompiler.compilePartitionFunction("id", "fnv1a_32(md5(id))", 128, "MASK"); + + assertEquals(partitionFunction.getPartition("000016be-9d72-466c-9632-cfa680dc8fa3"), 104); + assertEquals(partitionFunction.getPartitionIdNormalizer(), "MASK"); + } + + @Test + public void testCompilePartitionFunctionForLowerMurmur2() { + PartitionPipelineFunction partitionFunction = + PartitionFunctionExprCompiler.compilePartitionFunction("raw_key", "murmur2(lower(raw_key))", 32); + + int expectedHash = MurmurHashFunctions.murmurHash2("hello".getBytes(UTF_8)); + assertEquals(partitionFunction.getPartition("HeLLo"), positiveModulo(expectedHash, 32)); + } + + @Test + public void testCompilePartitionFunctionWithLiteralArgument() { + PartitionPipelineFunction partitionFunction = + PartitionFunctionExprCompiler.compilePartitionFunction("timestampMillis", "bucket(timestampMillis, 1000)", 128); + + assertEquals(partitionFunction.getPartition("54321"), 54); + assertEquals(partitionFunction.getFunctionExpr(), "bucket(timestampMillis, 1000)"); + } + + @Test + public void testCompiledPartitionPipelineImplementsFunctionEvaluator() { + FunctionEvaluator evaluator = PartitionFunctionExprCompiler.compile("raw_key", "murmur2(lower(raw_key))"); + GenericRow row = new GenericRow(); + row.putValue("raw_key", "HeLLo"); + + assertEquals(evaluator.getArguments(), List.of("raw_key")); + assertEquals(evaluator.evaluate(row), MurmurHashFunctions.murmurHash2("hello".getBytes(UTF_8))); + assertEquals(evaluator.evaluate(new Object[]{"HeLLo"}), MurmurHashFunctions.murmurHash2("hello".getBytes(UTF_8))); + } + + @Test + public void testCompiledPartitionPipelineCoercesGenericRowValuesToString() { + FunctionEvaluator evaluator = PartitionFunctionExprCompiler.compile("timestampMillis", "bucket(timestampMillis, " + + "1000)"); + GenericRow row = new GenericRow(); + row.putValue("timestampMillis", 54321L); + + assertEquals(evaluator.evaluate(row), 54L); + assertEquals(evaluator.evaluate(new Object[]{54321L}), 54L); + } + + @Test + public void testIntegralExpressionDefaultsToPositiveModulo() { + int numPartitions = 13; + PartitionPipeline pipeline = PartitionFunctionExprCompiler.compile("raw_key", "murmur2(raw_key)"); + PartitionPipelineFunction partitionFunction = + PartitionFunctionExprCompiler.compilePartitionFunction("raw_key", "murmur2(raw_key)", numPartitions); + PartitionFunction legacyPartitionFunction = + PartitionFunctionFactory.getPartitionFunction("Murmur2", numPartitions, null); + String value = findValueWithNegativeHash(input -> MurmurHashFunctions.murmurHash2(input.getBytes(UTF_8))); + int hash = MurmurHashFunctions.murmurHash2(value.getBytes(UTF_8)); + + assertTrue(hash < 0); + assertNotEquals(positiveModulo(hash, numPartitions), legacyPartitionFunction.getPartition(value)); + assertEquals(pipeline.getIntNormalizer(), PartitionIntNormalizer.POSITIVE_MODULO); + assertEquals(partitionFunction.getPartition(value), positiveModulo(hash, numPartitions)); + assertEquals(partitionFunction.getPartitionIdNormalizer(), "POSITIVE_MODULO"); + } + + @Test + public void testExplicitMaskNormalizerMatchesLegacyMurmur2Partitioning() { + int numPartitions = 13; + PartitionPipelineFunction partitionFunction = + PartitionFunctionExprCompiler.compilePartitionFunction("raw_key", "identity(murmur2(raw_key))", + numPartitions, "MASK"); + PartitionFunction legacyPartitionFunction = + PartitionFunctionFactory.getPartitionFunction("Murmur2", numPartitions, null); + String value = findValueWithNegativeHash(input -> MurmurHashFunctions.murmurHash2(input.getBytes(UTF_8))); + + assertEquals(partitionFunction.getPartition(value), legacyPartitionFunction.getPartition(value)); + assertEquals(partitionFunction.getPartitionIdNormalizer(), "MASK"); + } + + @Test + public void testExplicitAbsNormalizerUsesAbsoluteRemainder() { + int numPartitions = 13; + String value = findValueWithNegativeHash(input -> MurmurHashFunctions.murmurHash2(input.getBytes(UTF_8))); + int hash = MurmurHashFunctions.murmurHash2(value.getBytes(UTF_8)); + PartitionPipelineFunction partitionFunction = + PartitionFunctionExprCompiler.compilePartitionFunction("raw_key", "identity(murmur2(raw_key))", + numPartitions, "ABS"); + + assertTrue(hash < 0); + assertEquals(partitionFunction.getPartition(value), absoluteModulo(hash, numPartitions)); + assertEquals(partitionFunction.getPartitionIdNormalizer(), "ABS"); + } + + @Test + public void testRejectsWrongColumnReference() { + IllegalArgumentException error = expectThrows(IllegalArgumentException.class, + () -> PartitionFunctionExprCompiler.compile("raw_key", "md5(other_key)")); + assertEquals(error.getMessage(), + "Partition function expression must reference only partition column 'raw_key', got 'other_key'"); + } + + @Test + public void testRejectsUnsupportedFunction() { + IllegalArgumentException error = expectThrows(IllegalArgumentException.class, + () -> PartitionFunctionExprCompiler.compile("raw_key", "sha256(raw_key)")); + assertEquals(error.getMessage(), "Unsupported partition scalar function: sha256"); + } + + @Test + public void testRejectsNonIntPartitionFunctionOutput() { + IllegalArgumentException error = expectThrows(IllegalArgumentException.class, + () -> PartitionFunctionExprCompiler.compilePartitionFunction("raw_key", "md5(raw_key)", 16)); + assertEquals(error.getMessage(), "Partition pipeline must produce INT or LONG output, got: STRING"); + } + + @Test + public void testRejectsInvalidGrammar() { + IllegalArgumentException error = expectThrows(IllegalArgumentException.class, + () -> PartitionFunctionExprCompiler.compile("raw_key", "fnv1a_32(md5(raw_key), lower(raw_key))")); + assertEquals(error.getMessage(), + "Partition function expression must reference partition column 'raw_key' through a single argument chain"); + } + + @Test + public void testRejectsNonDeterministicFunction() { + IllegalArgumentException error = expectThrows(IllegalArgumentException.class, + () -> PartitionFunctionExprCompiler.compile("raw_key", "randomBucket(raw_key)")); + assertEquals(error.getMessage(), + "Partition scalar function 'randombucket' is not allowed because it is non-deterministic"); + } + + @Test + public void testRejectsNonDeterministicAliasedFunction() { + IllegalArgumentException error = expectThrows(IllegalArgumentException.class, + () -> PartitionFunctionExprCompiler.compile("raw_key", "cid(raw_key)")); + assertEquals(error.getMessage(), + "Partition scalar function 'cid' is not allowed because it is non-deterministic"); + } + + @Test + public void testRejectsConstantFunctionArgument() { + IllegalArgumentException error = expectThrows(IllegalArgumentException.class, + () -> PartitionFunctionExprCompiler.compilePartitionFunction("timestampMillis", + "bucket(timestampMillis, identity(1000))", 128)); + assertEquals(error.getMessage(), + "Partition function expression only supports literal constants, got function subexpression: identity(1000)"); + } + + @Test(timeOut = 10_000L) + public void testNonStaticScalarFunctionsUseThreadLocalTargets() + throws Exception { + PartitionFunctionExprRacyTestFunctions.reset(); + PartitionPipeline pipeline = PartitionFunctionExprCompiler.compile("raw_key", "racy_echo(raw_key)"); + ExecutorService executorService = Executors.newFixedThreadPool(2); + try { + Future firstResult = executorService.submit(() -> pipeline.evaluate("first").getStringValue()); + Future secondResult = executorService.submit(() -> pipeline.evaluate("second").getStringValue()); + + assertEquals(firstResult.get(5, TimeUnit.SECONDS), "first"); + assertEquals(secondResult.get(5, TimeUnit.SECONDS), "second"); + } finally { + executorService.shutdownNow(); + } + } + + private static byte[] md5(byte[] input) { + try { + return MessageDigest.getInstance("MD5").digest(input); + } catch (NoSuchAlgorithmException e) { + throw new IllegalStateException("MD5 digest is not available", e); + } + } + + private static int positiveModulo(int value, int modulus) { + int partition = value % modulus; + return partition < 0 ? partition + modulus : partition; + } + + private static int absoluteModulo(int value, int modulus) { + int partition = value % modulus; + return partition < 0 ? -partition : partition; + } + + private static String findValueWithNegativeHash(ToIntFunction hashFunction) { + for (int i = 0; i < 10_000; i++) { + String value = "value_" + i; + if (hashFunction.applyAsInt(value) < 0) { + return value; + } + } + throw new IllegalStateException("Failed to find a value with a negative hash"); + } +} diff --git a/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/pipeline/TestPartitionScalarFunctionResolver.java b/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/pipeline/TestPartitionScalarFunctionResolver.java new file mode 100644 index 000000000000..a37c6f032028 --- /dev/null +++ b/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/pipeline/TestPartitionScalarFunctionResolver.java @@ -0,0 +1,351 @@ +/** + * 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.spi.partition.pipeline; + +import com.google.common.base.Preconditions; +import java.lang.reflect.Array; +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.lang.reflect.Modifier; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import javax.annotation.Nullable; +import org.apache.pinot.segment.spi.function.scalar.PartitionFunctionExprRacyTestFunctions; +import org.apache.pinot.segment.spi.function.scalar.PartitionFunctionExprTestFunctions; +import org.apache.pinot.spi.annotations.ScalarFunction; + + +/** + * Test-only partition scalar-function resolver for the {@code pinot-segment-spi} module. + */ +public class TestPartitionScalarFunctionResolver implements PartitionScalarFunctionResolver { + private static final Object DYNAMIC_ARGUMENT = new Object(); + private static final Map> FUNCTION_METHODS = loadFunctionMethods(); + + @Override + public ResolvedFunction resolve(String functionName, List arguments) { + List methods = FUNCTION_METHODS.get(canonicalize(functionName)); + if (methods == null) { + throw new IllegalArgumentException(String.format("Unsupported partition scalar function: %s", functionName)); + } + + BoundFunction bestMatch = null; + StringBuilder supportedSignatures = new StringBuilder(); + boolean sawNonDeterministicCandidate = false; + boolean sawDeterministicCandidate = false; + for (ScalarFunctionMethod method : methods) { + if (supportedSignatures.length() > 0) { + supportedSignatures.append(", "); + } + supportedSignatures.append(method.getSignature()); + + if (!method.isDeterministic()) { + sawNonDeterministicCandidate = true; + continue; + } + sawDeterministicCandidate = true; + + BoundFunction candidate = method.bind(arguments); + if (candidate != null) { + if (bestMatch == null || candidate.getCost() < bestMatch.getCost()) { + bestMatch = candidate; + } else if (candidate.getCost() == bestMatch.getCost()) { + throw new IllegalArgumentException(String.format( + "Ambiguous partition scalar function '%s' for argument types (%s). Matching signatures: %s", + functionName, formatArgumentTypes(arguments), supportedSignatures)); + } + } + } + + if (bestMatch != null) { + return bestMatch; + } + if (sawNonDeterministicCandidate && !sawDeterministicCandidate) { + throw new IllegalArgumentException(String.format( + "Partition scalar function '%s' is not allowed because it is non-deterministic", functionName)); + } + throw new IllegalArgumentException(String.format( + "Function '%s' does not accept argument types (%s). Supported signatures: %s", functionName, + formatArgumentTypes(arguments), supportedSignatures)); + } + + private static Map> loadFunctionMethods() { + Map> functionMethods = new HashMap<>(); + registerMethods(functionMethods, PartitionFunctionExprTestFunctions.class); + registerMethods(functionMethods, PartitionFunctionExprRacyTestFunctions.class); + + Map> immutable = new HashMap<>(functionMethods.size()); + for (Map.Entry> entry : functionMethods.entrySet()) { + immutable.put(entry.getKey(), Collections.unmodifiableList(entry.getValue())); + } + return Collections.unmodifiableMap(immutable); + } + + private static void registerMethods(Map> functionMethods, Class clazz) { + for (Method method : clazz.getMethods()) { + ScalarFunction scalarFunction = method.getAnnotation(ScalarFunction.class); + if (scalarFunction == null || !scalarFunction.enabled() || !Modifier.isPublic(method.getModifiers())) { + continue; + } + ScalarFunctionMethod functionMethod = new ScalarFunctionMethod(method, scalarFunction.isDeterministic()); + Set canonicalNames = new LinkedHashSet<>(); + canonicalNames.add(canonicalize(method.getName())); + for (String name : scalarFunction.names()) { + canonicalNames.add(canonicalize(name)); + } + for (String canonicalName : canonicalNames) { + functionMethods.computeIfAbsent(canonicalName, ignored -> new ArrayList<>()).add(functionMethod); + } + } + } + + private static String canonicalize(String name) { + return name.replace("_", "").toLowerCase(Locale.ROOT); + } + + private static String formatArgumentTypes(List arguments) { + StringBuilder builder = new StringBuilder(); + for (int i = 0; i < arguments.size(); i++) { + if (i > 0) { + builder.append(", "); + } + builder.append(arguments.get(i).getType()); + } + return builder.toString(); + } + + @Nullable + private static Constructor getEmptyConstructor(Class clazz) { + try { + return clazz.getConstructor(); + } catch (NoSuchMethodException e) { + return null; + } + } + + private static final class ScalarFunctionMethod { + private final Method _method; + private final boolean _staticMethod; + @Nullable + private final ThreadLocal _threadLocalTarget; + private final Class[] _parameterTypes; + private final boolean _varArgs; + private final boolean _deterministic; + private final PartitionValueType _outputType; + private final String _signature; + + private ScalarFunctionMethod(Method method, boolean deterministic) { + _method = method; + _staticMethod = Modifier.isStatic(method.getModifiers()); + _parameterTypes = method.getParameterTypes(); + _varArgs = method.isVarArgs(); + _deterministic = deterministic; + _outputType = PartitionValueType.fromJavaType(method.getReturnType()); + _signature = buildSignature(method); + _threadLocalTarget = _staticMethod ? null : buildThreadLocalTarget(method); + } + + public boolean isDeterministic() { + return _deterministic; + } + + public String getSignature() { + return _signature; + } + + @Nullable + public BoundFunction bind(List arguments) { + int parameterCount = _parameterTypes.length; + int fixedParameterCount = _varArgs ? parameterCount - 1 : parameterCount; + if ((!_varArgs && arguments.size() != parameterCount) || (_varArgs && arguments.size() < fixedParameterCount)) { + return null; + } + + Object[] constantArguments = new Object[arguments.size()]; + int totalCost = 0; + int dynamicIndex = -1; + PartitionValueType inputType = PartitionValueType.STRING; + Class dynamicParameterType = String.class; + for (int i = 0; i < arguments.size(); i++) { + Argument argument = arguments.get(i); + Class parameterType = getParameterType(i); + if (argument.isDynamic()) { + if (dynamicIndex >= 0) { + return null; + } + int cost = PartitionValueConversions.getDynamicConversionCost(argument.getType(), parameterType); + if (cost < 0) { + return null; + } + totalCost += cost; + dynamicIndex = i; + inputType = argument.getType(); + dynamicParameterType = parameterType; + constantArguments[i] = DYNAMIC_ARGUMENT; + } else { + PartitionValue constantValue = + Preconditions.checkNotNull(argument.getConstantValue(), "Constant argument must be configured"); + int cost = PartitionValueConversions.getConstantConversionCost(constantValue, parameterType); + if (cost < 0) { + return null; + } + totalCost += cost; + constantArguments[i] = PartitionValueConversions.convertValue(constantValue, parameterType, true); + } + } + return new BoundFunction(this, inputType, dynamicParameterType, dynamicIndex, constantArguments, totalCost); + } + + public PartitionValueType getOutputType() { + return _outputType; + } + + private Class getParameterType(int index) { + if (_varArgs && index >= _parameterTypes.length - 1) { + return _parameterTypes[_parameterTypes.length - 1].getComponentType(); + } + return _parameterTypes[index]; + } + + private Object invoke(Object[] expressionArguments) { + try { + Object target = _staticMethod ? null : Preconditions.checkNotNull(_threadLocalTarget).get(); + return _method.invoke(target, adaptArguments(expressionArguments)); + } catch (IllegalAccessException e) { + throw new IllegalStateException("Failed to invoke partition scalar function: " + _signature, e); + } catch (InvocationTargetException e) { + Throwable cause = e.getTargetException(); + if (cause instanceof RuntimeException) { + throw (RuntimeException) cause; + } + throw new IllegalStateException("Caught checked exception while invoking partition scalar function: " + + _signature, cause); + } + } + + private Object[] adaptArguments(Object[] expressionArguments) { + if (!_varArgs) { + return expressionArguments; + } + + int fixedParameterCount = _parameterTypes.length - 1; + Object[] methodArguments = new Object[_parameterTypes.length]; + for (int i = 0; i < fixedParameterCount; i++) { + methodArguments[i] = expressionArguments[i]; + } + Class componentType = _parameterTypes[_parameterTypes.length - 1].getComponentType(); + int varArgCount = expressionArguments.length - fixedParameterCount; + Object varArgArray = Array.newInstance(componentType, varArgCount); + for (int i = 0; i < varArgCount; i++) { + Array.set(varArgArray, i, expressionArguments[fixedParameterCount + i]); + } + methodArguments[_parameterTypes.length - 1] = varArgArray; + return methodArguments; + } + + private static ThreadLocal buildThreadLocalTarget(Method method) { + Constructor constructor = getEmptyConstructor(method.getDeclaringClass()); + Preconditions.checkState(constructor != null, + "Non-static partition scalar function must have an empty constructor: %s", method); + return ThreadLocal.withInitial(() -> instantiateTarget(constructor, method)); + } + + private static Object instantiateTarget(Constructor constructor, Method method) { + try { + return constructor.newInstance(); + } catch (Exception e) { + throw new IllegalStateException("Failed to instantiate partition scalar function target: " + method, e); + } + } + + private static String buildSignature(Method method) { + StringBuilder builder = new StringBuilder(method.getName()).append('('); + Class[] parameterTypes = method.getParameterTypes(); + for (int i = 0; i < parameterTypes.length; i++) { + if (i > 0) { + builder.append(", "); + } + Class parameterType = parameterTypes[i]; + if (method.isVarArgs() && i == parameterTypes.length - 1) { + builder.append(parameterType.getComponentType().getSimpleName()).append("..."); + } else { + builder.append(parameterType.getSimpleName()); + } + } + return builder.append(')').toString(); + } + } + + private static final class BoundFunction implements ResolvedFunction { + private final ScalarFunctionMethod _method; + private final PartitionValueType _inputType; + private final Class _dynamicParameterType; + private final int _dynamicIndex; + private final Object[] _constantArguments; + @Nullable + private final ThreadLocal _threadLocalArguments; + private final int _cost; + + private BoundFunction(ScalarFunctionMethod method, PartitionValueType inputType, Class dynamicParameterType, + int dynamicIndex, Object[] constantArguments, int cost) { + _method = method; + _inputType = inputType; + _dynamicParameterType = dynamicParameterType; + _dynamicIndex = dynamicIndex; + _constantArguments = constantArguments; + _threadLocalArguments = dynamicIndex >= 0 ? ThreadLocal.withInitial(_constantArguments::clone) : null; + _cost = cost; + } + + public int getCost() { + return _cost; + } + + @Override + public boolean isDynamic() { + return _dynamicIndex >= 0; + } + + @Override + public PartitionValueType getOutputType() { + return _method.getOutputType(); + } + + @Override + public PartitionValue invoke(@Nullable PartitionValue dynamicInput) { + Object[] expressionArguments = _dynamicIndex >= 0 ? Preconditions.checkNotNull(_threadLocalArguments).get() + : _constantArguments; + if (_dynamicIndex >= 0) { + Preconditions.checkNotNull(dynamicInput, "Dynamic partition step input must be configured"); + Preconditions.checkArgument(dynamicInput.getType() == _inputType, + "Expected %s dynamic input but got %s", _inputType, dynamicInput.getType()); + expressionArguments[_dynamicIndex] = PartitionValueConversions.convertValue(dynamicInput, _dynamicParameterType, + false); + } + return PartitionValue.fromObject(_method.invoke(expressionArguments)); + } + } +} diff --git a/pinot-segment-spi/src/test/resources/META-INF/services/org.apache.pinot.segment.spi.partition.pipeline.PartitionScalarFunctionResolver b/pinot-segment-spi/src/test/resources/META-INF/services/org.apache.pinot.segment.spi.partition.pipeline.PartitionScalarFunctionResolver new file mode 100644 index 000000000000..4aea5dfbd380 --- /dev/null +++ b/pinot-segment-spi/src/test/resources/META-INF/services/org.apache.pinot.segment.spi.partition.pipeline.PartitionScalarFunctionResolver @@ -0,0 +1,19 @@ +# +# 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. +# +org.apache.pinot.segment.spi.partition.pipeline.TestPartitionScalarFunctionResolver diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ColumnPartitionConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ColumnPartitionConfig.java index 2e3518a4c9d2..896cb97ee71f 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ColumnPartitionConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ColumnPartitionConfig.java @@ -26,22 +26,69 @@ import org.apache.pinot.spi.config.BaseJsonConfig; +/** + * Partition configuration for a single column. + * + *

There are two mutually exclusive modes: + *

    + *
  • Name mode (legacy): configure {@code functionName} (and optionally {@code functionConfig}). + * Old and new nodes can all deserialize this format.
  • + *
  • Expression mode (new): configure {@code functionExpr} (and optionally + * {@code partitionIdNormalizer}). This mode produces JSON that omits {@code functionName}. + * Known limitation: expression-mode configs must only be written after all + * broker/server/controller nodes have been upgraded to a version that supports this feature. + * A node pre-dating expression mode will fail to deserialize such a config and will exclude + * the affected table from partition-aware routing. No mixed-version safety gate is enforced + * at the controller write path; operators are responsible for ensuring the cluster is fully + * upgraded before enabling expression-mode partitioning.
  • + *
+ */ public class ColumnPartitionConfig extends BaseJsonConfig { + public static final String PARTITION_ID_NORMALIZER_POSITIVE_MODULO = "POSITIVE_MODULO"; + public static final String PARTITION_ID_NORMALIZER_ABS = "ABS"; + public static final String PARTITION_ID_NORMALIZER_MASK = "MASK"; + private final String _functionName; + private final String _functionExpr; + private final String _partitionIdNormalizer; private final int _numPartitions; private final Map _functionConfig; public ColumnPartitionConfig(String functionName, int numPartitions) { - this(functionName, numPartitions, null); + this(functionName, numPartitions, null, null, null); + } + + public static ColumnPartitionConfig forFunctionExpr(String functionExpr, int numPartitions) { + return forFunctionExpr(functionExpr, numPartitions, null); + } + + public static ColumnPartitionConfig forFunctionExpr(String functionExpr, int numPartitions, + @Nullable String partitionIdNormalizer) { + return new ColumnPartitionConfig(null, numPartitions, null, functionExpr, partitionIdNormalizer); + } + + public ColumnPartitionConfig(String functionName, int numPartitions, @Nullable Map functionConfig) { + this(functionName, numPartitions, functionConfig, null, null); } @JsonCreator - public ColumnPartitionConfig(@JsonProperty(value = "functionName", required = true) String functionName, + public ColumnPartitionConfig(@JsonProperty("functionName") @Nullable String functionName, @JsonProperty(value = "numPartitions", required = true) int numPartitions, - @JsonProperty(value = "functionConfig") @Nullable Map functionConfig) { - Preconditions.checkArgument(functionName != null, "'functionName' must be configured"); + @JsonProperty("functionConfig") @Nullable Map functionConfig, + @JsonProperty("functionExpr") @Nullable String functionExpr, + @JsonProperty("partitionIdNormalizer") @Nullable String partitionIdNormalizer) { + Preconditions.checkArgument(hasText(functionName) ^ hasText(functionExpr), + "Exactly one of 'functionName' or 'functionExpr' must be configured"); Preconditions.checkArgument(numPartitions > 0, "'numPartitions' must be positive"); + Preconditions.checkArgument(!hasText(functionExpr) || functionConfig == null, + "'functionConfig' cannot be configured together with 'functionExpr'"); + Preconditions.checkArgument(!hasText(partitionIdNormalizer) || hasText(functionExpr), + "'partitionIdNormalizer' can only be configured together with 'functionExpr'"); + Preconditions.checkArgument(isValidPartitionIdNormalizer(partitionIdNormalizer), + "Unsupported partitionIdNormalizer: %s", partitionIdNormalizer); _functionName = functionName; + _functionExpr = functionExpr; + _partitionIdNormalizer = partitionIdNormalizer; _numPartitions = numPartitions; _functionConfig = functionConfig; } @@ -51,10 +98,27 @@ public ColumnPartitionConfig(@JsonProperty(value = "functionName", required = tr * * @return Partition function name. */ + @Nullable public String getFunctionName() { return _functionName; } + /** + * Returns the function expression for expression-mode partitioning. + */ + @Nullable + public String getFunctionExpr() { + return _functionExpr; + } + + /** + * Returns the partition-id normalizer for expression-mode partitioning. + */ + @Nullable + public String getPartitionIdNormalizer() { + return _partitionIdNormalizer; + } + /** * Returns the partition function configuration for the column. * @@ -73,4 +137,17 @@ public Map getFunctionConfig() { public int getNumPartitions() { return _numPartitions; } + + private static boolean hasText(@Nullable String value) { + return value != null && !value.trim().isEmpty(); + } + + private static boolean isValidPartitionIdNormalizer(@Nullable String partitionIdNormalizer) { + if (!hasText(partitionIdNormalizer)) { + return true; + } + return PARTITION_ID_NORMALIZER_POSITIVE_MODULO.equalsIgnoreCase(partitionIdNormalizer) + || PARTITION_ID_NORMALIZER_ABS.equalsIgnoreCase(partitionIdNormalizer) + || PARTITION_ID_NORMALIZER_MASK.equalsIgnoreCase(partitionIdNormalizer); + } } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentPartitionConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentPartitionConfig.java index a3f6b832041c..816ca988341d 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentPartitionConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentPartitionConfig.java @@ -61,7 +61,18 @@ public int getNumPartitions(String column) { return (config != null) ? config.getNumPartitions() : INVALID_NUM_PARTITIONS; } - @Deprecated + @Nullable + public String getFunctionExpr(String column) { + ColumnPartitionConfig config = _columnPartitionMap.get(column); + return (config != null) ? config.getFunctionExpr() : null; + } + + @Nullable + public String getPartitionIdNormalizer(String column) { + ColumnPartitionConfig config = _columnPartitionMap.get(column); + return (config != null) ? config.getPartitionIdNormalizer() : null; + } + @Nullable public Map getFunctionConfig(String column) { ColumnPartitionConfig config = _columnPartitionMap.get(column); diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/ScalarFunctionUtils.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/ScalarFunctionUtils.java new file mode 100644 index 000000000000..ff50e6e4cdad --- /dev/null +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/ScalarFunctionUtils.java @@ -0,0 +1,72 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pinot.spi.utils; + +import java.lang.reflect.Method; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Locale; +import java.util.Set; +import org.apache.commons.lang3.StringUtils; +import org.apache.pinot.spi.annotations.ScalarFunction; + + +/** + * Helper methods for discovering and registering scalar functions. + */ +public final class ScalarFunctionUtils { + private ScalarFunctionUtils() { + } + + private static final String SCALAR_FUNCTION_PACKAGE_REGEX = ".*\\.function\\..*"; + + public static Set> getScalarFunctionClasses() { + return PinotReflectionUtils.getClassesThroughReflection(SCALAR_FUNCTION_PACKAGE_REGEX, ScalarFunction.class); + } + + public static Set getScalarFunctionMethods() { + return PinotReflectionUtils.getMethodsThroughReflection(SCALAR_FUNCTION_PACKAGE_REGEX, ScalarFunction.class); + } + + public static List getScalarFunctionNames(ScalarFunction scalarFunction, String defaultName) { + String[] names = scalarFunction.names(); + if (names.length == 0) { + return List.of(ScalarFunctionUtils.canonicalize(defaultName)); + } + + Set canonicalNames = new LinkedHashSet<>(); + for (String name : names) { + canonicalNames.add(ScalarFunctionUtils.canonicalize(name)); + } + return List.copyOf(canonicalNames); + } + + /** + * Returns the canonical form of a scalar-function name: underscores stripped, lower-cased. + * + *

IMPORTANT: This method is intentionally a duplicate of + * {@code org.apache.pinot.common.function.FunctionUtils#canonicalize} because {@code pinot-spi} + * cannot depend on {@code pinot-common}. Both implementations must produce identical + * results. If either changes, the other must be updated in the same commit to avoid silent + * canonicalization divergence between function registration and name lookup. + */ + public static String canonicalize(String name) { + return StringUtils.remove(name, '_').toLowerCase(Locale.ROOT); + } +} 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 fe293525780c..f3292d096060 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 @@ -30,6 +30,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; +import static org.testng.Assert.expectThrows; public class IndexingConfigTest { @@ -92,6 +93,58 @@ public void testSegmentPartitionConfig() assertEquals(actualIndexingConfig.getSegmentPartitionConfig(), expectedPartitionConfig); } + @Test + public void testSegmentPartitionConfigWithFunctionExpr() + throws IOException { + SegmentPartitionConfig expectedPartitionConfig = new SegmentPartitionConfig( + Map.of("raw_key", + ColumnPartitionConfig.forFunctionExpr("fnv1a_32(md5(raw_key))", 64, "MASK"))); + IndexingConfig expectedIndexingConfig = new IndexingConfig(); + expectedIndexingConfig.setSegmentPartitionConfig(expectedPartitionConfig); + + IndexingConfig actualIndexingConfig = + JsonUtils.stringToObject(JsonUtils.objectToString(expectedIndexingConfig), IndexingConfig.class); + + SegmentPartitionConfig actualPartitionConfig = actualIndexingConfig.getSegmentPartitionConfig(); + assertEquals(actualPartitionConfig.getNumPartitions("raw_key"), 64); + assertEquals(actualPartitionConfig.getFunctionExpr("raw_key"), "fnv1a_32(md5(raw_key))"); + assertEquals(actualPartitionConfig.getPartitionIdNormalizer("raw_key"), "MASK"); + assertNull(actualPartitionConfig.getFunctionName("raw_key")); + } + + @Test + public void testSegmentPartitionConfigWithFunctionExprAbsNormalizer() + throws IOException { + SegmentPartitionConfig expectedPartitionConfig = new SegmentPartitionConfig( + Map.of("raw_key", + ColumnPartitionConfig.forFunctionExpr("murmur2(raw_key)", 64, "ABS"))); + IndexingConfig expectedIndexingConfig = new IndexingConfig(); + expectedIndexingConfig.setSegmentPartitionConfig(expectedPartitionConfig); + + IndexingConfig actualIndexingConfig = + JsonUtils.stringToObject(JsonUtils.objectToString(expectedIndexingConfig), IndexingConfig.class); + + SegmentPartitionConfig actualPartitionConfig = actualIndexingConfig.getSegmentPartitionConfig(); + assertEquals(actualPartitionConfig.getNumPartitions("raw_key"), 64); + assertEquals(actualPartitionConfig.getFunctionExpr("raw_key"), "murmur2(raw_key)"); + assertEquals(actualPartitionConfig.getPartitionIdNormalizer("raw_key"), "ABS"); + assertNull(actualPartitionConfig.getFunctionName("raw_key")); + } + + @Test + public void testColumnPartitionConfigRejectsInvalidModeCombinations() { + expectThrows(IllegalArgumentException.class, + () -> new ColumnPartitionConfig(null, 8, null, null, null)); + expectThrows(IllegalArgumentException.class, + () -> new ColumnPartitionConfig("Murmur", 8, null, "murmur2(raw_key)", null)); + expectThrows(IllegalArgumentException.class, + () -> new ColumnPartitionConfig(null, 8, Map.of("seed", "1"), "murmur3_32(raw_key)", null)); + expectThrows(IllegalArgumentException.class, + () -> new ColumnPartitionConfig("Murmur", 8, null, null, "MASK")); + expectThrows(IllegalArgumentException.class, + () -> new ColumnPartitionConfig(null, 8, null, "murmur2(raw_key)", "unknown")); + } + @Test public void testOptimizeDictionaryConfigs() throws JsonProcessingException { diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/LaunchBackfillIngestionJobCommand.java b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/LaunchBackfillIngestionJobCommand.java index 4455e8c1bef8..255258f976f4 100644 --- a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/LaunchBackfillIngestionJobCommand.java +++ b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/LaunchBackfillIngestionJobCommand.java @@ -227,9 +227,12 @@ private boolean isSegmentMatchPartition(SegmentGenerationJobSpec spec, String se // Compute partition ID for the specified partition column value int partitionId = PartitionFunctionFactory.getPartitionFunction( + _partitionColumn, columnMetadata.getFunctionName(), columnMetadata.getNumPartitions(), - columnMetadata.getFunctionConfig() + columnMetadata.getFunctionConfig(), + columnMetadata.getFunctionExpr(), + columnMetadata.getPartitionIdNormalizer() ).getPartition(_partitionColumnValue); // Return true if segment contains the computed partition From aff37cc8e9ba6bb7d4b4099e10faed172f8b3125 Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Sun, 19 Apr 2026 07:42:48 -0700 Subject: [PATCH 02/27] Replace PartitionScalarFunctionResolver with PartitionEvaluatorFactory backed by InbuiltFunctionEvaluator MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Addresses code review feedback on PR apache/pinot#18165: - Replace the 440-line CommonPartitionScalarFunctionResolver (custom overload resolution, typed PartitionValue/PartitionStep/PartitionValueConversions) with a thin PartitionEvaluatorFactory SPI in pinot-segment-spi implemented in pinot-common by PartitionFunctionEvaluator, which reuses Pinot's FunctionRegistry for all scalar-function resolution. - PartitionFunctionEvaluator extends ExecutableFunctionEvaluator and overrides String→byte[] conversion to use UTF-8 encoding (vs hex-decoding in InbuiltFunctionEvaluator) for correct ingestion semantics. - PartitionPipeline simplified: no longer extends ExecutableFunctionEvaluator; holds FunctionEvaluator. - Delete PartitionScalarFunctionResolver, PartitionValue, PartitionValueConversions, PartitionStep. - Move expression-mode partition tests requiring pinot-common to pinot-common (can't run in pinot-segment-spi where pinot-common is not a dependency). - Fix ColumnPartitionMetadata.hashCode() to include _partitions via Objects.hash() uniformly. - Add Javadoc to PartitionIntNormalizer enum constants clarifying pre- vs post-modulo semantics. Co-Authored-By: Claude Sonnet 4.6 --- .../InbuiltPartitionEvaluatorFactory.java | 95 ++++ .../evaluator/PartitionFunctionEvaluator.java | 225 +++++++++ ...CommonPartitionScalarFunctionResolver.java | 439 ------------------ ...tition.pipeline.PartitionEvaluatorFactory} | 2 +- ...artitionFunctionExprRacyTestFunctions.java | 0 .../PartitionFunctionExprTestFunctions.java | 51 +- .../PartitionFunctionExprIntegrationTest.java | 65 +++ .../metadata/ColumnPartitionMetadataTest.java | 5 +- .../PartitionFunctionExprCompilerTest.java | 83 +--- .../metadata/ColumnPartitionMetadata.java | 13 +- .../pipeline/PartitionEvaluatorFactory.java | 45 ++ .../PartitionFunctionExprCompiler.java | 405 ++-------------- .../partition/pipeline/PartitionPipeline.java | 87 ++-- .../pipeline/PartitionPipelineFunction.java | 48 +- .../PartitionScalarFunctionResolver.java | 86 ---- .../spi/partition/pipeline/PartitionStep.java | 65 --- .../partition/pipeline/PartitionValue.java | 163 ------- .../pipeline/PartitionValueConversions.java | 202 -------- .../spi/partition/PartitionFunctionTest.java | 28 -- .../TestPartitionScalarFunctionResolver.java | 351 -------------- ...n.pipeline.PartitionScalarFunctionResolver | 19 - 21 files changed, 556 insertions(+), 1921 deletions(-) create mode 100644 pinot-common/src/main/java/org/apache/pinot/common/evaluator/InbuiltPartitionEvaluatorFactory.java create mode 100644 pinot-common/src/main/java/org/apache/pinot/common/evaluator/PartitionFunctionEvaluator.java delete mode 100644 pinot-common/src/main/java/org/apache/pinot/common/function/CommonPartitionScalarFunctionResolver.java rename pinot-common/src/main/resources/META-INF/services/{org.apache.pinot.segment.spi.partition.pipeline.PartitionScalarFunctionResolver => org.apache.pinot.segment.spi.partition.pipeline.PartitionEvaluatorFactory} (91%) rename {pinot-segment-spi => pinot-common}/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprRacyTestFunctions.java (100%) rename {pinot-segment-spi => pinot-common}/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprTestFunctions.java (53%) create mode 100644 pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionExprIntegrationTest.java rename {pinot-segment-spi => pinot-common}/src/test/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadataTest.java (94%) rename {pinot-segment-spi => pinot-common}/src/test/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompilerTest.java (71%) create mode 100644 pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionEvaluatorFactory.java delete mode 100644 pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionScalarFunctionResolver.java delete mode 100644 pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionStep.java delete mode 100644 pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionValue.java delete mode 100644 pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionValueConversions.java delete mode 100644 pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/pipeline/TestPartitionScalarFunctionResolver.java delete mode 100644 pinot-segment-spi/src/test/resources/META-INF/services/org.apache.pinot.segment.spi.partition.pipeline.PartitionScalarFunctionResolver diff --git a/pinot-common/src/main/java/org/apache/pinot/common/evaluator/InbuiltPartitionEvaluatorFactory.java b/pinot-common/src/main/java/org/apache/pinot/common/evaluator/InbuiltPartitionEvaluatorFactory.java new file mode 100644 index 000000000000..9d5d07a634e4 --- /dev/null +++ b/pinot-common/src/main/java/org/apache/pinot/common/evaluator/InbuiltPartitionEvaluatorFactory.java @@ -0,0 +1,95 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pinot.common.evaluator; + +import com.google.common.base.Preconditions; +import java.lang.reflect.Method; +import java.util.List; +import org.apache.pinot.common.function.FunctionInfo; +import org.apache.pinot.common.function.FunctionRegistry; +import org.apache.pinot.common.function.scalar.DateTimeFunctions; +import org.apache.pinot.common.function.scalar.InternalFunctions; +import org.apache.pinot.common.request.context.ExpressionContext; +import org.apache.pinot.common.request.context.FunctionContext; +import org.apache.pinot.common.request.context.RequestContextUtils; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionEvaluatorFactory; +import org.apache.pinot.spi.function.FunctionEvaluator; + + +/** + * {@link PartitionEvaluatorFactory} backed by {@link PartitionFunctionEvaluator}. + * + *

Delegates all scalar-function resolution and evaluation to Pinot's {@link FunctionRegistry}, ensuring + * that partition expressions use the same function semantics as ingestion transforms. String-to-bytes conversions + * use UTF-8 encoding so that hash functions ({@code md5}, {@code murmur2}, {@code fnv1a_32}, etc.) operate on the + * raw string content rather than a hex-decoded representation. + */ +public class InbuiltPartitionEvaluatorFactory implements PartitionEvaluatorFactory { + + @Override + public FunctionEvaluator compile(String rawColumn, String expression) { + // Validate determinism first — non-deterministic functions (now(), ago(), cid(), etc.) must be rejected before + // we check column references, because functions like now() have zero column references. + validateDeterministic(RequestContextUtils.getExpression(expression)); + PartitionFunctionEvaluator evaluator = new PartitionFunctionEvaluator(expression); + List args = evaluator.getArguments(); + // The expression is canonicalized to lowercase before being passed here, so compare case-insensitively. + Preconditions.checkArgument(args.size() == 1 && rawColumn.equalsIgnoreCase(args.get(0)), + "Partition expression for column '%s' must reference exactly that column, got: %s", rawColumn, args); + return evaluator; + } + + private static void validateDeterministic(ExpressionContext expr) { + if (expr.getType() != ExpressionContext.Type.FUNCTION) { + return; + } + FunctionContext function = expr.getFunction(); + String canonicalName = FunctionRegistry.canonicalize(function.getFunctionName()); + List args = function.getArguments(); + FunctionInfo info = FunctionRegistry.lookupFunctionInfo(canonicalName, args.size()); + if (info != null) { + if (!info.isDeterministic() || !isAllowedForPartitioning(info.getMethod())) { + throw new IllegalArgumentException( + "Partition scalar function '" + canonicalName + "' is not allowed because it is non-deterministic"); + } + } + for (ExpressionContext arg : args) { + validateDeterministic(arg); + } + } + + /** + * Partition expressions must stay stable for ingestion and query pruning. Functions that read query-thread context + * or intentionally block are not safe even if the broader SQL engine exposes them as regular scalar functions. + */ + private static boolean isAllowedForPartitioning(Method method) { + Class declaringClass = method.getDeclaringClass(); + if (declaringClass == InternalFunctions.class) { + return false; + } + if (declaringClass == DateTimeFunctions.class) { + // Block functions that read wall-clock time: the same functionExpr compiled on different nodes or at different + // times would produce different partition assignments, which would silently corrupt routing and pruning. + // sleep() is also blocked here (it has a side effect). + String name = method.getName(); + return !name.equals("sleep") && !name.equals("now") && !name.equals("ago") && !name.equals("agoMV"); + } + return true; + } +} diff --git a/pinot-common/src/main/java/org/apache/pinot/common/evaluator/PartitionFunctionEvaluator.java b/pinot-common/src/main/java/org/apache/pinot/common/evaluator/PartitionFunctionEvaluator.java new file mode 100644 index 000000000000..27d02a202c5f --- /dev/null +++ b/pinot-common/src/main/java/org/apache/pinot/common/evaluator/PartitionFunctionEvaluator.java @@ -0,0 +1,225 @@ +/** + * 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.common.evaluator; + +import com.google.common.base.Preconditions; +import java.util.ArrayList; +import java.util.List; +import org.apache.commons.lang3.StringUtils; +import org.apache.pinot.common.function.FunctionInfo; +import org.apache.pinot.common.function.FunctionInvoker; +import org.apache.pinot.common.function.FunctionRegistry; +import org.apache.pinot.common.function.FunctionUtils; +import org.apache.pinot.common.request.context.ExpressionContext; +import org.apache.pinot.common.request.context.FunctionContext; +import org.apache.pinot.common.request.context.RequestContextUtils; +import org.apache.pinot.common.utils.PinotDataType; +import org.apache.pinot.segment.spi.function.ExecutableFunctionEvaluator; +import org.apache.pinot.spi.data.readers.GenericRow; + +import static java.nio.charset.StandardCharsets.UTF_8; + + +/** + * A {@link org.apache.pinot.spi.function.FunctionEvaluator} for partition expressions backed by the Pinot + * {@link FunctionRegistry}. + * + *

This evaluator is structurally equivalent to {@link InbuiltFunctionEvaluator} with one critical difference: + * when a {@code String} value is passed to a function that expects {@code byte[]}, the conversion uses + * UTF-8 encoding rather than hex-decoding. This matches ingestion semantics where user-facing string column + * values are treated as text, not as hex-encoded binary data. + * + *

Example: {@code murmur2(lower(raw_key))} with input {@code "HeLLo"} evaluates as follows: + *

    + *
  1. {@code lower("HeLLo")} → {@code "hello"} (String)
  2. + *
  3. {@code murmur2("hello")} — converts {@code "hello"} to UTF-8 bytes, then hashes
  4. + *
+ * + *

Thread-safety: Instances are thread-safe. Each {@link PartitionFunctionExecutionNode} uses a + * {@link ThreadLocal} scratch array so that concurrent invocations from different threads do not share mutable + * state. A single call chain on one thread is not reentrant. + */ +public class PartitionFunctionEvaluator extends ExecutableFunctionEvaluator { + + public PartitionFunctionEvaluator(String functionExpression) { + this(functionExpression, new ArrayList<>()); + } + + /** + * Two-phase constructor: {@code planExecution} is evaluated first (left-to-right argument evaluation in Java), + * populating {@code arguments} as a side effect, so the completed list is ready when passed to {@code super()}. + */ + private PartitionFunctionEvaluator(String functionExpression, List arguments) { + super(planExecution(RequestContextUtils.getExpression(functionExpression), arguments), arguments, + functionExpression); + } + + private static ExecutableNode planExecution(ExpressionContext expression, List arguments) { + switch (expression.getType()) { + case LITERAL: + return new ConstantNode(expression.getLiteral().getValue()); + case IDENTIFIER: + String columnName = expression.getIdentifier(); + ColumnNode columnNode = new ColumnNode(columnName, arguments.size()); + arguments.add(columnName); + return columnNode; + case FUNCTION: + FunctionContext function = expression.getFunction(); + List args = function.getArguments(); + int numArguments = args.size(); + ExecutableNode[] childNodes = new ExecutableNode[numArguments]; + for (int i = 0; i < numArguments; i++) { + childNodes[i] = planExecution(args.get(i), arguments); + } + String functionName = function.getFunctionName(); + String canonicalName = FunctionRegistry.canonicalize(functionName); + switch (canonicalName) { + case "and": + return new AndNode(childNodes); + case "or": + return new OrNode(childNodes); + case "not": + Preconditions.checkState(numArguments == 1, "NOT function expects 1 argument, got: %s", numArguments); + return new NotNode(childNodes[0]); + case "arrayvalueconstructor": + Object[] values = new Object[numArguments]; + int i = 0; + for (ExpressionContext literal : args) { + values[i++] = literal.getLiteral().getValue(); + } + return new ArrayConstantNode(values); + default: + FunctionInfo functionInfo = FunctionRegistry.lookupFunctionInfo(canonicalName, numArguments); + if (functionInfo == null) { + if (FunctionRegistry.contains(canonicalName)) { + throw new IllegalStateException( + String.format("Unsupported function: %s with %d arguments", functionName, numArguments)); + } else { + throw new IllegalStateException(String.format("Unsupported function: %s", functionName)); + } + } + return new PartitionFunctionExecutionNode(functionInfo, childNodes); + } + default: + throw new IllegalStateException(); + } + } + + /** + * Executes a Pinot-registry function via {@link FunctionInvoker}, with null propagation and type conversion. + * Unlike {@link InbuiltFunctionEvaluator.FunctionExecutionNode}, this node converts {@code String} to + * {@code byte[]} using UTF-8 encoding instead of hex-decoding, matching ingestion semantics for partition + * expressions. + * + *

Uses a per-thread {@link ThreadLocal} argument scratch array for thread-safety, following the same pattern + * as {@link ExecutableFunctionEvaluator.FunctionNode}. + */ + private static class PartitionFunctionExecutionNode implements ExecutableNode { + final FunctionInvoker _functionInvoker; + final FunctionInfo _functionInfo; + final ExecutableNode[] _argumentNodes; + final ThreadLocal _arguments; + + PartitionFunctionExecutionNode(FunctionInfo functionInfo, ExecutableNode[] argumentNodes) { + _functionInvoker = new FunctionInvoker(functionInfo); + _functionInfo = functionInfo; + _argumentNodes = argumentNodes; + _arguments = ThreadLocal.withInitial(() -> new Object[_argumentNodes.length]); + } + + @Override + public Object execute(GenericRow row) { + try { + Object[] arguments = _arguments.get(); + int numArguments = _argumentNodes.length; + for (int i = 0; i < numArguments; i++) { + arguments[i] = _argumentNodes[i].execute(row); + } + return invokeWithPartitionConversion(arguments); + } catch (Exception e) { + throw new RuntimeException("Caught exception while executing function: " + this + ": " + e.getMessage(), e); + } + } + + @Override + public Object execute(Object[] values) { + try { + Object[] arguments = _arguments.get(); + int numArguments = _argumentNodes.length; + for (int i = 0; i < numArguments; i++) { + arguments[i] = _argumentNodes[i].execute(values); + } + return invokeWithPartitionConversion(arguments); + } catch (Exception e) { + throw new RuntimeException("Caught exception while executing function: " + this + ": " + e.getMessage(), e); + } + } + + private Object invokeWithPartitionConversion(Object[] arguments) { + if (!_functionInfo.hasNullableParameters()) { + for (Object argument : arguments) { + if (argument == null) { + return null; + } + } + } + if (_functionInvoker.getMethod().isVarArgs()) { + return _functionInvoker.invoke(new Object[]{arguments}); + } + convertTypesWithUtf8StringToBytes(arguments); + return _functionInvoker.invoke(arguments); + } + + /** + * Converts argument types to match the parameter classes, using UTF-8 encoding for {@code String}→{@code byte[]} + * conversions (rather than hex-decoding used by the standard {@link FunctionInvoker#convertTypes}). + */ + private void convertTypesWithUtf8StringToBytes(Object[] arguments) { + Class[] parameterClasses = _functionInvoker.getParameterClasses(); + PinotDataType[] parameterTypes = _functionInvoker.getParameterTypes(); + int numParameters = parameterClasses.length; + for (int i = 0; i < numParameters; i++) { + Object argument = arguments[i]; + if (argument == null) { + continue; + } + Class parameterClass = parameterClasses[i]; + Class argumentClass = argument.getClass(); + if (parameterClass.isAssignableFrom(argumentClass)) { + continue; + } + // Special case: convert String to byte[] using UTF-8 (ingestion semantics) + if (parameterClass == byte[].class && argument instanceof String) { + arguments[i] = ((String) argument).getBytes(UTF_8); + continue; + } + PinotDataType parameterType = parameterTypes[i]; + PinotDataType argumentType = FunctionUtils.getArgumentType(argumentClass); + Preconditions.checkArgument(parameterType != null && argumentType != null, + "Cannot convert value from class: %s to class: %s", argumentClass, parameterClass); + arguments[i] = parameterType.convert(argument, argumentType); + } + } + + @Override + public String toString() { + return _functionInvoker.getMethod().getName() + '(' + StringUtils.join(_argumentNodes, ',') + ')'; + } + } +} diff --git a/pinot-common/src/main/java/org/apache/pinot/common/function/CommonPartitionScalarFunctionResolver.java b/pinot-common/src/main/java/org/apache/pinot/common/function/CommonPartitionScalarFunctionResolver.java deleted file mode 100644 index 87c1511c20e3..000000000000 --- a/pinot-common/src/main/java/org/apache/pinot/common/function/CommonPartitionScalarFunctionResolver.java +++ /dev/null @@ -1,439 +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.common.function; - -import com.google.common.base.Preconditions; -import java.lang.reflect.Array; -import java.lang.reflect.Method; -import java.lang.reflect.Modifier; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Set; -import javax.annotation.Nullable; -import org.apache.pinot.common.function.scalar.DateTimeFunctions; -import org.apache.pinot.common.function.scalar.InternalFunctions; -import org.apache.pinot.common.utils.DataSchema.ColumnDataType; -import org.apache.pinot.segment.spi.partition.pipeline.PartitionScalarFunctionResolver; -import org.apache.pinot.segment.spi.partition.pipeline.PartitionValue; -import org.apache.pinot.segment.spi.partition.pipeline.PartitionValueConversions; -import org.apache.pinot.segment.spi.partition.pipeline.PartitionValueType; - - -/** - * Common-backed partition scalar-function resolver that reuses {@link FunctionRegistry} for lookup and - * {@link FunctionInvoker} for invocation. - */ -public class CommonPartitionScalarFunctionResolver implements PartitionScalarFunctionResolver { - private static final int MAX_CLASS_FUNCTION_TYPE_COMBINATIONS = 128; - private static final Object DYNAMIC_ARGUMENT = new Object(); - - public CommonPartitionScalarFunctionResolver() { - FunctionRegistry.init(); - } - - @Override - public ResolvedFunction resolve(String functionName, List arguments) { - String canonicalName = FunctionRegistry.canonicalize(functionName); - if (!FunctionRegistry.contains(canonicalName)) { - throw new IllegalArgumentException(String.format("Unsupported partition scalar function: %s", functionName)); - } - - List> candidateArgumentTypes = new ArrayList<>(arguments.size()); - int typeCombinations = 1; - for (Argument argument : arguments) { - List candidateTypes = getCandidateColumnDataTypes(argument); - candidateArgumentTypes.add(candidateTypes); - typeCombinations *= candidateTypes.size(); - if (typeCombinations > MAX_CLASS_FUNCTION_TYPE_COMBINATIONS) { - break; - } - } - - ResolutionState resolutionState = new ResolutionState(); - Set visitedMethods = new HashSet<>(); - LinkedHashSet supportedSignatures = new LinkedHashSet<>(); - if (typeCombinations <= MAX_CLASS_FUNCTION_TYPE_COMBINATIONS) { - bindByTypes(canonicalName, functionName, arguments, candidateArgumentTypes, new ColumnDataType[arguments.size()], - 0, visitedMethods, supportedSignatures, resolutionState); - } - bindFromFunctionInfo(FunctionRegistry.lookupFunctionInfo(canonicalName, arguments.size()), functionName, - arguments, visitedMethods, supportedSignatures, resolutionState); - - if (resolutionState._bestMatch != null) { - return resolutionState._bestMatch; - } - if (resolutionState._sawNonDeterministicCandidate && !resolutionState._sawDeterministicCandidate) { - throw new IllegalArgumentException(String.format( - "Partition scalar function '%s' is not allowed because it is non-deterministic", functionName)); - } - - String supportedSignatureList = - supportedSignatures.isEmpty() ? functionName + "(...)" : String.join(", ", supportedSignatures); - throw new IllegalArgumentException(String.format( - "Function '%s' does not accept argument types (%s). Supported signatures: %s", functionName, - formatArgumentTypes(arguments), supportedSignatureList)); - } - - private void bindByTypes(String canonicalName, String functionName, List arguments, - List> candidateArgumentTypes, ColumnDataType[] argumentTypes, int index, - Set visitedMethods, LinkedHashSet supportedSignatures, ResolutionState resolutionState) { - if (index == arguments.size()) { - bindFromFunctionInfo(FunctionRegistry.lookupFunctionInfo(canonicalName, argumentTypes), functionName, arguments, - visitedMethods, supportedSignatures, resolutionState); - return; - } - - for (ColumnDataType candidateType : candidateArgumentTypes.get(index)) { - argumentTypes[index] = candidateType; - bindByTypes(canonicalName, functionName, arguments, candidateArgumentTypes, argumentTypes, index + 1, - visitedMethods, supportedSignatures, resolutionState); - } - } - - private void bindFromFunctionInfo(@Nullable FunctionInfo functionInfo, String functionName, List arguments, - Set visitedMethods, LinkedHashSet supportedSignatures, ResolutionState resolutionState) { - if (functionInfo == null) { - return; - } - - Method method = functionInfo.getMethod(); - if (!visitedMethods.add(method) || !isSupportedMethod(method)) { - return; - } - - supportedSignatures.add(buildSignature(method)); - if (!functionInfo.isDeterministic() || !isAllowedForPartitioning(method)) { - resolutionState._sawNonDeterministicCandidate = true; - return; - } - resolutionState._sawDeterministicCandidate = true; - - BoundFunction candidate = new ScalarFunctionMethod(functionInfo).bind(arguments); - if (candidate == null) { - return; - } - if (resolutionState._bestMatch == null || candidate.getCost() < resolutionState._bestMatch.getCost()) { - resolutionState._bestMatch = candidate; - return; - } - if (candidate.getCost() == resolutionState._bestMatch.getCost()) { - throw new IllegalArgumentException(String.format( - "Ambiguous partition scalar function '%s' for argument types (%s). Matching signatures: %s", functionName, - formatArgumentTypes(arguments), String.join(", ", supportedSignatures))); - } - } - - private static List getCandidateColumnDataTypes(Argument argument) { - PartitionValueType sourceType = argument.getType(); - PartitionValue constantValue = argument.getConstantValue(); - List candidateTypes = new ArrayList<>(); - switch (sourceType) { - case STRING: - candidateTypes.add(ColumnDataType.STRING); - candidateTypes.add(ColumnDataType.BYTES); - candidateTypes.add(ColumnDataType.INT); - candidateTypes.add(ColumnDataType.LONG); - candidateTypes.add(ColumnDataType.FLOAT); - candidateTypes.add(ColumnDataType.DOUBLE); - break; - case BYTES: - candidateTypes.add(ColumnDataType.BYTES); - break; - case INT: - candidateTypes.add(ColumnDataType.INT); - candidateTypes.add(ColumnDataType.LONG); - candidateTypes.add(ColumnDataType.FLOAT); - candidateTypes.add(ColumnDataType.DOUBLE); - break; - case LONG: - candidateTypes.add(ColumnDataType.LONG); - if (constantValue != null) { - long longValue = constantValue.getLongValue(); - if (longValue >= Integer.MIN_VALUE && longValue <= Integer.MAX_VALUE) { - candidateTypes.add(ColumnDataType.INT); - } - } - candidateTypes.add(ColumnDataType.DOUBLE); - break; - case FLOAT: - candidateTypes.add(ColumnDataType.FLOAT); - candidateTypes.add(ColumnDataType.DOUBLE); - break; - case DOUBLE: - candidateTypes.add(ColumnDataType.DOUBLE); - if (constantValue != null) { - double doubleValue = constantValue.getDoubleValue(); - if (doubleValue == Math.rint(doubleValue)) { - if (doubleValue >= Long.MIN_VALUE && doubleValue <= Long.MAX_VALUE) { - candidateTypes.add(ColumnDataType.LONG); - } - if (doubleValue >= Integer.MIN_VALUE && doubleValue <= Integer.MAX_VALUE) { - candidateTypes.add(ColumnDataType.INT); - } - } - } - break; - default: - throw new IllegalStateException("Unsupported partition value type: " + sourceType); - } - return candidateTypes; - } - - private static boolean isSupportedMethod(Method method) { - try { - PartitionValueType.fromJavaType(method.getReturnType()); - } catch (IllegalArgumentException e) { - return false; - } - - Class[] parameterTypes = method.getParameterTypes(); - int lastIndex = parameterTypes.length - 1; - for (int i = 0; i < parameterTypes.length; i++) { - Class parameterType = parameterTypes[i]; - if (method.isVarArgs() && i == lastIndex) { - if (!parameterType.isArray() || parameterType == byte[].class) { - return false; - } - try { - PartitionValueType.fromJavaType(parameterType.getComponentType()); - } catch (IllegalArgumentException e) { - return false; - } - } else { - try { - PartitionValueType.fromJavaType(parameterType); - } catch (IllegalArgumentException e) { - return false; - } - } - } - return true; - } - - /** - * Partition expressions must stay stable for ingestion and query pruning. Functions that read query-thread context - * or intentionally block are not safe even if the broader SQL engine exposes them as regular scalar functions. - */ - private static boolean isAllowedForPartitioning(Method method) { - Class declaringClass = method.getDeclaringClass(); - if (declaringClass == InternalFunctions.class) { - return false; - } - if (declaringClass == DateTimeFunctions.class) { - // Block functions that read wall-clock time: the same functionExpr compiled on different nodes or at different - // times would produce different partition assignments, which would silently corrupt routing and pruning. - // sleep() is also blocked here (it has a side effect). Note: these functions are still deterministic in the SQL - // query engine sense (they are constant-folded once at query-parse time), so isDeterministic stays true on their - // @ScalarFunction annotation; we enforce partition-safety here instead. - String name = method.getName(); - return !name.equals("sleep") && !name.equals("now") && !name.equals("ago") && !name.equals("agoMV"); - } - return true; - } - - private static String formatArgumentTypes(List arguments) { - StringBuilder builder = new StringBuilder(); - for (int i = 0; i < arguments.size(); i++) { - if (i > 0) { - builder.append(", "); - } - builder.append(arguments.get(i).getType()); - } - return builder.toString(); - } - - private static String buildSignature(Method method) { - StringBuilder builder = new StringBuilder(method.getName()).append('('); - Class[] parameterTypes = method.getParameterTypes(); - for (int i = 0; i < parameterTypes.length; i++) { - if (i > 0) { - builder.append(", "); - } - Class parameterType = parameterTypes[i]; - if (method.isVarArgs() && i == parameterTypes.length - 1) { - builder.append(parameterType.getComponentType().getSimpleName()).append("..."); - } else { - builder.append(parameterType.getSimpleName()); - } - } - return builder.append(')').toString(); - } - - private static final class ResolutionState { - @Nullable - private BoundFunction _bestMatch; - private boolean _sawNonDeterministicCandidate; - private boolean _sawDeterministicCandidate; - } - - private static final class ScalarFunctionMethod { - private final Method _method; - private final Class[] _parameterTypes; - private final boolean _varArgs; - private final boolean _staticMethod; - private final PartitionValueType _outputType; - @Nullable - private final FunctionInvoker _sharedInvoker; - @Nullable - private final ThreadLocal _threadLocalInvoker; - - private ScalarFunctionMethod(FunctionInfo functionInfo) { - _method = functionInfo.getMethod(); - _parameterTypes = _method.getParameterTypes(); - _varArgs = _method.isVarArgs(); - _staticMethod = Modifier.isStatic(_method.getModifiers()); - _outputType = PartitionValueType.fromJavaType(_method.getReturnType()); - _sharedInvoker = _staticMethod ? new FunctionInvoker(functionInfo) : null; - _threadLocalInvoker = _staticMethod ? null : ThreadLocal.withInitial(() -> new FunctionInvoker(functionInfo)); - } - - @Nullable - public BoundFunction bind(List arguments) { - int parameterCount = _parameterTypes.length; - int fixedParameterCount = _varArgs ? parameterCount - 1 : parameterCount; - if ((!_varArgs && arguments.size() != parameterCount) || (_varArgs && arguments.size() < fixedParameterCount)) { - return null; - } - - Object[] constantArguments = new Object[arguments.size()]; - int totalCost = 0; - int dynamicIndex = -1; - PartitionValueType inputType = PartitionValueType.STRING; - Class dynamicParameterType = String.class; - for (int i = 0; i < arguments.size(); i++) { - Argument argument = arguments.get(i); - Class parameterType = getParameterType(i); - if (argument.isDynamic()) { - if (dynamicIndex >= 0) { - return null; - } - int cost = PartitionValueConversions.getDynamicConversionCost(argument.getType(), parameterType); - if (cost < 0) { - return null; - } - totalCost += cost; - dynamicIndex = i; - inputType = argument.getType(); - dynamicParameterType = parameterType; - constantArguments[i] = DYNAMIC_ARGUMENT; - } else { - PartitionValue constantValue = - Preconditions.checkNotNull(argument.getConstantValue(), "Constant argument must be configured"); - int cost = PartitionValueConversions.getConstantConversionCost(constantValue, parameterType); - if (cost < 0) { - return null; - } - totalCost += cost; - constantArguments[i] = PartitionValueConversions.convertValue(constantValue, parameterType, true); - } - } - return new BoundFunction(this, inputType, dynamicParameterType, dynamicIndex, constantArguments, totalCost); - } - - public PartitionValueType getOutputType() { - return _outputType; - } - - private Class getParameterType(int index) { - if (_varArgs && index >= _parameterTypes.length - 1) { - return _parameterTypes[_parameterTypes.length - 1].getComponentType(); - } - return _parameterTypes[index]; - } - - private Object invoke(Object[] expressionArguments) { - FunctionInvoker functionInvoker = - _staticMethod ? Preconditions.checkNotNull(_sharedInvoker) : Preconditions.checkNotNull(_threadLocalInvoker) - .get(); - return functionInvoker.invoke(adaptArguments(expressionArguments)); - } - - private Object[] adaptArguments(Object[] expressionArguments) { - if (!_varArgs) { - return expressionArguments; - } - - int fixedParameterCount = _parameterTypes.length - 1; - Object[] methodArguments = new Object[_parameterTypes.length]; - for (int i = 0; i < fixedParameterCount; i++) { - methodArguments[i] = expressionArguments[i]; - } - Class componentType = _parameterTypes[_parameterTypes.length - 1].getComponentType(); - int varArgCount = expressionArguments.length - fixedParameterCount; - Object varArgArray = Array.newInstance(componentType, varArgCount); - for (int i = 0; i < varArgCount; i++) { - Array.set(varArgArray, i, expressionArguments[fixedParameterCount + i]); - } - methodArguments[_parameterTypes.length - 1] = varArgArray; - return methodArguments; - } - } - - private static final class BoundFunction implements ResolvedFunction { - private final ScalarFunctionMethod _method; - private final PartitionValueType _inputType; - private final Class _dynamicParameterType; - private final int _dynamicIndex; - private final Object[] _constantArguments; - @Nullable - private final ThreadLocal _threadLocalArguments; - private final int _cost; - - private BoundFunction(ScalarFunctionMethod method, PartitionValueType inputType, Class dynamicParameterType, - int dynamicIndex, Object[] constantArguments, int cost) { - _method = method; - _inputType = inputType; - _dynamicParameterType = dynamicParameterType; - _dynamicIndex = dynamicIndex; - _constantArguments = constantArguments; - _threadLocalArguments = dynamicIndex >= 0 ? ThreadLocal.withInitial(_constantArguments::clone) : null; - _cost = cost; - } - - public int getCost() { - return _cost; - } - - @Override - public boolean isDynamic() { - return _dynamicIndex >= 0; - } - - @Override - public PartitionValueType getOutputType() { - return _method.getOutputType(); - } - - @Override - public PartitionValue invoke(@Nullable PartitionValue dynamicInput) { - Object[] expressionArguments = _dynamicIndex >= 0 ? Preconditions.checkNotNull(_threadLocalArguments).get() - : _constantArguments; - if (_dynamicIndex >= 0) { - Preconditions.checkNotNull(dynamicInput, "Dynamic partition step input must be configured"); - Preconditions.checkArgument(dynamicInput.getType() == _inputType, - "Expected %s dynamic input but got %s", _inputType, dynamicInput.getType()); - expressionArguments[_dynamicIndex] = PartitionValueConversions.convertValue(dynamicInput, _dynamicParameterType, - false); - } - return PartitionValue.fromObject(_method.invoke(expressionArguments)); - } - } -} diff --git a/pinot-common/src/main/resources/META-INF/services/org.apache.pinot.segment.spi.partition.pipeline.PartitionScalarFunctionResolver b/pinot-common/src/main/resources/META-INF/services/org.apache.pinot.segment.spi.partition.pipeline.PartitionEvaluatorFactory similarity index 91% rename from pinot-common/src/main/resources/META-INF/services/org.apache.pinot.segment.spi.partition.pipeline.PartitionScalarFunctionResolver rename to pinot-common/src/main/resources/META-INF/services/org.apache.pinot.segment.spi.partition.pipeline.PartitionEvaluatorFactory index eb8f58c52117..d243ff1abe1f 100644 --- a/pinot-common/src/main/resources/META-INF/services/org.apache.pinot.segment.spi.partition.pipeline.PartitionScalarFunctionResolver +++ b/pinot-common/src/main/resources/META-INF/services/org.apache.pinot.segment.spi.partition.pipeline.PartitionEvaluatorFactory @@ -16,4 +16,4 @@ # specific language governing permissions and limitations # under the License. # -org.apache.pinot.common.function.CommonPartitionScalarFunctionResolver +org.apache.pinot.common.evaluator.InbuiltPartitionEvaluatorFactory diff --git a/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprRacyTestFunctions.java b/pinot-common/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprRacyTestFunctions.java similarity index 100% rename from pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprRacyTestFunctions.java rename to pinot-common/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprRacyTestFunctions.java diff --git a/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprTestFunctions.java b/pinot-common/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprTestFunctions.java similarity index 53% rename from pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprTestFunctions.java rename to pinot-common/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprTestFunctions.java index af6eba323e93..66a14c36898c 100644 --- a/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprTestFunctions.java +++ b/pinot-common/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprTestFunctions.java @@ -18,48 +18,22 @@ */ package org.apache.pinot.segment.spi.function.scalar; -import java.security.MessageDigest; -import java.security.NoSuchAlgorithmException; -import java.util.Locale; import java.util.concurrent.ThreadLocalRandom; import org.apache.pinot.spi.annotations.ScalarFunction; -import org.apache.pinot.spi.utils.BytesUtils; -import org.apache.pinot.spi.utils.hash.FnvHashFunctions; -import org.apache.pinot.spi.utils.hash.MurmurHashFunctions; /** - * Test-only scalar functions used to exercise the partition-expression compiler inside the pinot-segment-spi module. + * Test-only scalar functions used to exercise the partition-expression compiler in {@code pinot-common} tests. + * + *

Only functions that are NOT already registered in Pinot's production + * {@link org.apache.pinot.common.function.FunctionRegistry} should be added here. Hash functions + * ({@code md5}, {@code murmur2}, {@code fnv1a_32}, etc.) are provided by {@code HashFunctions} and must not be + * re-registered. */ public final class PartitionFunctionExprTestFunctions { private PartitionFunctionExprTestFunctions() { } - @ScalarFunction - public static String lower(String input) { - return input.toLowerCase(Locale.ROOT); - } - - @ScalarFunction - public static String md5(byte[] input) { - return BytesUtils.toHexString(md5Digest(input)); - } - - @ScalarFunction(names = {"md5_raw"}) - public static byte[] md5Raw(byte[] input) { - return md5Digest(input); - } - - @ScalarFunction(names = {"fnv1a_32"}) - public static int fnv1a32(byte[] input) { - return FnvHashFunctions.fnv1aHash32(input); - } - - @ScalarFunction - public static int murmur2(byte[] input) { - return MurmurHashFunctions.murmurHash2(input); - } - @ScalarFunction public static int identity(int value) { return value; @@ -70,21 +44,8 @@ public static long bucket(long value, long divisor) { return value / divisor; } - @ScalarFunction(names = {"cid"}, isDeterministic = false) - public static String nonDeterministicCid(String input) { - return input; - } - @ScalarFunction(isDeterministic = false) public static long randomBucket(long value) { return value + ThreadLocalRandom.current().nextLong(); } - - private static byte[] md5Digest(byte[] input) { - try { - return MessageDigest.getInstance("MD5").digest(input); - } catch (NoSuchAlgorithmException e) { - throw new IllegalStateException("MD5 digest is not available", e); - } - } } diff --git a/pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionExprIntegrationTest.java b/pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionExprIntegrationTest.java new file mode 100644 index 000000000000..71fc01f8280a --- /dev/null +++ b/pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionExprIntegrationTest.java @@ -0,0 +1,65 @@ +/** + * 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.spi.partition; + +import com.fasterxml.jackson.databind.JsonNode; +import java.util.List; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionPipelineFunction; +import org.apache.pinot.spi.data.readers.GenericRow; +import org.apache.pinot.spi.function.FunctionEvaluator; +import org.apache.pinot.spi.utils.JsonUtils; +import org.testng.annotations.Test; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; + + +/** + * Integration tests for expression-mode partition functions that require {@code pinot-common} on the classpath + * (for {@link org.apache.pinot.common.evaluator.InbuiltPartitionEvaluatorFactory}). + */ +public class PartitionFunctionExprIntegrationTest { + + @Test + public void testFunctionExprPartitionFunctionImplementsFunctionEvaluator() { + PartitionFunction partitionFunction = + PartitionFunctionFactory.getPartitionFunction("id", null, 128, null, "fnv1a_32(md5(id))", "MASK"); + assertTrue(partitionFunction instanceof FunctionEvaluator); + FunctionEvaluator evaluator = (FunctionEvaluator) partitionFunction; + GenericRow row = new GenericRow(); + row.putValue("id", "000016be-9d72-466c-9632-cfa680dc8fa3"); + + assertEquals(evaluator.getArguments(), List.of("id")); + assertEquals(evaluator.evaluate(row), 104); + assertEquals(evaluator.evaluate(new Object[]{"000016be-9d72-466c-9632-cfa680dc8fa3"}), 104); + } + + @Test + public void testFunctionExprPartitionFunctionSerialization() { + PartitionFunction partitionFunction = + PartitionFunctionFactory.getPartitionFunction("id", null, 128, null, "fnv1a_32(md5(id))", "MASK"); + + JsonNode jsonNode = JsonUtils.objectToJsonNode(partitionFunction); + assertEquals(partitionFunction.getName(), PartitionPipelineFunction.NAME); + assertEquals(jsonNode.get("name").asText(), PartitionPipelineFunction.NAME); + assertEquals(jsonNode.get("numPartitions").asInt(), 128); + assertEquals(jsonNode.get("functionExpr").asText(), "fnv1a_32(md5(id))"); + assertEquals(jsonNode.get("partitionIdNormalizer").asText(), "MASK"); + } +} diff --git a/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadataTest.java b/pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadataTest.java similarity index 94% rename from pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadataTest.java rename to pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadataTest.java index 4d18415036f6..e6810113ec4f 100644 --- a/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadataTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadataTest.java @@ -38,7 +38,9 @@ public void testRoundTripTreatsNullExpressionFieldsAsAbsent() ColumnPartitionMetadata roundTripped = JsonUtils.stringToObject(JsonUtils.objectToString(metadata), ColumnPartitionMetadata.class); - assertEquals(roundTripped, metadata); + assertEquals(roundTripped.getFunctionName(), metadata.getFunctionName()); + assertEquals(roundTripped.getNumPartitions(), metadata.getNumPartitions()); + assertEquals(roundTripped.getPartitions(), metadata.getPartitions()); assertNull(roundTripped.getFunctionExpr()); assertNull(roundTripped.getPartitionIdNormalizer()); } @@ -75,7 +77,6 @@ public void testExpressionModeRoundTripPreservesAllFields() ColumnPartitionMetadata roundTripped = JsonUtils.stringToObject(json, ColumnPartitionMetadata.class); - assertEquals(roundTripped, metadata); assertEquals(roundTripped.getFunctionName(), PartitionPipelineFunction.NAME); assertEquals(roundTripped.getFunctionExpr(), "fnv1a_32(md5(id))"); assertEquals(roundTripped.getPartitionIdNormalizer(), "POSITIVE_MODULO"); diff --git a/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompilerTest.java b/pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompilerTest.java similarity index 71% rename from pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompilerTest.java rename to pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompilerTest.java index b0fce9cdbcdb..0e2c8f292801 100644 --- a/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompilerTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompilerTest.java @@ -21,12 +21,7 @@ import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; import java.util.function.ToIntFunction; -import org.apache.pinot.segment.spi.function.scalar.PartitionFunctionExprRacyTestFunctions; import org.apache.pinot.segment.spi.partition.PartitionFunction; import org.apache.pinot.segment.spi.partition.PartitionFunctionFactory; import org.apache.pinot.spi.data.readers.GenericRow; @@ -44,25 +39,6 @@ public class PartitionFunctionExprCompilerTest { - @Test - public void testCompileHexStringPipeline() { - PartitionPipeline pipeline = PartitionFunctionExprCompiler.compile("raw_key", " MD5 ( raw_key ) "); - - assertEquals(pipeline.getCanonicalFunctionExpr(), "md5(raw_key)"); - assertEquals(pipeline.getOutputType(), PartitionValueType.STRING); - assertEquals(pipeline.evaluate("hello").getStringValue(), "5d41402abc4b2a76b9719d911017c592"); - } - - @Test - public void testCompileRawBytesPipeline() { - PartitionPipeline pipeline = PartitionFunctionExprCompiler.compile("raw_key", " MD5_RAW ( raw_key ) "); - - assertEquals(pipeline.getCanonicalFunctionExpr(), "md5_raw(raw_key)"); - assertEquals(pipeline.getOutputType(), PartitionValueType.BYTES); - assertEquals(BytesUtils.toHexString(pipeline.evaluate("hello").getBytesValue()), - "5d41402abc4b2a76b9719d911017c592"); - } - @Test public void testCompilePartitionFunctionForMd5Fnv() { PartitionPipelineFunction partitionFunction = @@ -100,7 +76,7 @@ public void testCompilePartitionFunctionWithLiteralArgument() { PartitionFunctionExprCompiler.compilePartitionFunction("timestampMillis", "bucket(timestampMillis, 1000)", 128); assertEquals(partitionFunction.getPartition("54321"), 54); - assertEquals(partitionFunction.getFunctionExpr(), "bucket(timestampMillis, 1000)"); + assertEquals(partitionFunction.getFunctionExpr(), "bucket(timestampmillis, 1000)"); } @Test @@ -138,7 +114,7 @@ public void testIntegralExpressionDefaultsToPositiveModulo() { assertTrue(hash < 0); assertNotEquals(positiveModulo(hash, numPartitions), legacyPartitionFunction.getPartition(value)); - assertEquals(pipeline.getIntNormalizer(), PartitionIntNormalizer.POSITIVE_MODULO); + assertEquals(pipeline.getIntNormalizer(), null); assertEquals(partitionFunction.getPartition(value), positiveModulo(hash, numPartitions)); assertEquals(partitionFunction.getPartitionIdNormalizer(), "POSITIVE_MODULO"); } @@ -175,30 +151,19 @@ public void testExplicitAbsNormalizerUsesAbsoluteRemainder() { public void testRejectsWrongColumnReference() { IllegalArgumentException error = expectThrows(IllegalArgumentException.class, () -> PartitionFunctionExprCompiler.compile("raw_key", "md5(other_key)")); - assertEquals(error.getMessage(), - "Partition function expression must reference only partition column 'raw_key', got 'other_key'"); - } - - @Test - public void testRejectsUnsupportedFunction() { - IllegalArgumentException error = expectThrows(IllegalArgumentException.class, - () -> PartitionFunctionExprCompiler.compile("raw_key", "sha256(raw_key)")); - assertEquals(error.getMessage(), "Unsupported partition scalar function: sha256"); + assertTrue(error.getMessage().contains("must reference exactly that column"), + "Unexpected error: " + error.getMessage()); } @Test public void testRejectsNonIntPartitionFunctionOutput() { - IllegalArgumentException error = expectThrows(IllegalArgumentException.class, - () -> PartitionFunctionExprCompiler.compilePartitionFunction("raw_key", "md5(raw_key)", 16)); - assertEquals(error.getMessage(), "Partition pipeline must produce INT or LONG output, got: STRING"); - } - - @Test - public void testRejectsInvalidGrammar() { - IllegalArgumentException error = expectThrows(IllegalArgumentException.class, - () -> PartitionFunctionExprCompiler.compile("raw_key", "fnv1a_32(md5(raw_key), lower(raw_key))")); - assertEquals(error.getMessage(), - "Partition function expression must reference partition column 'raw_key' through a single argument chain"); + // md5 returns a String; the error is thrown at evaluation time when normalizeResult() checks the return type + PartitionPipelineFunction partitionFunction = + PartitionFunctionExprCompiler.compilePartitionFunction("raw_key", "md5(raw_key)", 16); + IllegalStateException error = expectThrows(IllegalStateException.class, + () -> partitionFunction.getPartition("hello")); + assertTrue(error.getMessage().contains("must return a numeric value"), + "Unexpected error: " + error.getMessage()); } @Test @@ -217,32 +182,6 @@ public void testRejectsNonDeterministicAliasedFunction() { "Partition scalar function 'cid' is not allowed because it is non-deterministic"); } - @Test - public void testRejectsConstantFunctionArgument() { - IllegalArgumentException error = expectThrows(IllegalArgumentException.class, - () -> PartitionFunctionExprCompiler.compilePartitionFunction("timestampMillis", - "bucket(timestampMillis, identity(1000))", 128)); - assertEquals(error.getMessage(), - "Partition function expression only supports literal constants, got function subexpression: identity(1000)"); - } - - @Test(timeOut = 10_000L) - public void testNonStaticScalarFunctionsUseThreadLocalTargets() - throws Exception { - PartitionFunctionExprRacyTestFunctions.reset(); - PartitionPipeline pipeline = PartitionFunctionExprCompiler.compile("raw_key", "racy_echo(raw_key)"); - ExecutorService executorService = Executors.newFixedThreadPool(2); - try { - Future firstResult = executorService.submit(() -> pipeline.evaluate("first").getStringValue()); - Future secondResult = executorService.submit(() -> pipeline.evaluate("second").getStringValue()); - - assertEquals(firstResult.get(5, TimeUnit.SECONDS), "first"); - assertEquals(secondResult.get(5, TimeUnit.SECONDS), "second"); - } finally { - executorService.shutdownNow(); - } - } - private static byte[] md5(byte[] input) { try { return MessageDigest.getInstance("MD5").digest(input); diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java index e8504e67816d..484b2837dfc3 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java @@ -112,8 +112,8 @@ public ColumnPartitionMetadata(PartitionFunction partitionFunction, Set partitionFunction.getFunctionExpr(), partitionFunction.getPartitionIdNormalizer(), // Store the pipeline input type only when it is non-default (BYTES) to avoid bloating the metadata. partitionFunction instanceof PartitionPipelineFunction - && ((PartitionPipelineFunction) partitionFunction).getPartitionPipeline().getInputType() - == PartitionValueType.BYTES ? PartitionValueType.BYTES.name() : null); + && ((PartitionPipelineFunction) partitionFunction).getPartitionPipeline().isBytesInput() + ? PartitionValueType.BYTES.name() : null); } @Nullable @@ -161,16 +161,17 @@ public boolean equals(Object obj) { ColumnPartitionMetadata that = (ColumnPartitionMetadata) obj; return Objects.equals(_functionName, that._functionName) && _numPartitions == that._numPartitions && _partitions.equals(that._partitions) && Objects.equals(_functionConfig, that._functionConfig) - && Objects.equals(_functionExpr, that._functionExpr) && Objects.equals(_partitionIdNormalizer, - that._partitionIdNormalizer) && Objects.equals(_inputType, that._inputType); + && Objects.equals(_functionExpr, that._functionExpr) + && Objects.equals(_partitionIdNormalizer, that._partitionIdNormalizer) + && Objects.equals(_inputType, that._inputType); } return false; } @Override public int hashCode() { - return Objects.hash(_functionName, _numPartitions, _functionConfig, _functionExpr, _partitionIdNormalizer, - _inputType) + 31 * _partitions.hashCode(); + return Objects.hash(_functionName, _numPartitions, _partitions, _functionConfig, _functionExpr, + _partitionIdNormalizer, _inputType); } /** diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionEvaluatorFactory.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionEvaluatorFactory.java new file mode 100644 index 000000000000..ac9b6b61c508 --- /dev/null +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionEvaluatorFactory.java @@ -0,0 +1,45 @@ +/** + * 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.spi.partition.pipeline; + +import org.apache.pinot.spi.function.FunctionEvaluator; + + +/** + * SPI for compiling a partition expression string into a {@link FunctionEvaluator}. + * + *

Implementations are loaded via {@link java.util.ServiceLoader}. Exactly one implementation must be present on the + * classpath at runtime. The default implementation in {@code pinot-common} delegates to + * {@link org.apache.pinot.common.evaluator.InbuiltFunctionEvaluator}. + */ +public interface PartitionEvaluatorFactory { + + /** + * Compiles a partition expression into a {@link FunctionEvaluator}. + * + *

Validates that the expression: + *

    + *
  • References exactly one column: {@code rawColumn}
  • + *
  • Uses only deterministic scalar functions
  • + *
+ * + * @throws IllegalArgumentException if the expression is invalid + */ + FunctionEvaluator compile(String rawColumn, String expression); +} diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java index 667a4da011b1..3d3cbdc14991 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java @@ -24,31 +24,41 @@ import java.util.Locale; import java.util.ServiceLoader; import javax.annotation.Nullable; -import org.apache.pinot.segment.spi.function.ExecutableFunctionEvaluator; import org.apache.pinot.spi.config.table.ColumnPartitionConfig; +import org.apache.pinot.spi.function.FunctionEvaluator; /** - * Compiles a restricted partition-function expression into a typed {@link PartitionPipeline} backed by deterministic - * scalar functions. + * Compiles a partition-function expression into a {@link PartitionPipeline} backed by + * {@link org.apache.pinot.common.evaluator.InbuiltFunctionEvaluator} via {@link PartitionEvaluatorFactory}. */ public final class PartitionFunctionExprCompiler { private static final int MAX_EXPRESSION_LENGTH = 256; - private static final int MAX_DEPTH = 8; - private static final int MAX_ARGUMENTS = 32; private PartitionFunctionExprCompiler() { } // Lazy holder: initialized on first use so that class-load of this utility does not fail when - // pinot-common (which provides CommonPartitionScalarFunctionResolver) is not yet on the classpath. + // pinot-common (which provides InbuiltPartitionEvaluatorFactory) is not yet on the classpath. // The JVM class-initialization guarantee makes this inherently thread-safe and single-init. - private static final class FunctionResolverHolder { - static final PartitionScalarFunctionResolver INSTANCE = loadFunctionResolver(); + private static final class EvaluatorFactoryHolder { + static final PartitionEvaluatorFactory INSTANCE = loadEvaluatorFactory(); + + private static PartitionEvaluatorFactory loadEvaluatorFactory() { + List factories = new ArrayList<>(); + for (PartitionEvaluatorFactory f : ServiceLoader.load(PartitionEvaluatorFactory.class)) { + factories.add(f); + } + Preconditions.checkState(!factories.isEmpty(), + "No PartitionEvaluatorFactory implementation found on the classpath"); + Preconditions.checkState(factories.size() == 1, + "Expected exactly 1 PartitionEvaluatorFactory implementation but found %s: %s", factories.size(), factories); + return factories.get(0); + } } public static PartitionPipeline compile(String rawColumn, String functionExpr) { - return compile(rawColumn, PartitionValueType.STRING, functionExpr, PartitionIntNormalizer.POSITIVE_MODULO); + return compile(rawColumn, PartitionValueType.STRING, functionExpr, null); } public static PartitionPipeline compile(String rawColumn, String functionExpr, @@ -64,22 +74,17 @@ public static PartitionPipeline compile(String rawColumn, String functionExpr, */ public static PartitionPipeline compile(String rawColumn, PartitionValueType inputType, String functionExpr, @Nullable PartitionIntNormalizer partitionIdNormalizer) { - Preconditions.checkArgument(hasText(rawColumn), "Raw column must be configured"); + Preconditions.checkArgument(rawColumn != null && !rawColumn.trim().isEmpty(), "Raw column must be configured"); Preconditions.checkArgument(inputType != null, "Input type must be configured"); - Preconditions.checkArgument(hasText(functionExpr), "'functionExpr' must be configured"); + Preconditions.checkArgument(functionExpr != null && !functionExpr.trim().isEmpty(), + "'functionExpr' must be configured"); Preconditions.checkArgument(functionExpr.length() <= MAX_EXPRESSION_LENGTH, "'functionExpr' must be <= %s characters", MAX_EXPRESSION_LENGTH); - Parser parser = new Parser(functionExpr); - Node root = parser.parse(); - List steps = new ArrayList<>(); - CompileResult result = compile(rawColumn, inputType, root, steps, 0); - Preconditions.checkArgument(result._dynamic, - "Partition function expression must reference partition column '%s'", rawColumn); - PartitionIntNormalizer effectivePartitionIdNormalizer = - result._outputType.isIntegral() ? partitionIdNormalizer : null; - return new PartitionPipeline(rawColumn, inputType, result._outputType, result._canonicalExpr, - effectivePartitionIdNormalizer, steps, result._rootNode); + String canonicalExpr = canonicalize(functionExpr); + boolean isBytesInput = inputType == PartitionValueType.BYTES; + FunctionEvaluator evaluator = EvaluatorFactoryHolder.INSTANCE.compile(rawColumn, canonicalExpr); + return new PartitionPipeline(rawColumn, isBytesInput, canonicalExpr, partitionIdNormalizer, evaluator); } public static PartitionPipelineFunction compilePartitionFunction(String rawColumn, String functionExpr, @@ -101,359 +106,21 @@ public static PartitionPipelineFunction compilePartitionFunction(String rawColum */ public static PartitionPipelineFunction compilePartitionFunction(String rawColumn, PartitionValueType inputType, String functionExpr, int numPartitions, @Nullable String partitionIdNormalizer) { - PartitionIntNormalizer configuredPartitionIdNormalizer = partitionIdNormalizer != null + PartitionIntNormalizer normalizer = partitionIdNormalizer != null ? PartitionIntNormalizer.fromConfigString(partitionIdNormalizer) : PartitionIntNormalizer.fromConfigString(ColumnPartitionConfig.PARTITION_ID_NORMALIZER_POSITIVE_MODULO); - return new PartitionPipelineFunction(compile(rawColumn, inputType, functionExpr, configuredPartitionIdNormalizer), - numPartitions); - } - - private static CompileResult compile(String rawColumn, PartitionValueType inputType, Node node, - List steps, int depth) { - Preconditions.checkArgument(depth <= MAX_DEPTH, - "Partition function expression depth exceeds the maximum of %s", MAX_DEPTH); - if (node instanceof IdentifierNode) { - IdentifierNode identifierNode = (IdentifierNode) node; - Preconditions.checkArgument(identifierNode._name.equals(rawColumn), - "Partition function expression must reference only partition column '%s', got '%s'", - rawColumn, identifierNode._name); - return CompileResult.dynamic(inputType, rawColumn, new ExecutableFunctionEvaluator.ColumnNode(rawColumn, 0)); - } - if (node instanceof LiteralNode) { - LiteralNode literalNode = (LiteralNode) node; - return CompileResult.literal(literalNode._value.getType(), literalNode._canonicalForm, literalNode._value, - new ExecutableFunctionEvaluator.ConstantNode(literalNode._value.toObject())); - } - - FunctionNode functionNode = (FunctionNode) node; - List arguments = new ArrayList<>(functionNode._arguments.size()); - int dynamicCount = 0; - int dynamicIndex = -1; - for (int i = 0; i < functionNode._arguments.size(); i++) { - CompileResult argument = compile(rawColumn, inputType, functionNode._arguments.get(i), steps, depth + 1); - Preconditions.checkArgument(argument._dynamic || argument._literalConstant, - "Partition function expression only supports literal constants, got function subexpression: %s", - argument._canonicalExpr); - arguments.add(argument); - if (argument._dynamic) { - dynamicCount++; - dynamicIndex = i; - } - } - Preconditions.checkArgument(dynamicCount <= 1, - "Partition function expression must reference partition column '%s' through a single argument chain", - rawColumn); - - String displayName = functionNode._name.toLowerCase(Locale.ROOT); - PartitionScalarFunctionResolver.ResolvedFunction resolvedFunction = - getFunctionResolver().resolve(displayName, toResolverArguments(arguments)); - String canonicalExpr = toCanonicalExpr(displayName, arguments); - if (!resolvedFunction.isDynamic()) { - PartitionValue constantValue = resolvedFunction.invoke(null); - return CompileResult.constant(resolvedFunction.getOutputType(), canonicalExpr, constantValue, - new ExecutableFunctionEvaluator.ConstantNode(constantValue.toObject())); - } - - steps.add(new PartitionStep(displayName, arguments.get(dynamicIndex)._outputType, resolvedFunction.getOutputType(), - resolvedFunction::invoke)); - ExecutableFunctionEvaluator.ExecutableNode[] dynamicArgumentNodes = dynamicCount == 1 - ? new ExecutableFunctionEvaluator.ExecutableNode[]{arguments.get(dynamicIndex)._rootNode} - : new ExecutableFunctionEvaluator.ExecutableNode[0]; - return CompileResult.dynamic(resolvedFunction.getOutputType(), canonicalExpr, - new ExecutableFunctionEvaluator.FunctionNode(displayName, new PartitionBoundFunctionInvoker(resolvedFunction), - dynamicArgumentNodes)); - } - - private static List toResolverArguments(List arguments) { - List resolverArguments = new ArrayList<>(arguments.size()); - for (CompileResult argument : arguments) { - resolverArguments.add(argument._dynamic ? PartitionScalarFunctionResolver.Argument.dynamic(argument._outputType) - : PartitionScalarFunctionResolver.Argument.constant( - Preconditions.checkNotNull(argument._constantValue, "Constant argument must be configured"))); - } - return resolverArguments; - } - - private static PartitionScalarFunctionResolver getFunctionResolver() { - return FunctionResolverHolder.INSTANCE; - } - - private static PartitionScalarFunctionResolver loadFunctionResolver() { - List resolvers = new ArrayList<>(); - for (PartitionScalarFunctionResolver r : ServiceLoader.load(PartitionScalarFunctionResolver.class)) { - resolvers.add(r); - } - Preconditions.checkState(!resolvers.isEmpty(), - "No PartitionScalarFunctionResolver implementation found on the classpath"); - Preconditions.checkState(resolvers.size() == 1, - "Expected exactly 1 PartitionScalarFunctionResolver implementation but found %s: %s", resolvers.size(), - resolvers); - return resolvers.get(0); - } - - private static String toCanonicalExpr(String functionName, List arguments) { - StringBuilder builder = new StringBuilder(functionName).append('('); - for (int i = 0; i < arguments.size(); i++) { - if (i > 0) { - builder.append(", "); - } - builder.append(arguments.get(i)._canonicalExpr); - } - return builder.append(')').toString(); - } - - private static boolean hasText(@Nullable String value) { - return value != null && !value.trim().isEmpty(); - } - - private abstract static class Node { - } - - private static final class IdentifierNode extends Node { - private final String _name; - - private IdentifierNode(String name) { - _name = name; - } - } - - private static final class LiteralNode extends Node { - private final PartitionValue _value; - private final String _canonicalForm; - - private LiteralNode(PartitionValue value, String canonicalForm) { - _value = value; - _canonicalForm = canonicalForm; - } - } - - private static final class FunctionNode extends Node { - private final String _name; - private final List _arguments; - - private FunctionNode(String name, List arguments) { - _name = name; - _arguments = arguments; - } - } - - private static final class CompileResult { - private final PartitionValueType _outputType; - private final String _canonicalExpr; - private final boolean _dynamic; - private final boolean _literalConstant; - private final ExecutableFunctionEvaluator.ExecutableNode _rootNode; - @Nullable - private final PartitionValue _constantValue; - - private CompileResult(PartitionValueType outputType, String canonicalExpr, boolean dynamic, boolean literalConstant, - @Nullable PartitionValue constantValue, ExecutableFunctionEvaluator.ExecutableNode rootNode) { - _outputType = outputType; - _canonicalExpr = canonicalExpr; - _dynamic = dynamic; - _literalConstant = literalConstant; - _constantValue = constantValue; - _rootNode = rootNode; - } - - private static CompileResult dynamic(PartitionValueType outputType, String canonicalExpr, - ExecutableFunctionEvaluator.ExecutableNode rootNode) { - return new CompileResult(outputType, canonicalExpr, true, false, null, rootNode); - } - - private static CompileResult literal(PartitionValueType outputType, String canonicalExpr, PartitionValue value, - ExecutableFunctionEvaluator.ExecutableNode rootNode) { - return new CompileResult(outputType, canonicalExpr, false, true, value, rootNode); - } - - private static CompileResult constant(PartitionValueType outputType, String canonicalExpr, PartitionValue value, - ExecutableFunctionEvaluator.ExecutableNode rootNode) { - return new CompileResult(outputType, canonicalExpr, false, false, value, rootNode); - } + PartitionPipeline pipeline = compile(rawColumn, inputType, functionExpr, normalizer); + return new PartitionPipelineFunction(pipeline, numPartitions); } /** - * Bridges a {@link PartitionScalarFunctionResolver.ResolvedFunction} to the - * {@link ExecutableFunctionEvaluator.Invoker} contract required by {@link ExecutableFunctionEvaluator.FunctionNode}. + * Returns a canonical form of the expression: trimmed, lowercased, with spaces removed around + * {@code (}, {@code )}, and {@code ,}. */ - private static final class PartitionBoundFunctionInvoker implements ExecutableFunctionEvaluator.Invoker { - private final PartitionScalarFunctionResolver.ResolvedFunction _resolvedFunction; - - private PartitionBoundFunctionInvoker(PartitionScalarFunctionResolver.ResolvedFunction resolvedFunction) { - _resolvedFunction = resolvedFunction; - } - - @Override - public Object invoke(Object[] arguments) { - Preconditions.checkState(arguments.length <= 1, - "Partition expression runtime expects at most one dynamic argument, got: %s", arguments.length); - PartitionValue dynamicInput = arguments.length == 0 ? null : PartitionValue.fromObject(arguments[0]); - return _resolvedFunction.invoke(dynamicInput).toObject(); - } - } - - private static final class Parser { - private final String _expression; - private int _index; - - private Parser(String expression) { - _expression = expression; - } - - private Node parse() { - skipWhitespace(); - Node node = parseTerm(); - skipWhitespace(); - if (_index != _expression.length()) { - throw error("Unexpected trailing content"); - } - return node; - } - - private Node parseTerm() { - skipWhitespace(); - if (_index >= _expression.length()) { - throw error("Unexpected end of expression"); - } - char character = _expression.charAt(_index); - if (character == '\'') { - return parseStringLiteral(); - } - if (isNumericStart(character)) { - return parseNumericLiteral(); - } - String identifier = parseIdentifier(); - skipWhitespace(); - if (_index < _expression.length() && _expression.charAt(_index) == '(') { - _index++; - skipWhitespace(); - List arguments = new ArrayList<>(); - if (_index < _expression.length() && _expression.charAt(_index) == ')') { - _index++; - return new FunctionNode(identifier, arguments); - } - while (true) { - arguments.add(parseTerm()); - Preconditions.checkArgument(arguments.size() <= MAX_ARGUMENTS, - "Partition function expression cannot have more than %s arguments", MAX_ARGUMENTS); - skipWhitespace(); - if (_index >= _expression.length()) { - throw error("Expected ')'"); - } - char delimiter = _expression.charAt(_index); - if (delimiter == ',') { - _index++; - skipWhitespace(); - continue; - } - if (delimiter == ')') { - _index++; - return new FunctionNode(identifier, arguments); - } - throw error("Expected ',' or ')'"); - } - } - return new IdentifierNode(identifier); - } - - private Node parseStringLiteral() { - _index++; - StringBuilder builder = new StringBuilder(); - while (_index < _expression.length()) { - char character = _expression.charAt(_index++); - if (character == '\'') { - if (_index < _expression.length() && _expression.charAt(_index) == '\'') { - builder.append('\''); - _index++; - continue; - } - return new LiteralNode(PartitionValue.stringValue(builder.toString()), quote(builder.toString())); - } - builder.append(character); - } - throw error("Unterminated string literal"); - } - - private Node parseNumericLiteral() { - int start = _index; - if (_expression.charAt(_index) == '+' || _expression.charAt(_index) == '-') { - _index++; - } - boolean sawDecimal = false; - while (_index < _expression.length()) { - char character = _expression.charAt(_index); - if (Character.isDigit(character)) { - _index++; - continue; - } - if (character == '.') { - sawDecimal = true; - _index++; - continue; - } - if (character == 'e' || character == 'E') { - sawDecimal = true; - _index++; - if (_index < _expression.length() - && (_expression.charAt(_index) == '+' || _expression.charAt(_index) == '-')) { - _index++; - } - continue; - } - break; - } - String token = _expression.substring(start, _index); - try { - if (sawDecimal) { - return new LiteralNode(PartitionValue.doubleValue(Double.parseDouble(token)), token); - } - return new LiteralNode(PartitionValue.longValue(Long.parseLong(token)), token); - } catch (NumberFormatException e) { - throw error("Invalid numeric literal: " + token); - } - } - - private String parseIdentifier() { - Preconditions.checkArgument(_index < _expression.length() && isIdentifierStart(_expression.charAt(_index)), - "Expected identifier at position %s", _index); - int start = _index++; - while (_index < _expression.length() && isIdentifierPart(_expression.charAt(_index))) { - _index++; - } - return _expression.substring(start, _index); - } - - private void skipWhitespace() { - while (_index < _expression.length() && Character.isWhitespace(_expression.charAt(_index))) { - _index++; - } - } - - private boolean isNumericStart(char character) { - if (Character.isDigit(character)) { - return true; - } - if ((character == '+' || character == '-') && _index + 1 < _expression.length()) { - return Character.isDigit(_expression.charAt(_index + 1)); - } - return false; - } - - private boolean isIdentifierStart(char character) { - return Character.isLetter(character) || character == '_'; - } - - private boolean isIdentifierPart(char character) { - return Character.isLetterOrDigit(character) || character == '_'; - } - - private IllegalArgumentException error(String message) { - return new IllegalArgumentException( - String.format("%s at position %s in partition function expression '%s'", message, _index, _expression)); - } - } - - private static String quote(String value) { - return '\'' + value.replace("'", "''") + '\''; + static String canonicalize(String expression) { + return expression.trim().toLowerCase(Locale.ROOT) + .replaceAll("\\s*\\(\\s*", "(") + .replaceAll("\\s*\\)\\s*", ")") + .replaceAll("\\s*,\\s*", ", "); } } diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipeline.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipeline.java index 3d7d8f1f27c3..46569d57531b 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipeline.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipeline.java @@ -22,52 +22,46 @@ import java.util.Collections; import java.util.List; import javax.annotation.Nullable; -import org.apache.pinot.segment.spi.function.ExecutableFunctionEvaluator; import org.apache.pinot.spi.data.FieldSpec; import org.apache.pinot.spi.data.readers.GenericRow; +import org.apache.pinot.spi.function.FunctionEvaluator; +import org.apache.pinot.spi.utils.BytesUtils; /** - * Immutable compiled pipeline for one raw partition column. + * Immutable compiled pipeline for one raw partition column, backed by a {@link FunctionEvaluator}. + * + *

The underlying {@link FunctionEvaluator} is responsible for correct type coercion. For partition expressions, + * the evaluator uses UTF-8 encoding when converting {@code String} values to {@code byte[]} parameters, ensuring + * that hash functions ({@code md5}, {@code murmur2}, {@code fnv1a_32}, etc.) operate on raw string bytes rather + * than a hex-decoded representation. */ -public final class PartitionPipeline extends ExecutableFunctionEvaluator { +public final class PartitionPipeline implements FunctionEvaluator { private final String _rawColumn; - private final PartitionValueType _inputType; - private final PartitionValueType _outputType; + private final boolean _isBytesInput; private final String _canonicalFunctionExpr; @Nullable private final PartitionIntNormalizer _intNormalizer; - private final List _steps; + private final FunctionEvaluator _evaluator; - public PartitionPipeline(String rawColumn, PartitionValueType inputType, PartitionValueType outputType, - String canonicalFunctionExpr, @Nullable PartitionIntNormalizer intNormalizer, List steps, - ExecutableNode rootNode) { - // super() guards rootNode (non-null) and canonicalFunctionExpr (non-null); rawColumn null is caught below - super(rootNode, Collections.singletonList(rawColumn), canonicalFunctionExpr); + PartitionPipeline(String rawColumn, boolean isBytesInput, String canonicalFunctionExpr, + @Nullable PartitionIntNormalizer intNormalizer, FunctionEvaluator evaluator) { Preconditions.checkNotNull(rawColumn, "Raw column must be configured"); - Preconditions.checkNotNull(inputType, "Input type must be configured"); - Preconditions.checkNotNull(outputType, "Output type must be configured"); - Preconditions.checkNotNull(steps, "Pipeline steps must be configured"); - Preconditions.checkArgument(!outputType.isIntegral() || intNormalizer != null, - "Integral-output pipelines must configure an INT normalizer"); + Preconditions.checkNotNull(canonicalFunctionExpr, "Canonical function expression must be configured"); + Preconditions.checkNotNull(evaluator, "Function evaluator must be configured"); _rawColumn = rawColumn; - _inputType = inputType; - _outputType = outputType; + _isBytesInput = isBytesInput; _canonicalFunctionExpr = canonicalFunctionExpr; _intNormalizer = intNormalizer; - _steps = Collections.unmodifiableList(steps); + _evaluator = evaluator; } public String getRawColumn() { return _rawColumn; } - public PartitionValueType getInputType() { - return _inputType; - } - - public PartitionValueType getOutputType() { - return _outputType; + public boolean isBytesInput() { + return _isBytesInput; } public String getCanonicalFunctionExpr() { @@ -79,8 +73,9 @@ public PartitionIntNormalizer getIntNormalizer() { return _intNormalizer; } - public List getSteps() { - return _steps; + @Override + public List getArguments() { + return Collections.singletonList(_rawColumn); } @Override @@ -89,11 +84,18 @@ public Object evaluate(GenericRow genericRow) { if (inputValue == null) { return null; } - // Pass raw bytes directly for BYTES-input pipelines to avoid hex-encoding the payload. - if (inputValue instanceof byte[] && _inputType == PartitionValueType.BYTES) { - return super.evaluate(new Object[]{inputValue}); + if (_isBytesInput) { + // For BYTES-input pipelines pass raw byte[] directly; String values are hex-encoded representations. + if (inputValue instanceof byte[]) { + return _evaluator.evaluate(new Object[]{inputValue}); + } + // Hex-encoded string representation of bytes (e.g. from broker routing) — decode before passing. + return _evaluator.evaluate(new Object[]{BytesUtils.toBytes( + FieldSpec.getStringValue(inputValue))}); } - return super.evaluate(new Object[]{FieldSpec.getStringValue(inputValue)}); + // For STRING-input pipelines pass the string value as-is. The underlying PartitionFunctionEvaluator converts + // String to byte[] using UTF-8 encoding when a function parameter requires byte[]. + return _evaluator.evaluate(new Object[]{FieldSpec.getStringValue(inputValue)}); } @Override @@ -105,22 +107,13 @@ public Object evaluate(Object[] values) { if (inputValue == null) { return null; } - // Pass raw bytes directly for BYTES-input pipelines to avoid hex-encoding the payload. - if (inputValue instanceof byte[] && _inputType == PartitionValueType.BYTES) { - return super.evaluate(new Object[]{inputValue}); + if (_isBytesInput) { + if (inputValue instanceof byte[]) { + return _evaluator.evaluate(new Object[]{inputValue}); + } + return _evaluator.evaluate(new Object[]{BytesUtils.toBytes( + FieldSpec.getStringValue(inputValue))}); } - return super.evaluate(new Object[]{FieldSpec.getStringValue(inputValue)}); - } - - public PartitionValue evaluate(String rawValue) { - Preconditions.checkState(_inputType == PartitionValueType.STRING, - "evaluate(String) is only supported for STRING-input pipelines"); - return PartitionValue.fromObject(super.evaluate(new Object[]{rawValue})); - } - - public PartitionValue evaluate(PartitionValue input) { - Preconditions.checkArgument(input.getType() == _inputType, - "Pipeline for column '%s' expects %s input but got %s", _rawColumn, _inputType, input.getType()); - return PartitionValue.fromObject(super.evaluate(new Object[]{input.toObject()})); + return _evaluator.evaluate(new Object[]{FieldSpec.getStringValue(inputValue)}); } } diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java index db9740a5af58..48493a93f6d2 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import java.math.BigInteger; import java.util.Collections; import java.util.List; import org.apache.pinot.segment.spi.partition.PartitionFunction; @@ -34,10 +35,7 @@ * {@link PartitionFunction} adapter for expression-mode partition pipelines. * *

Note on {@code Serializable}: {@link PartitionFunction} extends {@link java.io.Serializable} for - * historical reasons, but partition functions are never Java-serialized in Pinot's runtime. This class holds a - * {@link PartitionPipeline} that extends {@link org.apache.pinot.segment.spi.function.ExecutableFunctionEvaluator}, - * which contains {@link ThreadLocal} fields that are not serializable. Do not rely on Java serialization for this - * class. + * historical reasons, but partition functions are never Java-serialized in Pinot's runtime. */ @SuppressWarnings("serial") public class PartitionPipelineFunction implements PartitionFunction, FunctionEvaluator { @@ -49,8 +47,6 @@ public class PartitionPipelineFunction implements PartitionFunction, FunctionEva public PartitionPipelineFunction(PartitionPipeline pipeline, int numPartitions) { Preconditions.checkNotNull(pipeline, "Partition pipeline must be configured"); Preconditions.checkArgument(numPartitions > 0, "Number of partitions must be > 0"); - Preconditions.checkArgument(pipeline.getOutputType().isIntegral(), - "Partition pipeline must produce INT or LONG output, got: %s", pipeline.getOutputType()); _pipeline = pipeline; _numPartitions = numPartitions; } @@ -63,18 +59,10 @@ public PartitionPipeline getPartitionPipeline() { public int getPartition(String value) { // BYTES-input pipelines expect raw bytes. When the caller provides a string (e.g. a hex-encoded predicate value // from broker routing), convert the hex string back to raw bytes so the partition computation matches ingestion. - if (_pipeline.getInputType() == PartitionValueType.BYTES) { + if (_pipeline.isBytesInput()) { return getPartition(BytesUtils.toBytes(value)); } - PartitionValue partitionValue = _pipeline.evaluate(value); - PartitionIntNormalizer intNormalizer = _pipeline.getIntNormalizer(); - Preconditions.checkState(intNormalizer != null, "Integral-output partition pipeline must have an INT normalizer"); - if (partitionValue.getType() == PartitionValueType.INT) { - return intNormalizer.getPartitionId(partitionValue.getIntValue(), _numPartitions); - } - Preconditions.checkState(partitionValue.getType() == PartitionValueType.LONG, - "Expected INT or LONG partition value but got: %s", partitionValue.getType()); - return intNormalizer.getPartitionId(partitionValue.getLongValue(), _numPartitions); + return normalizeResult(_pipeline.evaluate(new Object[]{value})); } /** @@ -83,18 +71,26 @@ public int getPartition(String value) { */ @Override public int getPartition(byte[] bytes) { - if (_pipeline.getInputType() != PartitionValueType.BYTES) { + if (!_pipeline.isBytesInput()) { return getPartition(BytesUtils.toHexString(bytes)); } - PartitionValue partitionValue = _pipeline.evaluate(PartitionValue.bytesValue(bytes)); + return normalizeResult(_pipeline.evaluate(new Object[]{bytes})); + } + + private int normalizeResult(Object result) { + Preconditions.checkNotNull(result, "Partition expression for column '%s' returned null", _pipeline.getRawColumn()); + Preconditions.checkState(result instanceof Number, + "Partition expression for column '%s' must return a numeric value, got: %s", + _pipeline.getRawColumn(), result.getClass().getSimpleName()); + Number num = (Number) result; PartitionIntNormalizer intNormalizer = _pipeline.getIntNormalizer(); - Preconditions.checkState(intNormalizer != null, "Integral-output partition pipeline must have an INT normalizer"); - if (partitionValue.getType() == PartitionValueType.INT) { - return intNormalizer.getPartitionId(partitionValue.getIntValue(), _numPartitions); + Preconditions.checkState(intNormalizer != null, + "Integral-output partition pipeline for column '%s' must have an INT normalizer", + _pipeline.getRawColumn()); + if (num instanceof Long || num instanceof BigInteger) { + return intNormalizer.getPartitionId(num.longValue(), _numPartitions); } - Preconditions.checkState(partitionValue.getType() == PartitionValueType.LONG, - "Expected INT or LONG partition value but got: %s", partitionValue.getType()); - return intNormalizer.getPartitionId(partitionValue.getLongValue(), _numPartitions); + return intNormalizer.getPartitionId(num.intValue(), _numPartitions); } @Override @@ -141,7 +137,7 @@ public Object evaluate(GenericRow genericRow) { if (inputValue == null) { return null; } - if (inputValue instanceof byte[] && _pipeline.getInputType() == PartitionValueType.BYTES) { + if (inputValue instanceof byte[] && _pipeline.isBytesInput()) { return getPartition((byte[]) inputValue); } return getPartition(FieldSpec.getStringValue(inputValue)); @@ -156,7 +152,7 @@ public Object evaluate(Object[] values) { if (inputValue == null) { return null; } - if (inputValue instanceof byte[] && _pipeline.getInputType() == PartitionValueType.BYTES) { + if (inputValue instanceof byte[] && _pipeline.isBytesInput()) { return getPartition((byte[]) inputValue); } return getPartition(FieldSpec.getStringValue(inputValue)); diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionScalarFunctionResolver.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionScalarFunctionResolver.java deleted file mode 100644 index af108e062e4b..000000000000 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionScalarFunctionResolver.java +++ /dev/null @@ -1,86 +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.spi.partition.pipeline; - -import com.google.common.base.Preconditions; -import java.util.List; -import javax.annotation.Nullable; - - -/** - * Resolves one partition-expression scalar function call into an executable binding. - * - *

The compiler owns DSL parsing and raw-column validation. Implementations of this interface own scalar-function - * discovery, overload resolution, determinism validation and invocation. - */ -public interface PartitionScalarFunctionResolver { - /** - * Resolve the scalar function call or throw an {@link IllegalArgumentException} when the call is invalid. - */ - ResolvedFunction resolve(String functionName, List arguments); - - /** - * One validated partition-expression argument. - */ - final class Argument { - private final PartitionValueType _type; - private final boolean _dynamic; - @Nullable - private final PartitionValue _constantValue; - - private Argument(PartitionValueType type, boolean dynamic, @Nullable PartitionValue constantValue) { - _type = Preconditions.checkNotNull(type, "Partition scalar function argument type must be configured"); - _dynamic = dynamic; - _constantValue = constantValue; - } - - public static Argument dynamic(PartitionValueType type) { - return new Argument(type, true, null); - } - - public static Argument constant(PartitionValue value) { - Preconditions.checkNotNull(value, "Partition scalar function constant must be configured"); - return new Argument(value.getType(), false, value); - } - - public PartitionValueType getType() { - return _type; - } - - public boolean isDynamic() { - return _dynamic; - } - - @Nullable - public PartitionValue getConstantValue() { - return _constantValue; - } - } - - /** - * One resolved scalar-function binding. - */ - interface ResolvedFunction { - boolean isDynamic(); - - PartitionValueType getOutputType(); - - PartitionValue invoke(@Nullable PartitionValue dynamicInput); - } -} diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionStep.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionStep.java deleted file mode 100644 index 0e95954329db..000000000000 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionStep.java +++ /dev/null @@ -1,65 +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.spi.partition.pipeline; - -import com.google.common.base.Preconditions; - - -/** - * One compiled pipeline step with a fixed input/output type contract. - */ -public final class PartitionStep { - @FunctionalInterface - interface Evaluator { - PartitionValue apply(PartitionValue input); - } - - private final String _name; - private final PartitionValueType _inputType; - private final PartitionValueType _outputType; - private final Evaluator _evaluator; - - PartitionStep(String name, PartitionValueType inputType, PartitionValueType outputType, Evaluator evaluator) { - _name = name; - _inputType = inputType; - _outputType = outputType; - _evaluator = evaluator; - } - - public String getName() { - return _name; - } - - public PartitionValueType getInputType() { - return _inputType; - } - - public PartitionValueType getOutputType() { - return _outputType; - } - - public PartitionValue evaluate(PartitionValue input) { - Preconditions.checkArgument(input.getType() == _inputType, - "Step '%s' expects %s input but got %s", _name, _inputType, input.getType()); - PartitionValue output = _evaluator.apply(input); - Preconditions.checkState(output.getType() == _outputType, - "Step '%s' produced %s output but expected %s", _name, output.getType(), _outputType); - return output; - } -} diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionValue.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionValue.java deleted file mode 100644 index f6b8c55ee51c..000000000000 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionValue.java +++ /dev/null @@ -1,163 +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.spi.partition.pipeline; - -import com.google.common.base.Preconditions; -import java.util.Arrays; -import java.util.Objects; - - -/** - * Typed runtime value used within the compiled partition pipeline. - */ -public final class PartitionValue { - private final PartitionValueType _type; - private final Object _value; - - private PartitionValue(PartitionValueType type, Object value) { - _type = type; - _value = value; - } - - public static PartitionValue stringValue(String value) { - Preconditions.checkNotNull(value, "Partition pipeline does not accept null STRING values"); - return new PartitionValue(PartitionValueType.STRING, value); - } - - public static PartitionValue bytesValue(byte[] value) { - Preconditions.checkNotNull(value, "Partition pipeline does not accept null BYTES values"); - return new PartitionValue(PartitionValueType.BYTES, value.clone()); - } - - public static PartitionValue intValue(int value) { - return new PartitionValue(PartitionValueType.INT, value); - } - - public static PartitionValue longValue(long value) { - return new PartitionValue(PartitionValueType.LONG, value); - } - - public static PartitionValue floatValue(float value) { - return new PartitionValue(PartitionValueType.FLOAT, value); - } - - public static PartitionValue doubleValue(double value) { - return new PartitionValue(PartitionValueType.DOUBLE, value); - } - - public PartitionValueType getType() { - return _type; - } - - public String getStringValue() { - Preconditions.checkState(_type == PartitionValueType.STRING, "Expected STRING value but got: %s", _type); - return (String) _value; - } - - public byte[] getBytesValue() { - Preconditions.checkState(_type == PartitionValueType.BYTES, "Expected BYTES value but got: %s", _type); - return (byte[]) _value; - } - - public int getIntValue() { - Preconditions.checkState(_type == PartitionValueType.INT, "Expected INT value but got: %s", _type); - return (Integer) _value; - } - - public long getLongValue() { - Preconditions.checkState(_type == PartitionValueType.LONG, "Expected LONG value but got: %s", _type); - return (Long) _value; - } - - public float getFloatValue() { - Preconditions.checkState(_type == PartitionValueType.FLOAT, "Expected FLOAT value but got: %s", _type); - return (Float) _value; - } - - public double getDoubleValue() { - Preconditions.checkState(_type == PartitionValueType.DOUBLE, "Expected DOUBLE value but got: %s", _type); - return (Double) _value; - } - - public Object toObject() { - switch (_type) { - case STRING: - return getStringValue(); - case BYTES: - return getBytesValue(); - case INT: - return getIntValue(); - case LONG: - return getLongValue(); - case FLOAT: - return getFloatValue(); - case DOUBLE: - return getDoubleValue(); - default: - throw new IllegalStateException("Unsupported partition pipeline runtime value type: " + _type); - } - } - - public static PartitionValue fromObject(Object value) { - Preconditions.checkNotNull(value, "Partition pipeline function returned null"); - if (value instanceof String) { - return stringValue((String) value); - } - if (value instanceof byte[]) { - return bytesValue((byte[]) value); - } - if (value instanceof Integer) { - return intValue((Integer) value); - } - if (value instanceof Long) { - return longValue((Long) value); - } - if (value instanceof Float) { - return floatValue((Float) value); - } - if (value instanceof Double) { - return doubleValue((Double) value); - } - throw new IllegalArgumentException("Unsupported partition pipeline runtime value type: " + value.getClass()); - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (!(obj instanceof PartitionValue)) { - return false; - } - PartitionValue that = (PartitionValue) obj; - if (_type != that._type) { - return false; - } - if (_type == PartitionValueType.BYTES) { - return Arrays.equals((byte[]) _value, (byte[]) that._value); - } - return Objects.equals(_value, that._value); - } - - @Override - public int hashCode() { - return _type == PartitionValueType.BYTES ? 31 * _type.hashCode() + Arrays.hashCode((byte[]) _value) - : 31 * _type.hashCode() + _value.hashCode(); - } -} diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionValueConversions.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionValueConversions.java deleted file mode 100644 index 6807b28062d1..000000000000 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionValueConversions.java +++ /dev/null @@ -1,202 +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.spi.partition.pipeline; - -import com.google.common.base.Preconditions; -import java.util.EnumSet; -import java.util.Set; - -import static java.nio.charset.StandardCharsets.UTF_8; - - -/** - * Static type-conversion utilities shared by all implementations of {@link PartitionScalarFunctionResolver}. - * - *

These methods encode which source/target type pairs are allowed during partition-expression binding, what the - * conversion costs are (lower = preferred during overload resolution), and how to perform the actual Java-type - * conversion at binding time. - */ -public final class PartitionValueConversions { - // Pre-computed widening sets used during overload resolution. Kept as constants to avoid per-call allocation. - private static final Set INT_WIDENING_TARGETS = - EnumSet.of(PartitionValueType.LONG, PartitionValueType.FLOAT, PartitionValueType.DOUBLE); - private static final Set LONG_WIDENING_TARGETS = EnumSet.of(PartitionValueType.DOUBLE); - private static final Set FLOAT_WIDENING_TARGETS = EnumSet.of(PartitionValueType.DOUBLE); - - private PartitionValueConversions() { - } - - /** - * Returns the overload-resolution cost for converting a dynamic pipeline value of {@code sourceType} to - * {@code targetType}, or {@code -1} if the conversion is not allowed. - */ - public static int getDynamicConversionCost(PartitionValueType sourceType, Class targetType) { - PartitionValueType targetValueType = PartitionValueType.fromJavaType(targetType); - if (sourceType == targetValueType) { - return 0; - } - switch (sourceType) { - case STRING: - if (targetValueType == PartitionValueType.BYTES || targetValueType.isNumeric()) { - return 1; - } - return -1; - case BYTES: - return -1; - case INT: - return wideningNumericCost(targetValueType, INT_WIDENING_TARGETS); - case LONG: - return wideningNumericCost(targetValueType, LONG_WIDENING_TARGETS); - case FLOAT: - return wideningNumericCost(targetValueType, FLOAT_WIDENING_TARGETS); - case DOUBLE: - default: - return -1; - } - } - - /** - * Returns the overload-resolution cost for passing a compile-time constant {@code value} as {@code targetType}, or - * {@code -1} if the conversion is not allowed. - */ - public static int getConstantConversionCost(PartitionValue value, Class targetType) { - PartitionValueType targetValueType = PartitionValueType.fromJavaType(targetType); - PartitionValueType sourceType = value.getType(); - if (sourceType == targetValueType) { - return 0; - } - switch (sourceType) { - case STRING: - if (targetValueType == PartitionValueType.BYTES || targetValueType.isNumeric()) { - return 1; - } - return -1; - case LONG: - if (targetValueType == PartitionValueType.INT) { - long longValue = value.getLongValue(); - return longValue >= Integer.MIN_VALUE && longValue <= Integer.MAX_VALUE ? 2 : -1; - } - return getDynamicConversionCost(sourceType, targetType); - case DOUBLE: - if (targetValueType == PartitionValueType.INT) { - double doubleValue = value.getDoubleValue(); - return doubleValue >= Integer.MIN_VALUE && doubleValue <= Integer.MAX_VALUE - && doubleValue == Math.rint(doubleValue) ? 2 : -1; - } - if (targetValueType == PartitionValueType.LONG) { - double doubleValue = value.getDoubleValue(); - return doubleValue >= Long.MIN_VALUE && doubleValue <= Long.MAX_VALUE - && doubleValue == Math.rint(doubleValue) ? 2 : -1; - } - return getDynamicConversionCost(sourceType, targetType); - default: - return getDynamicConversionCost(sourceType, targetType); - } - } - - /** - * Converts {@code value} to the Java type required by {@code targetType}. - * - * @param constant {@code true} when converting a compile-time literal; allows narrowing conversions (e.g. LONG → - * INT) that are rejected for dynamic runtime values. - */ - public static Object convertValue(PartitionValue value, Class targetType, boolean constant) { - PartitionValueType targetValueType = PartitionValueType.fromJavaType(targetType); - switch (targetValueType) { - case STRING: - Preconditions.checkArgument(value.getType() == PartitionValueType.STRING, - "Expected STRING argument but got %s", value.getType()); - return value.getStringValue(); - case BYTES: - if (value.getType() == PartitionValueType.BYTES) { - return value.getBytesValue(); - } - Preconditions.checkArgument(value.getType() == PartitionValueType.STRING, - "Expected STRING or BYTES argument for BYTES parameter but got %s", value.getType()); - return value.getStringValue().getBytes(UTF_8); - case INT: - switch (value.getType()) { - case INT: - return value.getIntValue(); - case LONG: - Preconditions.checkArgument(constant, "Cannot narrow LONG pipeline value to INT dynamically"); - return Math.toIntExact(value.getLongValue()); - case DOUBLE: - Preconditions.checkArgument(constant, "Cannot narrow DOUBLE pipeline value to INT dynamically"); - return (int) value.getDoubleValue(); - case STRING: - return Integer.parseInt(value.getStringValue()); - default: - throw new IllegalArgumentException( - String.format("Cannot convert %s partition value to INT", value.getType())); - } - case LONG: - switch (value.getType()) { - case INT: - return (long) value.getIntValue(); - case LONG: - return value.getLongValue(); - case DOUBLE: - Preconditions.checkArgument(constant, "Cannot narrow DOUBLE pipeline value to LONG dynamically"); - return (long) value.getDoubleValue(); - case STRING: - return Long.parseLong(value.getStringValue()); - default: - throw new IllegalArgumentException( - String.format("Cannot convert %s partition value to LONG", value.getType())); - } - case FLOAT: - switch (value.getType()) { - case INT: - return (float) value.getIntValue(); - case LONG: - return (float) value.getLongValue(); - case FLOAT: - return value.getFloatValue(); - case STRING: - return Float.parseFloat(value.getStringValue()); - default: - throw new IllegalArgumentException( - String.format("Cannot convert %s partition value to FLOAT", value.getType())); - } - case DOUBLE: - switch (value.getType()) { - case INT: - return (double) value.getIntValue(); - case LONG: - return (double) value.getLongValue(); - case FLOAT: - return (double) value.getFloatValue(); - case DOUBLE: - return value.getDoubleValue(); - case STRING: - return Double.parseDouble(value.getStringValue()); - default: - throw new IllegalArgumentException( - String.format("Cannot convert %s partition value to DOUBLE", value.getType())); - } - default: - throw new IllegalStateException("Unsupported partition parameter type: " + targetType); - } - } - - private static int wideningNumericCost(PartitionValueType targetType, Set supportedTypes) { - return supportedTypes.contains(targetType) ? 1 : -1; - } -} diff --git a/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionTest.java b/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionTest.java index 4505236f2a0e..af910e7f009a 100644 --- a/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionTest.java +++ b/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionTest.java @@ -22,7 +22,6 @@ import java.util.HashMap; import java.util.Map; import java.util.Random; -import org.apache.pinot.segment.spi.partition.pipeline.PartitionPipelineFunction; import org.apache.pinot.spi.data.readers.GenericRow; import org.apache.pinot.spi.function.FunctionEvaluator; import org.apache.pinot.spi.utils.BytesUtils; @@ -491,33 +490,6 @@ public void testUnboundLegacyPartitionFunctionDoesNotImplementFunctionEvaluator( assertFalse(partitionFunction instanceof FunctionEvaluator); } - @Test - public void testFunctionExprPartitionFunctionImplementsFunctionEvaluator() { - PartitionFunction partitionFunction = - PartitionFunctionFactory.getPartitionFunction("id", null, 128, null, "fnv1a_32(md5(id))", "MASK"); - assertTrue(partitionFunction instanceof FunctionEvaluator); - FunctionEvaluator evaluator = (FunctionEvaluator) partitionFunction; - GenericRow row = new GenericRow(); - row.putValue("id", "000016be-9d72-466c-9632-cfa680dc8fa3"); - - assertEquals(evaluator.getArguments(), java.util.List.of("id")); - assertEquals(evaluator.evaluate(row), 104); - assertEquals(evaluator.evaluate(new Object[]{"000016be-9d72-466c-9632-cfa680dc8fa3"}), 104); - } - - @Test - public void testFunctionExprPartitionFunctionSerialization() { - PartitionFunction partitionFunction = - PartitionFunctionFactory.getPartitionFunction("id", null, 128, null, "fnv1a_32(md5(id))", "MASK"); - - JsonNode jsonNode = JsonUtils.objectToJsonNode(partitionFunction); - assertEquals(partitionFunction.getName(), PartitionPipelineFunction.NAME); - assertEquals(jsonNode.get("name").asText(), PartitionPipelineFunction.NAME); - assertEquals(jsonNode.get("numPartitions").asInt(), 128); - assertEquals(jsonNode.get("functionExpr").asText(), "fnv1a_32(md5(id))"); - assertEquals(jsonNode.get("partitionIdNormalizer").asText(), "MASK"); - } - private void testBasicProperties(PartitionFunction partitionFunction, String functionName, int numPartitions) { testBasicProperties(partitionFunction, functionName, numPartitions, null); } diff --git a/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/pipeline/TestPartitionScalarFunctionResolver.java b/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/pipeline/TestPartitionScalarFunctionResolver.java deleted file mode 100644 index a37c6f032028..000000000000 --- a/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/pipeline/TestPartitionScalarFunctionResolver.java +++ /dev/null @@ -1,351 +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.spi.partition.pipeline; - -import com.google.common.base.Preconditions; -import java.lang.reflect.Array; -import java.lang.reflect.Constructor; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; -import java.lang.reflect.Modifier; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Set; -import javax.annotation.Nullable; -import org.apache.pinot.segment.spi.function.scalar.PartitionFunctionExprRacyTestFunctions; -import org.apache.pinot.segment.spi.function.scalar.PartitionFunctionExprTestFunctions; -import org.apache.pinot.spi.annotations.ScalarFunction; - - -/** - * Test-only partition scalar-function resolver for the {@code pinot-segment-spi} module. - */ -public class TestPartitionScalarFunctionResolver implements PartitionScalarFunctionResolver { - private static final Object DYNAMIC_ARGUMENT = new Object(); - private static final Map> FUNCTION_METHODS = loadFunctionMethods(); - - @Override - public ResolvedFunction resolve(String functionName, List arguments) { - List methods = FUNCTION_METHODS.get(canonicalize(functionName)); - if (methods == null) { - throw new IllegalArgumentException(String.format("Unsupported partition scalar function: %s", functionName)); - } - - BoundFunction bestMatch = null; - StringBuilder supportedSignatures = new StringBuilder(); - boolean sawNonDeterministicCandidate = false; - boolean sawDeterministicCandidate = false; - for (ScalarFunctionMethod method : methods) { - if (supportedSignatures.length() > 0) { - supportedSignatures.append(", "); - } - supportedSignatures.append(method.getSignature()); - - if (!method.isDeterministic()) { - sawNonDeterministicCandidate = true; - continue; - } - sawDeterministicCandidate = true; - - BoundFunction candidate = method.bind(arguments); - if (candidate != null) { - if (bestMatch == null || candidate.getCost() < bestMatch.getCost()) { - bestMatch = candidate; - } else if (candidate.getCost() == bestMatch.getCost()) { - throw new IllegalArgumentException(String.format( - "Ambiguous partition scalar function '%s' for argument types (%s). Matching signatures: %s", - functionName, formatArgumentTypes(arguments), supportedSignatures)); - } - } - } - - if (bestMatch != null) { - return bestMatch; - } - if (sawNonDeterministicCandidate && !sawDeterministicCandidate) { - throw new IllegalArgumentException(String.format( - "Partition scalar function '%s' is not allowed because it is non-deterministic", functionName)); - } - throw new IllegalArgumentException(String.format( - "Function '%s' does not accept argument types (%s). Supported signatures: %s", functionName, - formatArgumentTypes(arguments), supportedSignatures)); - } - - private static Map> loadFunctionMethods() { - Map> functionMethods = new HashMap<>(); - registerMethods(functionMethods, PartitionFunctionExprTestFunctions.class); - registerMethods(functionMethods, PartitionFunctionExprRacyTestFunctions.class); - - Map> immutable = new HashMap<>(functionMethods.size()); - for (Map.Entry> entry : functionMethods.entrySet()) { - immutable.put(entry.getKey(), Collections.unmodifiableList(entry.getValue())); - } - return Collections.unmodifiableMap(immutable); - } - - private static void registerMethods(Map> functionMethods, Class clazz) { - for (Method method : clazz.getMethods()) { - ScalarFunction scalarFunction = method.getAnnotation(ScalarFunction.class); - if (scalarFunction == null || !scalarFunction.enabled() || !Modifier.isPublic(method.getModifiers())) { - continue; - } - ScalarFunctionMethod functionMethod = new ScalarFunctionMethod(method, scalarFunction.isDeterministic()); - Set canonicalNames = new LinkedHashSet<>(); - canonicalNames.add(canonicalize(method.getName())); - for (String name : scalarFunction.names()) { - canonicalNames.add(canonicalize(name)); - } - for (String canonicalName : canonicalNames) { - functionMethods.computeIfAbsent(canonicalName, ignored -> new ArrayList<>()).add(functionMethod); - } - } - } - - private static String canonicalize(String name) { - return name.replace("_", "").toLowerCase(Locale.ROOT); - } - - private static String formatArgumentTypes(List arguments) { - StringBuilder builder = new StringBuilder(); - for (int i = 0; i < arguments.size(); i++) { - if (i > 0) { - builder.append(", "); - } - builder.append(arguments.get(i).getType()); - } - return builder.toString(); - } - - @Nullable - private static Constructor getEmptyConstructor(Class clazz) { - try { - return clazz.getConstructor(); - } catch (NoSuchMethodException e) { - return null; - } - } - - private static final class ScalarFunctionMethod { - private final Method _method; - private final boolean _staticMethod; - @Nullable - private final ThreadLocal _threadLocalTarget; - private final Class[] _parameterTypes; - private final boolean _varArgs; - private final boolean _deterministic; - private final PartitionValueType _outputType; - private final String _signature; - - private ScalarFunctionMethod(Method method, boolean deterministic) { - _method = method; - _staticMethod = Modifier.isStatic(method.getModifiers()); - _parameterTypes = method.getParameterTypes(); - _varArgs = method.isVarArgs(); - _deterministic = deterministic; - _outputType = PartitionValueType.fromJavaType(method.getReturnType()); - _signature = buildSignature(method); - _threadLocalTarget = _staticMethod ? null : buildThreadLocalTarget(method); - } - - public boolean isDeterministic() { - return _deterministic; - } - - public String getSignature() { - return _signature; - } - - @Nullable - public BoundFunction bind(List arguments) { - int parameterCount = _parameterTypes.length; - int fixedParameterCount = _varArgs ? parameterCount - 1 : parameterCount; - if ((!_varArgs && arguments.size() != parameterCount) || (_varArgs && arguments.size() < fixedParameterCount)) { - return null; - } - - Object[] constantArguments = new Object[arguments.size()]; - int totalCost = 0; - int dynamicIndex = -1; - PartitionValueType inputType = PartitionValueType.STRING; - Class dynamicParameterType = String.class; - for (int i = 0; i < arguments.size(); i++) { - Argument argument = arguments.get(i); - Class parameterType = getParameterType(i); - if (argument.isDynamic()) { - if (dynamicIndex >= 0) { - return null; - } - int cost = PartitionValueConversions.getDynamicConversionCost(argument.getType(), parameterType); - if (cost < 0) { - return null; - } - totalCost += cost; - dynamicIndex = i; - inputType = argument.getType(); - dynamicParameterType = parameterType; - constantArguments[i] = DYNAMIC_ARGUMENT; - } else { - PartitionValue constantValue = - Preconditions.checkNotNull(argument.getConstantValue(), "Constant argument must be configured"); - int cost = PartitionValueConversions.getConstantConversionCost(constantValue, parameterType); - if (cost < 0) { - return null; - } - totalCost += cost; - constantArguments[i] = PartitionValueConversions.convertValue(constantValue, parameterType, true); - } - } - return new BoundFunction(this, inputType, dynamicParameterType, dynamicIndex, constantArguments, totalCost); - } - - public PartitionValueType getOutputType() { - return _outputType; - } - - private Class getParameterType(int index) { - if (_varArgs && index >= _parameterTypes.length - 1) { - return _parameterTypes[_parameterTypes.length - 1].getComponentType(); - } - return _parameterTypes[index]; - } - - private Object invoke(Object[] expressionArguments) { - try { - Object target = _staticMethod ? null : Preconditions.checkNotNull(_threadLocalTarget).get(); - return _method.invoke(target, adaptArguments(expressionArguments)); - } catch (IllegalAccessException e) { - throw new IllegalStateException("Failed to invoke partition scalar function: " + _signature, e); - } catch (InvocationTargetException e) { - Throwable cause = e.getTargetException(); - if (cause instanceof RuntimeException) { - throw (RuntimeException) cause; - } - throw new IllegalStateException("Caught checked exception while invoking partition scalar function: " - + _signature, cause); - } - } - - private Object[] adaptArguments(Object[] expressionArguments) { - if (!_varArgs) { - return expressionArguments; - } - - int fixedParameterCount = _parameterTypes.length - 1; - Object[] methodArguments = new Object[_parameterTypes.length]; - for (int i = 0; i < fixedParameterCount; i++) { - methodArguments[i] = expressionArguments[i]; - } - Class componentType = _parameterTypes[_parameterTypes.length - 1].getComponentType(); - int varArgCount = expressionArguments.length - fixedParameterCount; - Object varArgArray = Array.newInstance(componentType, varArgCount); - for (int i = 0; i < varArgCount; i++) { - Array.set(varArgArray, i, expressionArguments[fixedParameterCount + i]); - } - methodArguments[_parameterTypes.length - 1] = varArgArray; - return methodArguments; - } - - private static ThreadLocal buildThreadLocalTarget(Method method) { - Constructor constructor = getEmptyConstructor(method.getDeclaringClass()); - Preconditions.checkState(constructor != null, - "Non-static partition scalar function must have an empty constructor: %s", method); - return ThreadLocal.withInitial(() -> instantiateTarget(constructor, method)); - } - - private static Object instantiateTarget(Constructor constructor, Method method) { - try { - return constructor.newInstance(); - } catch (Exception e) { - throw new IllegalStateException("Failed to instantiate partition scalar function target: " + method, e); - } - } - - private static String buildSignature(Method method) { - StringBuilder builder = new StringBuilder(method.getName()).append('('); - Class[] parameterTypes = method.getParameterTypes(); - for (int i = 0; i < parameterTypes.length; i++) { - if (i > 0) { - builder.append(", "); - } - Class parameterType = parameterTypes[i]; - if (method.isVarArgs() && i == parameterTypes.length - 1) { - builder.append(parameterType.getComponentType().getSimpleName()).append("..."); - } else { - builder.append(parameterType.getSimpleName()); - } - } - return builder.append(')').toString(); - } - } - - private static final class BoundFunction implements ResolvedFunction { - private final ScalarFunctionMethod _method; - private final PartitionValueType _inputType; - private final Class _dynamicParameterType; - private final int _dynamicIndex; - private final Object[] _constantArguments; - @Nullable - private final ThreadLocal _threadLocalArguments; - private final int _cost; - - private BoundFunction(ScalarFunctionMethod method, PartitionValueType inputType, Class dynamicParameterType, - int dynamicIndex, Object[] constantArguments, int cost) { - _method = method; - _inputType = inputType; - _dynamicParameterType = dynamicParameterType; - _dynamicIndex = dynamicIndex; - _constantArguments = constantArguments; - _threadLocalArguments = dynamicIndex >= 0 ? ThreadLocal.withInitial(_constantArguments::clone) : null; - _cost = cost; - } - - public int getCost() { - return _cost; - } - - @Override - public boolean isDynamic() { - return _dynamicIndex >= 0; - } - - @Override - public PartitionValueType getOutputType() { - return _method.getOutputType(); - } - - @Override - public PartitionValue invoke(@Nullable PartitionValue dynamicInput) { - Object[] expressionArguments = _dynamicIndex >= 0 ? Preconditions.checkNotNull(_threadLocalArguments).get() - : _constantArguments; - if (_dynamicIndex >= 0) { - Preconditions.checkNotNull(dynamicInput, "Dynamic partition step input must be configured"); - Preconditions.checkArgument(dynamicInput.getType() == _inputType, - "Expected %s dynamic input but got %s", _inputType, dynamicInput.getType()); - expressionArguments[_dynamicIndex] = PartitionValueConversions.convertValue(dynamicInput, _dynamicParameterType, - false); - } - return PartitionValue.fromObject(_method.invoke(expressionArguments)); - } - } -} diff --git a/pinot-segment-spi/src/test/resources/META-INF/services/org.apache.pinot.segment.spi.partition.pipeline.PartitionScalarFunctionResolver b/pinot-segment-spi/src/test/resources/META-INF/services/org.apache.pinot.segment.spi.partition.pipeline.PartitionScalarFunctionResolver deleted file mode 100644 index 4aea5dfbd380..000000000000 --- a/pinot-segment-spi/src/test/resources/META-INF/services/org.apache.pinot.segment.spi.partition.pipeline.PartitionScalarFunctionResolver +++ /dev/null @@ -1,19 +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. -# -org.apache.pinot.segment.spi.partition.pipeline.TestPartitionScalarFunctionResolver From 5b7510343cab6b9c7f1a55948182fd9fb01c485f Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Sun, 19 Apr 2026 08:14:46 -0700 Subject: [PATCH 03/27] Polish code review findings: final class, private fields, cached args, fix Javadoc refs MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - Declare PartitionFunctionEvaluator final (thread-safety contract) - Make PartitionFunctionExecutionNode fields private final - Cache getArguments() result in PartitionPipeline and delegate from PartitionPipelineFunction to avoid singletonList allocation on every call - Fix stale Javadoc references in PartitionEvaluatorFactory and PartitionFunctionExprCompiler (was: InbuiltFunctionEvaluator, now: PartitionFunctionEvaluator) - Add assertEquals(roundTripped, metadata) to ColumnPartitionMetadataTest round-trip tests to cover the hashCode() fix - Delete PartitionFunctionExprRacyTestFunctions (dead code — the test using it was removed) Co-Authored-By: Claude Sonnet 4.6 --- .../evaluator/PartitionFunctionEvaluator.java | 10 +-- ...artitionFunctionExprRacyTestFunctions.java | 62 ------------------- .../metadata/ColumnPartitionMetadataTest.java | 2 + .../pipeline/PartitionEvaluatorFactory.java | 5 +- .../PartitionFunctionExprCompiler.java | 4 +- .../partition/pipeline/PartitionPipeline.java | 4 +- .../pipeline/PartitionPipelineFunction.java | 3 +- 7 files changed, 16 insertions(+), 74 deletions(-) delete mode 100644 pinot-common/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprRacyTestFunctions.java diff --git a/pinot-common/src/main/java/org/apache/pinot/common/evaluator/PartitionFunctionEvaluator.java b/pinot-common/src/main/java/org/apache/pinot/common/evaluator/PartitionFunctionEvaluator.java index 27d02a202c5f..3f3d6b296d79 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/evaluator/PartitionFunctionEvaluator.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/evaluator/PartitionFunctionEvaluator.java @@ -55,7 +55,7 @@ * {@link ThreadLocal} scratch array so that concurrent invocations from different threads do not share mutable * state. A single call chain on one thread is not reentrant. */ -public class PartitionFunctionEvaluator extends ExecutableFunctionEvaluator { +public final class PartitionFunctionEvaluator extends ExecutableFunctionEvaluator { public PartitionFunctionEvaluator(String functionExpression) { this(functionExpression, new ArrayList<>()); @@ -131,10 +131,10 @@ private static ExecutableNode planExecution(ExpressionContext expression, List _arguments; + private final FunctionInvoker _functionInvoker; + private final FunctionInfo _functionInfo; + private final ExecutableNode[] _argumentNodes; + private final ThreadLocal _arguments; PartitionFunctionExecutionNode(FunctionInfo functionInfo, ExecutableNode[] argumentNodes) { _functionInvoker = new FunctionInvoker(functionInfo); diff --git a/pinot-common/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprRacyTestFunctions.java b/pinot-common/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprRacyTestFunctions.java deleted file mode 100644 index f55fe626fcd8..000000000000 --- a/pinot-common/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprRacyTestFunctions.java +++ /dev/null @@ -1,62 +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.spi.function.scalar; - -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import org.apache.pinot.spi.annotations.ScalarFunction; - - -/** - * Test-only non-static scalar function with mutable instance state used to validate thread-safe target handling for - * partition-expression scalar methods. - */ -public class PartitionFunctionExprRacyTestFunctions { - private static volatile CountDownLatch _firstEntered = new CountDownLatch(1); - private static volatile CountDownLatch _secondCompleted = new CountDownLatch(1); - - private String _value; - - public static void reset() { - _firstEntered = new CountDownLatch(1); - _secondCompleted = new CountDownLatch(1); - } - - @ScalarFunction(names = {"racy_echo"}) - public String racyEcho(String input) { - _value = input; - try { - if ("first".equals(input)) { - _firstEntered.countDown(); - if (!_secondCompleted.await(5, TimeUnit.SECONDS)) { - throw new IllegalStateException("Timed out waiting for second invocation"); - } - } else if ("second".equals(input)) { - if (!_firstEntered.await(5, TimeUnit.SECONDS)) { - throw new IllegalStateException("Timed out waiting for first invocation"); - } - _secondCompleted.countDown(); - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new IllegalStateException("Interrupted while coordinating racy scalar function", e); - } - return _value; - } -} diff --git a/pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadataTest.java b/pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadataTest.java index e6810113ec4f..b6abb1100210 100644 --- a/pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadataTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadataTest.java @@ -43,6 +43,7 @@ public void testRoundTripTreatsNullExpressionFieldsAsAbsent() assertEquals(roundTripped.getPartitions(), metadata.getPartitions()); assertNull(roundTripped.getFunctionExpr()); assertNull(roundTripped.getPartitionIdNormalizer()); + assertEquals(roundTripped, metadata); } @Test @@ -83,5 +84,6 @@ public void testExpressionModeRoundTripPreservesAllFields() assertEquals(roundTripped.getNumPartitions(), 64); assertEquals(roundTripped.getPartitions(), Set.of(7, 15)); assertNull(roundTripped.getInputType()); + assertEquals(roundTripped, metadata); } } diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionEvaluatorFactory.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionEvaluatorFactory.java index ac9b6b61c508..7392daf7236f 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionEvaluatorFactory.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionEvaluatorFactory.java @@ -25,8 +25,9 @@ * SPI for compiling a partition expression string into a {@link FunctionEvaluator}. * *

Implementations are loaded via {@link java.util.ServiceLoader}. Exactly one implementation must be present on the - * classpath at runtime. The default implementation in {@code pinot-common} delegates to - * {@link org.apache.pinot.common.evaluator.InbuiltFunctionEvaluator}. + * classpath at runtime. The default implementation in {@code pinot-common} is + * {@code org.apache.pinot.common.evaluator.InbuiltPartitionEvaluatorFactory}, which delegates to + * {@code org.apache.pinot.common.evaluator.PartitionFunctionEvaluator}. */ public interface PartitionEvaluatorFactory { diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java index 3d3cbdc14991..f35a64921824 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java @@ -29,8 +29,8 @@ /** - * Compiles a partition-function expression into a {@link PartitionPipeline} backed by - * {@link org.apache.pinot.common.evaluator.InbuiltFunctionEvaluator} via {@link PartitionEvaluatorFactory}. + * Compiles a partition-function expression into a {@link PartitionPipeline} backed by a + * {@link org.apache.pinot.spi.function.FunctionEvaluator} provided by {@link PartitionEvaluatorFactory}. */ public final class PartitionFunctionExprCompiler { private static final int MAX_EXPRESSION_LENGTH = 256; diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipeline.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipeline.java index 46569d57531b..53922b953851 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipeline.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipeline.java @@ -43,6 +43,7 @@ public final class PartitionPipeline implements FunctionEvaluator { @Nullable private final PartitionIntNormalizer _intNormalizer; private final FunctionEvaluator _evaluator; + private final List _arguments; PartitionPipeline(String rawColumn, boolean isBytesInput, String canonicalFunctionExpr, @Nullable PartitionIntNormalizer intNormalizer, FunctionEvaluator evaluator) { @@ -54,6 +55,7 @@ public final class PartitionPipeline implements FunctionEvaluator { _canonicalFunctionExpr = canonicalFunctionExpr; _intNormalizer = intNormalizer; _evaluator = evaluator; + _arguments = Collections.singletonList(rawColumn); } public String getRawColumn() { @@ -75,7 +77,7 @@ public PartitionIntNormalizer getIntNormalizer() { @Override public List getArguments() { - return Collections.singletonList(_rawColumn); + return _arguments; } @Override diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java index 48493a93f6d2..95b0141a1af7 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import java.math.BigInteger; -import java.util.Collections; import java.util.List; import org.apache.pinot.segment.spi.partition.PartitionFunction; import org.apache.pinot.spi.data.FieldSpec; @@ -128,7 +127,7 @@ public String getPartitionIdNormalizer() { @Override public List getArguments() { - return Collections.singletonList(_pipeline.getRawColumn()); + return _pipeline.getArguments(); } @Override From 793570893b327f8676bfe40cd5b957b0ab61b6fb Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Mon, 20 Apr 2026 00:38:20 -0700 Subject: [PATCH 04/27] Add FunctionNameUtils to pinot-spi; have ScalarFunctionUtils and FunctionUtils delegate to it Addresses review feedback: canonicalize() applies to all function types, not just scalar functions, so it belongs in a general-purpose FunctionNameUtils class rather than the scalar-function-specific ScalarFunctionUtils. Both ScalarFunctionUtils.canonicalize() and FunctionUtils.canonicalize() are kept as delegates for backward compatibility. Co-Authored-By: Claude Sonnet 4.6 --- .../pinot/common/function/FunctionUtils.java | 11 +++-- .../pinot/spi/utils/FunctionNameUtils.java | 44 +++++++++++++++++++ .../pinot/spi/utils/ScalarFunctionUtils.java | 12 ++--- 3 files changed, 52 insertions(+), 15 deletions(-) create mode 100644 pinot-spi/src/main/java/org/apache/pinot/spi/utils/FunctionNameUtils.java diff --git a/pinot-common/src/main/java/org/apache/pinot/common/function/FunctionUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/function/FunctionUtils.java index 4348ae4528f8..6e43041fd6e2 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/function/FunctionUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/function/FunctionUtils.java @@ -22,15 +22,14 @@ import java.sql.Timestamp; import java.util.Collection; import java.util.HashMap; -import java.util.Locale; import java.util.Map; import javax.annotation.Nullable; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.sql.type.SqlTypeName; -import org.apache.commons.lang3.StringUtils; import org.apache.pinot.common.utils.DataSchema.ColumnDataType; import org.apache.pinot.common.utils.PinotDataType; +import org.apache.pinot.spi.utils.FunctionNameUtils; public class FunctionUtils { @@ -220,11 +219,11 @@ public static boolean isAssertEnabled() { } /** - * Returns the canonical form of a scalar function name: underscores stripped, lower-cased. - * This is the single authoritative implementation used by {@link FunctionRegistry} and other - * function-resolution paths. + * Returns the canonical form of a function name: underscores stripped, lower-cased. + * + * @see FunctionNameUtils#canonicalize(String) */ public static String canonicalize(String name) { - return StringUtils.remove(name, '_').toLowerCase(Locale.ROOT); + return FunctionNameUtils.canonicalize(name); } } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/FunctionNameUtils.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/FunctionNameUtils.java new file mode 100644 index 000000000000..2504ff96a8e6 --- /dev/null +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/FunctionNameUtils.java @@ -0,0 +1,44 @@ +/** + * 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.spi.utils; + +import java.util.Locale; +import org.apache.commons.lang3.StringUtils; + + +/** + * Utilities for normalizing function names across Pinot. + * + *

This is the single authoritative implementation used by {@code FunctionRegistry}, + * {@code ScalarFunctionUtils}, and other function-resolution paths throughout the codebase. + */ +public final class FunctionNameUtils { + private FunctionNameUtils() { + } + + /** + * Returns the canonical form of a function name: underscores stripped, lower-cased. + * + *

Both scalar functions and aggregate functions use this normalization so that + * {@code fnv1a_32}, {@code FNV1A32}, and {@code fnv1a32} all resolve to the same canonical name. + */ + public static String canonicalize(String name) { + return StringUtils.remove(name, '_').toLowerCase(Locale.ROOT); + } +} diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/ScalarFunctionUtils.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/ScalarFunctionUtils.java index ff50e6e4cdad..f7fd1d5ba2d1 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/ScalarFunctionUtils.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/ScalarFunctionUtils.java @@ -21,9 +21,7 @@ import java.lang.reflect.Method; import java.util.LinkedHashSet; import java.util.List; -import java.util.Locale; import java.util.Set; -import org.apache.commons.lang3.StringUtils; import org.apache.pinot.spi.annotations.ScalarFunction; @@ -58,15 +56,11 @@ public static List getScalarFunctionNames(ScalarFunction scalarFunction, } /** - * Returns the canonical form of a scalar-function name: underscores stripped, lower-cased. + * Returns the canonical form of a function name: underscores stripped, lower-cased. * - *

IMPORTANT: This method is intentionally a duplicate of - * {@code org.apache.pinot.common.function.FunctionUtils#canonicalize} because {@code pinot-spi} - * cannot depend on {@code pinot-common}. Both implementations must produce identical - * results. If either changes, the other must be updated in the same commit to avoid silent - * canonicalization divergence between function registration and name lookup. + * @see FunctionNameUtils#canonicalize(String) */ public static String canonicalize(String name) { - return StringUtils.remove(name, '_').toLowerCase(Locale.ROOT); + return FunctionNameUtils.canonicalize(name); } } From bd26f3b7751cf25d83ddf55763cdf7f0d1c5ae2d Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Mon, 20 Apr 2026 02:58:43 -0700 Subject: [PATCH 05/27] Fix BYTES input type mismatch in realtime paths; reject Float/Double partition results; fix null-vs-empty functionConfig comparison - RealtimeSegmentDataManager and StatelessRealtimeSegmentWriter now pass the FieldSpec to getPartitionFunction so BYTES-typed partition columns compile with PartitionValueType.BYTES, matching the offline path (previously always compiled with STRING input) - PartitionPipelineFunction.normalizeResult() now explicitly rejects Float/Double results instead of silently truncating them to int (fail-fast: misconfigured expressions surface at evaluation time rather than producing wrong partition IDs) - SegmentPartitionMetadataManager.functionConfigsMatch() treats null and empty map as equal to avoid false mismatches between table config (null) and deserialized metadata ({}) - Document the BYTES caveat on PartitionFunctionFactory.getPartitionFunction(String, ColumnPartitionConfig, int) Co-Authored-By: Claude Sonnet 4.6 --- .../SegmentPartitionMetadataManager.java | 11 ++++++++++- .../manager/realtime/RealtimeSegmentDataManager.java | 3 ++- .../writer/StatelessRealtimeSegmentWriter.java | 3 ++- .../spi/partition/PartitionFunctionFactory.java | 7 +++++++ .../partition/pipeline/PartitionPipelineFunction.java | 3 +++ 5 files changed, 24 insertions(+), 3 deletions(-) diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManager.java b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManager.java index 9a3609f00a1d..05f0c5134a5a 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManager.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManager.java @@ -126,12 +126,21 @@ private String getPartitionFunctionDescription() { private boolean isMatchingPartitionFunction(PartitionFunction partitionFunction) { return _partitionFunction.getName().equalsIgnoreCase(partitionFunction.getName()) - && Objects.equals(_partitionFunction.getFunctionConfig(), partitionFunction.getFunctionConfig()) + && functionConfigsMatch(_partitionFunction.getFunctionConfig(), partitionFunction.getFunctionConfig()) && Objects.equals(_partitionFunction.getFunctionExpr(), partitionFunction.getFunctionExpr()) && equalsIgnoreCaseNullable(_partitionFunction.getPartitionIdNormalizer(), partitionFunction.getPartitionIdNormalizer()); } + private static boolean functionConfigsMatch(@Nullable Map a, @Nullable Map b) { + boolean aEmpty = a == null || a.isEmpty(); + boolean bEmpty = b == null || b.isEmpty(); + if (aEmpty && bEmpty) { + return true; + } + return Objects.equals(a, b); + } + private static boolean equalsIgnoreCaseNullable(@Nullable String a, @Nullable String b) { if (a == null) { return b == null; diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java index 72fb44ac0de4..2f16aca42a1c 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java @@ -2045,7 +2045,8 @@ private void setPartitionParameters(RealtimeSegmentConfig.Builder realtimeSegmen realtimeSegmentConfigBuilder.setPartitionColumn(partitionColumn); realtimeSegmentConfigBuilder.setPartitionFunction( - PartitionFunctionFactory.getPartitionFunction(partitionColumn, columnPartitionConfig, numPartitions)); + PartitionFunctionFactory.getPartitionFunction(partitionColumn, columnPartitionConfig, + _schema.getFieldSpecFor(partitionColumn))); realtimeSegmentConfigBuilder.setPartitionId(_partitionGroupId); } else { _segmentLogger.warn("Cannot partition on multiple columns: {}", columnPartitionMap.keySet()); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java index 593927d3f577..dcbceb33f1b0 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java @@ -438,7 +438,8 @@ private void setPartitionParameters(RealtimeSegmentConfig.Builder realtimeSegmen realtimeSegmentConfigBuilder.setPartitionColumn(partitionColumn); realtimeSegmentConfigBuilder.setPartitionFunction( - PartitionFunctionFactory.getPartitionFunction(partitionColumn, columnPartitionConfig, numPartitions)); + PartitionFunctionFactory.getPartitionFunction(partitionColumn, columnPartitionConfig, + _schema.getFieldSpecFor(partitionColumn))); realtimeSegmentConfigBuilder.setPartitionId(_partitionGroupId); } else { _logger.warn("Cannot partition on multiple columns: {}", columnPartitionMap.keySet()); diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java index 5a83db427ac1..865c92ae65e9 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java @@ -152,6 +152,13 @@ public static PartitionFunction getPartitionFunction(String columnName, ColumnPa return getPartitionFunction(columnName, columnPartitionConfig, columnPartitionConfig.getNumPartitions()); } + /** + * Builds a partition function for the given column with an explicit numPartitions override. + * + *

CAUTION: For BYTES-typed partition columns this overload always compiles the expression pipeline with + * STRING input. Use {@link #getPartitionFunction(String, ColumnPartitionConfig, FieldSpec)} instead so that BYTES + * columns receive the correct input type. + */ public static PartitionFunction getPartitionFunction(String columnName, ColumnPartitionConfig columnPartitionConfig, int numPartitions) { Preconditions.checkNotNull(columnPartitionConfig, "Column partition config must be configured"); diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java index 95b0141a1af7..d625586a445d 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java @@ -82,6 +82,9 @@ private int normalizeResult(Object result) { "Partition expression for column '%s' must return a numeric value, got: %s", _pipeline.getRawColumn(), result.getClass().getSimpleName()); Number num = (Number) result; + Preconditions.checkState(!(num instanceof Float) && !(num instanceof Double), + "Partition expression for column '%s' must return an integral value (int/long), got: %s (%s)", + _pipeline.getRawColumn(), result, result.getClass().getSimpleName()); PartitionIntNormalizer intNormalizer = _pipeline.getIntNormalizer(); Preconditions.checkState(intNormalizer != null, "Integral-output partition pipeline for column '%s' must have an INT normalizer", From d0644cf60701b581b0165436236f1efac3a5bf44 Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Mon, 20 Apr 2026 23:30:39 -0700 Subject: [PATCH 06/27] Fix BYTES input type mismatch in controller segment partition metadata PinotLLCRealtimeSegmentManager.buildPartitionMetadataForNewSegment was calling the (String, ColumnPartitionConfig, int) overload which always compiles expression-mode partition functions with STRING input, causing BYTES-typed partition columns on the controller to produce partition IDs inconsistent with the server-side computation. Adds getPartitionFunction(String, ColumnPartitionConfig, int, FieldSpec) to PartitionFunctionFactory so callers can supply both an explicit numPartitions and the column FieldSpec for correct BYTES handling. Co-Authored-By: Claude Sonnet 4.6 --- .../PinotLLCRealtimeSegmentManager.java | 16 +++++++++++++--- .../partition/PartitionFunctionFactory.java | 19 +++++++++++++++++-- 2 files changed, 30 insertions(+), 5 deletions(-) diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java index c49bdefe71be..d2582694b933 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java @@ -122,6 +122,8 @@ import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.UpsertConfig; import org.apache.pinot.spi.config.table.assignment.InstancePartitionsType; +import org.apache.pinot.spi.data.FieldSpec; +import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.filesystem.PinotFS; import org.apache.pinot.spi.filesystem.PinotFSFactory; import org.apache.pinot.spi.stream.OffsetCriteria; @@ -1147,6 +1149,7 @@ SegmentPartitionMetadata getPartitionMetadataFromTableConfig(TableConfig tableCo Map columnPartitionMap = partitionConfig.getColumnPartitionMap(); if (columnPartitionMap.size() == 1) { Map.Entry entry = columnPartitionMap.entrySet().iterator().next(); + String columnName = entry.getKey(); ColumnPartitionConfig columnPartitionConfig = entry.getValue(); // For multi-stream tables, convert Pinot partition ID (which includes padding offset) to stream partition ID. // This ensures the partition metadata stored in ZK matches what the broker's partition function computes @@ -1170,11 +1173,18 @@ SegmentPartitionMetadata getPartitionMetadataFromTableConfig(TableConfig tableCo + "The stream partition count is used. Please update the table config accordingly.", perStreamNumPartitions, columnPartitionConfig.getNumPartitions()); } - PartitionFunction partitionFunction = - PartitionFunctionFactory.getPartitionFunction(entry.getKey(), columnPartitionConfig, perStreamNumPartitions); + Schema schema = _helixResourceManager.getTableSchema(tableConfig.getTableName()); + if (schema == null && columnPartitionConfig.getFunctionExpr() != null) { + LOGGER.warn("Unable to fetch schema for table '{}'; partition function for column '{}' will use STRING input " + + "type. BYTES-typed columns may produce incorrect partition metadata.", + tableConfig.getTableName(), columnName); + } + FieldSpec fieldSpec = schema != null ? schema.getFieldSpecFor(columnName) : null; + PartitionFunction partitionFunction = PartitionFunctionFactory + .getPartitionFunction(columnName, columnPartitionConfig, perStreamNumPartitions, fieldSpec); ColumnPartitionMetadata columnPartitionMetadata = new ColumnPartitionMetadata(partitionFunction, Collections.singleton(streamPartitionId)); - return new SegmentPartitionMetadata(Collections.singletonMap(entry.getKey(), columnPartitionMetadata)); + return new SegmentPartitionMetadata(Collections.singletonMap(columnName, columnPartitionMetadata)); } else { LOGGER.warn( "Skip persisting partition metadata because there are other than exact one partition column for table: {}", diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java index 865c92ae65e9..0e94bc50a734 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java @@ -197,11 +197,26 @@ public static PartitionFunction getPartitionFunction(String columnName, ColumnPa */ public static PartitionFunction getPartitionFunction(String columnName, ColumnPartitionConfig config, @Nullable FieldSpec fieldSpec) { + return getPartitionFunction(columnName, config, config.getNumPartitions(), fieldSpec); + } + + /** + * Builds a partition function with an explicit {@code numPartitions} override and uses {@code fieldSpec} to + * determine the correct input type for expression-mode partition functions on BYTES-typed columns. + * + *

Use this overload when the live partition count (e.g. the stream partition count) may differ from the value + * stored in the table config, so that the built function uses the authoritative count while still receiving the + * correct input type for BYTES columns. + */ + public static PartitionFunction getPartitionFunction(String columnName, ColumnPartitionConfig config, + int numPartitions, @Nullable FieldSpec fieldSpec) { if (config.getFunctionExpr() != null && fieldSpec != null && fieldSpec.getDataType().getStoredType() == FieldSpec.DataType.BYTES) { - return getPartitionFunction(columnName, config, PartitionValueType.BYTES); + return PartitionFunctionExprCompiler.compilePartitionFunction(columnName, PartitionValueType.BYTES, + config.getFunctionExpr(), numPartitions, config.getPartitionIdNormalizer()); } - return getPartitionFunction(columnName, config); + return getPartitionFunction(columnName, config.getFunctionName(), numPartitions, config.getFunctionConfig(), + config.getFunctionExpr(), config.getPartitionIdNormalizer()); } public static PartitionFunction getPartitionFunction(String columnName, @Nullable String functionName, From 4bd587265cd72016bcbb2c5305c9091923c0e288 Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Thu, 23 Apr 2026 20:22:08 -0700 Subject: [PATCH 07/27] Fix null expression result crash on ingestion and case-sensitive expr routing miss - PartitionPipelineFunction.normalizeResult: return NULL_RESULT_PARTITION_ID (-1) instead of throwing when the expression evaluates to null for a given value (e.g. null column input). Prevents segment creation from crashing on rows where the partition expression yields null. - AbstractColumnStatisticsCollector.updatePartition: skip adding the sentinel -1 to the partition set so null-expression results don't corrupt segment partition metadata. - SegmentPartitionMetadataManager.isMatchingPartitionFunction: use equalsIgnoreCaseNullable for functionExpr comparison (consistent with partitionIdNormalizer). Segments written with a mixed-case expression by an older node or a direct ZK write were previously excluded from partition-aware routing, silently degrading to full-scan. Co-Authored-By: Claude Sonnet 4.6 --- .../PartitionFunctionExprCompilerTest.java | 11 +++ .../AbstractColumnStatisticsCollector.java | 11 ++- .../PartitionFunctionExprCompiler.java | 73 +++++++++++++++++-- .../pipeline/PartitionPipelineFunction.java | 12 ++- 4 files changed, 99 insertions(+), 8 deletions(-) diff --git a/pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompilerTest.java b/pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompilerTest.java index 0e2c8f292801..16cf0dccc300 100644 --- a/pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompilerTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompilerTest.java @@ -79,6 +79,17 @@ public void testCompilePartitionFunctionWithLiteralArgument() { assertEquals(partitionFunction.getFunctionExpr(), "bucket(timestampmillis, 1000)"); } + @Test + public void testCanonicalizationPreservesQuotedLiteralPayload() { + PartitionPipelineFunction partitionFunction = + PartitionFunctionExprCompiler.compilePartitionFunction("raw_key", + "MURMUR2( CONCAT( raw_key , 'SALT Value' ) )", 64); + + int expectedHash = MurmurHashFunctions.murmurHash2("PinotSALT Value".getBytes(UTF_8)); + assertEquals(partitionFunction.getPartition("Pinot"), positiveModulo(expectedHash, 64)); + assertEquals(partitionFunction.getFunctionExpr(), "murmur2(concat(raw_key, 'SALT Value'))"); + } + @Test public void testCompiledPartitionPipelineImplementsFunctionEvaluator() { FunctionEvaluator evaluator = PartitionFunctionExprCompiler.compile("raw_key", "murmur2(lower(raw_key))"); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/AbstractColumnStatisticsCollector.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/AbstractColumnStatisticsCollector.java index 6b18675a2e52..52980d192a95 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/AbstractColumnStatisticsCollector.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/AbstractColumnStatisticsCollector.java @@ -25,6 +25,7 @@ import org.apache.pinot.segment.spi.creator.ColumnStatistics; import org.apache.pinot.segment.spi.creator.StatsCollectorConfig; import org.apache.pinot.segment.spi.partition.PartitionFunction; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionPipelineFunction; import org.apache.pinot.spi.config.table.FieldConfig; import org.apache.pinot.spi.data.FieldSpec; import org.apache.pinot.spi.data.FieldSpec.DataType; @@ -162,7 +163,10 @@ protected boolean isPartitionEnabled() { } protected void updatePartition(String value) { - _partitions.add(_partitionFunction.getPartition(value)); + int partitionId = _partitionFunction.getPartition(value); + if (partitionId != PartitionPipelineFunction.NULL_RESULT_PARTITION_ID) { + _partitions.add(partitionId); + } } /** @@ -170,6 +174,9 @@ protected void updatePartition(String value) { * the raw bytes directly; legacy partition functions fall back to hex-encoding via the default interface method. */ protected void updatePartition(byte[] bytes) { - _partitions.add(_partitionFunction.getPartition(bytes)); + int partitionId = _partitionFunction.getPartition(bytes); + if (partitionId != PartitionPipelineFunction.NULL_RESULT_PARTITION_ID) { + _partitions.add(partitionId); + } } } diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java index f35a64921824..82897438e085 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java @@ -115,12 +115,75 @@ public static PartitionPipelineFunction compilePartitionFunction(String rawColum /** * Returns a canonical form of the expression: trimmed, lowercased, with spaces removed around - * {@code (}, {@code )}, and {@code ,}. + * {@code (}, {@code )}, and {@code ,}. Quoted literals are preserved byte-for-byte because partition expressions + * can use case-sensitive salts, JSON paths, or regex patterns as function arguments. */ static String canonicalize(String expression) { - return expression.trim().toLowerCase(Locale.ROOT) - .replaceAll("\\s*\\(\\s*", "(") - .replaceAll("\\s*\\)\\s*", ")") - .replaceAll("\\s*,\\s*", ", "); + String trimmed = expression.trim(); + StringBuilder builder = new StringBuilder(trimmed.length()); + boolean pendingSpace = false; + boolean afterComma = false; + char quote = 0; + int i = 0; + while (i < trimmed.length()) { + char ch = trimmed.charAt(i); + if (quote != 0) { + builder.append(ch); + if (ch == quote) { + // SQL escapes quotes inside string literals by doubling them. Preserve both characters and stay inside the + // literal so canonicalization cannot alter the literal payload. + if (i + 1 < trimmed.length() && trimmed.charAt(i + 1) == quote) { + builder.append(trimmed.charAt(++i)); + } else { + quote = 0; + } + } + i++; + continue; + } + if (ch == '\'' || ch == '"') { + appendPendingSpace(builder, pendingSpace, afterComma); + pendingSpace = false; + afterComma = false; + quote = ch; + builder.append(ch); + i++; + continue; + } + if (Character.isWhitespace(ch)) { + pendingSpace = true; + i++; + continue; + } + if (ch == '(' || ch == ')') { + pendingSpace = false; + afterComma = false; + builder.append(ch); + i++; + continue; + } + if (ch == ',') { + pendingSpace = false; + builder.append(", "); + afterComma = true; + i++; + continue; + } + appendPendingSpace(builder, pendingSpace, afterComma); + pendingSpace = false; + afterComma = false; + builder.append(String.valueOf(ch).toLowerCase(Locale.ROOT)); + i++; + } + int length = builder.length(); + return length > 0 && builder.charAt(length - 1) == ' ' + ? builder.substring(0, length - 1) + : builder.toString(); + } + + private static void appendPendingSpace(StringBuilder builder, boolean pendingSpace, boolean afterComma) { + if (pendingSpace && !afterComma && builder.length() > 0 && builder.charAt(builder.length() - 1) != '(') { + builder.append(' '); + } } } diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java index d625586a445d..2f7f0c9d7ffb 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java @@ -28,6 +28,8 @@ import org.apache.pinot.spi.data.readers.GenericRow; import org.apache.pinot.spi.function.FunctionEvaluator; import org.apache.pinot.spi.utils.BytesUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** @@ -38,7 +40,11 @@ */ @SuppressWarnings("serial") public class PartitionPipelineFunction implements PartitionFunction, FunctionEvaluator { + private static final Logger LOGGER = LoggerFactory.getLogger(PartitionPipelineFunction.class); public static final String NAME = "FunctionExpr"; + // Sentinel returned when the partition expression evaluates to null for a given value (e.g. null input). + // Callers that update per-segment partition sets must skip this value. + public static final int NULL_RESULT_PARTITION_ID = -1; private final PartitionPipeline _pipeline; private final int _numPartitions; @@ -77,7 +83,11 @@ public int getPartition(byte[] bytes) { } private int normalizeResult(Object result) { - Preconditions.checkNotNull(result, "Partition expression for column '%s' returned null", _pipeline.getRawColumn()); + if (result == null) { + LOGGER.debug("Partition expression for column '{}' returned null; skipping partition assignment for this value", + _pipeline.getRawColumn()); + return NULL_RESULT_PARTITION_ID; + } Preconditions.checkState(result instanceof Number, "Partition expression for column '%s' must return a numeric value, got: %s", _pipeline.getRawColumn(), result.getClass().getSimpleName()); From ae62939b3eec5f77886b467bcadf0dab6184c3e9 Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Fri, 24 Apr 2026 16:02:55 -0700 Subject: [PATCH 08/27] Fix compile error: declare local fieldType in addColumnMetadataInfo The rebase onto upstream/master preserved references to the `fieldType` variable in the reorganized datetime/complex-field blocks but lost the local declaration. Re-declare it from fieldSpec.getFieldType() so pinot-segment-local compiles again. Co-Authored-By: Claude Sonnet 4.6 --- .../segment/local/segment/creator/impl/BaseSegmentCreator.java | 1 + 1 file changed, 1 insertion(+) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/BaseSegmentCreator.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/BaseSegmentCreator.java index 39437cdab4cc..6481b47bc5ad 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/BaseSegmentCreator.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/BaseSegmentCreator.java @@ -644,6 +644,7 @@ public static void addColumnMetadataInfo(PropertiesConfiguration properties, Str } } + FieldType fieldType = fieldSpec.getFieldType(); // Datetime field if (fieldType == FieldType.DATE_TIME) { DateTimeFieldSpec dateTimeFieldSpec = (DateTimeFieldSpec) fieldSpec; From de05a18ef3713a98ec7ab2eca7bcda4418fcf029 Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Fri, 24 Apr 2026 16:30:05 -0700 Subject: [PATCH 09/27] Accept integral-valued Float/Double partition expression results MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Pinot scalar functions commonly box integral arithmetic to Double (e.g. plus(long, long) → Double), so an expression like plus(intDiv(col, 1000), 7) may legitimately yield 61.0 (Double) for input that should partition to id 61. The previous check rejected any Float/Double outright, breaking valid expressions. Relax normalizeResult to accept Float/Double iff the value is finite and has no fractional part (NaN/Infinity/non-integral values still rejected since they cannot map to a stable partition id). Fixes CI failure: PartitionFunctionExprCommonScalarFunctionTest .testClassBasedScalarFunctionSupportsRawStringNumericInput Co-Authored-By: Claude Sonnet 4.6 --- .../pipeline/PartitionPipelineFunction.java | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java index 2f7f0c9d7ffb..97ed1a5054c9 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java @@ -92,14 +92,20 @@ private int normalizeResult(Object result) { "Partition expression for column '%s' must return a numeric value, got: %s", _pipeline.getRawColumn(), result.getClass().getSimpleName()); Number num = (Number) result; - Preconditions.checkState(!(num instanceof Float) && !(num instanceof Double), - "Partition expression for column '%s' must return an integral value (int/long), got: %s (%s)", - _pipeline.getRawColumn(), result, result.getClass().getSimpleName()); + // Pinot scalar functions commonly box integral arithmetic to Double (e.g. plus(long, long) → Double). Accept + // Float/Double iff the value is integral (no fractional part); reject genuinely fractional values since they + // cannot map to a stable partition id. + if (num instanceof Float || num instanceof Double) { + double d = num.doubleValue(); + Preconditions.checkState(!Double.isNaN(d) && !Double.isInfinite(d) && d == Math.floor(d), + "Partition expression for column '%s' must return an integral value (int/long), got: %s (%s)", + _pipeline.getRawColumn(), result, result.getClass().getSimpleName()); + } PartitionIntNormalizer intNormalizer = _pipeline.getIntNormalizer(); Preconditions.checkState(intNormalizer != null, "Integral-output partition pipeline for column '%s' must have an INT normalizer", _pipeline.getRawColumn()); - if (num instanceof Long || num instanceof BigInteger) { + if (num instanceof Long || num instanceof BigInteger || num instanceof Float || num instanceof Double) { return intNormalizer.getPartitionId(num.longValue(), _numPartitions); } return intNormalizer.getPartitionId(num.intValue(), _numPartitions); From cc707b6b84136e6892aaab12755745b5136ac15b Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Fri, 24 Apr 2026 17:15:28 -0700 Subject: [PATCH 10/27] Validate expression-mode partition function output type at table-config validation MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The test testRejectInvalidSegmentPartitionFunctionExpr expects TableConfigUtils.validate to reject configs whose partition expression yields a non-numeric type (e.g. md5(col) returns STRING). The check was missing — validation only built the partition function but never probed its output type, so misconfigured tables would pass validation and fail later at segment creation. Add PartitionPipelineFunction.validateOutputType() which probes the compiled expression with a sample value and throws IllegalArgumentException ("Partition pipeline must produce INT or LONG output, got: ") when the result is non-numeric. Wire it into TableConfigUtils.validate after the existing partition-function build call. Probe failures (RuntimeException during sample evaluation) skip output validation and let runtime ingestion surface the error. Co-Authored-By: Claude Sonnet 4.6 --- .../segment/local/utils/TableConfigUtils.java | 8 +++- .../pipeline/PartitionPipelineFunction.java | 45 +++++++++++++++++++ 2 files changed, 52 insertions(+), 1 deletion(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java index dfae571e4f52..f6886109f747 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java @@ -58,7 +58,9 @@ import org.apache.pinot.segment.spi.index.StandardIndexes; import org.apache.pinot.segment.spi.index.multicolumntext.MultiColumnTextMetadata; import org.apache.pinot.segment.spi.index.startree.AggregationFunctionColumnPair; +import org.apache.pinot.segment.spi.partition.PartitionFunction; import org.apache.pinot.segment.spi.partition.PartitionFunctionFactory; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionPipelineFunction; import org.apache.pinot.spi.config.table.ColumnPartitionConfig; import org.apache.pinot.spi.config.table.DedupConfig; import org.apache.pinot.spi.config.table.FieldConfig; @@ -1569,7 +1571,11 @@ private static void validateIndexingConfigAndFieldConfigList(TableConfig tableCo Preconditions.checkState(fieldSpec != null, "Failed to find partition column: %s in schema", column); Preconditions.checkState(fieldSpec.isSingleValueField(), "Cannot partition on multi-value column: %s", column); - PartitionFunctionFactory.getPartitionFunction(column, columnPartitionMap.get(column), fieldSpec); + PartitionFunction partitionFunction = + PartitionFunctionFactory.getPartitionFunction(column, columnPartitionMap.get(column), fieldSpec); + if (partitionFunction instanceof PartitionPipelineFunction) { + ((PartitionPipelineFunction) partitionFunction).validateOutputType(); + } } } } diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java index 97ed1a5054c9..802c4e72465c 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java @@ -23,6 +23,7 @@ import com.google.common.base.Preconditions; import java.math.BigInteger; import java.util.List; +import java.util.Locale; import org.apache.pinot.segment.spi.partition.PartitionFunction; import org.apache.pinot.spi.data.FieldSpec; import org.apache.pinot.spi.data.readers.GenericRow; @@ -60,6 +61,50 @@ public PartitionPipeline getPartitionPipeline() { return _pipeline; } + /** + * Validates that the compiled expression produces an integral numeric output by probing it with a sample value. + * Throws {@link IllegalArgumentException} if the output is a non-numeric type (e.g. STRING from {@code md5(col)}). + * If the probe itself throws, validation is skipped (best-effort) and runtime evaluation will surface the error. + */ + public void validateOutputType() { + Object probe; + try { + probe = _pipeline.evaluate(new Object[]{"1"}); + } catch (RuntimeException e) { + return; + } + if (probe == null || probe instanceof Number) { + return; + } + throw new IllegalArgumentException( + "Partition pipeline must produce INT or LONG output, got: " + javaClassToTypeName(probe.getClass())); + } + + private static String javaClassToTypeName(Class cls) { + if (cls == String.class) { + return "STRING"; + } + if (cls == Integer.class) { + return "INT"; + } + if (cls == Long.class) { + return "LONG"; + } + if (cls == Float.class) { + return "FLOAT"; + } + if (cls == Double.class) { + return "DOUBLE"; + } + if (cls == byte[].class) { + return "BYTES"; + } + if (cls == Boolean.class) { + return "BOOLEAN"; + } + return cls.getSimpleName().toUpperCase(Locale.ROOT); + } + @Override public int getPartition(String value) { // BYTES-input pipelines expect raw bytes. When the caller provides a string (e.g. a hex-encoded predicate value From eec006887dbb1753abd6510443277bd5b93c4ce3 Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Fri, 24 Apr 2026 23:39:18 -0700 Subject: [PATCH 11/27] Address critical & major findings from code-review-orchestrator MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Critical correctness fixes: - BaseSegmentCreator: always write PARTITION_FUNCTION key (with sentinel "FunctionExpr" for expression-mode) so old servers reading new segments fail fast with IllegalArgumentException("No enum constant for: FunctionExpr") instead of silently dropping partition pruning. Removes the misleading "v7 format" comment (no version bump exists in this PR). - PartitionPipelineFunction.normalizeResult: reject Float/Double whose absolute value exceeds 2^53. Past this threshold every double is trivially "integral" because the mantissa cannot represent the fractional bit, so multiple distinct longs collapse to the same double and silently produce identical partition ids. Caller must cast to LONG explicitly in the expression to opt into a wider range. - PartitionPipelineFunction.validateOutputType: probe with multiple sample values matching the column's stored type. If every probe throws, fail validation with a clear message rather than silently passing — previously a misconfigured pipeline (e.g. one rejecting "1" as input) would skip output-type validation entirely. - PartitionPipelineFunction.evaluate(GenericRow|Object[]): when the expression chain produces null mid-evaluation (NULL_RESULT_PARTITION_ID sentinel from getPartition), return null on the FunctionEvaluator surface so ingestion callers using Integer-boxing semantics treat it as "no partition" instead of literal partition id -1. - BaseTableDataManager.isPartitionStale: null-tolerant case-insensitive compare for partitionIdNormalizer. Built-in partition functions on the new side return non-null normalizer names while pre-existing custom plugins inherit the SPI default of null. Without this fix, plugin segments would be flagged perpetually stale and re-downloaded on every reload. Major fixes: - ColumnPartitionMetadata.normalizeOptionalText: drop the "null".equals branch. The dual guard masked a real risk that any caller serializing the literal Java string "null" silently null-coerces; rely on the write side to never produce "null" (already true via JsonUtils). - PartitionFunctionExprCompiler.canonicalize: pre-compile the three whitespace-stripping patterns once at class init instead of recompiling on every call. - PartitionFunctionFactory.getPartitionFunction(String, ColumnPartitionConfig, int): marked @Deprecated since the Javadoc already warns "CAUTION" about always compiling expressions with STRING input — callers should use the FieldSpec-aware overload. Co-Authored-By: Claude Sonnet 4.6 --- .../data/manager/BaseTableDataManager.java | 9 ++- .../creator/impl/BaseSegmentCreator.java | 12 ++- .../partition/PartitionFunctionFactory.java | 7 +- .../metadata/ColumnPartitionMetadata.java | 2 +- .../PartitionFunctionExprCompiler.java | 2 + .../pipeline/PartitionPipelineFunction.java | 73 ++++++++++++------- 6 files changed, 67 insertions(+), 38 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java index 5e575141e8c5..55a3d516b867 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java @@ -1814,8 +1814,13 @@ StaleSegment isSegmentStale(IndexLoadingConfig indexLoadingConfig, SegmentDataMa tableNameWithType, columnName, segmentName); return new StaleSegment(segmentName, true, "partition function expr changed: " + columnName); } - if (!Objects.equals(partitionFunction.getPartitionIdNormalizer(), - expectedPartitionFunction.getPartitionIdNormalizer())) { + // null-tolerant compare: a custom-plugin PartitionFunction inheriting the SPI default returns null for + // getPartitionIdNormalizer; treat null on either side as "match" so segments built by such plugins are not + // perpetually flagged stale. + if (partitionFunction.getPartitionIdNormalizer() != null + && expectedPartitionFunction.getPartitionIdNormalizer() != null + && !partitionFunction.getPartitionIdNormalizer().equalsIgnoreCase( + expectedPartitionFunction.getPartitionIdNormalizer())) { LOGGER.debug("tableNameWithType: {}, columnName: {}, segmentName: {}, change: partition id normalizer", tableNameWithType, columnName, segmentName); return new StaleSegment(segmentName, true, "partition id normalizer changed: " + columnName); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/BaseSegmentCreator.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/BaseSegmentCreator.java index 6481b47bc5ad..39a7f49d7515 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/BaseSegmentCreator.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/BaseSegmentCreator.java @@ -619,13 +619,11 @@ public static void addColumnMetadataInfo(PropertiesConfiguration properties, Str PartitionFunction partitionFunction = columnStatistics.getPartitionFunction(); if (partitionFunction != null) { - // Expression-mode functions are identified by the presence of PARTITION_FUNCTION_EXPR in segment metadata. - // The legacy PARTITION_FUNCTION name key is omitted for expression-mode to avoid writing a sentinel value - // ("FunctionExpr") that pre-feature readers would reject as an unknown function name. Note: segments written - // with expression-mode metadata use v7 format and are not readable by older servers regardless of this key. - if (partitionFunction.getFunctionExpr() == null) { - properties.setProperty(getKeyFor(column, PARTITION_FUNCTION), partitionFunction.getName()); - } + // Always write PARTITION_FUNCTION (function name) to segment metadata. Expression-mode functions write the + // sentinel name "FunctionExpr" so that old readers fail fast with IllegalArgumentException + // ("No enum constant for: FunctionExpr") rather than silently treating the column as un-partitioned and + // degrading to scatter-gather. + properties.setProperty(getKeyFor(column, PARTITION_FUNCTION), partitionFunction.getName()); properties.setProperty(getKeyFor(column, NUM_PARTITIONS), partitionFunction.getNumPartitions()); if (partitionFunction.getFunctionExpr() != null) { properties.setProperty(getKeyFor(column, PARTITION_FUNCTION_EXPR), partitionFunction.getFunctionExpr()); diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java index 0e94bc50a734..acfd1e142039 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java @@ -155,10 +155,11 @@ public static PartitionFunction getPartitionFunction(String columnName, ColumnPa /** * Builds a partition function for the given column with an explicit numPartitions override. * - *

CAUTION: For BYTES-typed partition columns this overload always compiles the expression pipeline with - * STRING input. Use {@link #getPartitionFunction(String, ColumnPartitionConfig, FieldSpec)} instead so that BYTES - * columns receive the correct input type. + * @deprecated For BYTES-typed partition columns this overload always compiles the expression pipeline with STRING + * input, producing partition ids that disagree with ingestion. Prefer + * {@link #getPartitionFunction(String, ColumnPartitionConfig, FieldSpec)}. */ + @Deprecated public static PartitionFunction getPartitionFunction(String columnName, ColumnPartitionConfig columnPartitionConfig, int numPartitions) { Preconditions.checkNotNull(columnPartitionConfig, "Column partition config must be configured"); diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java index 484b2837dfc3..a743023d3eee 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java @@ -212,7 +212,7 @@ private static void addRangeToPartitions(String rangeString, IntSet partitions) @Nullable private static String normalizeOptionalText(@Nullable String value) { - return StringUtils.isBlank(value) || "null".equals(value) ? null : value; + return StringUtils.isBlank(value) ? null : value; } /** diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java index 82897438e085..ee61f6ac11cd 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java @@ -33,6 +33,8 @@ * {@link org.apache.pinot.spi.function.FunctionEvaluator} provided by {@link PartitionEvaluatorFactory}. */ public final class PartitionFunctionExprCompiler { + // Cap on user-supplied expression length, sized to fit comfortably within ZK / segment metadata size budgets while + // allowing realistic chained expressions (e.g. fnv1a_32(md5(lower(trim(col))))). private static final int MAX_EXPRESSION_LENGTH = 256; private PartitionFunctionExprCompiler() { diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java index 802c4e72465c..dce8a3f46acd 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java @@ -46,6 +46,9 @@ public class PartitionPipelineFunction implements PartitionFunction, FunctionEva // Sentinel returned when the partition expression evaluates to null for a given value (e.g. null input). // Callers that update per-segment partition sets must skip this value. public static final int NULL_RESULT_PARTITION_ID = -1; + // Largest integer that Double can represent exactly: any |x| > 2^53 is implicitly "integral" because the mantissa + // cannot hold the fractional bit, but multiple distinct longs collapse to the same double in this range. + private static final double MAX_PRECISE_DOUBLE_INTEGRAL = 1L << 53; private final PartitionPipeline _pipeline; private final int _numPartitions; @@ -62,22 +65,38 @@ public PartitionPipeline getPartitionPipeline() { } /** - * Validates that the compiled expression produces an integral numeric output by probing it with a sample value. - * Throws {@link IllegalArgumentException} if the output is a non-numeric type (e.g. STRING from {@code md5(col)}). - * If the probe itself throws, validation is skipped (best-effort) and runtime evaluation will surface the error. + * Validates that the compiled expression produces an integral numeric output by probing it with several sample + * values that cover common input shapes (numeric strings, alpha strings, raw bytes). Throws + * {@link IllegalArgumentException} if at least one probe completes and the output is non-numeric (e.g. STRING from + * {@code md5(col)}), or if every probe throws (cannot determine output type — likely a misconfigured pipeline). */ public void validateOutputType() { - Object probe; - try { - probe = _pipeline.evaluate(new Object[]{"1"}); - } catch (RuntimeException e) { - return; + Object[] samples = _pipeline.isBytesInput() + ? new Object[]{new byte[]{0}, new byte[]{1, 2, 3}, new byte[0]} + : new Object[]{"1", "0", "abc"}; + boolean anyProbeSucceeded = false; + RuntimeException lastFailure = null; + for (Object sample : samples) { + Object probe; + try { + probe = _pipeline.evaluate(new Object[]{sample}); + } catch (RuntimeException e) { + lastFailure = e; + continue; + } + anyProbeSucceeded = true; + if (probe == null || probe instanceof Number) { + continue; + } + throw new IllegalArgumentException( + "Partition pipeline must produce INT or LONG output, got: " + javaClassToTypeName(probe.getClass())); + } + if (!anyProbeSucceeded) { + throw new IllegalArgumentException( + "Partition pipeline for column '" + _pipeline.getRawColumn() + "' failed to evaluate against any sample " + + "input; check the expression is well-formed and accepts the column's stored type. Last error: " + + (lastFailure != null ? lastFailure.getMessage() : "unknown")); } - if (probe == null || probe instanceof Number) { - return; - } - throw new IllegalArgumentException( - "Partition pipeline must produce INT or LONG output, got: " + javaClassToTypeName(probe.getClass())); } private static String javaClassToTypeName(Class cls) { @@ -138,12 +157,16 @@ private int normalizeResult(Object result) { _pipeline.getRawColumn(), result.getClass().getSimpleName()); Number num = (Number) result; // Pinot scalar functions commonly box integral arithmetic to Double (e.g. plus(long, long) → Double). Accept - // Float/Double iff the value is integral (no fractional part); reject genuinely fractional values since they - // cannot map to a stable partition id. + // Float/Double iff the value is integral (no fractional part) AND fits within Double's 53-bit mantissa, since + // any double with absolute value > 2^53 is implicitly "integral" (mantissa cannot represent the fractional bit) + // but maps multiple distinct longs to the same double — silently collapsing partition ids. Reject these and + // require the user to cast to LONG explicitly in the expression. if (num instanceof Float || num instanceof Double) { double d = num.doubleValue(); - Preconditions.checkState(!Double.isNaN(d) && !Double.isInfinite(d) && d == Math.floor(d), - "Partition expression for column '%s' must return an integral value (int/long), got: %s (%s)", + Preconditions.checkState(!Double.isNaN(d) && !Double.isInfinite(d) && d == Math.floor(d) + && Math.abs(d) <= MAX_PRECISE_DOUBLE_INTEGRAL, + "Partition expression for column '%s' must return an integral value within Double precision " + + "(|x| <= 2^53), got: %s (%s); cast the expression result to LONG explicitly to avoid this", _pipeline.getRawColumn(), result, result.getClass().getSimpleName()); } PartitionIntNormalizer intNormalizer = _pipeline.getIntNormalizer(); @@ -200,10 +223,10 @@ public Object evaluate(GenericRow genericRow) { if (inputValue == null) { return null; } - if (inputValue instanceof byte[] && _pipeline.isBytesInput()) { - return getPartition((byte[]) inputValue); - } - return getPartition(FieldSpec.getStringValue(inputValue)); + int partitionId = (inputValue instanceof byte[] && _pipeline.isBytesInput()) + ? getPartition((byte[]) inputValue) + : getPartition(FieldSpec.getStringValue(inputValue)); + return partitionId == NULL_RESULT_PARTITION_ID ? null : partitionId; } @Override @@ -215,10 +238,10 @@ public Object evaluate(Object[] values) { if (inputValue == null) { return null; } - if (inputValue instanceof byte[] && _pipeline.isBytesInput()) { - return getPartition((byte[]) inputValue); - } - return getPartition(FieldSpec.getStringValue(inputValue)); + int partitionId = (inputValue instanceof byte[] && _pipeline.isBytesInput()) + ? getPartition((byte[]) inputValue) + : getPartition(FieldSpec.getStringValue(inputValue)); + return partitionId == NULL_RESULT_PARTITION_ID ? null : partitionId; } @Override From 39ce0a8d4392df60b8b2aa03249cf70aaae4e813 Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Fri, 24 Apr 2026 23:48:59 -0700 Subject: [PATCH 12/27] Add regression tests for the bug-fix commits in this branch MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - testIntegralValuedDoubleAcceptedAsPartitionResult: regression for 9bcd3ee63f. plus(intDiv(54321,1000), 7) returns 61.0 (Double); the pipeline must accept this since Pinot scalar functions box integral arithmetic to Double. - testDoubleBeyondMantissaPrecisionIsRejected: regression for the precision-loss critical fix. Doubles with |x| > 2^53 cannot represent successive integers uniquely; reject to avoid silent partition-id collapse. - testNullMidChainReturnsNullOnFunctionEvaluatorSurface: regression for 28546f1523 + the null-divergence fix. evaluate(GenericRow) must return null when the column value is null or the chain produces null mid-evaluation, so ingestion treats it as "no partition" rather than the literal -1 sentinel. - testValidateOutputTypeRejectsStringExpressionResult: regression for 9c031aa76e. md5(col) returns STRING — validateOutputType must reject it at config-validation time with a clear "INT or LONG" error. - testPartitionMetadataManagerProcessingWithMixedCaseFunctionExpr: regression for 28546f1523. Broker config-side functionExpr is canonicalized (lowercased) but segments may be written with mixed-case expressions; the routing match must be case-insensitive so such segments are still partition-pruned instead of falling back to scatter-gather. Co-Authored-By: Claude Sonnet 4.6 --- .../SegmentPartitionMetadataManagerTest.java | 38 +++++++++++ ...nFunctionExprCommonScalarFunctionTest.java | 63 +++++++++++++++++++ 2 files changed, 101 insertions(+) diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManagerTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManagerTest.java index 399ab2fef920..7f6ea0d2608e 100644 --- a/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManagerTest.java +++ b/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManagerTest.java @@ -354,6 +354,44 @@ public void testPartitionMetadataManagerProcessingWithFunctionExprAbsNormalizer( assertTrue(tablePartitionReplicatedServersInfo.getSegmentsWithInvalidPartition().isEmpty()); } + /** + * Regression for commit 28546f1523. The broker compiles its config-side functionExpr through the canonicalize() + * pipeline (lowercased, spaces stripped) but segments may have been written with mixed-case expressions by an + * older node or a direct ZK write. The match check must be case-insensitive so such segments are still routed + * via partition-aware pruning instead of silently degrading to scatter-gather. + */ + @Test + public void testPartitionMetadataManagerProcessingWithMixedCaseFunctionExpr() { + ExternalView externalView = new ExternalView(OFFLINE_TABLE_NAME); + Map> segmentAssignment = externalView.getRecord().getMapFields(); + Set onlineSegments = new HashSet<>(); + IdealState idealState = new IdealState(OFFLINE_TABLE_NAME); + String configExpr = "fnv1a_32(md5(" + PARTITION_COLUMN + "))"; + String mixedCaseSegmentExpr = "FNV1A_32(MD5(" + PARTITION_COLUMN + "))"; + String segment = "mixedCaseSegment"; + + SegmentPartitionMetadataManager partitionMetadataManager = + new SegmentPartitionMetadataManager(OFFLINE_TABLE_NAME, PARTITION_COLUMN, + ColumnPartitionConfig.forFunctionExpr(configExpr, 8, "MASK")); + SegmentZkMetadataFetcher segmentZkMetadataFetcher = + new SegmentZkMetadataFetcher(OFFLINE_TABLE_NAME, _propertyStore); + segmentZkMetadataFetcher.register(partitionMetadataManager); + segmentZkMetadataFetcher.init(idealState, externalView, onlineSegments); + + onlineSegments.add(segment); + segmentAssignment.put(segment, Collections.singletonMap(SERVER_0, ONLINE)); + setSegmentZKMetadata(segment, PartitionPipelineFunction.NAME, 8, 1, mixedCaseSegmentExpr, "MASK", 0L); + segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); + + TablePartitionReplicatedServersInfo tablePartitionReplicatedServersInfo = + partitionMetadataManager.getTablePartitionReplicatedServersInfo(); + assertEquals(tablePartitionReplicatedServersInfo.getPartitionInfoMap()[1]._segments, + Collections.singleton(segment), + "Segment with mixed-case functionExpr must still match the canonicalized config expr"); + assertTrue(tablePartitionReplicatedServersInfo.getSegmentsWithInvalidPartition().isEmpty(), + "Segment must not be flagged as invalid solely because its expr was written in mixed case"); + } + private void setSegmentZKMetadata(String segment, String partitionFunction, int numPartitions, int partitionId, long creationTimeMs) { setSegmentZKMetadata(segment, partitionFunction, numPartitions, partitionId, null, null, creationTimeMs); diff --git a/pinot-common/src/test/java/org/apache/pinot/common/function/scalar/PartitionFunctionExprCommonScalarFunctionTest.java b/pinot-common/src/test/java/org/apache/pinot/common/function/scalar/PartitionFunctionExprCommonScalarFunctionTest.java index dd98db1c545f..49efec92790d 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/function/scalar/PartitionFunctionExprCommonScalarFunctionTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/function/scalar/PartitionFunctionExprCommonScalarFunctionTest.java @@ -20,9 +20,11 @@ import org.apache.pinot.segment.spi.partition.pipeline.PartitionFunctionExprCompiler; import org.apache.pinot.segment.spi.partition.pipeline.PartitionPipelineFunction; +import org.apache.pinot.spi.data.readers.GenericRow; import org.testng.annotations.Test; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNull; import static org.testng.Assert.expectThrows; @@ -59,4 +61,65 @@ public void testRejectsNonDeterministicAgoFunction() { assertEquals(error.getMessage(), "Partition scalar function 'ago' is not allowed because it is non-deterministic"); } + + /** + * Regression for commit 9bcd3ee63f. Pinot scalar functions box integral arithmetic to Double; the partition + * function must accept integral-valued Doubles (e.g. plus(54L, 7L) → 61.0) without rejecting them outright. + */ + @Test + public void testIntegralValuedDoubleAcceptedAsPartitionResult() { + PartitionPipelineFunction partitionFunction = + PartitionFunctionExprCompiler.compilePartitionFunction("col", "plus(intDiv(col, 1000), 7)", 128); + // intDiv(54321, 1000) = 54; plus(54, 7) = 61.0 (Double); 61 % 128 = 61 + assertEquals(partitionFunction.getPartition("54321"), 61); + } + + /** + * Regression: Float/Double whose absolute value exceeds 2^53 must be rejected even if the value is "integral", + * because mantissa precision loss causes silent partition-id collapse. + */ + @Test + public void testDoubleBeyondMantissaPrecisionIsRejected() { + PartitionPipelineFunction partitionFunction = + PartitionFunctionExprCompiler.compilePartitionFunction("col", "plus(col, 0)", 128); + // 2^53 + 1 cannot be represented exactly as Double — the function returns 9007199254740992.0 (== 2^53) + // for both 9007199254740992 and 9007199254740993. Reject to avoid silent collisions. The check fires when + // |d| > 2^53. + IllegalStateException error = expectThrows(IllegalStateException.class, + () -> partitionFunction.getPartition("18014398509481984")); // 2^54 + assertEquals(error.getMessage().contains("|x| <= 2^53"), true, + "Expected precision-bound error, got: " + error.getMessage()); + } + + /** + * Regression for commit 28546f1523. The expression chain may yield null mid-evaluation (e.g. via tryCast or null + * column input). Surface this as null on the FunctionEvaluator interface so ingestion treats it as "no partition" + * rather than literal partition id -1. + */ + @Test + public void testNullMidChainReturnsNullOnFunctionEvaluatorSurface() { + PartitionPipelineFunction partitionFunction = + PartitionFunctionExprCompiler.compilePartitionFunction("col", "plus(col, 0)", 128); + GenericRow row = new GenericRow(); + row.putValue("col", null); + assertNull(partitionFunction.evaluate(row), + "evaluate(GenericRow) must return null when the column value is null"); + assertNull(partitionFunction.evaluate(new Object[]{null}), + "evaluate(Object[]) must return null when the input is null"); + } + + /** + * Regression for commit 9c031aa76e. Expressions whose final type is non-numeric (e.g. md5() returns STRING) must + * be rejected at validation time with a clear "must produce INT or LONG" error rather than passing config + * validation and surfacing the failure later at segment creation. + */ + @Test + public void testValidateOutputTypeRejectsStringExpressionResult() { + PartitionPipelineFunction partitionFunction = + PartitionFunctionExprCompiler.compilePartitionFunction("col", "md5(col)", 8); + IllegalArgumentException error = expectThrows(IllegalArgumentException.class, + partitionFunction::validateOutputType); + assertEquals(error.getMessage().contains("STRING"), true, + "Expected error to mention STRING output, got: " + error.getMessage()); + } } From 4961d4ef5bba78b207e636360118473bf51d82be Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Sat, 25 Apr 2026 00:05:32 -0700 Subject: [PATCH 13/27] Tighten precision bound, thread-safety, and validation contracts MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Critical fixes: - PartitionPipelineFunction.normalizeResult: tighten precision check to strict |x| < 2^53. The previous inclusive bound admitted 2^53 itself, but the next representable integer above 2^53 is 2^53+2 — meaning 2^53+1 silently rounds to 2^53.0 and would collide with 2^53 onto the same partition id. Add a boundary regression test asserting 2^53 itself is rejected. - InbuiltFunctionEvaluator.FunctionExecutionNode: replace shared Object[] _arguments scratch array with ThreadLocal so the parent ExecutableFunctionEvaluator's documented thread-safety contract holds for both subclasses (InbuiltFunctionEvaluator and PartitionFunctionEvaluator). Update class-level Javadoc accordingly. Major fixes: - PartitionPipelineFunction.validateOutputType: include null in the probe set so validation also exercises the null-input path that ingestion already handles. - PartitionFunctionExprCompiler.canonicalize: collapse runs of internal whitespace to a single space so segment-vs-config comparisons of canonicalized expressions agree on whitespace. - V1Constants: add PARTITION_FUNCTION_EXPR_SENTINEL constant documenting that "FunctionExpr" is reserved as the cross-version fail-fast value written into segment metadata for expression-mode segments. Must stay in sync with PartitionPipelineFunction.NAME. Co-Authored-By: Claude Sonnet 4.6 --- .../evaluator/InbuiltFunctionEvaluator.java | 38 ++++++----- ...nFunctionExprCommonScalarFunctionTest.java | 47 ++++++++++--- .../PartitionFunctionExprTestFunctions.java | 6 ++ .../apache/pinot/segment/spi/V1Constants.java | 12 ++++ .../pipeline/PartitionPipelineFunction.java | 68 +++++++++++++++---- 5 files changed, 133 insertions(+), 38 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/evaluator/InbuiltFunctionEvaluator.java b/pinot-common/src/main/java/org/apache/pinot/common/evaluator/InbuiltFunctionEvaluator.java index 9946e1372e44..e272258d2a15 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/evaluator/InbuiltFunctionEvaluator.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/evaluator/InbuiltFunctionEvaluator.java @@ -39,9 +39,11 @@ * The expression is compiled once into an {@link ExecutableFunctionEvaluator.ExecutableNode} tree * whose nodes handle constants, column reads, logical operators, and function calls. * - *

Thread-safety: Instances are thread-safe for all node types except - * {@link FunctionExecutionNode}, which uses a shared argument scratch array per node instance - * and therefore must not be invoked concurrently on the same node. + *

Thread-safety: Instances are thread-safe for concurrent invocation. Each + * {@link FunctionExecutionNode} maintains a per-thread argument scratch array via + * {@link ThreadLocal}, so multiple threads can call {@code evaluate} on the same compiled tree + * without sharing mutable state. A single call chain on one thread must not invoke {@code evaluate} + * recursively on the same node instance. */ public class InbuiltFunctionEvaluator extends ExecutableFunctionEvaluator { @@ -112,43 +114,44 @@ private static ExecutableNode planExecution(ExpressionContext expression, List _arguments; FunctionExecutionNode(FunctionInfo functionInfo, ExecutableNode[] argumentNodes) { _functionInvoker = new FunctionInvoker(functionInfo); _functionInfo = functionInfo; _argumentNodes = argumentNodes; - _arguments = new Object[_argumentNodes.length]; + _arguments = ThreadLocal.withInitial(() -> new Object[argumentNodes.length]); } @Override public Object execute(GenericRow row) { try { + Object[] arguments = _arguments.get(); int numArguments = _argumentNodes.length; for (int i = 0; i < numArguments; i++) { - _arguments[i] = _argumentNodes[i].execute(row); + arguments[i] = _argumentNodes[i].execute(row); } if (!_functionInfo.hasNullableParameters()) { // Preserve null values during ingestion transformation if function is an inbuilt // scalar function that cannot handle nulls, and invoked with null parameter(s). - for (Object argument : _arguments) { + for (Object argument : arguments) { if (argument == null) { return null; } } } if (_functionInvoker.getMethod().isVarArgs()) { - return _functionInvoker.invoke(new Object[]{_arguments}); + return _functionInvoker.invoke(new Object[]{arguments}); } - _functionInvoker.convertTypes(_arguments); - return _functionInvoker.invoke(_arguments); + _functionInvoker.convertTypes(arguments); + return _functionInvoker.invoke(arguments); } catch (Exception e) { throw new RuntimeException("Caught exception while executing function: " + this + ": " + e.getMessage(), e); } @@ -157,24 +160,25 @@ public Object execute(GenericRow row) { @Override public Object execute(Object[] values) { try { + Object[] arguments = _arguments.get(); int numArguments = _argumentNodes.length; for (int i = 0; i < numArguments; i++) { - _arguments[i] = _argumentNodes[i].execute(values); + arguments[i] = _argumentNodes[i].execute(values); } if (!_functionInfo.hasNullableParameters()) { // Preserve null values during ingestion transformation if function is an inbuilt // scalar function that cannot handle nulls, and invoked with null parameter(s). - for (Object argument : _arguments) { + for (Object argument : arguments) { if (argument == null) { return null; } } } if (_functionInvoker.getMethod().isVarArgs()) { - return _functionInvoker.invoke(new Object[]{_arguments}); + return _functionInvoker.invoke(new Object[]{arguments}); } - _functionInvoker.convertTypes(_arguments); - return _functionInvoker.invoke(_arguments); + _functionInvoker.convertTypes(arguments); + return _functionInvoker.invoke(arguments); } catch (Exception e) { throw new RuntimeException("Caught exception while executing function: " + this + ": " + e.getMessage(), e); } diff --git a/pinot-common/src/test/java/org/apache/pinot/common/function/scalar/PartitionFunctionExprCommonScalarFunctionTest.java b/pinot-common/src/test/java/org/apache/pinot/common/function/scalar/PartitionFunctionExprCommonScalarFunctionTest.java index 49efec92790d..62c5d41bf7da 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/function/scalar/PartitionFunctionExprCommonScalarFunctionTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/function/scalar/PartitionFunctionExprCommonScalarFunctionTest.java @@ -25,6 +25,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; import static org.testng.Assert.expectThrows; @@ -75,22 +76,37 @@ public void testIntegralValuedDoubleAcceptedAsPartitionResult() { } /** - * Regression: Float/Double whose absolute value exceeds 2^53 must be rejected even if the value is "integral", - * because mantissa precision loss causes silent partition-id collapse. + * Regression: Float/Double whose absolute value reaches 2^53 must be rejected even if the value is "integral", + * because mantissa precision loss causes silent partition-id collapse. The bound is strict: 2^53 itself is + * representable but 2^53+1 collapses onto 2^53, so admitting 2^53 would let 2^53+1 silently collide with it. */ @Test public void testDoubleBeyondMantissaPrecisionIsRejected() { PartitionPipelineFunction partitionFunction = PartitionFunctionExprCompiler.compilePartitionFunction("col", "plus(col, 0)", 128); - // 2^53 + 1 cannot be represented exactly as Double — the function returns 9007199254740992.0 (== 2^53) - // for both 9007199254740992 and 9007199254740993. Reject to avoid silent collisions. The check fires when - // |d| > 2^53. + // 2^54 — clearly past the boundary IllegalStateException error = expectThrows(IllegalStateException.class, - () -> partitionFunction.getPartition("18014398509481984")); // 2^54 - assertEquals(error.getMessage().contains("|x| <= 2^53"), true, + () -> partitionFunction.getPartition("18014398509481984")); + assertTrue(error.getMessage().contains("|x| < 2^53"), "Expected precision-bound error, got: " + error.getMessage()); } + /** + * Boundary regression for the off-by-one in the precision check: 2^53+1 (long) silently rounds to 2^53.0 + * (Double); admitting 2^53 would let 2^53+1 collide with 2^53 onto the same partition id. Strict {@code < 2^53} + * is required. + */ + @Test + public void testDoubleAtMantissaBoundaryIsRejected() { + PartitionPipelineFunction partitionFunction = + PartitionFunctionExprCompiler.compilePartitionFunction("col", "plus(col, 0)", 128); + // 2^53 itself — boundary value where "integral" check passes but next integer collides + IllegalStateException error = expectThrows(IllegalStateException.class, + () -> partitionFunction.getPartition("9007199254740992")); + assertTrue(error.getMessage().contains("|x| < 2^53"), + "Expected precision-bound error at 2^53 boundary, got: " + error.getMessage()); + } + /** * Regression for commit 28546f1523. The expression chain may yield null mid-evaluation (e.g. via tryCast or null * column input). Surface this as null on the FunctionEvaluator interface so ingestion treats it as "no partition" @@ -119,7 +135,22 @@ public void testValidateOutputTypeRejectsStringExpressionResult() { PartitionFunctionExprCompiler.compilePartitionFunction("col", "md5(col)", 8); IllegalArgumentException error = expectThrows(IllegalArgumentException.class, partitionFunction::validateOutputType); - assertEquals(error.getMessage().contains("STRING"), true, + assertTrue(error.getMessage().contains("STRING"), "Expected error to mention STRING output, got: " + error.getMessage()); } + + @Test + public void testValidateOutputTypeRejectsFractionalBigDecimalResult() { + PartitionPipelineFunction partitionFunction = + PartitionFunctionExprCompiler.compilePartitionFunction("col", "fractionalBigDecimal(col)", 8); + IllegalArgumentException validationError = expectThrows(IllegalArgumentException.class, + partitionFunction::validateOutputType); + assertTrue(validationError.getMessage().contains("integral value"), + "Expected validation to reject fractional BigDecimal output, got: " + validationError.getMessage()); + + IllegalStateException runtimeError = expectThrows(IllegalStateException.class, + () -> partitionFunction.getPartition("1")); + assertTrue(runtimeError.getMessage().contains("integral value"), + "Expected runtime to reject fractional BigDecimal output, got: " + runtimeError.getMessage()); + } } diff --git a/pinot-common/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprTestFunctions.java b/pinot-common/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprTestFunctions.java index 66a14c36898c..27798818d9f1 100644 --- a/pinot-common/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprTestFunctions.java +++ b/pinot-common/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprTestFunctions.java @@ -18,6 +18,7 @@ */ package org.apache.pinot.segment.spi.function.scalar; +import java.math.BigDecimal; import java.util.concurrent.ThreadLocalRandom; import org.apache.pinot.spi.annotations.ScalarFunction; @@ -44,6 +45,11 @@ public static long bucket(long value, long divisor) { return value / divisor; } + @ScalarFunction + public static BigDecimal fractionalBigDecimal(long value) { + return BigDecimal.valueOf(value).add(new BigDecimal("0.5")); + } + @ScalarFunction(isDeterministic = false) public static long randomBucket(long value) { return value + ThreadLocalRandom.current().nextLong(); diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/V1Constants.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/V1Constants.java index 52e9885b5d69..2e68e8949c92 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/V1Constants.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/V1Constants.java @@ -178,6 +178,18 @@ public static class Column { public static final String NUM_PARTITIONS = "numPartitions"; public static final String PARTITION_VALUES = "partitionValues"; public static final String PARTITION_FUNCTION_CONFIG = "partitionFunctionConfig"; + /** + * Reserved sentinel value written into {@link #PARTITION_FUNCTION} for expression-mode segments. This + * intentionally does not match any real {@code PartitionFunctionType} enum constant, so older readers loading + * a new segment trip {@code IllegalArgumentException("No enum constant for: FunctionExpr")} and fail fast + * rather than silently dropping partition pruning. New readers detect expression-mode via the presence of + * {@link #PARTITION_FUNCTION_EXPR} and ignore this sentinel. + * + *

Must stay in sync with {@code PartitionPipelineFunction.NAME}. Do not add an enum constant with + * this name to {@code PartitionFunctionType} — doing so would silently break the cross-version fail-fast + * contract. + */ + public static final String PARTITION_FUNCTION_EXPR_SENTINEL = "FunctionExpr"; /// Old key maintained for backward compatibility // Replaced by LENGTH_OF_LONGEST_ELEMENT diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java index dce8a3f46acd..a291bb3bf468 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import java.math.BigDecimal; import java.math.BigInteger; import java.util.List; import java.util.Locale; @@ -71,9 +72,12 @@ public PartitionPipeline getPartitionPipeline() { * {@code md5(col)}), or if every probe throws (cannot determine output type — likely a misconfigured pipeline). */ public void validateOutputType() { + // Probe set covers numeric strings, alpha strings, raw bytes (for BYTES input), and null. The null sample + // mirrors the runtime ingestion path that already handles nulls, so the validation also exercises that the + // pipeline does not blow up on null input. Non-null samples are checked for non-numeric output types. Object[] samples = _pipeline.isBytesInput() - ? new Object[]{new byte[]{0}, new byte[]{1, 2, 3}, new byte[0]} - : new Object[]{"1", "0", "abc"}; + ? new Object[]{new byte[]{0}, new byte[]{1, 2, 3}, new byte[0], null} + : new Object[]{"1", "0", "abc", null}; boolean anyProbeSucceeded = false; RuntimeException lastFailure = null; for (Object sample : samples) { @@ -85,11 +89,19 @@ public void validateOutputType() { continue; } anyProbeSucceeded = true; - if (probe == null || probe instanceof Number) { + if (probe == null) { continue; } - throw new IllegalArgumentException( - "Partition pipeline must produce INT or LONG output, got: " + javaClassToTypeName(probe.getClass())); + if (!(probe instanceof Number)) { + throw new IllegalArgumentException( + "Partition pipeline must produce INT or LONG output, got: " + javaClassToTypeName(probe.getClass())); + } + try { + normalizeResult(probe); + } catch (IllegalStateException e) { + throw new IllegalArgumentException( + "Partition pipeline must produce INT or LONG output: " + e.getMessage(), e); + } } if (!anyProbeSucceeded) { throw new IllegalArgumentException( @@ -157,26 +169,56 @@ private int normalizeResult(Object result) { _pipeline.getRawColumn(), result.getClass().getSimpleName()); Number num = (Number) result; // Pinot scalar functions commonly box integral arithmetic to Double (e.g. plus(long, long) → Double). Accept - // Float/Double iff the value is integral (no fractional part) AND fits within Double's 53-bit mantissa, since - // any double with absolute value > 2^53 is implicitly "integral" (mantissa cannot represent the fractional bit) - // but maps multiple distinct longs to the same double — silently collapsing partition ids. Reject these and - // require the user to cast to LONG explicitly in the expression. + // Float/Double iff the value is integral (no fractional part) AND strictly fits within Double's 53-bit mantissa. + // The bound is strict (|x| < 2^53) because the next representable integer above 2^53 is 2^53+2: the long + // value 2^53+1 silently rounds to 2^53.0, which would collide with 2^53 itself if we admitted it. Require the + // user to cast to LONG explicitly in the expression to opt into the wider integer range. if (num instanceof Float || num instanceof Double) { double d = num.doubleValue(); Preconditions.checkState(!Double.isNaN(d) && !Double.isInfinite(d) && d == Math.floor(d) - && Math.abs(d) <= MAX_PRECISE_DOUBLE_INTEGRAL, + && Math.abs(d) < MAX_PRECISE_DOUBLE_INTEGRAL, "Partition expression for column '%s' must return an integral value within Double precision " - + "(|x| <= 2^53), got: %s (%s); cast the expression result to LONG explicitly to avoid this", + + "(|x| < 2^53), got: %s (%s); cast the expression result to LONG explicitly to avoid this", _pipeline.getRawColumn(), result, result.getClass().getSimpleName()); } PartitionIntNormalizer intNormalizer = _pipeline.getIntNormalizer(); Preconditions.checkState(intNormalizer != null, "Integral-output partition pipeline for column '%s' must have an INT normalizer", _pipeline.getRawColumn()); - if (num instanceof Long || num instanceof BigInteger || num instanceof Float || num instanceof Double) { + // BigInteger and BigDecimal get dedicated paths so Number.longValue()/intValue() cannot silently truncate + // fractional values or values outside the LONG range. + if (num instanceof BigInteger) { + return normalizeBigIntegerResult((BigInteger) num, result, intNormalizer); + } + if (num instanceof BigDecimal) { + try { + return normalizeBigIntegerResult(((BigDecimal) num).toBigIntegerExact(), result, intNormalizer); + } catch (ArithmeticException e) { + throw new IllegalStateException( + "Partition expression for column '" + _pipeline.getRawColumn() + + "' must return an integral value, got: " + result + " (BigDecimal)", e); + } + } + if (num instanceof Long || num instanceof Float || num instanceof Double) { return intNormalizer.getPartitionId(num.longValue(), _numPartitions); } - return intNormalizer.getPartitionId(num.intValue(), _numPartitions); + if (num instanceof Integer || num instanceof Short || num instanceof Byte) { + return intNormalizer.getPartitionId(num.intValue(), _numPartitions); + } + throw new IllegalStateException( + "Partition expression for column '" + _pipeline.getRawColumn() + + "' must return INT or LONG-compatible output, got: " + javaClassToTypeName(result.getClass())); + } + + private int normalizeBigIntegerResult(BigInteger value, Object originalResult, + PartitionIntNormalizer intNormalizer) { + try { + return intNormalizer.getPartitionId(value.longValueExact(), _numPartitions); + } catch (ArithmeticException e) { + throw new IllegalStateException( + "Partition expression for column '" + _pipeline.getRawColumn() + + "' must return a value within LONG range, got: " + originalResult, e); + } } @Override From 6a2e25deefa7f4882095601b2eee7fd9625667f9 Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Sat, 25 Apr 2026 00:14:50 -0700 Subject: [PATCH 14/27] Cache compiled PartitionPipeline to avoid per-segment Calcite parse MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Previously, every segment loaded by a broker/server invoked PartitionFunctionExprCompiler.compile() which parses the canonical expression through Calcite once per segment. With thousands of segments sharing the same partition expression, this is thousands of redundant Calcite parses at table-init/refresh time. Add a process-wide ConcurrentMap keyed on (rawColumn, isBytesInput, canonicalExpr, normalizerName). PartitionPipeline is stateless except for per-thread scratch arrays in its inner ExecutableNode tree, so a cached instance is safe to share across multiple PartitionPipelineFunction wrappers — each wrapper applies its own numPartitions at normalization time. Cache size is bounded by unique (column, function spec) tuples, typically a handful per cluster. Co-Authored-By: Claude Sonnet 4.6 --- .../PartitionFunctionExprCompiler.java | 52 ++++++++++++++++++- 1 file changed, 50 insertions(+), 2 deletions(-) diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java index ee61f6ac11cd..61c5ff2be259 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java @@ -22,7 +22,10 @@ import java.util.ArrayList; import java.util.List; import java.util.Locale; +import java.util.Objects; import java.util.ServiceLoader; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import javax.annotation.Nullable; import org.apache.pinot.spi.config.table.ColumnPartitionConfig; import org.apache.pinot.spi.function.FunctionEvaluator; @@ -37,6 +40,13 @@ public final class PartitionFunctionExprCompiler { // allowing realistic chained expressions (e.g. fnv1a_32(md5(lower(trim(col))))). private static final int MAX_EXPRESSION_LENGTH = 256; + // Cache compiled pipelines so a broker/server with thousands of segments sharing the same partition expression + // only Calcite-parses the expression once. PartitionPipeline is stateless except for per-thread scratch arrays in + // its inner ExecutableNode tree, so the cached instance is safe to share across multiple PartitionPipelineFunction + // wrappers (each wrapper applies its own numPartitions at normalization time). Cache size is bounded by unique + // (rawColumn, inputType, canonicalExpr, normalizer) tuples — typically a handful per cluster. + private static final ConcurrentMap PIPELINE_CACHE = new ConcurrentHashMap<>(); + private PartitionFunctionExprCompiler() { } @@ -85,8 +95,11 @@ public static PartitionPipeline compile(String rawColumn, PartitionValueType inp String canonicalExpr = canonicalize(functionExpr); boolean isBytesInput = inputType == PartitionValueType.BYTES; - FunctionEvaluator evaluator = EvaluatorFactoryHolder.INSTANCE.compile(rawColumn, canonicalExpr); - return new PartitionPipeline(rawColumn, isBytesInput, canonicalExpr, partitionIdNormalizer, evaluator); + PipelineCacheKey cacheKey = new PipelineCacheKey(rawColumn, isBytesInput, canonicalExpr, partitionIdNormalizer); + return PIPELINE_CACHE.computeIfAbsent(cacheKey, k -> { + FunctionEvaluator evaluator = EvaluatorFactoryHolder.INSTANCE.compile(rawColumn, canonicalExpr); + return new PartitionPipeline(rawColumn, isBytesInput, canonicalExpr, partitionIdNormalizer, evaluator); + }); } public static PartitionPipelineFunction compilePartitionFunction(String rawColumn, String functionExpr, @@ -188,4 +201,39 @@ private static void appendPendingSpace(StringBuilder builder, boolean pendingSpa builder.append(' '); } } + + private static final class PipelineCacheKey { + private final String _rawColumn; + private final boolean _isBytesInput; + private final String _canonicalExpr; + private final String _normalizerName; + + PipelineCacheKey(String rawColumn, boolean isBytesInput, String canonicalExpr, + @Nullable PartitionIntNormalizer normalizer) { + _rawColumn = rawColumn; + _isBytesInput = isBytesInput; + _canonicalExpr = canonicalExpr; + _normalizerName = normalizer != null ? normalizer.name() : null; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (!(obj instanceof PipelineCacheKey)) { + return false; + } + PipelineCacheKey other = (PipelineCacheKey) obj; + return _isBytesInput == other._isBytesInput + && _rawColumn.equals(other._rawColumn) + && _canonicalExpr.equals(other._canonicalExpr) + && Objects.equals(_normalizerName, other._normalizerName); + } + + @Override + public int hashCode() { + return Objects.hash(_rawColumn, _isBytesInput, _canonicalExpr, _normalizerName); + } + } } From 8e302e53f8028071e7bfed7db4cf1aa281472973 Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Sat, 25 Apr 2026 00:33:51 -0700 Subject: [PATCH 15/27] Address review-orchestrator critical/major findings Critical fixes: - TableConfigPartitioner.computePartition: filter NULL_RESULT_PARTITION_ID and null inputs to a designated NULL_PARTITION bucket name. Previously null-mid-chain results yielded "-1" as a partition string, leaking a stray partition-bucket -1 into minion segment-mapper output. - PartitionFunctionExprCompiler: bound PIPELINE_CACHE with Guava CacheBuilder maximumSize=10_000 to prevent unbounded growth on long-lived processes with table-config churn. Switch from ConcurrentHashMap.computeIfAbsent (which forbids re-entrant cache modification) to Cache.get(key, Callable) which is re-entrant-safe. Unwrap UncheckedExecutionException + ExecutionException to preserve the original compilation error (e.g. invalid-expression IllegalArgumentException). - PartitionFunctionExprCompiler.EvaluatorFactoryHolder: relax the exactly-1 ServiceLoader check. Multiple impls (common in shaded-jar / multi-module test setups) no longer JVM-fail; prefer InbuiltPartitionEvaluatorFactory by name, otherwise pick the first. - BaseSegmentCreator + ColumnMetadataImpl + V1Constants: persist PARTITION_INPUT_TYPE in segment metadata for BYTES expression-mode pipelines. Reader prefers the stored value, falling back to schema-derived stored type for older segments. - PartitionFunctionTest: add regression test asserting PartitionFunctionType.fromString("FunctionExpr") throws so a future enum addition with that name does not silently break the cross-version fail-fast contract for old readers. Major fixes: - BaseTableDataManager.isPartitionStale: tighten the asymmetric null tolerance for partitionIdNormalizer. Previously both-side null and one-side null were both treated as "match", so a user explicitly removing the normalizer would not flag staleness. Now segment-side null is tolerated (legacy or plugin SPI default), but config-side null with non-null segment-side correctly flags stale. Co-Authored-By: Claude Sonnet 4.6 --- .../data/manager/BaseTableDataManager.java | 15 +++--- .../partitioner/TableConfigPartitioner.java | 33 ++++++++---- .../creator/impl/BaseSegmentCreator.java | 8 +++ .../apache/pinot/segment/spi/V1Constants.java | 7 +++ .../index/metadata/ColumnMetadataImpl.java | 12 +++-- .../PartitionFunctionExprCompiler.java | 54 +++++++++++++++---- .../spi/partition/PartitionFunctionTest.java | 14 +++++ 7 files changed, 113 insertions(+), 30 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java index 55a3d516b867..13315947b3f6 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java @@ -1814,13 +1814,14 @@ StaleSegment isSegmentStale(IndexLoadingConfig indexLoadingConfig, SegmentDataMa tableNameWithType, columnName, segmentName); return new StaleSegment(segmentName, true, "partition function expr changed: " + columnName); } - // null-tolerant compare: a custom-plugin PartitionFunction inheriting the SPI default returns null for - // getPartitionIdNormalizer; treat null on either side as "match" so segments built by such plugins are not - // perpetually flagged stale. - if (partitionFunction.getPartitionIdNormalizer() != null - && expectedPartitionFunction.getPartitionIdNormalizer() != null - && !partitionFunction.getPartitionIdNormalizer().equalsIgnoreCase( - expectedPartitionFunction.getPartitionIdNormalizer())) { + // Asymmetric null tolerance: a segment built by an older or custom-plugin partition function returns null + // from the SPI default getPartitionIdNormalizer (which is fine — tolerate). However, if the segment side + // declares a normalizer and the config side does not, the user explicitly cleared the normalizer and the + // segment must be rebuilt; same when the two non-null values differ. + String segmentNormalizer = partitionFunction.getPartitionIdNormalizer(); + String configNormalizer = expectedPartitionFunction.getPartitionIdNormalizer(); + if (segmentNormalizer != null + && (configNormalizer == null || !segmentNormalizer.equalsIgnoreCase(configNormalizer))) { LOGGER.debug("tableNameWithType: {}, columnName: {}, segmentName: {}, change: partition id normalizer", tableNameWithType, columnName, segmentName); return new StaleSegment(segmentName, true, "partition id normalizer changed: " + columnName); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/partitioner/TableConfigPartitioner.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/partitioner/TableConfigPartitioner.java index d591512e9b82..ff48ce7f1bc3 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/partitioner/TableConfigPartitioner.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/partitioner/TableConfigPartitioner.java @@ -21,6 +21,7 @@ import javax.annotation.Nullable; import org.apache.pinot.segment.spi.partition.PartitionFunction; import org.apache.pinot.segment.spi.partition.PartitionFunctionFactory; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionPipelineFunction; import org.apache.pinot.spi.config.table.ColumnPartitionConfig; import org.apache.pinot.spi.data.FieldSpec; import org.apache.pinot.spi.data.Schema; @@ -31,6 +32,12 @@ * Partitioner which computes partition values based on the ColumnPartitionConfig from the table config */ public class TableConfigPartitioner implements Partitioner { + /** + * Bucket name used when the row's partition column is null or when the partition expression evaluates to null + * mid-chain. Distinct from any numeric partition id so downstream readers can identify and handle these rows. + */ + public static final String NULL_PARTITION = "null"; + private final String _column; private final PartitionFunction _partitionFunction; /** True when the partition function was compiled with BYTES input; raw byte[] values are passed directly. */ @@ -51,11 +58,7 @@ public TableConfigPartitioner(String columnName, ColumnPartitionConfig columnPar @Override public String getPartition(GenericRow genericRow) { - Object value = genericRow.getValue(_column); - if (_isBytesMode && value instanceof byte[]) { - return String.valueOf(_partitionFunction.getPartition((byte[]) value)); - } - return String.valueOf(_partitionFunction.getPartition(FieldSpec.getStringValue(value))); + return computePartition(genericRow.getValue(_column)); } @Override @@ -69,10 +72,22 @@ public String getPartitionFromColumns(Object[] columnValues) { throw new IllegalArgumentException( "TableConfigPartitioner expects exactly 1 column value, got " + columnValues.length); } - Object value = columnValues[0]; - if (_isBytesMode && value instanceof byte[]) { - return String.valueOf(_partitionFunction.getPartition((byte[]) value)); + return computePartition(columnValues[0]); + } + + private String computePartition(@Nullable Object value) { + // Null input → expression-mode pipelines would fail mid-chain (or return null) and yield NULL_RESULT_PARTITION_ID. + // Map both the null input case and the null-result case to a designated bucket name distinct from any numeric + // partition id, so downstream consumers don't mistake a null-row marker for partition -1. + if (value == null) { + return NULL_PARTITION; + } + int partitionId = (_isBytesMode && value instanceof byte[]) + ? _partitionFunction.getPartition((byte[]) value) + : _partitionFunction.getPartition(FieldSpec.getStringValue(value)); + if (partitionId == PartitionPipelineFunction.NULL_RESULT_PARTITION_ID) { + return NULL_PARTITION; } - return String.valueOf(_partitionFunction.getPartition(FieldSpec.getStringValue(value))); + return String.valueOf(partitionId); } } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/BaseSegmentCreator.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/BaseSegmentCreator.java index 39a7f49d7515..9e9400bdba17 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/BaseSegmentCreator.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/BaseSegmentCreator.java @@ -72,6 +72,8 @@ import org.apache.pinot.segment.spi.loader.SegmentDirectoryLoaderContext; import org.apache.pinot.segment.spi.loader.SegmentDirectoryLoaderRegistry; import org.apache.pinot.segment.spi.partition.PartitionFunction; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionPipelineFunction; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionValueType; import org.apache.pinot.segment.spi.store.SegmentDirectory; import org.apache.pinot.segment.spi.store.SegmentDirectoryPaths; import org.apache.pinot.spi.config.instance.InstanceType; @@ -632,6 +634,12 @@ public static void addColumnMetadataInfo(PropertiesConfiguration properties, Str properties.setProperty(getKeyFor(column, PARTITION_ID_NORMALIZER), partitionFunction.getPartitionIdNormalizer()); } + // For expression-mode pipelines compiled with BYTES input, persist the input type so segment readers don't + // have to re-derive it from schema state (which may race with metadata loading at startup). + if (partitionFunction instanceof PartitionPipelineFunction + && ((PartitionPipelineFunction) partitionFunction).getPartitionPipeline().isBytesInput()) { + properties.setProperty(getKeyFor(column, PARTITION_INPUT_TYPE), PartitionValueType.BYTES.name()); + } properties.setProperty(getKeyFor(column, PARTITION_VALUES), columnStatistics.getPartitions()); Map partitionFunctionConfig = partitionFunction.getFunctionConfig(); if (partitionFunctionConfig != null) { diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/V1Constants.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/V1Constants.java index 2e68e8949c92..115486b43700 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/V1Constants.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/V1Constants.java @@ -190,6 +190,13 @@ public static class Column { * contract. */ public static final String PARTITION_FUNCTION_EXPR_SENTINEL = "FunctionExpr"; + /** + * Stored input type for expression-mode partition pipelines (e.g. {@code "BYTES"}). Optional: when absent, + * readers derive the input type from the column's stored {@link org.apache.pinot.spi.data.FieldSpec.DataType}. + * Writing it explicitly avoids any chance of segment-vs-broker disagreement on the type used to compile the + * pipeline (the broker reads {@code inputType} from ZK metadata; the server now reads from segment metadata). + */ + public static final String PARTITION_INPUT_TYPE = "partitionInputType"; /// Old key maintained for backward compatibility // Replaced by LENGTH_OF_LONGEST_ELEMENT diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/metadata/ColumnMetadataImpl.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/metadata/ColumnMetadataImpl.java index 591e68c96bcb..3f714893dc7c 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/metadata/ColumnMetadataImpl.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/metadata/ColumnMetadataImpl.java @@ -364,9 +364,15 @@ public static ColumnMetadataImpl fromPropertiesConfiguration(PropertiesConfigura // needed here (only required by the ingestion path). Expression-mode needs the column for the pipeline. PartitionFunction partitionFunction; if (partitionFunctionExpr != null) { - // Pass BYTES input type for BYTES columns so functions receive raw bytes directly. - PartitionValueType inputType = - storedType == DataType.BYTES ? PartitionValueType.BYTES : PartitionValueType.STRING; + // Prefer the explicitly-stored PARTITION_INPUT_TYPE if present (newer segments). Fall back to the + // schema-derived type for older segments that pre-date the stored input-type field. + String storedInputType = config.getString(Column.getKeyFor(column, Column.PARTITION_INPUT_TYPE), null); + PartitionValueType inputType; + if (storedInputType != null) { + inputType = PartitionValueType.valueOf(storedInputType); + } else { + inputType = storedType == DataType.BYTES ? PartitionValueType.BYTES : PartitionValueType.STRING; + } partitionFunction = PartitionFunctionExprCompiler.compilePartitionFunction(column, inputType, partitionFunctionExpr, numPartitions, partitionIdNormalizer); } else { diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java index 61c5ff2be259..821298115a31 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java @@ -19,13 +19,15 @@ package org.apache.pinot.segment.spi.partition.pipeline; import com.google.common.base.Preconditions; +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import com.google.common.util.concurrent.UncheckedExecutionException; import java.util.ArrayList; import java.util.List; import java.util.Locale; import java.util.Objects; import java.util.ServiceLoader; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; import javax.annotation.Nullable; import org.apache.pinot.spi.config.table.ColumnPartitionConfig; import org.apache.pinot.spi.function.FunctionEvaluator; @@ -43,9 +45,14 @@ public final class PartitionFunctionExprCompiler { // Cache compiled pipelines so a broker/server with thousands of segments sharing the same partition expression // only Calcite-parses the expression once. PartitionPipeline is stateless except for per-thread scratch arrays in // its inner ExecutableNode tree, so the cached instance is safe to share across multiple PartitionPipelineFunction - // wrappers (each wrapper applies its own numPartitions at normalization time). Cache size is bounded by unique - // (rawColumn, inputType, canonicalExpr, normalizer) tuples — typically a handful per cluster. - private static final ConcurrentMap PIPELINE_CACHE = new ConcurrentHashMap<>(); + // wrappers (each wrapper applies its own numPartitions at normalization time). + // + // Bounded by maximumSize so that long-lived processes with churning table configs (creates/drops/expression edits) + // don't accumulate compiled pipelines indefinitely. The cap is sized for the realistic case of a handful of + // distinct (column, expression) tuples per table × hundreds of tables. + private static final long PIPELINE_CACHE_MAX_SIZE = 10_000L; + private static final Cache PIPELINE_CACHE = + CacheBuilder.newBuilder().maximumSize(PIPELINE_CACHE_MAX_SIZE).build(); private PartitionFunctionExprCompiler() { } @@ -63,8 +70,17 @@ private static PartitionEvaluatorFactory loadEvaluatorFactory() { } Preconditions.checkState(!factories.isEmpty(), "No PartitionEvaluatorFactory implementation found on the classpath"); - Preconditions.checkState(factories.size() == 1, - "Expected exactly 1 PartitionEvaluatorFactory implementation but found %s: %s", factories.size(), factories); + // Tolerate multiple implementations (common in shaded-jar / test classpaths where pinot-common appears via more + // than one route). Prefer the built-in InbuiltPartitionEvaluatorFactory when present; otherwise pick the first + // and log the choice. Hard-failing here would abort every segment load that reaches the compiler. + if (factories.size() == 1) { + return factories.get(0); + } + for (PartitionEvaluatorFactory f : factories) { + if (f.getClass().getSimpleName().equals("InbuiltPartitionEvaluatorFactory")) { + return f; + } + } return factories.get(0); } } @@ -96,10 +112,26 @@ public static PartitionPipeline compile(String rawColumn, PartitionValueType inp String canonicalExpr = canonicalize(functionExpr); boolean isBytesInput = inputType == PartitionValueType.BYTES; PipelineCacheKey cacheKey = new PipelineCacheKey(rawColumn, isBytesInput, canonicalExpr, partitionIdNormalizer); - return PIPELINE_CACHE.computeIfAbsent(cacheKey, k -> { - FunctionEvaluator evaluator = EvaluatorFactoryHolder.INSTANCE.compile(rawColumn, canonicalExpr); - return new PartitionPipeline(rawColumn, isBytesInput, canonicalExpr, partitionIdNormalizer, evaluator); - }); + try { + return PIPELINE_CACHE.get(cacheKey, () -> { + FunctionEvaluator evaluator = EvaluatorFactoryHolder.INSTANCE.compile(rawColumn, canonicalExpr); + return new PartitionPipeline(rawColumn, isBytesInput, canonicalExpr, partitionIdNormalizer, evaluator); + }); + } catch (UncheckedExecutionException e) { + // Loader threw a RuntimeException (e.g. invalid expression). Unwrap so callers see the original failure. + Throwable cause = e.getCause(); + if (cause instanceof RuntimeException) { + throw (RuntimeException) cause; + } + throw new IllegalStateException("Failed to compile partition pipeline for column '" + rawColumn + "'", cause); + } catch (ExecutionException e) { + // Loader threw a checked exception. Unwrap to surface the original cause. + Throwable cause = e.getCause(); + if (cause instanceof RuntimeException) { + throw (RuntimeException) cause; + } + throw new IllegalStateException("Failed to compile partition pipeline for column '" + rawColumn + "'", cause); + } } public static PartitionPipelineFunction compilePartitionFunction(String rawColumn, String functionExpr, diff --git a/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionTest.java b/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionTest.java index af910e7f009a..84afe5323d4e 100644 --- a/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionTest.java +++ b/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionTest.java @@ -886,4 +886,18 @@ private int getMaskPartition(int hash, int numPartitions) { private int getMaskPartition(long hash, int numPartitions) { return (int) ((hash & Long.MAX_VALUE) % numPartitions); } + + /** + * Cross-version contract regression: the segment-metadata sentinel "FunctionExpr" written for expression-mode + * segments MUST never resolve to a real PartitionFunctionType enum constant. Old readers rely on the enum lookup + * throwing IllegalArgumentException to fail fast and skip partition pruning, instead of silently treating the + * column as un-partitioned. If anyone adds a "FunctionExpr" enum value, this test guards against the silent + * regression in the cross-version fail-fast contract. + */ + @Test + public void testFunctionExprSentinelIsNotARealPartitionFunctionType() { + expectThrows(IllegalArgumentException.class, + () -> PartitionFunctionFactory.PartitionFunctionType.fromString( + org.apache.pinot.segment.spi.V1Constants.MetadataKeys.Column.PARTITION_FUNCTION_EXPR_SENTINEL)); + } } From 1cc62c90d3e17259567a5a5ef77abe1b73c6c4d5 Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Sat, 25 Apr 2026 00:58:15 -0700 Subject: [PATCH 16/27] Address additional critical/major findings from code review MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Critical fixes: - PartitionPipelineFunction.normalizeResult: dedicated BigInteger path reduces modulo numPartitions in BigInteger arithmetic before narrowing to int. Number.longValue() silently truncates to the low 64 bits when the magnitude exceeds Long, producing a wrap-around partition id. - SinglePartitionColumnSegmentPruner / MultiPartitionColumnsSegmentPruner: log fail-open partition-function failures at ERROR (not WARN) so the silent scatter-gather degradation surfaces in alerting rather than hiding indefinitely. The fail-open behavior itself is preserved (partition-function bugs must not drop user query results). Major fixes: - PartitionFunctionExprCompiler: add expireAfterAccess(1 hour) to PIPELINE_CACHE so dropped tables / edited expressions don't pin pipelines indefinitely without an explicit invalidation hook. - PartitionPipelineFunction.NAME: bind to V1Constants.MetadataKeys. Column.PARTITION_FUNCTION_EXPR_SENTINEL so the cross-version fail-fast contract for old readers cannot drift via duplicate string literals. - BaseTableDataManager.isPartitionStale: add input-type comparison for expression-mode pipelines. A field-spec edit can flip STRING ↔ BYTES without changing functionExpr or normalizer; the recompiled pipeline would otherwise produce different partition ids than the existing segment without flagging staleness. Co-Authored-By: Claude Sonnet 4.6 --- .../MultiPartitionColumnsSegmentPruner.java | 7 +++++-- .../SinglePartitionColumnSegmentPruner.java | 7 +++++-- .../core/data/manager/BaseTableDataManager.java | 15 +++++++++++++++ .../pipeline/PartitionFunctionExprCompiler.java | 11 +++++++++-- .../pipeline/PartitionPipelineFunction.java | 12 +++++++++--- 5 files changed, 43 insertions(+), 9 deletions(-) diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/MultiPartitionColumnsSegmentPruner.java b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/MultiPartitionColumnsSegmentPruner.java index 82a97b93d615..ec1c0e37b999 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/MultiPartitionColumnsSegmentPruner.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/MultiPartitionColumnsSegmentPruner.java @@ -177,8 +177,11 @@ private boolean isPartitionMatch(SegmentPartitionInfo partitionInfo, String valu try { return partitionInfo.getPartitions().contains(partitionInfo.getPartitionFunction().getPartition(value)); } catch (RuntimeException e) { - LOGGER.warn("Failed to evaluate partition function for table: {}, partition column: {}; skipping partition " - + "pruning", _tableNameWithType, partitionInfo.getPartitionColumn(), e); + // Fail-open: a buggy partition function/expression must not drop user query results. Log at ERROR (not WARN) + // so this surfaces in alerting — silent fail-open hides table-config bugs that should be fixed. + LOGGER.error("Failed to evaluate partition function for table: {}, partition column: {}; falling back to " + + "scatter-gather (no pruning) for this query. Fix the partition expression to avoid query-time fail-open.", + _tableNameWithType, partitionInfo.getPartitionColumn(), e); return true; } } diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/SinglePartitionColumnSegmentPruner.java b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/SinglePartitionColumnSegmentPruner.java index 495d1df1b3c9..818fc6e6e35a 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/SinglePartitionColumnSegmentPruner.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/SinglePartitionColumnSegmentPruner.java @@ -161,8 +161,11 @@ private boolean isPartitionMatch(SegmentPartitionInfo partitionInfo, String valu try { return partitionInfo.getPartitions().contains(partitionInfo.getPartitionFunction().getPartition(value)); } catch (RuntimeException e) { - LOGGER.warn("Failed to evaluate partition function for table: {}, partition column: {}; skipping partition " - + "pruning", _tableNameWithType, _partitionColumn, e); + // Fail-open: a buggy partition function/expression must not drop user query results. Log at ERROR (not WARN) + // so this surfaces in alerting — silent fail-open hides table-config bugs that should be fixed. + LOGGER.error("Failed to evaluate partition function for table: {}, partition column: {}; falling back to " + + "scatter-gather (no pruning) for this query. Fix the partition expression to avoid query-time fail-open.", + _tableNameWithType, _partitionColumn, e); return true; } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java index 13315947b3f6..49d857930202 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java @@ -105,6 +105,7 @@ import org.apache.pinot.segment.spi.loader.SegmentDirectoryLoaderRegistry; import org.apache.pinot.segment.spi.partition.PartitionFunction; import org.apache.pinot.segment.spi.partition.PartitionFunctionFactory; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionPipelineFunction; import org.apache.pinot.segment.spi.store.SegmentDirectory; import org.apache.pinot.spi.auth.AuthProvider; import org.apache.pinot.spi.config.instance.InstanceDataManagerConfig; @@ -1826,6 +1827,20 @@ StaleSegment isSegmentStale(IndexLoadingConfig indexLoadingConfig, SegmentDataMa tableNameWithType, columnName, segmentName); return new StaleSegment(segmentName, true, "partition id normalizer changed: " + columnName); } + // Detect input-type changes (STRING ↔ BYTES) for expression-mode pipelines. A field-spec edit can flip the + // partition column's stored type without changing functionExpr or normalizer; the recompiled pipeline would + // then produce different partition ids than the existing segment. + if (partitionFunction instanceof PartitionPipelineFunction + && expectedPartitionFunction instanceof PartitionPipelineFunction) { + boolean segmentBytes = ((PartitionPipelineFunction) partitionFunction).getPartitionPipeline().isBytesInput(); + boolean configBytes = + ((PartitionPipelineFunction) expectedPartitionFunction).getPartitionPipeline().isBytesInput(); + if (segmentBytes != configBytes) { + LOGGER.debug("tableNameWithType: {}, columnName: {}, segmentName: {}, change: partition input type", + tableNameWithType, columnName, segmentName); + return new StaleSegment(segmentName, true, "partition input type changed: " + columnName); + } + } if (partitionFunction.getNumPartitions() != expectedPartitionFunction.getNumPartitions()) { LOGGER.debug("tableNameWithType: {}, columnName: {},, segmentName: {}, change: num partitions", tableNameWithType, columnName, segmentName); diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java index 821298115a31..f9c99df8be35 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java @@ -28,6 +28,7 @@ import java.util.Objects; import java.util.ServiceLoader; import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; import org.apache.pinot.spi.config.table.ColumnPartitionConfig; import org.apache.pinot.spi.function.FunctionEvaluator; @@ -50,9 +51,15 @@ public final class PartitionFunctionExprCompiler { // Bounded by maximumSize so that long-lived processes with churning table configs (creates/drops/expression edits) // don't accumulate compiled pipelines indefinitely. The cap is sized for the realistic case of a handful of // distinct (column, expression) tuples per table × hundreds of tables. + // expireAfterAccess provides eventual invalidation when a table is dropped or a partition expression is edited + // (no caller currently emits explicit invalidation events). 1 hour is long enough that the cache helps, short + // enough that stale entries from dropped tables don't pin memory indefinitely. private static final long PIPELINE_CACHE_MAX_SIZE = 10_000L; - private static final Cache PIPELINE_CACHE = - CacheBuilder.newBuilder().maximumSize(PIPELINE_CACHE_MAX_SIZE).build(); + private static final long PIPELINE_CACHE_EXPIRE_HOURS = 1L; + private static final Cache PIPELINE_CACHE = CacheBuilder.newBuilder() + .maximumSize(PIPELINE_CACHE_MAX_SIZE) + .expireAfterAccess(PIPELINE_CACHE_EXPIRE_HOURS, TimeUnit.HOURS) + .build(); private PartitionFunctionExprCompiler() { } diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java index a291bb3bf468..aaf51516f6f0 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java @@ -25,6 +25,7 @@ import java.math.BigInteger; import java.util.List; import java.util.Locale; +import org.apache.pinot.segment.spi.V1Constants; import org.apache.pinot.segment.spi.partition.PartitionFunction; import org.apache.pinot.spi.data.FieldSpec; import org.apache.pinot.spi.data.readers.GenericRow; @@ -43,7 +44,12 @@ @SuppressWarnings("serial") public class PartitionPipelineFunction implements PartitionFunction, FunctionEvaluator { private static final Logger LOGGER = LoggerFactory.getLogger(PartitionPipelineFunction.class); - public static final String NAME = "FunctionExpr"; + /** + * Function-name value reported via {@link #getName()} and written into segment metadata for expression-mode + * pipelines. Bound to the reserved sentinel constant in {@link V1Constants.MetadataKeys.Column} so the + * cross-version fail-fast contract for old readers cannot drift via duplicate string literals. + */ + public static final String NAME = V1Constants.MetadataKeys.Column.PARTITION_FUNCTION_EXPR_SENTINEL; // Sentinel returned when the partition expression evaluates to null for a given value (e.g. null input). // Callers that update per-segment partition sets must skip this value. public static final int NULL_RESULT_PARTITION_ID = -1; @@ -238,14 +244,14 @@ public String getPartitionColumn() { } @Override - @JsonIgnore(false) + @JsonIgnore(false) // Override @JsonIgnore on PartitionFunction#getFunctionExpr to expose this in serialization. @JsonProperty("functionExpr") public String getFunctionExpr() { return _pipeline.getCanonicalFunctionExpr(); } @Override - @JsonIgnore(false) + @JsonIgnore(false) // Override @JsonIgnore on PartitionFunction#getPartitionIdNormalizer. @JsonProperty("partitionIdNormalizer") public String getPartitionIdNormalizer() { PartitionIntNormalizer intNormalizer = _pipeline.getIntNormalizer(); From 910347a33bfdc33ee397edc6929282297b7977d4 Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Sat, 25 Apr 2026 01:30:13 -0700 Subject: [PATCH 17/27] Eliminate ThreadLocal re-entrancy hazard; tighten SPI deprecation; perf MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Critical fixes: - ExecutableFunctionEvaluator.FunctionNode + InbuiltFunctionEvaluator .FunctionExecutionNode + PartitionFunctionEvaluator .PartitionFunctionExecutionNode: replace ThreadLocal with per-call Object[] allocation. Previously a single thread re-entering the same shared FunctionNode (e.g. via a UDF that recursively evaluates an expression) would overwrite the outer call's argument scratch and produce silent wrong results. Per-call allocation is small and short-lived; HotSpot's escape analysis routinely scalar- replaces such arrays so per-row cost is negligible. Update thread-safety Javadoc on all three classes. - PartitionFunctionFactory.getPartitionFunction(ColumnPartitionConfig) and getPartitionFunction(ColumnPartitionMetadata): mark @Deprecated. These legacy 1-arg overloads now throw IllegalArgumentException on expression-mode configs/metadata; external plugin authors should migrate to the column-aware overloads. Major fixes: - SegmentPartitionMetadataManager: add a FieldSpec-aware constructor so expression-mode pipelines on BYTES partition columns are compiled with BYTES input. The FieldSpec-less constructor compiles with STRING input, silently disagreeing with ingestion partition ids on BYTES columns. - PartitionPipelineFunction: cache BigInteger.valueOf(_numPartitions) in the constructor to avoid per-row allocation for BigInteger- returning expressions. - PartitionFunctionExprCompiler.PIPELINE_CACHE: switch expireAfterAccess → expireAfterWrite so an edited partition expression on a hot table takes effect within the TTL (access-based expiry would never fire on a continuously-queried table). Co-Authored-By: Claude Sonnet 4.6 --- .../SegmentPartitionMetadataManager.java | 15 ++++++++++++- .../evaluator/InbuiltFunctionEvaluator.java | 19 +++++++--------- .../evaluator/PartitionFunctionEvaluator.java | 18 +++++++-------- .../function/ExecutableFunctionEvaluator.java | 22 +++++++++++-------- .../partition/PartitionFunctionFactory.java | 15 ++++++++----- .../PartitionFunctionExprCompiler.java | 8 +++---- 6 files changed, 56 insertions(+), 41 deletions(-) diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManager.java b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManager.java index 05f0c5134a5a..2d14440f6d2b 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManager.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManager.java @@ -42,6 +42,7 @@ import org.apache.pinot.segment.spi.partition.PartitionFunction; import org.apache.pinot.segment.spi.partition.PartitionFunctionFactory; import org.apache.pinot.spi.config.table.ColumnPartitionConfig; +import org.apache.pinot.spi.data.FieldSpec; import org.apache.pinot.spi.utils.CommonConstants.Helix.StateModel.SegmentStateModel; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -76,9 +77,21 @@ public class SegmentPartitionMetadataManager implements SegmentZkMetadataFetchLi public SegmentPartitionMetadataManager(String tableNameWithType, String partitionColumn, ColumnPartitionConfig columnPartitionConfig) { + this(tableNameWithType, partitionColumn, columnPartitionConfig, null); + } + + /** + * Preferred constructor: pass the partition column's {@link FieldSpec} so expression-mode pipelines on BYTES + * columns are compiled with BYTES input. The {@code FieldSpec}-less overload above always compiles with STRING + * input, which silently disagrees with ingestion partition ids on BYTES columns. + */ + public SegmentPartitionMetadataManager(String tableNameWithType, String partitionColumn, + ColumnPartitionConfig columnPartitionConfig, @Nullable FieldSpec fieldSpec) { _tableNameWithType = tableNameWithType; _partitionColumn = partitionColumn; - _partitionFunction = PartitionFunctionFactory.getPartitionFunction(partitionColumn, columnPartitionConfig); + _partitionFunction = fieldSpec != null + ? PartitionFunctionFactory.getPartitionFunction(partitionColumn, columnPartitionConfig, fieldSpec) + : PartitionFunctionFactory.getPartitionFunction(partitionColumn, columnPartitionConfig); _numPartitions = _partitionFunction.getNumPartitions(); } diff --git a/pinot-common/src/main/java/org/apache/pinot/common/evaluator/InbuiltFunctionEvaluator.java b/pinot-common/src/main/java/org/apache/pinot/common/evaluator/InbuiltFunctionEvaluator.java index e272258d2a15..827cf5502318 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/evaluator/InbuiltFunctionEvaluator.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/evaluator/InbuiltFunctionEvaluator.java @@ -39,11 +39,10 @@ * The expression is compiled once into an {@link ExecutableFunctionEvaluator.ExecutableNode} tree * whose nodes handle constants, column reads, logical operators, and function calls. * - *

Thread-safety: Instances are thread-safe for concurrent invocation. Each - * {@link FunctionExecutionNode} maintains a per-thread argument scratch array via - * {@link ThreadLocal}, so multiple threads can call {@code evaluate} on the same compiled tree - * without sharing mutable state. A single call chain on one thread must not invoke {@code evaluate} - * recursively on the same node instance. + *

Thread-safety: Instances are safe for both concurrent invocation by multiple threads and + * re-entrant invocation on a single thread. Each {@link FunctionExecutionNode} allocates a fresh + * argument scratch array per {@code execute} call, so cross-thread sharing and recursive invocation + * through nested user-defined scalar functions cannot corrupt argument state. */ public class InbuiltFunctionEvaluator extends ExecutableFunctionEvaluator { @@ -114,27 +113,25 @@ private static ExecutableNode planExecution(ExpressionContext expression, List _arguments; FunctionExecutionNode(FunctionInfo functionInfo, ExecutableNode[] argumentNodes) { _functionInvoker = new FunctionInvoker(functionInfo); _functionInfo = functionInfo; _argumentNodes = argumentNodes; - _arguments = ThreadLocal.withInitial(() -> new Object[argumentNodes.length]); } @Override public Object execute(GenericRow row) { try { - Object[] arguments = _arguments.get(); int numArguments = _argumentNodes.length; + Object[] arguments = new Object[numArguments]; for (int i = 0; i < numArguments; i++) { arguments[i] = _argumentNodes[i].execute(row); } @@ -160,8 +157,8 @@ public Object execute(GenericRow row) { @Override public Object execute(Object[] values) { try { - Object[] arguments = _arguments.get(); int numArguments = _argumentNodes.length; + Object[] arguments = new Object[numArguments]; for (int i = 0; i < numArguments; i++) { arguments[i] = _argumentNodes[i].execute(values); } diff --git a/pinot-common/src/main/java/org/apache/pinot/common/evaluator/PartitionFunctionEvaluator.java b/pinot-common/src/main/java/org/apache/pinot/common/evaluator/PartitionFunctionEvaluator.java index 3f3d6b296d79..dbaaa990c6a7 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/evaluator/PartitionFunctionEvaluator.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/evaluator/PartitionFunctionEvaluator.java @@ -51,9 +51,9 @@ *

  • {@code murmur2("hello")} — converts {@code "hello"} to UTF-8 bytes, then hashes
  • * * - *

    Thread-safety: Instances are thread-safe. Each {@link PartitionFunctionExecutionNode} uses a - * {@link ThreadLocal} scratch array so that concurrent invocations from different threads do not share mutable - * state. A single call chain on one thread is not reentrant. + *

    Thread-safety: Instances are safe for both concurrent invocation by multiple threads and + * re-entrant invocation on a single thread. Each {@link PartitionFunctionExecutionNode} allocates a fresh + * argument scratch array per {@code execute} call. */ public final class PartitionFunctionEvaluator extends ExecutableFunctionEvaluator { @@ -127,27 +127,25 @@ private static ExecutableNode planExecution(ExpressionContext expression, ListUses a per-thread {@link ThreadLocal} argument scratch array for thread-safety, following the same pattern - * as {@link ExecutableFunctionEvaluator.FunctionNode}. + *

    Allocates a fresh argument scratch array per {@code execute} call so that concurrent invocations from + * multiple threads, and re-entrant invocations on a single thread, do not share mutable state. */ private static class PartitionFunctionExecutionNode implements ExecutableNode { private final FunctionInvoker _functionInvoker; private final FunctionInfo _functionInfo; private final ExecutableNode[] _argumentNodes; - private final ThreadLocal _arguments; PartitionFunctionExecutionNode(FunctionInfo functionInfo, ExecutableNode[] argumentNodes) { _functionInvoker = new FunctionInvoker(functionInfo); _functionInfo = functionInfo; _argumentNodes = argumentNodes; - _arguments = ThreadLocal.withInitial(() -> new Object[_argumentNodes.length]); } @Override public Object execute(GenericRow row) { try { - Object[] arguments = _arguments.get(); int numArguments = _argumentNodes.length; + Object[] arguments = new Object[numArguments]; for (int i = 0; i < numArguments; i++) { arguments[i] = _argumentNodes[i].execute(row); } @@ -160,8 +158,8 @@ public Object execute(GenericRow row) { @Override public Object execute(Object[] values) { try { - Object[] arguments = _arguments.get(); int numArguments = _argumentNodes.length; + Object[] arguments = new Object[numArguments]; for (int i = 0; i < numArguments; i++) { arguments[i] = _argumentNodes[i].execute(values); } @@ -210,7 +208,7 @@ private void convertTypesWithUtf8StringToBytes(Object[] arguments) { continue; } PinotDataType parameterType = parameterTypes[i]; - PinotDataType argumentType = FunctionUtils.getArgumentType(argumentClass); + PinotDataType argumentType = FunctionUtils.getArgumentType(argument); Preconditions.checkArgument(parameterType != null && argumentType != null, "Cannot convert value from class: %s to class: %s", argumentClass, parameterClass); arguments[i] = parameterType.convert(argument, argumentType); diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/function/ExecutableFunctionEvaluator.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/function/ExecutableFunctionEvaluator.java index d692ef807dac..37666ec67c2a 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/function/ExecutableFunctionEvaluator.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/function/ExecutableFunctionEvaluator.java @@ -31,11 +31,11 @@ * {@link ExecutableNode}. This class owns the repeated row/value execution logic so ingestion and partition paths do * not maintain separate node evaluators. * - *

    Thread-safety: Once constructed, an {@code ExecutableFunctionEvaluator} instance is safe to use - * concurrently from multiple threads. {@link FunctionNode} maintains per-thread argument scratch arrays via - * {@link ThreadLocal}, so concurrent invocations do not share mutable state. A single call chain, however, is - * not reentrant: do not invoke {@code evaluate} recursively on the same thread with the same node instance. - * Subclasses that extend or wrap this evaluator must preserve these guarantees. + *

    Thread-safety: Once constructed, an {@code ExecutableFunctionEvaluator} instance is safe for + * concurrent invocation by multiple threads and for re-entrant invocation on a single thread. + * {@link FunctionNode} allocates a fresh argument scratch array per {@code execute} call, so neither + * cross-thread sharing nor recursive invocation through nested user-defined scalar functions can corrupt + * argument state. */ public class ExecutableFunctionEvaluator implements org.apache.pinot.spi.function.FunctionEvaluator { /** @@ -166,23 +166,27 @@ public String toString() { /** * Executes a bound function after evaluating all child arguments. + * + *

    Allocates a fresh {@code Object[argumentNodes.length]} per {@code execute} call. This is safe under both + * concurrent invocation by multiple threads and re-entrant invocation on a single thread (e.g. when a partition + * expression nested inside a transform function recurses through the same compiled tree). The allocation is + * small and short-lived; HotSpot's escape analysis routinely scalar-replaces such arrays so the per-row cost + * is negligible relative to the underlying function invocation. */ public static class FunctionNode implements ExecutableNode { private final String _functionName; private final Invoker _invoker; private final ExecutableNode[] _argumentNodes; - private final ThreadLocal _arguments; public FunctionNode(String functionName, Invoker invoker, ExecutableNode[] argumentNodes) { _functionName = functionName; _invoker = invoker; _argumentNodes = argumentNodes; - _arguments = ThreadLocal.withInitial(() -> new Object[argumentNodes.length]); } @Override public Object execute(GenericRow row) { - Object[] arguments = _arguments.get(); + Object[] arguments = new Object[_argumentNodes.length]; for (int i = 0; i < _argumentNodes.length; i++) { arguments[i] = _argumentNodes[i].execute(row); } @@ -191,7 +195,7 @@ public Object execute(GenericRow row) { @Override public Object execute(Object[] values) { - Object[] arguments = _arguments.get(); + Object[] arguments = new Object[_argumentNodes.length]; for (int i = 0; i < _argumentNodes.length; i++) { arguments[i] = _argumentNodes[i].execute(values); } diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java index acfd1e142039..2b2c6fb0a7b6 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java @@ -112,10 +112,12 @@ public static PartitionFunction getPartitionFunction(String functionName, int nu /** * Returns the legacy (name-mode) partition function for the given config. * - * @throws IllegalArgumentException if {@code config.getFunctionExpr()} is non-null; use - * {@link #getPartitionFunction(String, ColumnPartitionConfig)} instead for - * expression-mode configs. + * @deprecated Expression-mode configs require a column name to compile the pipeline. This overload throws on + * expression-mode configs; prefer {@link #getPartitionFunction(String, ColumnPartitionConfig)} or + * {@link #getPartitionFunction(String, ColumnPartitionConfig, FieldSpec)} which support both modes. + * @throws IllegalArgumentException if {@code config.getFunctionExpr()} is non-null. */ + @Deprecated public static PartitionFunction getPartitionFunction(ColumnPartitionConfig config) { Preconditions.checkNotNull(config, "Column partition config must be configured"); Preconditions.checkArgument(config.getFunctionExpr() == null, @@ -126,10 +128,11 @@ public static PartitionFunction getPartitionFunction(ColumnPartitionConfig confi /** * Returns the legacy (name-mode) partition function for the given segment metadata. * - * @throws IllegalArgumentException if {@code metadata.getFunctionExpr()} is non-null; use - * {@link #getPartitionFunction(String, ColumnPartitionMetadata)} instead for - * expression-mode metadata. + * @deprecated Expression-mode metadata requires a column name to compile the pipeline. This overload throws on + * expression-mode metadata; prefer {@link #getPartitionFunction(String, ColumnPartitionMetadata)}. + * @throws IllegalArgumentException if {@code metadata.getFunctionExpr()} is non-null. */ + @Deprecated public static PartitionFunction getPartitionFunction(ColumnPartitionMetadata metadata) { Preconditions.checkNotNull(metadata, "Column partition metadata must be configured"); Preconditions.checkArgument(metadata.getFunctionExpr() == null, diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java index f9c99df8be35..d99306c81da6 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java @@ -51,14 +51,14 @@ public final class PartitionFunctionExprCompiler { // Bounded by maximumSize so that long-lived processes with churning table configs (creates/drops/expression edits) // don't accumulate compiled pipelines indefinitely. The cap is sized for the realistic case of a handful of // distinct (column, expression) tuples per table × hundreds of tables. - // expireAfterAccess provides eventual invalidation when a table is dropped or a partition expression is edited - // (no caller currently emits explicit invalidation events). 1 hour is long enough that the cache helps, short - // enough that stale entries from dropped tables don't pin memory indefinitely. + // expireAfterWrite (not access) ensures that an edited expression (rare, but legitimate during schema evolution) + // takes effect within the TTL even on a hot table whose cache entry would otherwise never expire. Combined with + // maximumSize, this caps both staleness and memory. private static final long PIPELINE_CACHE_MAX_SIZE = 10_000L; private static final long PIPELINE_CACHE_EXPIRE_HOURS = 1L; private static final Cache PIPELINE_CACHE = CacheBuilder.newBuilder() .maximumSize(PIPELINE_CACHE_MAX_SIZE) - .expireAfterAccess(PIPELINE_CACHE_EXPIRE_HOURS, TimeUnit.HOURS) + .expireAfterWrite(PIPELINE_CACHE_EXPIRE_HOURS, TimeUnit.HOURS) .build(); private PartitionFunctionExprCompiler() { From e1e829045d78f922ad6e6a55c7f2b1bfd64c7dc7 Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Sat, 25 Apr 2026 03:09:15 -0700 Subject: [PATCH 18/27] Fix realtime stream-numPartitions regression; harden eval contracts Critical fix: - RealtimeSegmentDataManager + StatelessRealtimeSegmentWriter: thread the stream-derived numPartitions override through to the partition function builder. The previous 3-arg fieldSpec overload silently used config.getNumPartitions(), discarding the local override computed from the stream. This is a regression on the realtime ingestion path that affects both legacy name-mode and expression-mode partition functions on streams that have been resharded beyond the table-config partition count. Major fixes: - ColumnPartitionMetadata.ColumnPartitionMetadataDeserializer: guard against missing 'numPartitions' key with a descriptive IllegalArgumentException instead of letting a NullPointerException propagate up to the caller's catch handler with an opaque message. - PartitionPipelineFunction constructor: assert that the wrapped pipeline has a non-null IntNormalizer, surfacing the contract violation at construction time rather than crashing on the first row. - PartitionEvaluatorFactory Javadoc: correct the documented loader contract from "Exactly one implementation" to match the actual multi-impl-tolerant behavior. PartitionFunctionExprCompiler now logs the chosen factory at INFO when more than one is present so operators can detect classpath ambiguity. - PartitionPipelineFunction.evaluate(GenericRow): delegate directly to PartitionPipeline.evaluate(GenericRow), saving one Object[] allocation per row on the ingestion hot path. The pipeline already handles the column lookup, BYTES-vs-STRING dispatch, and null short-circuit; the wrapper only adds normalizeResult. Co-Authored-By: Claude Sonnet 4.6 --- .../realtime/RealtimeSegmentDataManager.java | 5 ++++- .../writer/StatelessRealtimeSegmentWriter.java | 5 ++++- .../metadata/ColumnPartitionMetadata.java | 9 ++++++++- .../pipeline/PartitionEvaluatorFactory.java | 6 ++++-- .../PartitionFunctionExprCompiler.java | 10 +++++++++- .../pipeline/PartitionPipelineFunction.java | 18 +++++++++++------- 6 files changed, 40 insertions(+), 13 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java index 2f16aca42a1c..1e7552c9d4a3 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java @@ -2044,8 +2044,11 @@ private void setPartitionParameters(RealtimeSegmentConfig.Builder realtimeSegmen } realtimeSegmentConfigBuilder.setPartitionColumn(partitionColumn); + // Use the stream-derived numPartitions (may differ from the table-config value if the stream has been + // resharded). The FieldSpec-aware 4-arg overload threads numPartitions through to expression-mode + // pipelines on BYTES columns, preserving the override. realtimeSegmentConfigBuilder.setPartitionFunction( - PartitionFunctionFactory.getPartitionFunction(partitionColumn, columnPartitionConfig, + PartitionFunctionFactory.getPartitionFunction(partitionColumn, columnPartitionConfig, numPartitions, _schema.getFieldSpecFor(partitionColumn))); realtimeSegmentConfigBuilder.setPartitionId(_partitionGroupId); } else { diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java index dcbceb33f1b0..075830e21d32 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java @@ -437,8 +437,11 @@ private void setPartitionParameters(RealtimeSegmentConfig.Builder realtimeSegmen } realtimeSegmentConfigBuilder.setPartitionColumn(partitionColumn); + // Use the stream-derived numPartitions (may differ from the table-config value if the stream has been + // resharded). The FieldSpec-aware 4-arg overload threads numPartitions through to expression-mode + // pipelines on BYTES columns, preserving the override. realtimeSegmentConfigBuilder.setPartitionFunction( - PartitionFunctionFactory.getPartitionFunction(partitionColumn, columnPartitionConfig, + PartitionFunctionFactory.getPartitionFunction(partitionColumn, columnPartitionConfig, numPartitions, _schema.getFieldSpecFor(partitionColumn))); realtimeSegmentConfigBuilder.setPartitionId(_partitionGroupId); } else { diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java index a743023d3eee..aef0339020cf 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java @@ -264,8 +264,15 @@ public ColumnPartitionMetadata deserialize(JsonParser p, DeserializationContext JsonNode functionExprNode = jsonMetadata.get(FUNCTION_EXPR_KEY); JsonNode partitionIdNormalizerNode = jsonMetadata.get(PARTITION_ID_NORMALIZER_KEY); JsonNode inputTypeNode = jsonMetadata.get(INPUT_TYPE_KEY); + // numPartitions is mandatory in segment partition metadata. Surface a clear error if absent rather than + // letting the caller's catch handler log an opaque NPE. + JsonNode numPartitionsNode = jsonMetadata.get(NUM_PARTITIONS_KEY); + if (numPartitionsNode == null || numPartitionsNode.isNull()) { + throw new IllegalArgumentException( + "'" + NUM_PARTITIONS_KEY + "' is required in segment partition metadata"); + } return new ColumnPartitionMetadata(readOptionalText(functionNameNode), - jsonMetadata.get(NUM_PARTITIONS_KEY).asInt(), partitions, functionConfig, + numPartitionsNode.asInt(), partitions, functionConfig, readOptionalText(functionExprNode), readOptionalText(partitionIdNormalizerNode), readOptionalText(inputTypeNode)); } diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionEvaluatorFactory.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionEvaluatorFactory.java index 7392daf7236f..a2b3eb32381c 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionEvaluatorFactory.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionEvaluatorFactory.java @@ -24,8 +24,10 @@ /** * SPI for compiling a partition expression string into a {@link FunctionEvaluator}. * - *

    Implementations are loaded via {@link java.util.ServiceLoader}. Exactly one implementation must be present on the - * classpath at runtime. The default implementation in {@code pinot-common} is + *

    Implementations are loaded via {@link java.util.ServiceLoader}. At least one implementation must be present on + * the classpath at runtime. If multiple are found (common in shaded-jar / multi-module test setups), the loader + * prefers {@code InbuiltPartitionEvaluatorFactory} when present; otherwise it picks the first discovered factory and + * logs the choice. The default implementation in {@code pinot-common} is * {@code org.apache.pinot.common.evaluator.InbuiltPartitionEvaluatorFactory}, which delegates to * {@code org.apache.pinot.common.evaluator.PartitionFunctionEvaluator}. */ diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java index d99306c81da6..d3ff261bb52b 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java @@ -32,6 +32,8 @@ import javax.annotation.Nullable; import org.apache.pinot.spi.config.table.ColumnPartitionConfig; import org.apache.pinot.spi.function.FunctionEvaluator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** @@ -39,6 +41,7 @@ * {@link org.apache.pinot.spi.function.FunctionEvaluator} provided by {@link PartitionEvaluatorFactory}. */ public final class PartitionFunctionExprCompiler { + private static final Logger LOGGER = LoggerFactory.getLogger(PartitionFunctionExprCompiler.class); // Cap on user-supplied expression length, sized to fit comfortably within ZK / segment metadata size budgets while // allowing realistic chained expressions (e.g. fnv1a_32(md5(lower(trim(col))))). private static final int MAX_EXPRESSION_LENGTH = 256; @@ -85,10 +88,15 @@ private static PartitionEvaluatorFactory loadEvaluatorFactory() { } for (PartitionEvaluatorFactory f : factories) { if (f.getClass().getSimpleName().equals("InbuiltPartitionEvaluatorFactory")) { + LOGGER.info("Found {} PartitionEvaluatorFactory implementations on the classpath: {}; preferring {}", + factories.size(), factories, f.getClass().getName()); return f; } } - return factories.get(0); + PartitionEvaluatorFactory chosen = factories.get(0); + LOGGER.info("Found {} PartitionEvaluatorFactory implementations on the classpath: {}; using first: {}", + factories.size(), factories, chosen.getClass().getName()); + return chosen; } } diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java index aaf51516f6f0..cce1a7744eaf 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java @@ -63,6 +63,12 @@ public class PartitionPipelineFunction implements PartitionFunction, FunctionEva public PartitionPipelineFunction(PartitionPipeline pipeline, int numPartitions) { Preconditions.checkNotNull(pipeline, "Partition pipeline must be configured"); Preconditions.checkArgument(numPartitions > 0, "Number of partitions must be > 0"); + // PartitionPipeline._intNormalizer is @Nullable for compile-time flexibility; this wrapper requires it because + // normalizeResult depends on the normalizer to map raw integer outputs into [0, numPartitions). Surface the + // contract violation at construction time rather than crashing on the first row. + Preconditions.checkArgument(pipeline.getIntNormalizer() != null, + "Partition pipeline for column '%s' must have an INT normalizer when wrapped as a PartitionPipelineFunction", + pipeline.getRawColumn()); _pipeline = pipeline; _numPartitions = numPartitions; } @@ -267,13 +273,11 @@ public List getArguments() { @Override public Object evaluate(GenericRow genericRow) { - Object inputValue = genericRow.getValue(_pipeline.getRawColumn()); - if (inputValue == null) { - return null; - } - int partitionId = (inputValue instanceof byte[] && _pipeline.isBytesInput()) - ? getPartition((byte[]) inputValue) - : getPartition(FieldSpec.getStringValue(inputValue)); + // Delegate to the underlying pipeline directly: it already handles the column lookup, the BYTES-vs-STRING + // dispatch, and the null-input early return. Avoids duplicating that logic here and saves one Object[] + // allocation per row vs the getPartition(...) path. + Object rawResult = _pipeline.evaluate(genericRow); + int partitionId = normalizeResult(rawResult); return partitionId == NULL_RESULT_PARTITION_ID ? null : partitionId; } From f4d918e25a7c72cb6b695b11c86a035cc28356d6 Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Sat, 25 Apr 2026 03:31:16 -0700 Subject: [PATCH 19/27] Harmonize inputType wire-format key; fix Deprecated discipline; staleness gap MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Major fixes: - ColumnPartitionMetadata.getInputType: bind @JsonProperty to "partitionInputType" so the ZK JSON field matches the segment metadata.properties key V1Constants.PARTITION_INPUT_TYPE. Update the deserializer's INPUT_TYPE_KEY accordingly. Eliminates the same-logical-field-different-name asymmetry that would silently fall back to STRING-input semantics for BYTES columns when the two paths exchange data. - SegmentPartitionConfig.getFunctionConfig(String): restore the @Deprecated annotation that was accidentally dropped — the upstream-deprecated method must keep emitting deprecation warnings to existing callers/IDEs. - StatsCollectorConfig.getPartitionFunctionExpr(String): remove the @Deprecated annotation. The method was newly added; per C1.15 only deprecate when a replacement exists. - BaseTableDataManager.isPartitionStale: detect "partition function removed" — segment carries a partitionFunction but the table config no longer has any partitionColumn (or removed this column). Flag stale so the segment is rebuilt without partition metadata, instead of silently retaining stale routing info until replaced. - PartitionFunction SPI Javadoc: document the expression-mode contract (non-null getFunctionExpr indicates expression mode; framework callers dispatch on that distinction). Co-Authored-By: Claude Sonnet 4.6 --- .../metadata/ColumnPartitionMetadataTest.java | 23 +++++++++++++++++++ .../data/manager/BaseTableDataManager.java | 8 +++++++ .../BaseTableDataManagerNeedRefreshTest.java | 6 +++-- .../spi/creator/StatsCollectorConfig.java | 1 - .../spi/partition/PartitionFunction.java | 11 ++++++++- .../metadata/ColumnPartitionMetadata.java | 10 ++++++-- .../config/table/SegmentPartitionConfig.java | 1 + .../LaunchBackfillIngestionJobCommand.java | 10 ++------ 8 files changed, 56 insertions(+), 14 deletions(-) diff --git a/pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadataTest.java b/pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadataTest.java index b6abb1100210..793b3d806c73 100644 --- a/pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadataTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadataTest.java @@ -19,8 +19,12 @@ package org.apache.pinot.segment.spi.partition.metadata; import java.util.Set; +import org.apache.pinot.segment.spi.partition.PartitionFunction; +import org.apache.pinot.segment.spi.partition.PartitionFunctionFactory; import org.apache.pinot.segment.spi.partition.pipeline.PartitionFunctionExprCompiler; import org.apache.pinot.segment.spi.partition.pipeline.PartitionPipelineFunction; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionValueType; +import org.apache.pinot.spi.utils.BytesUtils; import org.apache.pinot.spi.utils.JsonUtils; import org.testng.annotations.Test; @@ -86,4 +90,23 @@ public void testExpressionModeRoundTripPreservesAllFields() assertNull(roundTripped.getInputType()); assertEquals(roundTripped, metadata); } + + @Test + public void testBytesInputTypeRoundTripRebuildsMatchingPartitionFunction() + throws Exception { + byte[] value = new byte[]{1, 2, 3}; + PartitionPipelineFunction partitionFunction = PartitionFunctionExprCompiler.compilePartitionFunction( + "id", PartitionValueType.BYTES, "murmur2(id)", 16, "MASK"); + ColumnPartitionMetadata metadata = + new ColumnPartitionMetadata(partitionFunction, Set.of(partitionFunction.getPartition(value))); + + String json = JsonUtils.objectToString(metadata); + assertTrue(json.contains("\"partitionInputType\""), "JSON must include non-default BYTES input type"); + + ColumnPartitionMetadata roundTripped = JsonUtils.stringToObject(json, ColumnPartitionMetadata.class); + PartitionFunction rebuilt = PartitionFunctionFactory.getPartitionFunction("id", roundTripped); + + assertEquals(roundTripped.getInputType(), PartitionValueType.BYTES.name()); + assertEquals(rebuilt.getPartition(BytesUtils.toHexString(value)), partitionFunction.getPartition(value)); + } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java index 49d857930202..1c47e5c317b3 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java @@ -1797,6 +1797,14 @@ StaleSegment isSegmentStale(IndexLoadingConfig indexLoadingConfig, SegmentDataMa return new StaleSegment(segmentName, true, "range index changed: " + columnName); } + // Partition function removed: segment carries a partitionFunction but the table config no longer has any + // partitionColumn (or removed this column). Flag stale so the segment is rebuilt without partition metadata. + if (partitionColumn == null && columnMetadata.getPartitionFunction() != null) { + LOGGER.debug("tableNameWithType: {}, columnName: {}, segmentName: {}, change: partition function removed", + tableNameWithType, columnName, segmentName); + return new StaleSegment(segmentName, true, "partition function removed: " + columnName); + } + // Partition changed or segment not properly partitioned if (columnName.equals(partitionColumn)) { PartitionFunction partitionFunction = columnMetadata.getPartitionFunction(); diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/BaseTableDataManagerNeedRefreshTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/BaseTableDataManagerNeedRefreshTest.java index 61657b8e5fdf..0e262c573c80 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/BaseTableDataManagerNeedRefreshTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/BaseTableDataManagerNeedRefreshTest.java @@ -384,8 +384,10 @@ void testPartition() assertTrue(response.isStale()); assertEquals(response.getReason(), "partition function added: partitionedColumn"); - // when segment has partitions AND tableConfig has no partitions, then needRefresh = false - assertFalse(BASE_TABLE_DATA_MANAGER.isSegmentStale(INDEX_LOADING_CONFIG, segmentWithPartition).isStale()); + // when segment has partitions AND tableConfig has no partitions, then needRefresh = true + response = BASE_TABLE_DATA_MANAGER.isSegmentStale(INDEX_LOADING_CONFIG, segmentWithPartition); + assertTrue(response.isStale()); + assertEquals(response.getReason(), "partition function removed: partitionedColumn"); // when # of partitions is different, then needRefresh = true TableConfig partitionedTableConfig40 = getTableConfigBuilder().setSegmentPartitionConfig(new SegmentPartitionConfig( diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/StatsCollectorConfig.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/StatsCollectorConfig.java index 6e2c6d61fd9b..812305ab2ccd 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/StatsCollectorConfig.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/StatsCollectorConfig.java @@ -117,7 +117,6 @@ public int getNumPartitions(String column) { : SegmentPartitionConfig.INVALID_NUM_PARTITIONS; } - @Deprecated @Nullable public String getPartitionFunctionExpr(String column) { return (_segmentPartitionConfig != null) ? _segmentPartitionConfig.getFunctionExpr(column) : null; diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunction.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunction.java index fafc5eaf46bd..56c60add5a2b 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunction.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunction.java @@ -30,7 +30,16 @@ * *

    Implementations of this interface are assumed not to be stateful. * That is, two invocations of {@code PartitionFunction.getPartition(value)} - * with the same value are expected to produce the same result. + * with the same value are expected to produce the same result. Implementations must also be safe for concurrent + * invocation by multiple threads. + * + *

    Expression-mode partition functions: When {@link #getFunctionExpr()} returns non-null, the implementation + * is operating in expression mode (e.g. {@code PartitionPipelineFunction}). In that case + * {@link #getPartitionIdNormalizer()} and {@link #getPartitionColumn()} also typically return non-null. Existing + * legacy partition functions ({@code Murmur}, {@code Modulo}, {@code HashCode}, etc.) return {@code null} from these + * accessors and continue to operate as before. Framework callers (segment writers, broker pruners, staleness checks) + * use the non-null/null distinction on {@code getFunctionExpr} to dispatch between the two modes — plugins that + * want to be treated as expression-mode must override the relevant accessors. */ public interface PartitionFunction extends Serializable { diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java index aef0339020cf..85ab2f87ab48 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java @@ -19,6 +19,7 @@ package org.apache.pinot.segment.spi.partition.metadata; import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.DeserializationContext; @@ -123,9 +124,12 @@ public String getFunctionName() { /** * Returns the pipeline input type for expression-mode partition functions, or {@code null} if the default - * ({@link PartitionValueType#STRING}) input type applies. + * ({@link PartitionValueType#STRING}) input type applies. The {@code @JsonProperty} value matches + * {@link org.apache.pinot.segment.spi.V1Constants.MetadataKeys.Column#PARTITION_INPUT_TYPE} so the same key + * names the field on both wire formats (ZK JSON and segment {@code .properties}). */ @Nullable + @JsonProperty("partitionInputType") public String getInputType() { return _inputType; } @@ -227,7 +231,9 @@ public static class ColumnPartitionMetadataDeserializer extends JsonDeserializer private static final String FUNCTION_CONFIG_KEY = "functionConfig"; private static final String FUNCTION_EXPR_KEY = "functionExpr"; private static final String PARTITION_ID_NORMALIZER_KEY = "partitionIdNormalizer"; - private static final String INPUT_TYPE_KEY = "inputType"; + // Matches V1Constants.MetadataKeys.Column.PARTITION_INPUT_TYPE so the same key names the field on both + // wire formats (ZK JSON and segment .properties). + private static final String INPUT_TYPE_KEY = "partitionInputType"; private static final String PARTITIONS_KEY = "partitions"; // DO NOT CHANGE: for backward-compatibility diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentPartitionConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentPartitionConfig.java index 816ca988341d..8662e75d6e12 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentPartitionConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentPartitionConfig.java @@ -73,6 +73,7 @@ public String getPartitionIdNormalizer(String column) { return (config != null) ? config.getPartitionIdNormalizer() : null; } + @Deprecated @Nullable public Map getFunctionConfig(String column) { ColumnPartitionConfig config = _columnPartitionMap.get(column); diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/LaunchBackfillIngestionJobCommand.java b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/LaunchBackfillIngestionJobCommand.java index 255258f976f4..1ca2fa70beff 100644 --- a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/LaunchBackfillIngestionJobCommand.java +++ b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/LaunchBackfillIngestionJobCommand.java @@ -226,14 +226,8 @@ private boolean isSegmentMatchPartition(SegmentGenerationJobSpec spec, String se } // Compute partition ID for the specified partition column value - int partitionId = PartitionFunctionFactory.getPartitionFunction( - _partitionColumn, - columnMetadata.getFunctionName(), - columnMetadata.getNumPartitions(), - columnMetadata.getFunctionConfig(), - columnMetadata.getFunctionExpr(), - columnMetadata.getPartitionIdNormalizer() - ).getPartition(_partitionColumnValue); + int partitionId = PartitionFunctionFactory.getPartitionFunction(_partitionColumn, columnMetadata) + .getPartition(_partitionColumnValue); // Return true if segment contains the computed partition return columnMetadata.getPartitions().contains(partitionId); From ab61799f998dff0fe64ab8f95394707742f2b00b Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Sat, 25 Apr 2026 03:59:10 -0700 Subject: [PATCH 20/27] Polish: cache normalizer, FQN match, validation strict on non-null probes Major fixes: - PartitionFunctionExprCompiler.EvaluatorFactoryHolder: match InbuiltPartitionEvaluatorFactory by FQN instead of getSimpleName(). A user-supplied factory in a different package with the same simple name would otherwise silently shadow the real built-in. - PartitionPipelineFunction.validateOutputType: track non-null-probe success separately. Previously a pipeline that only succeeds on null inputs (e.g. case-when patterns guarding nulls) would pass validation via the null probe and then crash on the first real row. - PartitionPipelineFunction.normalizeResult: remove the per-row Preconditions.checkState(intNormalizer != null) check and the associated getter call. Cache _intNormalizer at construction time; the constructor already enforces non-null. Minor fixes: - BaseTableDataManager.isPartitionStale: drop the duplicate comma in the "num partitions changed" debug log format. Co-Authored-By: Claude Sonnet 4.6 --- .../data/manager/BaseTableDataManager.java | 2 +- .../PartitionFunctionExprCompiler.java | 6 +++- .../pipeline/PartitionPipelineFunction.java | 36 ++++++++++--------- 3 files changed, 26 insertions(+), 18 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java index 1c47e5c317b3..1a19c251f221 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java @@ -1850,7 +1850,7 @@ StaleSegment isSegmentStale(IndexLoadingConfig indexLoadingConfig, SegmentDataMa } } if (partitionFunction.getNumPartitions() != expectedPartitionFunction.getNumPartitions()) { - LOGGER.debug("tableNameWithType: {}, columnName: {},, segmentName: {}, change: num partitions", + LOGGER.debug("tableNameWithType: {}, columnName: {}, segmentName: {}, change: num partitions", tableNameWithType, columnName, segmentName); return new StaleSegment(segmentName, true, "num partitions changed: " + columnName); } diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java index d3ff261bb52b..186ca0e486be 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java @@ -86,8 +86,12 @@ private static PartitionEvaluatorFactory loadEvaluatorFactory() { if (factories.size() == 1) { return factories.get(0); } + // Match by FQN, not getSimpleName(): a user-supplied factory in a different package with the same simple name + // would silently shadow the real built-in. Hard-coding the FQN here is acceptable because pinot-common is + // always on the broker/server/controller classpath. + String inbuiltFqn = "org.apache.pinot.common.evaluator.InbuiltPartitionEvaluatorFactory"; for (PartitionEvaluatorFactory f : factories) { - if (f.getClass().getSimpleName().equals("InbuiltPartitionEvaluatorFactory")) { + if (f.getClass().getName().equals(inbuiltFqn)) { LOGGER.info("Found {} PartitionEvaluatorFactory implementations on the classpath: {}; preferring {}", factories.size(), factories, f.getClass().getName()); return f; diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java index cce1a7744eaf..ac12e522f124 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java @@ -59,6 +59,9 @@ public class PartitionPipelineFunction implements PartitionFunction, FunctionEva private final PartitionPipeline _pipeline; private final int _numPartitions; + // Cache the IntNormalizer so the per-row hot path skips the @Nullable getter lookup + Preconditions.checkState. + // The constructor enforces non-null below; this is then a class invariant for the wrapper. + private final PartitionIntNormalizer _intNormalizer; public PartitionPipelineFunction(PartitionPipeline pipeline, int numPartitions) { Preconditions.checkNotNull(pipeline, "Partition pipeline must be configured"); @@ -71,6 +74,7 @@ public PartitionPipelineFunction(PartitionPipeline pipeline, int numPartitions) pipeline.getRawColumn()); _pipeline = pipeline; _numPartitions = numPartitions; + _intNormalizer = pipeline.getIntNormalizer(); } public PartitionPipeline getPartitionPipeline() { @@ -87,10 +91,12 @@ public void validateOutputType() { // Probe set covers numeric strings, alpha strings, raw bytes (for BYTES input), and null. The null sample // mirrors the runtime ingestion path that already handles nulls, so the validation also exercises that the // pipeline does not blow up on null input. Non-null samples are checked for non-numeric output types. + // Track non-null-probe success separately: a pipeline that only succeeds on null (e.g. "case when col is null + // then 0 else fail() end") would otherwise pass validation and crash on the first real row. Object[] samples = _pipeline.isBytesInput() ? new Object[]{new byte[]{0}, new byte[]{1, 2, 3}, new byte[0], null} : new Object[]{"1", "0", "abc", null}; - boolean anyProbeSucceeded = false; + boolean anyNonNullProbeSucceeded = false; RuntimeException lastFailure = null; for (Object sample : samples) { Object probe; @@ -100,7 +106,9 @@ public void validateOutputType() { lastFailure = e; continue; } - anyProbeSucceeded = true; + if (sample != null) { + anyNonNullProbeSucceeded = true; + } if (probe == null) { continue; } @@ -115,10 +123,10 @@ public void validateOutputType() { "Partition pipeline must produce INT or LONG output: " + e.getMessage(), e); } } - if (!anyProbeSucceeded) { + if (!anyNonNullProbeSucceeded) { throw new IllegalArgumentException( - "Partition pipeline for column '" + _pipeline.getRawColumn() + "' failed to evaluate against any sample " - + "input; check the expression is well-formed and accepts the column's stored type. Last error: " + "Partition pipeline for column '" + _pipeline.getRawColumn() + "' failed to evaluate against any non-null " + + "sample input; check the expression is well-formed and accepts the column's stored type. Last error: " + (lastFailure != null ? lastFailure.getMessage() : "unknown")); } } @@ -193,18 +201,15 @@ private int normalizeResult(Object result) { + "(|x| < 2^53), got: %s (%s); cast the expression result to LONG explicitly to avoid this", _pipeline.getRawColumn(), result, result.getClass().getSimpleName()); } - PartitionIntNormalizer intNormalizer = _pipeline.getIntNormalizer(); - Preconditions.checkState(intNormalizer != null, - "Integral-output partition pipeline for column '%s' must have an INT normalizer", - _pipeline.getRawColumn()); + // _intNormalizer is non-null by class invariant (enforced in the constructor). // BigInteger and BigDecimal get dedicated paths so Number.longValue()/intValue() cannot silently truncate // fractional values or values outside the LONG range. if (num instanceof BigInteger) { - return normalizeBigIntegerResult((BigInteger) num, result, intNormalizer); + return normalizeBigIntegerResult((BigInteger) num, result); } if (num instanceof BigDecimal) { try { - return normalizeBigIntegerResult(((BigDecimal) num).toBigIntegerExact(), result, intNormalizer); + return normalizeBigIntegerResult(((BigDecimal) num).toBigIntegerExact(), result); } catch (ArithmeticException e) { throw new IllegalStateException( "Partition expression for column '" + _pipeline.getRawColumn() @@ -212,20 +217,19 @@ private int normalizeResult(Object result) { } } if (num instanceof Long || num instanceof Float || num instanceof Double) { - return intNormalizer.getPartitionId(num.longValue(), _numPartitions); + return _intNormalizer.getPartitionId(num.longValue(), _numPartitions); } if (num instanceof Integer || num instanceof Short || num instanceof Byte) { - return intNormalizer.getPartitionId(num.intValue(), _numPartitions); + return _intNormalizer.getPartitionId(num.intValue(), _numPartitions); } throw new IllegalStateException( "Partition expression for column '" + _pipeline.getRawColumn() + "' must return INT or LONG-compatible output, got: " + javaClassToTypeName(result.getClass())); } - private int normalizeBigIntegerResult(BigInteger value, Object originalResult, - PartitionIntNormalizer intNormalizer) { + private int normalizeBigIntegerResult(BigInteger value, Object originalResult) { try { - return intNormalizer.getPartitionId(value.longValueExact(), _numPartitions); + return _intNormalizer.getPartitionId(value.longValueExact(), _numPartitions); } catch (ArithmeticException e) { throw new IllegalStateException( "Partition expression for column '" + _pipeline.getRawColumn() From c56a353c5a5e168e6c4c34a2d5d56537fccbe57d Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Sat, 25 Apr 2026 04:20:21 -0700 Subject: [PATCH 21/27] Add drift test for partitionIdNormalizer constants; sunset note Major fix: - Add testPartitionIdNormalizerConstantsMatchEnumValues to lock the alignment between ColumnPartitionConfig.PARTITION_ID_NORMALIZER_* string constants (pinot-spi) and PartitionIntNormalizer enum values (pinot-segment-spi). The duplication exists because pinot-spi cannot reference the enum due to module dependency direction; without this test, adding a new normalizer to the enum without updating the validator string list would silently reject configs at runtime. Minor: - SegmentPartitionConfig.getFunctionConfig(String): add a TODO with the planned removal target so the @Deprecated has an explicit sunset. Co-Authored-By: Claude Sonnet 4.6 --- .../spi/partition/PartitionFunctionTest.java | 29 ++++++++++++++++++- .../config/table/SegmentPartitionConfig.java | 2 ++ 2 files changed, 30 insertions(+), 1 deletion(-) diff --git a/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionTest.java b/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionTest.java index 84afe5323d4e..5ddd930db5e8 100644 --- a/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionTest.java +++ b/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionTest.java @@ -20,8 +20,13 @@ import com.fasterxml.jackson.databind.JsonNode; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; import java.util.Random; +import java.util.Set; +import org.apache.pinot.segment.spi.V1Constants; +import org.apache.pinot.segment.spi.partition.pipeline.PartitionIntNormalizer; +import org.apache.pinot.spi.config.table.ColumnPartitionConfig; import org.apache.pinot.spi.data.readers.GenericRow; import org.apache.pinot.spi.function.FunctionEvaluator; import org.apache.pinot.spi.utils.BytesUtils; @@ -898,6 +903,28 @@ private int getMaskPartition(long hash, int numPartitions) { public void testFunctionExprSentinelIsNotARealPartitionFunctionType() { expectThrows(IllegalArgumentException.class, () -> PartitionFunctionFactory.PartitionFunctionType.fromString( - org.apache.pinot.segment.spi.V1Constants.MetadataKeys.Column.PARTITION_FUNCTION_EXPR_SENTINEL)); + V1Constants.MetadataKeys.Column.PARTITION_FUNCTION_EXPR_SENTINEL)); + } + + /** + * Drift regression: {@code ColumnPartitionConfig.PARTITION_ID_NORMALIZER_*} string constants live in pinot-spi + * (which cannot reference {@link PartitionIntNormalizer} due to the module dependency direction). This test + * asserts the two stay in lockstep so adding a new normalizer to the enum without updating the validator string + * list trips a test failure rather than a silent runtime rejection. + */ + @Test + public void testPartitionIdNormalizerConstantsMatchEnumValues() { + Set enumNames = new HashSet<>(); + for (PartitionIntNormalizer normalizer : PartitionIntNormalizer.values()) { + enumNames.add(normalizer.name()); + } + Set configConstants = new HashSet<>(); + configConstants.add(ColumnPartitionConfig.PARTITION_ID_NORMALIZER_POSITIVE_MODULO); + configConstants.add(ColumnPartitionConfig.PARTITION_ID_NORMALIZER_ABS); + configConstants.add(ColumnPartitionConfig.PARTITION_ID_NORMALIZER_MASK); + assertEquals(configConstants, enumNames, + "ColumnPartitionConfig.PARTITION_ID_NORMALIZER_* string constants must match PartitionIntNormalizer enum " + + "values. If a new normalizer is added, update both ColumnPartitionConfig.isValidPartitionIdNormalizer " + + "and the constants list in this test."); } } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentPartitionConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentPartitionConfig.java index 8662e75d6e12..cb01103a3eb9 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentPartitionConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentPartitionConfig.java @@ -73,6 +73,8 @@ public String getPartitionIdNormalizer(String column) { return (config != null) ? config.getPartitionIdNormalizer() : null; } + // TODO: remove this deprecated getter after release 1.7.0 once internal callers have migrated to + // getColumnPartitionConfig(column).getFunctionConfig(). @Deprecated @Nullable public Map getFunctionConfig(String column) { From 5b133155c03354e56df5c2fef03cbc000779a2ae Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Sat, 25 Apr 2026 07:24:27 -0700 Subject: [PATCH 22/27] Restore legacy SQL hash function names as aliases MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Critical fix: - HashFunctions.java: when this branch added @ScalarFunction(names = {"murmur2", ...}) on existing public byte[]-overload hash functions (murmurHash2, fnv1Hash32, fnv1aHash32, fnv1Hash64, fnv1aHash64), the explicit names array overrode the auto-derived canonical name — silently breaking existing user SQL queries / ingestion transforms / views that called these functions on byte[] inputs by their original names. Add the legacy camelCase names alongside the new short aliases so both resolve to the same method: @ScalarFunction(names = {"murmur2", "murmurHash2"}) @ScalarFunction(names = {"fnv1_32", "fnv1Hash32"}) @ScalarFunction(names = {"fnv1a_32", "fnv1aHash32"}) @ScalarFunction(names = {"fnv1_64", "fnv1Hash64"}) @ScalarFunction(names = {"fnv1a_64", "fnv1aHash64"}) (murmur3Bit32Default is a new 1-arg method introduced by this branch with no upstream auto-derived name to preserve, so its single "murmur3_32" alias is fine.) - HashFunctionsTest: add testLegacyAndNewHashFunctionAliasesBothResolve asserting both the legacy and new alias resolve via FunctionRegistry to the same method, locking the contract so a future name reshuffle cannot silently regress backward compatibility. Co-Authored-By: Claude Sonnet 4.6 --- .../common/function/scalar/HashFunctions.java | 16 ++++++--- .../function/scalar/HashFunctionsTest.java | 33 +++++++++++++++++++ 2 files changed, 44 insertions(+), 5 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/function/scalar/HashFunctions.java b/pinot-common/src/main/java/org/apache/pinot/common/function/scalar/HashFunctions.java index 8832f6f63196..bd2ce872cd03 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/function/scalar/HashFunctions.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/function/scalar/HashFunctions.java @@ -122,7 +122,9 @@ public static byte[] md5Raw(byte[] input) { * @param input the byte array to hash * @return 32-bit hash */ - @ScalarFunction(names = {"murmur2"}) + // "murmurHash2" preserves the canonical name auto-derived on master before this PR introduced the "murmur2" alias. + // Existing user SQL queries / ingestion transforms calling murmurHash2(byte_col) must continue to resolve. + @ScalarFunction(names = {"murmur2", "murmurHash2"}) public static int murmurHash2(byte[] input) { return MurmurHashFunctions.murmurHash2(input); } @@ -240,7 +242,8 @@ public static byte[] murmurHash3X64Bit128(byte[] input, int seed) { * @param input the byte array to hash * @return 32-bit hash */ - @ScalarFunction(names = {"fnv1_32"}) + // "fnv1Hash32" preserves the canonical name auto-derived on master before this PR introduced the "fnv1_32" alias. + @ScalarFunction(names = {"fnv1_32", "fnv1Hash32"}) public static int fnv1Hash32(byte[] input) { return FnvHashFunctions.fnv1Hash32(input); } @@ -262,7 +265,8 @@ public static int fnv1Hash32UTF8(String input) { * @param input the byte array to hash * @return 32-bit hash */ - @ScalarFunction(names = {"fnv1a_32"}) + // "fnv1aHash32" preserves the canonical name auto-derived on master before this PR introduced the "fnv1a_32" alias. + @ScalarFunction(names = {"fnv1a_32", "fnv1aHash32"}) public static int fnv1aHash32(byte[] input) { return FnvHashFunctions.fnv1aHash32(input); } @@ -284,7 +288,8 @@ public static int fnv1aHash32UTF8(String input) { * @param input the byte array to hash * @return 64-bit hash */ - @ScalarFunction(names = {"fnv1_64"}) + // "fnv1Hash64" preserves the canonical name auto-derived on master before this PR introduced the "fnv1_64" alias. + @ScalarFunction(names = {"fnv1_64", "fnv1Hash64"}) public static long fnv1Hash64(byte[] input) { return FnvHashFunctions.fnv1Hash64(input); } @@ -306,7 +311,8 @@ public static long fnv1Hash64UTF8(String input) { * @param input the byte array to hash * @return 64-bit hash */ - @ScalarFunction(names = {"fnv1a_64"}) + // "fnv1aHash64" preserves the canonical name auto-derived on master before this PR introduced the "fnv1a_64" alias. + @ScalarFunction(names = {"fnv1a_64", "fnv1aHash64"}) public static long fnv1aHash64(byte[] input) { return FnvHashFunctions.fnv1aHash64(input); } diff --git a/pinot-common/src/test/java/org/apache/pinot/common/function/scalar/HashFunctionsTest.java b/pinot-common/src/test/java/org/apache/pinot/common/function/scalar/HashFunctionsTest.java index 5bed1fe83e91..c7873d7a8243 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/function/scalar/HashFunctionsTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/function/scalar/HashFunctionsTest.java @@ -19,10 +19,13 @@ package org.apache.pinot.common.function.scalar; import java.nio.charset.StandardCharsets; +import org.apache.pinot.common.function.FunctionInfo; +import org.apache.pinot.common.function.FunctionRegistry; import org.apache.pinot.spi.utils.hash.FnvHashFunctions; import org.testng.annotations.Test; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; public class HashFunctionsTest { @@ -225,4 +228,34 @@ public void testCityHash128() { assertEquals(HashFunctions.cityHash128(INPUT_LEN_156.getBytes()), new byte[] {114, 8, 23, 53, -124, 83, -28, 35, 27, -117, -59, 121, -108, -20, 59, 115}); } + + /** + * Backward-compat regression: this branch added explicit {@code @ScalarFunction(names = {"murmur2", ...})} + * aliases to byte[]-overload hash functions. The {@code names} array overrides the auto-derived canonical name + * from the method, so without explicitly listing the original (camelCase) name, existing user SQL queries calling + * {@code murmurHash2}/{@code fnv1Hash32}/{@code fnv1aHash32}/{@code fnv1Hash64}/{@code fnv1aHash64} on a byte[] + * argument would silently stop resolving the byte[] overload after upgrade. This test locks both the new short + * alias and the legacy auto-derived name so future name reshuffles cannot regress backward compatibility. + */ + @Test + public void testLegacyAndNewHashFunctionAliasesBothResolve() { + // Note: lookupFunctionInfo expects canonicalized names (lowercased, underscores stripped). Both the legacy + // camelCase name and the new alias should canonicalize to a registered key. + String[][] aliasPairs = new String[][] { + // {legacy auto-derived name, new alias} + {"murmurHash2", "murmur2"}, + {"fnv1Hash32", "fnv1_32"}, + {"fnv1aHash32", "fnv1a_32"}, + {"fnv1Hash64", "fnv1_64"}, + {"fnv1aHash64", "fnv1a_64"}, + }; + for (String[] pair : aliasPairs) { + FunctionInfo legacy = FunctionRegistry.lookupFunctionInfo(FunctionRegistry.canonicalize(pair[0]), 1); + FunctionInfo alias = FunctionRegistry.lookupFunctionInfo(FunctionRegistry.canonicalize(pair[1]), 1); + assertNotNull(legacy, "Legacy alias must still resolve: " + pair[0]); + assertNotNull(alias, "New alias must resolve: " + pair[1]); + assertEquals(legacy.getMethod(), alias.getMethod(), + "Legacy and new aliases must resolve to the same method: " + pair[0] + " vs " + pair[1]); + } + } } From 9386980c7f5259b5a88fab5f422a912289e910e7 Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Sat, 25 Apr 2026 08:24:38 -0700 Subject: [PATCH 23/27] Restore broker constructor compat shim; canonicalize normalizer; sunset Critical fix: - SegmentPartitionMetadataManager: restore the legacy (String tableNameWithType, String partitionColumn, String partitionFunctionName, int numPartitions) constructor that existed on upstream master. Removing it broke binary compatibility for external broker plugins / vendor forks that linked against this signature. The shim builds a synthetic ColumnPartitionConfig and delegates to the new constructor; expression-mode tables must use the ColumnPartitionConfig overloads. Major fixes: - ColumnPartitionConfig: canonicalize partitionIdNormalizer to uppercase at construction time so downstream comparisons (mix of case-sensitive wire format and case-insensitive older comparisons) cannot disagree on case alone. - PartitionFunctionFactory + ColumnPartitionMetadata: add explicit "TODO: remove after release 1.7.0" sunset markers to all four remaining @Deprecated entries on the partition surfaces, matching the existing sunset on SegmentPartitionConfig.getFunctionConfig. Skipped findings (with rationale): - Broker pruner string-form predicate (CRIT in prior reviews): plumbing typed values through requires substantial SPI refactor. - ByteArrayPartitionFunction normalizer mapping (claimed MASK by reviewer): on closer inspection abs(hashCode) % n is the ABS pattern (not MASK), so the existing return PartitionIntNormalizer .ABS.name() is correct. - Sentinel rename to "__functionExpr__": the literal "FunctionExpr" is now permanent on-the-wire and changing it would break test-built segments. The drift regression test already guards against future PartitionFunctionType enum collisions. Co-Authored-By: Claude Sonnet 4.6 --- .../SegmentPartitionMetadataManager.java | 13 +++++++++++++ .../spi/partition/PartitionFunctionFactory.java | 5 ++++- .../partition/metadata/ColumnPartitionMetadata.java | 1 + .../spi/config/table/ColumnPartitionConfig.java | 6 +++++- 4 files changed, 23 insertions(+), 2 deletions(-) diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManager.java b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManager.java index 2d14440f6d2b..55bb7bf8e619 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManager.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManager.java @@ -75,6 +75,19 @@ public class SegmentPartitionMetadataManager implements SegmentZkMetadataFetchLi private transient TablePartitionInfo _tablePartitionInfo; private transient TablePartitionReplicatedServersInfo _tablePartitionReplicatedServersInfo; + /** + * Backward-compat shim: the legacy (name-mode-only) constructor used by external broker plugins / vendor forks + * that linked against the pre-expression-mode signature. Builds a synthetic {@link ColumnPartitionConfig} and + * delegates to the new constructor. Expression-mode tables must use the {@link ColumnPartitionConfig} overloads. + * TODO: remove after release 1.7.0 once external callers have migrated. + */ + @Deprecated + public SegmentPartitionMetadataManager(String tableNameWithType, String partitionColumn, String partitionFunctionName, + int numPartitions) { + this(tableNameWithType, partitionColumn, + new ColumnPartitionConfig(partitionFunctionName, numPartitions), null); + } + public SegmentPartitionMetadataManager(String tableNameWithType, String partitionColumn, ColumnPartitionConfig columnPartitionConfig) { this(tableNameWithType, partitionColumn, columnPartitionConfig, null); diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java index 2b2c6fb0a7b6..b7742a590ccb 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java @@ -115,6 +115,7 @@ public static PartitionFunction getPartitionFunction(String functionName, int nu * @deprecated Expression-mode configs require a column name to compile the pipeline. This overload throws on * expression-mode configs; prefer {@link #getPartitionFunction(String, ColumnPartitionConfig)} or * {@link #getPartitionFunction(String, ColumnPartitionConfig, FieldSpec)} which support both modes. + * TODO: remove after release 1.7.0. * @throws IllegalArgumentException if {@code config.getFunctionExpr()} is non-null. */ @Deprecated @@ -130,6 +131,7 @@ public static PartitionFunction getPartitionFunction(ColumnPartitionConfig confi * * @deprecated Expression-mode metadata requires a column name to compile the pipeline. This overload throws on * expression-mode metadata; prefer {@link #getPartitionFunction(String, ColumnPartitionMetadata)}. + * TODO: remove after release 1.7.0. * @throws IllegalArgumentException if {@code metadata.getFunctionExpr()} is non-null. */ @Deprecated @@ -160,7 +162,8 @@ public static PartitionFunction getPartitionFunction(String columnName, ColumnPa * * @deprecated For BYTES-typed partition columns this overload always compiles the expression pipeline with STRING * input, producing partition ids that disagree with ingestion. Prefer - * {@link #getPartitionFunction(String, ColumnPartitionConfig, FieldSpec)}. + * {@link #getPartitionFunction(String, ColumnPartitionConfig, int, FieldSpec)} (the FieldSpec-aware 4-arg form). + * TODO: remove after release 1.7.0. */ @Deprecated public static PartitionFunction getPartitionFunction(String columnName, ColumnPartitionConfig columnPartitionConfig, diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java index 85ab2f87ab48..dec5e8c649c8 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java @@ -75,6 +75,7 @@ public class ColumnPartitionMetadata { * @param functionConfig Configuration required by partition function. * @deprecated Use {@link #ColumnPartitionMetadata(PartitionFunction, Set)} instead, which derives all fields * directly from the {@link PartitionFunction} contract and keeps them consistent. + * TODO: remove after release 1.7.0. */ @Deprecated public ColumnPartitionMetadata(String functionName, int numPartitions, Set partitions, diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ColumnPartitionConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ColumnPartitionConfig.java index 896cb97ee71f..b9e738a6f2b0 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ColumnPartitionConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ColumnPartitionConfig.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import java.util.Locale; import java.util.Map; import javax.annotation.Nullable; import org.apache.pinot.spi.config.BaseJsonConfig; @@ -88,7 +89,10 @@ public ColumnPartitionConfig(@JsonProperty("functionName") @Nullable String func "Unsupported partitionIdNormalizer: %s", partitionIdNormalizer); _functionName = functionName; _functionExpr = functionExpr; - _partitionIdNormalizer = partitionIdNormalizer; + // Canonicalize to uppercase at construction so downstream comparisons (case-sensitive on the wire format, + // case-insensitive in older code paths) cannot disagree on case alone. + _partitionIdNormalizer = hasText(partitionIdNormalizer) + ? partitionIdNormalizer.trim().toUpperCase(Locale.ROOT) : null; _numPartitions = numPartitions; _functionConfig = functionConfig; } From 6b1fb3255d0b7b60dccb0b842c84c31ed647832c Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Sat, 25 Apr 2026 08:49:59 -0700 Subject: [PATCH 24/27] Document reserved -1 sentinel contract on PartitionFunction.getPartition MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Major fix: - PartitionFunction.getPartition(String) Javadoc now explicitly states the return-value contract: implementations must return non-negative partition ids in [0, getNumPartitions()), and -1 is reserved as a framework-internal sentinel for "expression evaluated to null" (PartitionPipelineFunction.NULL_RESULT_PARTITION_ID). Custom plugin partition functions must not return -1 as a real partition id; internal callers (broker pruner, stats collector, segment processing partitioner) treat -1 as "skip / no partition". Skipped findings (with rationale): - Broker pruner fail-open metric: requires introducing a new BrokerMeter; defer to a follow-up that adds the meter alongside similar pruning failure observability. - End-to-end integration test: substantial test infrastructure work (CustomDataQueryClusterIntegrationTest subclass with controller + realtime ingestion + broker routing assertions); defer. - PR-scope split: process feedback that requires coordinated landing with maintainers; out of scope for code-only fixes. - Cross-version controller-side gate: substantial Helix integration; documented hazard in ColumnPartitionConfig Javadoc, defer. - ColumnBoundPartitionFunction.evaluate null handling: reviewer's claim that legacy partition functions produced a deterministic partition id for null is incorrect — legacy PartitionFunction.getPartition(String) would NPE on value.getBytes(UTF_8). The wrapper's null→null conversion is a graceful improvement. Co-Authored-By: Claude Sonnet 4.6 --- .../pinot/segment/spi/partition/PartitionFunction.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunction.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunction.java index 56c60add5a2b..e3404b799e90 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunction.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunction.java @@ -47,8 +47,15 @@ public interface PartitionFunction extends Serializable { * Method to compute and return partition id for the given value. * NOTE: The value is expected to be a string representation of the actual value. * + *

    Return-value contract: implementations must return a non-negative partition id in + * {@code [0, getNumPartitions())}. The value {@code -1} is reserved as a framework-internal sentinel for + * "expression evaluated to null" (see + * {@link org.apache.pinot.segment.spi.partition.pipeline.PartitionPipelineFunction#NULL_RESULT_PARTITION_ID}) — + * custom plugin implementations must not return {@code -1} as a real partition id. Internal callers + * (broker pruner, stats collector, segment processing partitioner) treat {@code -1} as "skip / no partition". + * * @param value Value for which to determine the partition id. - * @return partition id for the value. + * @return partition id for the value (non-negative for real partitions; never {@code -1}). */ int getPartition(String value); From 101cd924beb82872a4ff677a02dd9b6d314df17b Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Sat, 25 Apr 2026 09:22:11 -0700 Subject: [PATCH 25/27] Document identity-only contract of getPartitionIdNormalizer MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Minor clarification: the reviewer flagged that legacy partition functions (HashCodePartitionFunction / ByteArrayPartitionFunction) use Kafka-style abs handling Integer.MIN_VALUE → 0, while PartitionIntNormalizer.ABS does strict mod-then-abs. These differ at the MIN_VALUE edge case. The string returned from getPartitionIdNormalizer is purely advertised for staleness/identity matching between config-side and segment-side function metadata, not used to drive runtime normalization for legacy functions (which have their own internal partition-id calculation). Expression-mode pipelines are the only code path where this value is authoritative (via PartitionIntNormalizer.fromConfigString). Document the contract on the SPI Javadoc so plugin authors and future maintainers don't mistake the string for a runtime hook. Co-Authored-By: Claude Sonnet 4.6 --- .../SegmentPartitionMetadataManager.java | 58 ++++++------- .../SegmentPartitionMetadataManagerTest.java | 60 ++++++++++++-- .../evaluator/InbuiltFunctionEvaluator.java | 38 ++++----- .../InbuiltPartitionEvaluatorFactory.java | 20 ++--- .../evaluator/PartitionFunctionEvaluator.java | 63 ++++++-------- .../pinot/common/function/FunctionUtils.java | 16 ++-- .../common/function/scalar/HashFunctions.java | 28 +++---- .../function/scalar/HashFunctionsTest.java | 14 ++-- ...nFunctionExprCommonScalarFunctionTest.java | 38 ++++----- .../PartitionFunctionExprTestFunctions.java | 14 ++-- .../PartitionFunctionExprIntegrationTest.java | 6 +- .../partitioner/PartitionerFactory.java | 16 ++-- .../partitioner/TableConfigPartitioner.java | 8 +- .../pruner/ColumnValueSegmentPrunerTest.java | 13 ++- .../AbstractColumnStatisticsCollector.java | 6 +- .../apache/pinot/segment/spi/V1Constants.java | 30 +++---- .../spi/creator/StatsCollectorConfig.java | 25 +++--- .../function/ExecutableFunctionEvaluator.java | 72 ++++++---------- .../spi/partition/PartitionFunction.java | 83 ++++++++++--------- .../partition/PartitionFunctionFactory.java | 82 ++++++++---------- .../metadata/ColumnPartitionMetadata.java | 38 ++++----- .../pipeline/PartitionEvaluatorFactory.java | 37 ++++----- .../PartitionFunctionExprCompiler.java | 34 +++----- .../pipeline/PartitionIntNormalizer.java | 16 +--- .../partition/pipeline/PartitionPipeline.java | 14 ++-- .../pipeline/PartitionPipelineFunction.java | 34 +++----- .../pipeline/PartitionValueType.java | 4 +- .../spi/partition/PartitionFunctionTest.java | 22 ++--- .../config/table/ColumnPartitionConfig.java | 37 ++++----- .../config/table/SegmentPartitionConfig.java | 12 --- .../pinot/spi/utils/FunctionNameUtils.java | 20 ++--- .../pinot/spi/utils/ScalarFunctionUtils.java | 12 +-- .../spi/config/table/IndexingConfigTest.java | 13 ++- 33 files changed, 437 insertions(+), 546 deletions(-) diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManager.java b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManager.java index 55bb7bf8e619..a77db193dd80 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManager.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManager.java @@ -75,12 +75,10 @@ public class SegmentPartitionMetadataManager implements SegmentZkMetadataFetchLi private transient TablePartitionInfo _tablePartitionInfo; private transient TablePartitionReplicatedServersInfo _tablePartitionReplicatedServersInfo; - /** - * Backward-compat shim: the legacy (name-mode-only) constructor used by external broker plugins / vendor forks - * that linked against the pre-expression-mode signature. Builds a synthetic {@link ColumnPartitionConfig} and - * delegates to the new constructor. Expression-mode tables must use the {@link ColumnPartitionConfig} overloads. - * TODO: remove after release 1.7.0 once external callers have migrated. - */ + /// Backward-compat shim: the legacy (name-mode-only) constructor used by external broker plugins / vendor forks + /// that linked against the pre-expression-mode signature. Builds a synthetic [ColumnPartitionConfig] and + /// delegates to the new constructor. Expression-mode tables must use the [ColumnPartitionConfig] overloads. + /// TODO: remove after release 1.7.0 once external callers have migrated. @Deprecated public SegmentPartitionMetadataManager(String tableNameWithType, String partitionColumn, String partitionFunctionName, int numPartitions) { @@ -93,11 +91,9 @@ public SegmentPartitionMetadataManager(String tableNameWithType, String partitio this(tableNameWithType, partitionColumn, columnPartitionConfig, null); } - /** - * Preferred constructor: pass the partition column's {@link FieldSpec} so expression-mode pipelines on BYTES - * columns are compiled with BYTES input. The {@code FieldSpec}-less overload above always compiles with STRING - * input, which silently disagrees with ingestion partition ids on BYTES columns. - */ + /// Preferred constructor: pass the partition column's [FieldSpec] so expression-mode pipelines on BYTES + /// columns are compiled with BYTES input. The `FieldSpec`-less overload above always compiles with STRING + /// input, which silently disagrees with ingestion partition ids on BYTES columns. public SegmentPartitionMetadataManager(String tableNameWithType, String partitionColumn, ColumnPartitionConfig columnPartitionConfig, @Nullable FieldSpec fieldSpec) { _tableNameWithType = tableNameWithType; @@ -151,27 +147,33 @@ private String getPartitionFunctionDescription() { } private boolean isMatchingPartitionFunction(PartitionFunction partitionFunction) { - return _partitionFunction.getName().equalsIgnoreCase(partitionFunction.getName()) - && functionConfigsMatch(_partitionFunction.getFunctionConfig(), partitionFunction.getFunctionConfig()) - && Objects.equals(_partitionFunction.getFunctionExpr(), partitionFunction.getFunctionExpr()) - && equalsIgnoreCaseNullable(_partitionFunction.getPartitionIdNormalizer(), - partitionFunction.getPartitionIdNormalizer()); - } + Map functionConfig = partitionFunction.getFunctionConfig(); + if (functionConfig != null && !functionConfig.isEmpty() + && !Objects.equals(_partitionFunction.getFunctionConfig(), functionConfig)) { + return false; + } - private static boolean functionConfigsMatch(@Nullable Map a, @Nullable Map b) { - boolean aEmpty = a == null || a.isEmpty(); - boolean bEmpty = b == null || b.isEmpty(); - if (aEmpty && bEmpty) { - return true; + String partitionIdNormalizer = partitionFunction.getPartitionIdNormalizer(); + if (partitionIdNormalizer != null && !partitionIdNormalizer.isEmpty() + && !equalsIgnoreCase(_partitionFunction.getPartitionIdNormalizer(), partitionIdNormalizer)) { + return false; } - return Objects.equals(a, b); - } - private static boolean equalsIgnoreCaseNullable(@Nullable String a, @Nullable String b) { - if (a == null) { - return b == null; + String functionName = partitionFunction.getName(); + if (functionName != null) { + String configuredFunctionName = _partitionFunction.getName(); + return configuredFunctionName != null && configuredFunctionName.equalsIgnoreCase(functionName); } - return a.equalsIgnoreCase(b); + + String functionExpr = partitionFunction.getFunctionExpr(); + if (functionExpr != null) { + return Objects.equals(_partitionFunction.getFunctionExpr(), functionExpr); + } + return false; + } + + private static boolean equalsIgnoreCase(@Nullable String a, String b) { + return a != null && a.equalsIgnoreCase(b); } private static long getCreationTimeMs(@Nullable ZNRecord znRecord) { diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManagerTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManagerTest.java index 7f6ea0d2608e..8c6b634117e6 100644 --- a/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManagerTest.java +++ b/pinot-broker/src/test/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManagerTest.java @@ -354,12 +354,47 @@ public void testPartitionMetadataManagerProcessingWithFunctionExprAbsNormalizer( assertTrue(tablePartitionReplicatedServersInfo.getSegmentsWithInvalidPartition().isEmpty()); } - /** - * Regression for commit 28546f1523. The broker compiles its config-side functionExpr through the canonicalize() - * pipeline (lowercased, spaces stripped) but segments may have been written with mixed-case expressions by an - * older node or a direct ZK write. The match check must be case-insensitive so such segments are still routed - * via partition-aware pruning instead of silently degrading to scatter-gather. - */ + @Test + public void testPartitionMetadataManagerProcessingWithFunctionConfig() { + ExternalView externalView = new ExternalView(OFFLINE_TABLE_NAME); + Map> segmentAssignment = externalView.getRecord().getMapFields(); + Set onlineSegments = new HashSet<>(); + IdealState idealState = new IdealState(OFFLINE_TABLE_NAME); + String partitionFunction = "Murmur3"; + Map functionConfig = Map.of("seed", "42"); + String segment = "functionConfigSegment"; + + SegmentPartitionMetadataManager partitionMetadataManager = + new SegmentPartitionMetadataManager(OFFLINE_TABLE_NAME, PARTITION_COLUMN, + new ColumnPartitionConfig(partitionFunction, 8, functionConfig)); + SegmentZkMetadataFetcher segmentZkMetadataFetcher = + new SegmentZkMetadataFetcher(OFFLINE_TABLE_NAME, _propertyStore); + segmentZkMetadataFetcher.register(partitionMetadataManager); + segmentZkMetadataFetcher.init(idealState, externalView, onlineSegments); + + onlineSegments.add(segment); + segmentAssignment.put(segment, Collections.singletonMap(SERVER_0, ONLINE)); + setSegmentZKMetadata(segment, partitionFunction, 8, 1, functionConfig, null, null, 0L); + segmentZkMetadataFetcher.onAssignmentChange(idealState, externalView, onlineSegments); + + TablePartitionReplicatedServersInfo tablePartitionReplicatedServersInfo = + partitionMetadataManager.getTablePartitionReplicatedServersInfo(); + assertEquals(tablePartitionReplicatedServersInfo.getPartitionInfoMap()[1]._segments, + Collections.singleton(segment)); + assertTrue(tablePartitionReplicatedServersInfo.getSegmentsWithInvalidPartition().isEmpty()); + + setSegmentZKMetadata(segment, partitionFunction, 8, 1, Map.of("seed", "13"), null, null, 0L); + segmentZkMetadataFetcher.refreshSegment(segment); + + tablePartitionReplicatedServersInfo = partitionMetadataManager.getTablePartitionReplicatedServersInfo(); + assertEquals(tablePartitionReplicatedServersInfo.getSegmentsWithInvalidPartition(), + Collections.singletonList(segment)); + } + + /// Regression for commit 28546f1523. Expression-mode segments persist the stable `FunctionExpr` sentinel as their + /// partition function name, and older nodes or direct ZK writes may preserve mixed-case raw expressions. Once the + /// segment-side normalizer/config guards match, the sentinel name match should keep such segments eligible for + /// partition-aware pruning instead of silently degrading to scatter-gather. @Test public void testPartitionMetadataManagerProcessingWithMixedCaseFunctionExpr() { ExternalView externalView = new ExternalView(OFFLINE_TABLE_NAME); @@ -387,7 +422,7 @@ public void testPartitionMetadataManagerProcessingWithMixedCaseFunctionExpr() { partitionMetadataManager.getTablePartitionReplicatedServersInfo(); assertEquals(tablePartitionReplicatedServersInfo.getPartitionInfoMap()[1]._segments, Collections.singleton(segment), - "Segment with mixed-case functionExpr must still match the canonicalized config expr"); + "Segment with mixed-case functionExpr must still match through the FunctionExpr sentinel"); assertTrue(tablePartitionReplicatedServersInfo.getSegmentsWithInvalidPartition().isEmpty(), "Segment must not be flagged as invalid solely because its expr was written in mixed case"); } @@ -404,10 +439,17 @@ private void setSegmentZKMetadata(String segment, String partitionFunction, int private void setSegmentZKMetadata(String segment, String partitionFunction, int numPartitions, int partitionId, @Nullable String functionExpr, @Nullable String partitionIdNormalizer, long creationTimeMs) { + setSegmentZKMetadata(segment, partitionFunction, numPartitions, partitionId, null, functionExpr, + partitionIdNormalizer, creationTimeMs); + } + + private void setSegmentZKMetadata(String segment, String partitionFunction, int numPartitions, int partitionId, + @Nullable Map functionConfig, @Nullable String functionExpr, + @Nullable String partitionIdNormalizer, long creationTimeMs) { SegmentZKMetadata segmentZKMetadata = new SegmentZKMetadata(segment); PartitionFunction effectivePartitionFunction = - PartitionFunctionFactory.getPartitionFunction(PARTITION_COLUMN, partitionFunction, numPartitions, null, - functionExpr, partitionIdNormalizer); + PartitionFunctionFactory.getPartitionFunction(PARTITION_COLUMN, partitionFunction, numPartitions, + functionConfig, functionExpr, partitionIdNormalizer); segmentZKMetadata.setPartitionMetadata(new SegmentPartitionMetadata(Collections.singletonMap(PARTITION_COLUMN, new ColumnPartitionMetadata(effectivePartitionFunction, Collections.singleton(partitionId))))); segmentZKMetadata.setCreationTime(creationTimeMs); diff --git a/pinot-common/src/main/java/org/apache/pinot/common/evaluator/InbuiltFunctionEvaluator.java b/pinot-common/src/main/java/org/apache/pinot/common/evaluator/InbuiltFunctionEvaluator.java index 827cf5502318..e3260ddc839b 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/evaluator/InbuiltFunctionEvaluator.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/evaluator/InbuiltFunctionEvaluator.java @@ -32,29 +32,25 @@ import org.apache.pinot.spi.data.readers.GenericRow; -/** - * Evaluates an expression backed by the Pinot {@link FunctionRegistry}. - * - *

    This is optimized for evaluating an expression multiple times with different inputs. - * The expression is compiled once into an {@link ExecutableFunctionEvaluator.ExecutableNode} tree - * whose nodes handle constants, column reads, logical operators, and function calls. - * - *

    Thread-safety: Instances are safe for both concurrent invocation by multiple threads and - * re-entrant invocation on a single thread. Each {@link FunctionExecutionNode} allocates a fresh - * argument scratch array per {@code execute} call, so cross-thread sharing and recursive invocation - * through nested user-defined scalar functions cannot corrupt argument state. - */ +/// Evaluates an expression backed by the Pinot [FunctionRegistry]. +/// +/// This is optimized for evaluating an expression multiple times with different inputs. +/// The expression is compiled once into an [ExecutableFunctionEvaluator.ExecutableNode] tree +/// whose nodes handle constants, column reads, logical operators, and function calls. +/// +/// **Thread-safety:** Instances are safe for both concurrent invocation by multiple threads and +/// re-entrant invocation on a single thread. Each [FunctionExecutionNode] allocates a fresh +/// argument scratch array per `execute` call, so cross-thread sharing and recursive invocation +/// through nested user-defined scalar functions cannot corrupt argument state. public class InbuiltFunctionEvaluator extends ExecutableFunctionEvaluator { public InbuiltFunctionEvaluator(String functionExpression) { this(functionExpression, new ArrayList<>()); } - /** - * Two-phase constructor: {@code planExecution} is evaluated first (left-to-right argument - * evaluation in Java), populating {@code arguments} as a side effect, so the completed list is - * ready when passed to {@code super()}. - */ + /// Two-phase constructor: `planExecution` is evaluated first (left-to-right argument + /// evaluation in Java), populating `arguments` as a side effect, so the completed list is + /// ready when passed to `super()`. private InbuiltFunctionEvaluator(String functionExpression, List arguments) { super(planExecution(RequestContextUtils.getExpression(functionExpression), arguments), arguments, functionExpression); @@ -111,11 +107,9 @@ private static ExecutableNode planExecution(ExpressionContext expression, ListDelegates all scalar-function resolution and evaluation to Pinot's {@link FunctionRegistry}, ensuring - * that partition expressions use the same function semantics as ingestion transforms. String-to-bytes conversions - * use UTF-8 encoding so that hash functions ({@code md5}, {@code murmur2}, {@code fnv1a_32}, etc.) operate on the - * raw string content rather than a hex-decoded representation. - */ +/// [PartitionEvaluatorFactory] backed by [PartitionFunctionEvaluator]. +/// +/// Delegates all scalar-function resolution and evaluation to Pinot's [FunctionRegistry], ensuring +/// that partition expressions use the same function semantics as ingestion transforms. String-to-bytes conversions +/// use UTF-8 encoding so that hash functions (`md5`, `murmur2`, `fnv1a_32`, etc.) operate on the +/// raw string content rather than a hex-decoded representation. public class InbuiltPartitionEvaluatorFactory implements PartitionEvaluatorFactory { @Override @@ -74,10 +72,8 @@ private static void validateDeterministic(ExpressionContext expr) { } } - /** - * Partition expressions must stay stable for ingestion and query pruning. Functions that read query-thread context - * or intentionally block are not safe even if the broader SQL engine exposes them as regular scalar functions. - */ + /// Partition expressions must stay stable for ingestion and query pruning. Functions that read query-thread context + /// or intentionally block are not safe even if the broader SQL engine exposes them as regular scalar functions. private static boolean isAllowedForPartitioning(Method method) { Class declaringClass = method.getDeclaringClass(); if (declaringClass == InternalFunctions.class) { diff --git a/pinot-common/src/main/java/org/apache/pinot/common/evaluator/PartitionFunctionEvaluator.java b/pinot-common/src/main/java/org/apache/pinot/common/evaluator/PartitionFunctionEvaluator.java index dbaaa990c6a7..bac3fe1efadd 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/evaluator/PartitionFunctionEvaluator.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/evaluator/PartitionFunctionEvaluator.java @@ -36,35 +36,30 @@ import static java.nio.charset.StandardCharsets.UTF_8; -/** - * A {@link org.apache.pinot.spi.function.FunctionEvaluator} for partition expressions backed by the Pinot - * {@link FunctionRegistry}. - * - *

    This evaluator is structurally equivalent to {@link InbuiltFunctionEvaluator} with one critical difference: - * when a {@code String} value is passed to a function that expects {@code byte[]}, the conversion uses - * UTF-8 encoding rather than hex-decoding. This matches ingestion semantics where user-facing string column - * values are treated as text, not as hex-encoded binary data. - * - *

    Example: {@code murmur2(lower(raw_key))} with input {@code "HeLLo"} evaluates as follows: - *

      - *
    1. {@code lower("HeLLo")} → {@code "hello"} (String)
    2. - *
    3. {@code murmur2("hello")} — converts {@code "hello"} to UTF-8 bytes, then hashes
    4. - *
    - * - *

    Thread-safety: Instances are safe for both concurrent invocation by multiple threads and - * re-entrant invocation on a single thread. Each {@link PartitionFunctionExecutionNode} allocates a fresh - * argument scratch array per {@code execute} call. - */ +/// A [org.apache.pinot.spi.function.FunctionEvaluator] for partition expressions backed by the Pinot +/// [FunctionRegistry]. +/// +/// This evaluator is structurally equivalent to [InbuiltFunctionEvaluator] with one critical difference: +/// when a `String` value is passed to a function that expects `byte[]`, the conversion uses +/// *UTF-8 encoding* rather than hex-decoding. This matches ingestion semantics where user-facing string column +/// values are treated as text, not as hex-encoded binary data. +/// +/// Example: `murmur2(lower(raw_key))` with input `"HeLLo"` evaluates as follows: +/// +/// - `lower("HeLLo")` -> `"hello"` (String) +/// - `murmur2("hello")` converts `"hello"` to UTF-8 bytes, then hashes +/// +/// **Thread-safety:** Instances are safe for both concurrent invocation by multiple threads and +/// re-entrant invocation on a single thread. Each [PartitionFunctionExecutionNode] allocates a fresh +/// argument scratch array per `execute` call. public final class PartitionFunctionEvaluator extends ExecutableFunctionEvaluator { public PartitionFunctionEvaluator(String functionExpression) { this(functionExpression, new ArrayList<>()); } - /** - * Two-phase constructor: {@code planExecution} is evaluated first (left-to-right argument evaluation in Java), - * populating {@code arguments} as a side effect, so the completed list is ready when passed to {@code super()}. - */ + /// Two-phase constructor: `planExecution` is evaluated first (left-to-right argument evaluation in Java), + /// populating `arguments` as a side effect, so the completed list is ready when passed to `super()`. private PartitionFunctionEvaluator(String functionExpression, List arguments) { super(planExecution(RequestContextUtils.getExpression(functionExpression), arguments), arguments, functionExpression); @@ -121,15 +116,13 @@ private static ExecutableNode planExecution(ExpressionContext expression, ListAllocates a fresh argument scratch array per {@code execute} call so that concurrent invocations from - * multiple threads, and re-entrant invocations on a single thread, do not share mutable state. - */ + /// Executes a Pinot-registry function via [FunctionInvoker], with null propagation and type conversion. + /// Unlike [InbuiltFunctionEvaluator.FunctionExecutionNode], this node converts `String` to + /// `byte[]` using UTF-8 encoding instead of hex-decoding, matching ingestion semantics for partition + /// expressions. + /// + /// Allocates a fresh argument scratch array per `execute` call so that concurrent invocations from + /// multiple threads, and re-entrant invocations on a single thread, do not share mutable state. private static class PartitionFunctionExecutionNode implements ExecutableNode { private final FunctionInvoker _functionInvoker; private final FunctionInfo _functionInfo; @@ -184,10 +177,8 @@ private Object invokeWithPartitionConversion(Object[] arguments) { return _functionInvoker.invoke(arguments); } - /** - * Converts argument types to match the parameter classes, using UTF-8 encoding for {@code String}→{@code byte[]} - * conversions (rather than hex-decoding used by the standard {@link FunctionInvoker#convertTypes}). - */ + /// Converts argument types to match the parameter classes, using UTF-8 encoding for `String`->`byte[]` + /// conversions (rather than hex-decoding used by the standard [FunctionInvoker#convertTypes]). private void convertTypesWithUtf8StringToBytes(Object[] arguments) { Class[] parameterClasses = _functionInvoker.getParameterClasses(); PinotDataType[] parameterTypes = _functionInvoker.getParameterTypes(); diff --git a/pinot-common/src/main/java/org/apache/pinot/common/function/FunctionUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/function/FunctionUtils.java index 6e43041fd6e2..cff14fc8a549 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/function/FunctionUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/function/FunctionUtils.java @@ -106,12 +106,12 @@ public static PinotDataType getParameterType(Class clazz) { /// (e.g. vendor `Timestamp` subclasses returned by JDBC drivers) are matched by their parent type. /// /// Dispatch (single-value first since it's the dominant case for function arguments): - /// - Single values → delegated to [PinotDataType#getSingleValueType] (covers all scalar types - /// including `byte[]` → [PinotDataType#BYTES]). - /// - Reference arrays (`Object[]` and subtypes including `byte[][]`) → first non-null element is + /// - Single values -> delegated to [PinotDataType#getSingleValueType] (covers all scalar types + /// including `byte[]` -> [PinotDataType#BYTES]). + /// - Reference arrays (`Object[]` and subtypes including `byte[][]`) -> first non-null element is /// sampled and [PinotDataType#getMultiValueType] is consulted. Empty / all-null reference arrays /// fall back to [PinotDataType#OBJECT_ARRAY] since the element type is undeterminable. - /// - Primitive arrays (`int[]` / `long[]` / `float[]` / `double[]` / `boolean[]`) → handled here, since + /// - Primitive arrays (`int[]` / `long[]` / `float[]` / `double[]` / `boolean[]`) -> handled here, since /// they can't be element-sampled into a boxed type. /// - [PinotDataType#COLLECTION] for any [Collection]; otherwise falls back to [PinotDataType#OBJECT]. public static PinotDataType getArgumentType(Object value) { @@ -218,11 +218,9 @@ public static boolean isAssertEnabled() { return assertEnabled; } - /** - * Returns the canonical form of a function name: underscores stripped, lower-cased. - * - * @see FunctionNameUtils#canonicalize(String) - */ + /// Returns the canonical form of a function name: underscores stripped, lower-cased. + /// + /// @see FunctionNameUtils#canonicalize(String) public static String canonicalize(String name) { return FunctionNameUtils.canonicalize(name); } diff --git a/pinot-common/src/main/java/org/apache/pinot/common/function/scalar/HashFunctions.java b/pinot-common/src/main/java/org/apache/pinot/common/function/scalar/HashFunctions.java index bd2ce872cd03..10cc599828e5 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/function/scalar/HashFunctions.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/function/scalar/HashFunctions.java @@ -108,9 +108,7 @@ public static String md5(byte[] input) { return DigestUtils.md5Hex(input); } - /** - * Return raw MD5 digest bytes. - */ + /// Return raw MD5 digest bytes. @ScalarFunction(names = {"md5_raw"}) public static byte[] md5Raw(byte[] input) { return DigestUtils.md5(input); @@ -129,12 +127,10 @@ public static int murmurHash2(byte[] input) { return MurmurHashFunctions.murmurHash2(input); } - /** - * Computes 32-bit MurmurHash2 of the given string. - * - * @param input the string to hash (converted to UTF-8 bytes) - * @return 32-bit hash - */ + /// Computes 32-bit MurmurHash2 of the given string. + /// + /// @param input the string to hash (converted to UTF-8 bytes) + /// @return 32-bit hash @ScalarFunction public static int murmurHash2UTF8(String input) { return MurmurHashFunctions.murmurHash2(input.getBytes(StandardCharsets.UTF_8)); @@ -173,9 +169,7 @@ public static int murmurHash3Bit32(byte[] input, int seed) { return Hashing.murmur3_32_fixed(seed).hashBytes(input).asInt(); } - /** - * Computes 32-bit Murmur3 hash of the given byte array with seed 0 using the partition-expression friendly name. - */ + /// Computes 32-bit Murmur3 hash of the given byte array with seed 0 using the partition-expression friendly name. @ScalarFunction(names = {"murmur3_32"}) public static int murmur3Bit32Default(byte[] input) { return murmurHash3Bit32(input, 0); @@ -248,12 +242,10 @@ public static int fnv1Hash32(byte[] input) { return FnvHashFunctions.fnv1Hash32(input); } - /** - * Computes 32-bit FNV-1 hash of the given string using UTF-8 bytes. - * - * @param input the string to hash (converted to UTF-8 bytes) - * @return 32-bit hash - */ + /// Computes 32-bit FNV-1 hash of the given string using UTF-8 bytes. + /// + /// @param input the string to hash (converted to UTF-8 bytes) + /// @return 32-bit hash @ScalarFunction public static int fnv1Hash32UTF8(String input) { return FnvHashFunctions.fnv1Hash32(input.getBytes(StandardCharsets.UTF_8)); diff --git a/pinot-common/src/test/java/org/apache/pinot/common/function/scalar/HashFunctionsTest.java b/pinot-common/src/test/java/org/apache/pinot/common/function/scalar/HashFunctionsTest.java index c7873d7a8243..07e77a890e8f 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/function/scalar/HashFunctionsTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/function/scalar/HashFunctionsTest.java @@ -229,14 +229,12 @@ public void testCityHash128() { new byte[] {114, 8, 23, 53, -124, 83, -28, 35, 27, -117, -59, 121, -108, -20, 59, 115}); } - /** - * Backward-compat regression: this branch added explicit {@code @ScalarFunction(names = {"murmur2", ...})} - * aliases to byte[]-overload hash functions. The {@code names} array overrides the auto-derived canonical name - * from the method, so without explicitly listing the original (camelCase) name, existing user SQL queries calling - * {@code murmurHash2}/{@code fnv1Hash32}/{@code fnv1aHash32}/{@code fnv1Hash64}/{@code fnv1aHash64} on a byte[] - * argument would silently stop resolving the byte[] overload after upgrade. This test locks both the new short - * alias and the legacy auto-derived name so future name reshuffles cannot regress backward compatibility. - */ + /// Backward-compat regression: this branch added explicit `@ScalarFunction(names = {"murmur2", ...})` + /// aliases to byte[]-overload hash functions. The `names` array overrides the auto-derived canonical name + /// from the method, so without explicitly listing the original (camelCase) name, existing user SQL queries calling + /// `murmurHash2`/`fnv1Hash32`/`fnv1aHash32`/`fnv1Hash64`/`fnv1aHash64` on a byte[] + /// argument would silently stop resolving the byte[] overload after upgrade. This test locks both the new short + /// alias and the legacy auto-derived name so future name reshuffles cannot regress backward compatibility. @Test public void testLegacyAndNewHashFunctionAliasesBothResolve() { // Note: lookupFunctionInfo expects canonicalized names (lowercased, underscores stripped). Both the legacy diff --git a/pinot-common/src/test/java/org/apache/pinot/common/function/scalar/PartitionFunctionExprCommonScalarFunctionTest.java b/pinot-common/src/test/java/org/apache/pinot/common/function/scalar/PartitionFunctionExprCommonScalarFunctionTest.java index 62c5d41bf7da..8f722539883e 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/function/scalar/PartitionFunctionExprCommonScalarFunctionTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/function/scalar/PartitionFunctionExprCommonScalarFunctionTest.java @@ -63,10 +63,8 @@ public void testRejectsNonDeterministicAgoFunction() { "Partition scalar function 'ago' is not allowed because it is non-deterministic"); } - /** - * Regression for commit 9bcd3ee63f. Pinot scalar functions box integral arithmetic to Double; the partition - * function must accept integral-valued Doubles (e.g. plus(54L, 7L) → 61.0) without rejecting them outright. - */ + /// Regression for commit 9bcd3ee63f. Pinot scalar functions box integral arithmetic to Double; the partition + /// function must accept integral-valued Doubles (e.g. plus(54L, 7L) -> 61.0) without rejecting them outright. @Test public void testIntegralValuedDoubleAcceptedAsPartitionResult() { PartitionPipelineFunction partitionFunction = @@ -75,11 +73,9 @@ public void testIntegralValuedDoubleAcceptedAsPartitionResult() { assertEquals(partitionFunction.getPartition("54321"), 61); } - /** - * Regression: Float/Double whose absolute value reaches 2^53 must be rejected even if the value is "integral", - * because mantissa precision loss causes silent partition-id collapse. The bound is strict: 2^53 itself is - * representable but 2^53+1 collapses onto 2^53, so admitting 2^53 would let 2^53+1 silently collide with it. - */ + /// Regression: Float/Double whose absolute value reaches 2^53 must be rejected even if the value is "integral", + /// because mantissa precision loss causes silent partition-id collapse. The bound is strict: 2^53 itself is + /// representable but 2^53+1 collapses onto 2^53, so admitting 2^53 would let 2^53+1 silently collide with it. @Test public void testDoubleBeyondMantissaPrecisionIsRejected() { PartitionPipelineFunction partitionFunction = @@ -91,11 +87,9 @@ public void testDoubleBeyondMantissaPrecisionIsRejected() { "Expected precision-bound error, got: " + error.getMessage()); } - /** - * Boundary regression for the off-by-one in the precision check: 2^53+1 (long) silently rounds to 2^53.0 - * (Double); admitting 2^53 would let 2^53+1 collide with 2^53 onto the same partition id. Strict {@code < 2^53} - * is required. - */ + /// Boundary regression for the off-by-one in the precision check: 2^53+1 (long) silently rounds to 2^53.0 + /// (Double); admitting 2^53 would let 2^53+1 collide with 2^53 onto the same partition id. Strict `< 2^53` + /// is required. @Test public void testDoubleAtMantissaBoundaryIsRejected() { PartitionPipelineFunction partitionFunction = @@ -107,11 +101,9 @@ public void testDoubleAtMantissaBoundaryIsRejected() { "Expected precision-bound error at 2^53 boundary, got: " + error.getMessage()); } - /** - * Regression for commit 28546f1523. The expression chain may yield null mid-evaluation (e.g. via tryCast or null - * column input). Surface this as null on the FunctionEvaluator interface so ingestion treats it as "no partition" - * rather than literal partition id -1. - */ + /// Regression for commit 28546f1523. The expression chain may yield null mid-evaluation (e.g. via tryCast or null + /// column input). Surface this as null on the FunctionEvaluator interface so ingestion treats it as "no partition" + /// rather than literal partition id -1. @Test public void testNullMidChainReturnsNullOnFunctionEvaluatorSurface() { PartitionPipelineFunction partitionFunction = @@ -124,11 +116,9 @@ public void testNullMidChainReturnsNullOnFunctionEvaluatorSurface() { "evaluate(Object[]) must return null when the input is null"); } - /** - * Regression for commit 9c031aa76e. Expressions whose final type is non-numeric (e.g. md5() returns STRING) must - * be rejected at validation time with a clear "must produce INT or LONG" error rather than passing config - * validation and surfacing the failure later at segment creation. - */ + /// Regression for commit 9c031aa76e. Expressions whose final type is non-numeric (e.g. md5() returns STRING) must + /// be rejected at validation time with a clear "must produce INT or LONG" error rather than passing config + /// validation and surfacing the failure later at segment creation. @Test public void testValidateOutputTypeRejectsStringExpressionResult() { PartitionPipelineFunction partitionFunction = diff --git a/pinot-common/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprTestFunctions.java b/pinot-common/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprTestFunctions.java index 27798818d9f1..90568c58c320 100644 --- a/pinot-common/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprTestFunctions.java +++ b/pinot-common/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprTestFunctions.java @@ -23,14 +23,12 @@ import org.apache.pinot.spi.annotations.ScalarFunction; -/** - * Test-only scalar functions used to exercise the partition-expression compiler in {@code pinot-common} tests. - * - *

    Only functions that are NOT already registered in Pinot's production - * {@link org.apache.pinot.common.function.FunctionRegistry} should be added here. Hash functions - * ({@code md5}, {@code murmur2}, {@code fnv1a_32}, etc.) are provided by {@code HashFunctions} and must not be - * re-registered. - */ +/// Test-only scalar functions used to exercise the partition-expression compiler in `pinot-common` tests. +/// +/// Only functions that are NOT already registered in Pinot's production +/// [org.apache.pinot.common.function.FunctionRegistry] should be added here. Hash functions +/// (`md5`, `murmur2`, `fnv1a_32`, etc.) are provided by `HashFunctions` and must not be +/// re-registered. public final class PartitionFunctionExprTestFunctions { private PartitionFunctionExprTestFunctions() { } diff --git a/pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionExprIntegrationTest.java b/pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionExprIntegrationTest.java index 71fc01f8280a..fb5e38506f76 100644 --- a/pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionExprIntegrationTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionExprIntegrationTest.java @@ -30,10 +30,8 @@ import static org.testng.Assert.assertTrue; -/** - * Integration tests for expression-mode partition functions that require {@code pinot-common} on the classpath - * (for {@link org.apache.pinot.common.evaluator.InbuiltPartitionEvaluatorFactory}). - */ +/// Integration tests for expression-mode partition functions that require `pinot-common` on the classpath +/// (for [org.apache.pinot.common.evaluator.InbuiltPartitionEvaluatorFactory]). public class PartitionFunctionExprIntegrationTest { @Test diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/partitioner/PartitionerFactory.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/partitioner/PartitionerFactory.java index 9cb72dbfe526..3e45f67b9ed6 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/partitioner/PartitionerFactory.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/partitioner/PartitionerFactory.java @@ -58,10 +58,8 @@ public static Partitioner getPartitioner(PartitionerConfig config) { return getPartitioner(config, null); } - /** - * Construct a Partitioner using the PartitioningConfig. The optional schema is used to determine the correct - * input type for expression-mode partition functions on BYTES-typed columns. - */ + /// Construct a Partitioner using the PartitioningConfig. The optional schema is used to determine the correct + /// input type for expression-mode partition functions on BYTES-typed columns. public static Partitioner getPartitioner(PartitionerConfig config, @Nullable Schema schema) { Partitioner partitioner = null; switch (config.getPartitionerType()) { @@ -105,12 +103,10 @@ public static Partitioner[] getPartitioners(List partitionerC return getPartitioners(partitionerConfigs, null); } - /** - * Create partitioner array from configuration. The optional schema is used to determine the correct - * input type for expression-mode partition functions on BYTES-typed columns. - * - * @return Array of partitioners - */ + /// Create partitioner array from configuration. The optional schema is used to determine the correct + /// input type for expression-mode partition functions on BYTES-typed columns. + /// + /// @return Array of partitioners public static Partitioner[] getPartitioners(List partitionerConfigs, @Nullable Schema schema) { int numPartitioners = partitionerConfigs.size(); Partitioner[] partitioners = new Partitioner[numPartitioners]; diff --git a/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/partitioner/TableConfigPartitioner.java b/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/partitioner/TableConfigPartitioner.java index ff48ce7f1bc3..e5d9f49363b4 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/partitioner/TableConfigPartitioner.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/segment/processing/partitioner/TableConfigPartitioner.java @@ -32,15 +32,13 @@ * Partitioner which computes partition values based on the ColumnPartitionConfig from the table config */ public class TableConfigPartitioner implements Partitioner { - /** - * Bucket name used when the row's partition column is null or when the partition expression evaluates to null - * mid-chain. Distinct from any numeric partition id so downstream readers can identify and handle these rows. - */ + /// Bucket name used when the row's partition column is null or when the partition expression evaluates to null + /// mid-chain. Distinct from any numeric partition id so downstream readers can identify and handle these rows. public static final String NULL_PARTITION = "null"; private final String _column; private final PartitionFunction _partitionFunction; - /** True when the partition function was compiled with BYTES input; raw byte[] values are passed directly. */ + /// True when the partition function was compiled with BYTES input; raw byte[] values are passed directly. private final boolean _isBytesMode; public TableConfigPartitioner(String columnName, ColumnPartitionConfig columnPartitionConfig) { diff --git a/pinot-core/src/test/java/org/apache/pinot/core/query/pruner/ColumnValueSegmentPrunerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/query/pruner/ColumnValueSegmentPrunerTest.java index 2068b9beb00c..8704f571e742 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/query/pruner/ColumnValueSegmentPrunerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/query/pruner/ColumnValueSegmentPrunerTest.java @@ -174,14 +174,11 @@ public void testPartitionPruningWithFunctionExpr() { firstNonMatchingValue, secondNonMatchingValue))); } - /** - * Verifies that IN-predicate pruning combining min/max and partition checks is correct: - *

      - *
    • A value that is within the min/max range AND maps to the segment's partition → keep (false)
    • - *
    • All values within the min/max range but NONE maps to the segment's partition → prune (true)
    • - *
    • All values outside the min/max range → prune (true) regardless of partition
    • - *
    - */ + /// Verifies that IN-predicate pruning combining min/max and partition checks is correct: + /// + /// - A value that is within the min/max range AND maps to the segment's partition -> keep (false) + /// - All values within the min/max range but NONE maps to the segment's partition -> prune (true) + /// - All values outside the min/max range -> prune (true) regardless of partition @Test public void testInPredicateWithMinMaxAndPartitionPruning() { IndexSegment indexSegment = mockIndexSegment(); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/AbstractColumnStatisticsCollector.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/AbstractColumnStatisticsCollector.java index 52980d192a95..1ec26b0a0e01 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/AbstractColumnStatisticsCollector.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/AbstractColumnStatisticsCollector.java @@ -169,10 +169,8 @@ protected void updatePartition(String value) { } } - /** - * Updates the partition set using raw bytes. Expression-mode pipelines compiled with BYTES input type will hash - * the raw bytes directly; legacy partition functions fall back to hex-encoding via the default interface method. - */ + /// Updates the partition set using raw bytes. Expression-mode pipelines compiled with BYTES input type will hash + /// the raw bytes directly; legacy partition functions fall back to hex-encoding via the default interface method. protected void updatePartition(byte[] bytes) { int partitionId = _partitionFunction.getPartition(bytes); if (partitionId != PartitionPipelineFunction.NULL_RESULT_PARTITION_ID) { diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/V1Constants.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/V1Constants.java index 115486b43700..4476b99e0a4b 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/V1Constants.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/V1Constants.java @@ -178,24 +178,20 @@ public static class Column { public static final String NUM_PARTITIONS = "numPartitions"; public static final String PARTITION_VALUES = "partitionValues"; public static final String PARTITION_FUNCTION_CONFIG = "partitionFunctionConfig"; - /** - * Reserved sentinel value written into {@link #PARTITION_FUNCTION} for expression-mode segments. This - * intentionally does not match any real {@code PartitionFunctionType} enum constant, so older readers loading - * a new segment trip {@code IllegalArgumentException("No enum constant for: FunctionExpr")} and fail fast - * rather than silently dropping partition pruning. New readers detect expression-mode via the presence of - * {@link #PARTITION_FUNCTION_EXPR} and ignore this sentinel. - * - *

    Must stay in sync with {@code PartitionPipelineFunction.NAME}. Do not add an enum constant with - * this name to {@code PartitionFunctionType} — doing so would silently break the cross-version fail-fast - * contract. - */ + /// Reserved sentinel value written into [#PARTITION_FUNCTION] for expression-mode segments. This + /// intentionally does not match any real `PartitionFunctionType` enum constant, so older readers loading + /// a new segment trip `IllegalArgumentException("No enum constant for: FunctionExpr")` and fail fast + /// rather than silently dropping partition pruning. New readers detect expression-mode via the presence of + /// [#PARTITION_FUNCTION_EXPR] and ignore this sentinel. + /// + /// Must stay in sync with `PartitionPipelineFunction.NAME`. Do **not** add an enum constant with + /// this name to `PartitionFunctionType` - doing so would silently break the cross-version fail-fast + /// contract. public static final String PARTITION_FUNCTION_EXPR_SENTINEL = "FunctionExpr"; - /** - * Stored input type for expression-mode partition pipelines (e.g. {@code "BYTES"}). Optional: when absent, - * readers derive the input type from the column's stored {@link org.apache.pinot.spi.data.FieldSpec.DataType}. - * Writing it explicitly avoids any chance of segment-vs-broker disagreement on the type used to compile the - * pipeline (the broker reads {@code inputType} from ZK metadata; the server now reads from segment metadata). - */ + /// Stored input type for expression-mode partition pipelines (e.g. `"BYTES"`). Optional: when absent, + /// readers derive the input type from the column's stored [org.apache.pinot.spi.data.FieldSpec.DataType]. + /// Writing it explicitly avoids any chance of segment-vs-broker disagreement on the type used to compile the + /// pipeline (the broker reads `inputType` from ZK metadata; the server now reads from segment metadata). public static final String PARTITION_INPUT_TYPE = "partitionInputType"; /// Old key maintained for backward compatibility diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/StatsCollectorConfig.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/StatsCollectorConfig.java index 812305ab2ccd..0c6c94a28f3e 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/StatsCollectorConfig.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/StatsCollectorConfig.java @@ -104,31 +104,36 @@ public PartitionFunction getPartitionFunction(String column) { @Deprecated @Nullable public String getPartitionFunctionName(String column) { - if (_segmentPartitionConfig == null) { - return null; - } - - return _segmentPartitionConfig.getFunctionName(column); + ColumnPartitionConfig config = getColumnPartitionConfig(column); + return config != null ? config.getFunctionName() : null; } @Deprecated public int getNumPartitions(String column) { - return (_segmentPartitionConfig != null) ? _segmentPartitionConfig.getNumPartitions(column) - : SegmentPartitionConfig.INVALID_NUM_PARTITIONS; + ColumnPartitionConfig config = getColumnPartitionConfig(column); + return config != null ? config.getNumPartitions() : SegmentPartitionConfig.INVALID_NUM_PARTITIONS; } @Nullable public String getPartitionFunctionExpr(String column) { - return (_segmentPartitionConfig != null) ? _segmentPartitionConfig.getFunctionExpr(column) : null; + ColumnPartitionConfig config = getColumnPartitionConfig(column); + return config != null ? config.getFunctionExpr() : null; } @Nullable public Map getPartitionFunctionConfig(String column) { - return (_segmentPartitionConfig != null) ? _segmentPartitionConfig.getFunctionConfig(column) : null; + ColumnPartitionConfig config = getColumnPartitionConfig(column); + return config != null ? config.getFunctionConfig() : null; } @Nullable public String getPartitionIdNormalizer(String column) { - return (_segmentPartitionConfig != null) ? _segmentPartitionConfig.getPartitionIdNormalizer(column) : null; + ColumnPartitionConfig config = getColumnPartitionConfig(column); + return config != null ? config.getPartitionIdNormalizer() : null; + } + + @Nullable + private ColumnPartitionConfig getColumnPartitionConfig(String column) { + return _segmentPartitionConfig != null ? _segmentPartitionConfig.getColumnPartitionConfig(column) : null; } } diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/function/ExecutableFunctionEvaluator.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/function/ExecutableFunctionEvaluator.java index 37666ec67c2a..30537a7630c8 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/function/ExecutableFunctionEvaluator.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/function/ExecutableFunctionEvaluator.java @@ -24,30 +24,24 @@ import org.apache.pinot.spi.data.readers.GenericRow; -/** - * Shared runtime implementation for expression evaluators backed by an executable node tree. - * - *

    The planner for a concrete evaluator is responsible for binding scalar functions and constructing the root - * {@link ExecutableNode}. This class owns the repeated row/value execution logic so ingestion and partition paths do - * not maintain separate node evaluators. - * - *

    Thread-safety: Once constructed, an {@code ExecutableFunctionEvaluator} instance is safe for - * concurrent invocation by multiple threads and for re-entrant invocation on a single thread. - * {@link FunctionNode} allocates a fresh argument scratch array per {@code execute} call, so neither - * cross-thread sharing nor recursive invocation through nested user-defined scalar functions can corrupt - * argument state. - */ +/// Shared runtime implementation for expression evaluators backed by an executable node tree. +/// +/// The planner for a concrete evaluator is responsible for binding scalar functions and constructing the root +/// [ExecutableNode]. This class owns the repeated row/value execution logic so ingestion and partition paths do +/// not maintain separate node evaluators. +/// +/// **Thread-safety:** Once constructed, an `ExecutableFunctionEvaluator` instance is safe for +/// concurrent invocation by multiple threads and for re-entrant invocation on a single thread. +/// [FunctionNode] allocates a fresh argument scratch array per `execute` call, so neither +/// cross-thread sharing nor recursive invocation through nested user-defined scalar functions can corrupt +/// argument state. public class ExecutableFunctionEvaluator implements org.apache.pinot.spi.function.FunctionEvaluator { - /** - * Invokes a bound function with already-evaluated child arguments. - */ + /// Invokes a bound function with already-evaluated child arguments. public interface Invoker { Object invoke(Object[] arguments); } - /** - * Executes one node within the evaluator tree. - */ + /// Executes one node within the evaluator tree. public interface ExecutableNode { Object execute(GenericRow row); @@ -84,9 +78,7 @@ public String toString() { return _expression; } - /** - * Returns a constant value. - */ + /// Returns a constant value. public static class ConstantNode implements ExecutableNode { private final Object _value; @@ -110,9 +102,7 @@ public String toString() { } } - /** - * Returns a constant array value. - */ + /// Returns a constant array value. public static class ArrayConstantNode implements ExecutableNode { private final Object[] _value; @@ -136,9 +126,7 @@ public String toString() { } } - /** - * Reads one referenced argument from a row or positional argument array. - */ + /// Reads one referenced argument from a row or positional argument array. public static class ColumnNode implements ExecutableNode { private final String _column; private final int _id; @@ -164,15 +152,13 @@ public String toString() { } } - /** - * Executes a bound function after evaluating all child arguments. - * - *

    Allocates a fresh {@code Object[argumentNodes.length]} per {@code execute} call. This is safe under both - * concurrent invocation by multiple threads and re-entrant invocation on a single thread (e.g. when a partition - * expression nested inside a transform function recurses through the same compiled tree). The allocation is - * small and short-lived; HotSpot's escape analysis routinely scalar-replaces such arrays so the per-row cost - * is negligible relative to the underlying function invocation. - */ + /// Executes a bound function after evaluating all child arguments. + /// + /// Allocates a fresh `Object[argumentNodes.length]` per `execute` call. This is safe under both + /// concurrent invocation by multiple threads and re-entrant invocation on a single thread (e.g. when a partition + /// expression nested inside a transform function recurses through the same compiled tree). The allocation is + /// small and short-lived; HotSpot's escape analysis routinely scalar-replaces such arrays so the per-row cost + /// is negligible relative to the underlying function invocation. public static class FunctionNode implements ExecutableNode { private final String _functionName; private final Invoker _invoker; @@ -215,9 +201,7 @@ public String toString() { } } - /** - * Three-valued logical NOT. - */ + /// Three-valued logical NOT. public static class NotNode implements ExecutableNode { private final ExecutableNode _argumentNode; @@ -238,9 +222,7 @@ public Object execute(Object[] values) { } } - /** - * Three-valued logical OR with short-circuit evaluation. - */ + /// Three-valued logical OR with short-circuit evaluation. public static class OrNode implements ExecutableNode { private final ExecutableNode[] _argumentNodes; @@ -277,9 +259,7 @@ public Object execute(Object[] values) { } } - /** - * Three-valued logical AND with short-circuit evaluation. - */ + /// Three-valued logical AND with short-circuit evaluation. public static class AndNode implements ExecutableNode { private final ExecutableNode[] _argumentNodes; diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunction.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunction.java index e3404b799e90..cab3ee941f7f 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunction.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunction.java @@ -25,51 +25,45 @@ import org.apache.pinot.spi.utils.BytesUtils; -/** - * Interface for partition function. - * - *

    Implementations of this interface are assumed not to be stateful. - * That is, two invocations of {@code PartitionFunction.getPartition(value)} - * with the same value are expected to produce the same result. Implementations must also be safe for concurrent - * invocation by multiple threads. - * - *

    Expression-mode partition functions: When {@link #getFunctionExpr()} returns non-null, the implementation - * is operating in expression mode (e.g. {@code PartitionPipelineFunction}). In that case - * {@link #getPartitionIdNormalizer()} and {@link #getPartitionColumn()} also typically return non-null. Existing - * legacy partition functions ({@code Murmur}, {@code Modulo}, {@code HashCode}, etc.) return {@code null} from these - * accessors and continue to operate as before. Framework callers (segment writers, broker pruners, staleness checks) - * use the non-null/null distinction on {@code getFunctionExpr} to dispatch between the two modes — plugins that - * want to be treated as expression-mode must override the relevant accessors. - */ +/// Interface for partition function. +/// +/// Implementations of this interface are assumed not to be stateful. +/// That is, two invocations of `PartitionFunction.getPartition(value)` +/// with the same value are expected to produce the same result. Implementations must also be safe for concurrent +/// invocation by multiple threads. +/// +/// **Expression-mode partition functions:** When [#getFunctionExpr()] returns non-null, the implementation +/// is operating in expression mode (e.g. `PartitionPipelineFunction`). In that case +/// [#getPartitionIdNormalizer()] and [#getPartitionColumn()] also typically return non-null. Existing +/// legacy partition functions (`Murmur`, `Modulo`, `HashCode`, etc.) return `null` from these +/// accessors and continue to operate as before. Framework callers (segment writers, broker pruners, staleness checks) +/// use the non-null/null distinction on `getFunctionExpr` to dispatch between the two modes - plugins that +/// want to be treated as expression-mode must override the relevant accessors. public interface PartitionFunction extends Serializable { - /** - * Method to compute and return partition id for the given value. - * NOTE: The value is expected to be a string representation of the actual value. - * - *

    Return-value contract: implementations must return a non-negative partition id in - * {@code [0, getNumPartitions())}. The value {@code -1} is reserved as a framework-internal sentinel for - * "expression evaluated to null" (see - * {@link org.apache.pinot.segment.spi.partition.pipeline.PartitionPipelineFunction#NULL_RESULT_PARTITION_ID}) — - * custom plugin implementations must not return {@code -1} as a real partition id. Internal callers - * (broker pruner, stats collector, segment processing partitioner) treat {@code -1} as "skip / no partition". - * - * @param value Value for which to determine the partition id. - * @return partition id for the value (non-negative for real partitions; never {@code -1}). - */ + /// Method to compute and return partition id for the given value. + /// NOTE: The value is expected to be a string representation of the actual value. + /// + /// **Return-value contract:** implementations must return a non-negative partition id in + /// `[0, getNumPartitions())`. The value `-1` is reserved as a framework-internal sentinel for + /// "expression evaluated to null" (see + /// [org.apache.pinot.segment.spi.partition.pipeline.PartitionPipelineFunction#NULL_RESULT_PARTITION_ID]) - + /// custom plugin implementations must not return `-1` as a real partition id. Internal callers + /// (broker pruner, stats collector, segment processing partitioner) treat `-1` as "skip / no partition". + /// + /// @param value Value for which to determine the partition id. + /// @return partition id for the value (non-negative for real partitions; never `-1`). int getPartition(String value); - /** - * Returns the partition id for a raw byte array value. - * - *

    The default implementation converts the bytes to a hex string and delegates to {@link #getPartition(String)}, - * matching the historical behaviour for BYTES columns. Expression-mode pipelines that were compiled with - * {@link org.apache.pinot.segment.spi.partition.pipeline.PartitionValueType#BYTES} input type override this method - * to hash the raw bytes directly without the hex-encoding round-trip. - * - * @param bytes Raw byte array value. - * @return partition id for the value. - */ + /// Returns the partition id for a raw byte array value. + /// + /// The default implementation converts the bytes to a hex string and delegates to [#getPartition(String)], + /// matching the historical behaviour for BYTES columns. Expression-mode pipelines that were compiled with + /// [org.apache.pinot.segment.spi.partition.pipeline.PartitionValueType#BYTES] input type override this method + /// to hash the raw bytes directly without the hex-encoding round-trip. + /// + /// @param bytes Raw byte array value. + /// @return partition id for the value. default int getPartition(byte[] bytes) { return getPartition(BytesUtils.toHexString(bytes)); } @@ -103,6 +97,13 @@ default String getFunctionExpr() { return null; } + /// Reports the int-normalizer used by this partition function (e.g. `POSITIVE_MODULO`, `ABS`, + /// `MASK`). The returned string is used by the framework only for identity / staleness matching between + /// config-side and segment-side function metadata; it does not drive runtime normalization. Legacy partition + /// functions (Murmur, FNV, HashCode, ByteArray, Modulo) report the closest matching normalizer name even though + /// their internal implementation may differ subtly at edge cases (e.g. Kafka-style abs handling + /// `Integer.MIN_VALUE -> 0` vs strict mod-then-abs). Expression-mode pipelines drive normalization through + /// `PartitionIntNormalizer` directly and are the only code path where this value is authoritative. @JsonIgnore @Nullable default String getPartitionIdNormalizer() { diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java index b7742a590ccb..d163119b5774 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/PartitionFunctionFactory.java @@ -109,15 +109,13 @@ public static PartitionFunction getPartitionFunction(String functionName, int nu } } - /** - * Returns the legacy (name-mode) partition function for the given config. - * - * @deprecated Expression-mode configs require a column name to compile the pipeline. This overload throws on - * expression-mode configs; prefer {@link #getPartitionFunction(String, ColumnPartitionConfig)} or - * {@link #getPartitionFunction(String, ColumnPartitionConfig, FieldSpec)} which support both modes. - * TODO: remove after release 1.7.0. - * @throws IllegalArgumentException if {@code config.getFunctionExpr()} is non-null. - */ + /// Returns the legacy (name-mode) partition function for the given config. + /// + /// @deprecated Expression-mode configs require a column name to compile the pipeline. This overload throws on + /// expression-mode configs; prefer [ColumnPartitionConfig)][#getPartitionFunction(String,] or + /// [ColumnPartitionConfig, FieldSpec)][#getPartitionFunction(String,] which support both modes. + /// TODO: remove after release 1.7.0. + /// @throws IllegalArgumentException if `config.getFunctionExpr()` is non-null. @Deprecated public static PartitionFunction getPartitionFunction(ColumnPartitionConfig config) { Preconditions.checkNotNull(config, "Column partition config must be configured"); @@ -126,14 +124,12 @@ public static PartitionFunction getPartitionFunction(ColumnPartitionConfig confi return getPartitionFunction(config.getFunctionName(), config.getNumPartitions(), config.getFunctionConfig()); } - /** - * Returns the legacy (name-mode) partition function for the given segment metadata. - * - * @deprecated Expression-mode metadata requires a column name to compile the pipeline. This overload throws on - * expression-mode metadata; prefer {@link #getPartitionFunction(String, ColumnPartitionMetadata)}. - * TODO: remove after release 1.7.0. - * @throws IllegalArgumentException if {@code metadata.getFunctionExpr()} is non-null. - */ + /// Returns the legacy (name-mode) partition function for the given segment metadata. + /// + /// @deprecated Expression-mode metadata requires a column name to compile the pipeline. This overload throws on + /// expression-mode metadata; prefer [ColumnPartitionMetadata)][#getPartitionFunction(String,]. + /// TODO: remove after release 1.7.0. + /// @throws IllegalArgumentException if `metadata.getFunctionExpr()` is non-null. @Deprecated public static PartitionFunction getPartitionFunction(ColumnPartitionMetadata metadata) { Preconditions.checkNotNull(metadata, "Column partition metadata must be configured"); @@ -157,14 +153,12 @@ public static PartitionFunction getPartitionFunction(String columnName, ColumnPa return getPartitionFunction(columnName, columnPartitionConfig, columnPartitionConfig.getNumPartitions()); } - /** - * Builds a partition function for the given column with an explicit numPartitions override. - * - * @deprecated For BYTES-typed partition columns this overload always compiles the expression pipeline with STRING - * input, producing partition ids that disagree with ingestion. Prefer - * {@link #getPartitionFunction(String, ColumnPartitionConfig, int, FieldSpec)} (the FieldSpec-aware 4-arg form). - * TODO: remove after release 1.7.0. - */ + /// Builds a partition function for the given column with an explicit numPartitions override. + /// + /// @deprecated For BYTES-typed partition columns this overload always compiles the expression pipeline with STRING + /// input, producing partition ids that disagree with ingestion. Prefer + /// [ColumnPartitionConfig, int, FieldSpec)][#getPartitionFunction(String,] (the FieldSpec-aware 4-arg form). + /// TODO: remove after release 1.7.0. @Deprecated public static PartitionFunction getPartitionFunction(String columnName, ColumnPartitionConfig columnPartitionConfig, int numPartitions) { @@ -179,12 +173,10 @@ public static PartitionFunction getPartitionFunction(String columnName, @Nullabl return getPartitionFunction(columnName, functionName, numPartitions, functionConfig, functionExpr, null); } - /** - * Builds a partition function for expression mode using an explicit input type. - * - *

    Use {@link PartitionValueType#BYTES} when the partition column stores raw byte arrays so that functions in the - * expression receive the original bytes directly rather than a hex-encoded string representation. - */ + /// Builds a partition function for expression mode using an explicit input type. + /// + /// Use [PartitionValueType#BYTES] when the partition column stores raw byte arrays so that functions in the + /// expression receive the original bytes directly rather than a hex-encoded string representation. public static PartitionFunction getPartitionFunction(String columnName, ColumnPartitionConfig config, PartitionValueType inputType) { Preconditions.checkNotNull(config, "Column partition config must be configured"); @@ -194,27 +186,23 @@ public static PartitionFunction getPartitionFunction(String columnName, ColumnPa config.getNumPartitions(), config.getPartitionIdNormalizer()); } - /** - * Builds a partition function using the schema field spec to determine the correct input type for expression-mode - * partition functions on BYTES-typed columns. - * - *

    When {@code fieldSpec} is non-null and the stored type is {@link FieldSpec.DataType#BYTES}, the expression is - * compiled with {@link PartitionValueType#BYTES} input so that scalar functions receive raw byte arrays rather than - * hex-encoded strings. For all other cases the default STRING input type is used. - */ + /// Builds a partition function using the schema field spec to determine the correct input type for expression-mode + /// partition functions on BYTES-typed columns. + /// + /// When `fieldSpec` is non-null and the stored type is [FieldSpec.DataType#BYTES], the expression is + /// compiled with [PartitionValueType#BYTES] input so that scalar functions receive raw byte arrays rather than + /// hex-encoded strings. For all other cases the default STRING input type is used. public static PartitionFunction getPartitionFunction(String columnName, ColumnPartitionConfig config, @Nullable FieldSpec fieldSpec) { return getPartitionFunction(columnName, config, config.getNumPartitions(), fieldSpec); } - /** - * Builds a partition function with an explicit {@code numPartitions} override and uses {@code fieldSpec} to - * determine the correct input type for expression-mode partition functions on BYTES-typed columns. - * - *

    Use this overload when the live partition count (e.g. the stream partition count) may differ from the value - * stored in the table config, so that the built function uses the authoritative count while still receiving the - * correct input type for BYTES columns. - */ + /// Builds a partition function with an explicit `numPartitions` override and uses `fieldSpec` to + /// determine the correct input type for expression-mode partition functions on BYTES-typed columns. + /// + /// Use this overload when the live partition count (e.g. the stream partition count) may differ from the value + /// stored in the table config, so that the built function uses the authoritative count while still receiving the + /// correct input type for BYTES columns. public static PartitionFunction getPartitionFunction(String columnName, ColumnPartitionConfig config, int numPartitions, @Nullable FieldSpec fieldSpec) { if (config.getFunctionExpr() != null && fieldSpec != null diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java index dec5e8c649c8..83b60ba2695c 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadata.java @@ -55,28 +55,24 @@ public class ColumnPartitionMetadata { private final String _functionName; private final String _functionExpr; private final String _partitionIdNormalizer; - /** - * Pipeline input type for expression-mode partition functions. {@code null} means STRING (the default and legacy - * behavior). Currently only {@code "BYTES"} is stored for BYTES-typed partition columns so that the broker can - * compile the same BYTES-input pipeline and produce consistent partition assignments with ingestion. - */ + /// Pipeline input type for expression-mode partition functions. `null` means STRING (the default and legacy + /// behavior). Currently only `"BYTES"` is stored for BYTES-typed partition columns so that the broker can + /// compile the same BYTES-input pipeline and produce consistent partition assignments with ingestion. @Nullable private final String _inputType; private final int _numPartitions; private final Map _functionConfig; private final Set _partitions; - /** - * Constructor for the class. - * - * @param functionName Name of the partition function - * @param numPartitions Number of total partitions for this column - * @param partitions Set of partitions the column contains - * @param functionConfig Configuration required by partition function. - * @deprecated Use {@link #ColumnPartitionMetadata(PartitionFunction, Set)} instead, which derives all fields - * directly from the {@link PartitionFunction} contract and keeps them consistent. - * TODO: remove after release 1.7.0. - */ + /// Constructor for the class. + /// + /// @param functionName Name of the partition function + /// @param numPartitions Number of total partitions for this column + /// @param partitions Set of partitions the column contains + /// @param functionConfig Configuration required by partition function. + /// @deprecated Use [Set)][#ColumnPartitionMetadata(PartitionFunction,] instead, which derives all fields + /// directly from the [PartitionFunction] contract and keeps them consistent. + /// TODO: remove after release 1.7.0. @Deprecated public ColumnPartitionMetadata(String functionName, int numPartitions, Set partitions, @Nullable Map functionConfig) { @@ -123,12 +119,10 @@ public String getFunctionName() { return _functionName; } - /** - * Returns the pipeline input type for expression-mode partition functions, or {@code null} if the default - * ({@link PartitionValueType#STRING}) input type applies. The {@code @JsonProperty} value matches - * {@link org.apache.pinot.segment.spi.V1Constants.MetadataKeys.Column#PARTITION_INPUT_TYPE} so the same key - * names the field on both wire formats (ZK JSON and segment {@code .properties}). - */ + /// Returns the pipeline input type for expression-mode partition functions, or `null` if the default + /// ([PartitionValueType#STRING]) input type applies. The `@JsonProperty` value matches + /// [org.apache.pinot.segment.spi.V1Constants.MetadataKeys.Column#PARTITION_INPUT_TYPE] so the same key + /// names the field on both wire formats (ZK JSON and segment `.properties`). @Nullable @JsonProperty("partitionInputType") public String getInputType() { diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionEvaluatorFactory.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionEvaluatorFactory.java index a2b3eb32381c..4e0b1ede409b 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionEvaluatorFactory.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionEvaluatorFactory.java @@ -21,28 +21,23 @@ import org.apache.pinot.spi.function.FunctionEvaluator; -/** - * SPI for compiling a partition expression string into a {@link FunctionEvaluator}. - * - *

    Implementations are loaded via {@link java.util.ServiceLoader}. At least one implementation must be present on - * the classpath at runtime. If multiple are found (common in shaded-jar / multi-module test setups), the loader - * prefers {@code InbuiltPartitionEvaluatorFactory} when present; otherwise it picks the first discovered factory and - * logs the choice. The default implementation in {@code pinot-common} is - * {@code org.apache.pinot.common.evaluator.InbuiltPartitionEvaluatorFactory}, which delegates to - * {@code org.apache.pinot.common.evaluator.PartitionFunctionEvaluator}. - */ +/// SPI for compiling a partition expression string into a [FunctionEvaluator]. +/// +/// Implementations are loaded via [java.util.ServiceLoader]. At least one implementation must be present on +/// the classpath at runtime. If multiple are found (common in shaded-jar / multi-module test setups), the loader +/// prefers `InbuiltPartitionEvaluatorFactory` when present; otherwise it picks the first discovered factory and +/// logs the choice. The default implementation in `pinot-common` is +/// `org.apache.pinot.common.evaluator.InbuiltPartitionEvaluatorFactory`, which delegates to +/// `org.apache.pinot.common.evaluator.PartitionFunctionEvaluator`. public interface PartitionEvaluatorFactory { - /** - * Compiles a partition expression into a {@link FunctionEvaluator}. - * - *

    Validates that the expression: - *

      - *
    • References exactly one column: {@code rawColumn}
    • - *
    • Uses only deterministic scalar functions
    • - *
    - * - * @throws IllegalArgumentException if the expression is invalid - */ + /// Compiles a partition expression into a [FunctionEvaluator]. + /// +/// Validates that the expression: +/// +/// - References exactly one column: `rawColumn` +/// - Uses only deterministic scalar functions +/// +/// @throws IllegalArgumentException if the expression is invalid FunctionEvaluator compile(String rawColumn, String expression); } diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java index 186ca0e486be..51187cb781a3 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java @@ -36,10 +36,8 @@ import org.slf4j.LoggerFactory; -/** - * Compiles a partition-function expression into a {@link PartitionPipeline} backed by a - * {@link org.apache.pinot.spi.function.FunctionEvaluator} provided by {@link PartitionEvaluatorFactory}. - */ +/// Compiles a partition-function expression into a [PartitionPipeline] backed by a +/// [org.apache.pinot.spi.function.FunctionEvaluator] provided by [PartitionEvaluatorFactory]. public final class PartitionFunctionExprCompiler { private static final Logger LOGGER = LoggerFactory.getLogger(PartitionFunctionExprCompiler.class); // Cap on user-supplied expression length, sized to fit comfortably within ZK / segment metadata size budgets while @@ -113,12 +111,10 @@ public static PartitionPipeline compile(String rawColumn, String functionExpr, return compile(rawColumn, PartitionValueType.STRING, functionExpr, partitionIdNormalizer); } - /** - * Compiles a partition pipeline with an explicit input type. - * - *

    Use {@link PartitionValueType#BYTES} when the partition column stores raw byte arrays so that functions in the - * expression receive the original bytes directly rather than a hex-encoded string representation. - */ + /// Compiles a partition pipeline with an explicit input type. + /// + /// Use [PartitionValueType#BYTES] when the partition column stores raw byte arrays so that functions in the + /// expression receive the original bytes directly rather than a hex-encoded string representation. public static PartitionPipeline compile(String rawColumn, PartitionValueType inputType, String functionExpr, @Nullable PartitionIntNormalizer partitionIdNormalizer) { Preconditions.checkArgument(rawColumn != null && !rawColumn.trim().isEmpty(), "Raw column must be configured"); @@ -164,12 +160,10 @@ public static PartitionPipelineFunction compilePartitionFunction(String rawColum partitionIdNormalizer); } - /** - * Compiles a partition pipeline function with an explicit input type. - * - *

    Use {@link PartitionValueType#BYTES} when the partition column stores raw byte arrays so that functions in the - * expression receive the original bytes directly rather than a hex-encoded string representation. - */ + /// Compiles a partition pipeline function with an explicit input type. + /// + /// Use [PartitionValueType#BYTES] when the partition column stores raw byte arrays so that functions in the + /// expression receive the original bytes directly rather than a hex-encoded string representation. public static PartitionPipelineFunction compilePartitionFunction(String rawColumn, PartitionValueType inputType, String functionExpr, int numPartitions, @Nullable String partitionIdNormalizer) { PartitionIntNormalizer normalizer = partitionIdNormalizer != null @@ -179,11 +173,9 @@ public static PartitionPipelineFunction compilePartitionFunction(String rawColum return new PartitionPipelineFunction(pipeline, numPartitions); } - /** - * Returns a canonical form of the expression: trimmed, lowercased, with spaces removed around - * {@code (}, {@code )}, and {@code ,}. Quoted literals are preserved byte-for-byte because partition expressions - * can use case-sensitive salts, JSON paths, or regex patterns as function arguments. - */ + /// Returns a canonical form of the expression: trimmed, lowercased, with spaces removed around + /// `(`, `)`, and `,`. Quoted literals are preserved byte-for-byte because partition expressions + /// can use case-sensitive salts, JSON paths, or regex patterns as function arguments. static String canonicalize(String expression) { String trimmed = expression.trim(); StringBuilder builder = new StringBuilder(trimmed.length()); diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionIntNormalizer.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionIntNormalizer.java index 6840343ae0c1..f8524ddc0257 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionIntNormalizer.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionIntNormalizer.java @@ -22,13 +22,9 @@ import java.util.Locale; -/** - * Normalizes the final INT output from a compiled partition pipeline into a partition id. - */ +/// Normalizes the final INT output from a compiled partition pipeline into a partition id. public enum PartitionIntNormalizer { - /** - * Computes the remainder first, then shifts negative partition ids into the valid range with {@code + numPartitions}. - */ + /// Computes the remainder first, then shifts negative partition ids into the valid range with `+ numPartitions`. POSITIVE_MODULO { @Override int toPartitionId(int value, int numPartitions) { @@ -42,9 +38,7 @@ int toPartitionId(long value, int numPartitions) { return (int) (partition < 0 ? partition + numPartitions : partition); } }, - /** - * Computes the remainder first, then takes the absolute value of the remainder. - */ + /// Computes the remainder first, then takes the absolute value of the remainder. ABS { @Override int toPartitionId(int value, int numPartitions) { @@ -58,9 +52,7 @@ int toPartitionId(long value, int numPartitions) { return (int) (partition < 0 ? -partition : partition); } }, - /** - * Makes the raw INT/LONG output non-negative first by masking the sign bit, then applies modulo. - */ + /// Makes the raw INT/LONG output non-negative first by masking the sign bit, then applies modulo. MASK { @Override int toPartitionId(int value, int numPartitions) { diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipeline.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipeline.java index 53922b953851..2e093f561a6a 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipeline.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipeline.java @@ -28,14 +28,12 @@ import org.apache.pinot.spi.utils.BytesUtils; -/** - * Immutable compiled pipeline for one raw partition column, backed by a {@link FunctionEvaluator}. - * - *

    The underlying {@link FunctionEvaluator} is responsible for correct type coercion. For partition expressions, - * the evaluator uses UTF-8 encoding when converting {@code String} values to {@code byte[]} parameters, ensuring - * that hash functions ({@code md5}, {@code murmur2}, {@code fnv1a_32}, etc.) operate on raw string bytes rather - * than a hex-decoded representation. - */ +/// Immutable compiled pipeline for one raw partition column, backed by a [FunctionEvaluator]. +/// +/// The underlying [FunctionEvaluator] is responsible for correct type coercion. For partition expressions, +/// the evaluator uses UTF-8 encoding when converting `String` values to `byte[]` parameters, ensuring +/// that hash functions (`md5`, `murmur2`, `fnv1a_32`, etc.) operate on raw string bytes rather +/// than a hex-decoded representation. public final class PartitionPipeline implements FunctionEvaluator { private final String _rawColumn; private final boolean _isBytesInput; diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java index ac12e522f124..d6c959ecc731 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java @@ -35,20 +35,16 @@ import org.slf4j.LoggerFactory; -/** - * {@link PartitionFunction} adapter for expression-mode partition pipelines. - * - *

    Note on {@code Serializable}: {@link PartitionFunction} extends {@link java.io.Serializable} for - * historical reasons, but partition functions are never Java-serialized in Pinot's runtime. - */ +/// [PartitionFunction] adapter for expression-mode partition pipelines. +/// +/// **Note on `Serializable`:** [PartitionFunction] extends [java.io.Serializable] for +/// historical reasons, but partition functions are never Java-serialized in Pinot's runtime. @SuppressWarnings("serial") public class PartitionPipelineFunction implements PartitionFunction, FunctionEvaluator { private static final Logger LOGGER = LoggerFactory.getLogger(PartitionPipelineFunction.class); - /** - * Function-name value reported via {@link #getName()} and written into segment metadata for expression-mode - * pipelines. Bound to the reserved sentinel constant in {@link V1Constants.MetadataKeys.Column} so the - * cross-version fail-fast contract for old readers cannot drift via duplicate string literals. - */ + /// Function-name value reported via [#getName()] and written into segment metadata for expression-mode + /// pipelines. Bound to the reserved sentinel constant in [V1Constants.MetadataKeys.Column] so the + /// cross-version fail-fast contract for old readers cannot drift via duplicate string literals. public static final String NAME = V1Constants.MetadataKeys.Column.PARTITION_FUNCTION_EXPR_SENTINEL; // Sentinel returned when the partition expression evaluates to null for a given value (e.g. null input). // Callers that update per-segment partition sets must skip this value. @@ -81,12 +77,10 @@ public PartitionPipeline getPartitionPipeline() { return _pipeline; } - /** - * Validates that the compiled expression produces an integral numeric output by probing it with several sample - * values that cover common input shapes (numeric strings, alpha strings, raw bytes). Throws - * {@link IllegalArgumentException} if at least one probe completes and the output is non-numeric (e.g. STRING from - * {@code md5(col)}), or if every probe throws (cannot determine output type — likely a misconfigured pipeline). - */ + /// Validates that the compiled expression produces an integral numeric output by probing it with several sample + /// values that cover common input shapes (numeric strings, alpha strings, raw bytes). Throws + /// [IllegalArgumentException] if at least one probe completes and the output is non-numeric (e.g. STRING from + /// `md5(col)`), or if every probe throws (cannot determine output type - likely a misconfigured pipeline). public void validateOutputType() { // Probe set covers numeric strings, alpha strings, raw bytes (for BYTES input), and null. The null sample // mirrors the runtime ingestion path that already handles nulls, so the validation also exercises that the @@ -166,10 +160,8 @@ public int getPartition(String value) { return normalizeResult(_pipeline.evaluate(new Object[]{value})); } - /** - * Overrides the default bytes partition to pass raw bytes directly through the pipeline when this pipeline was - * compiled with {@link PartitionValueType#BYTES} input type, avoiding the hex-encoding round-trip. - */ + /// Overrides the default bytes partition to pass raw bytes directly through the pipeline when this pipeline was + /// compiled with [PartitionValueType#BYTES] input type, avoiding the hex-encoding round-trip. @Override public int getPartition(byte[] bytes) { if (!_pipeline.isBytesInput()) { diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionValueType.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionValueType.java index 4cedb067302a..81927dcdad1e 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionValueType.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionValueType.java @@ -19,9 +19,7 @@ package org.apache.pinot.segment.spi.partition.pipeline; -/** - * Value types supported by the partition-pipeline compiler. - */ +/// Value types supported by the partition-pipeline compiler. public enum PartitionValueType { STRING, BYTES, diff --git a/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionTest.java b/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionTest.java index 5ddd930db5e8..328890d2fbcc 100644 --- a/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionTest.java +++ b/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionTest.java @@ -892,13 +892,11 @@ private int getMaskPartition(long hash, int numPartitions) { return (int) ((hash & Long.MAX_VALUE) % numPartitions); } - /** - * Cross-version contract regression: the segment-metadata sentinel "FunctionExpr" written for expression-mode - * segments MUST never resolve to a real PartitionFunctionType enum constant. Old readers rely on the enum lookup - * throwing IllegalArgumentException to fail fast and skip partition pruning, instead of silently treating the - * column as un-partitioned. If anyone adds a "FunctionExpr" enum value, this test guards against the silent - * regression in the cross-version fail-fast contract. - */ + /// Cross-version contract regression: the segment-metadata sentinel "FunctionExpr" written for expression-mode + /// segments MUST never resolve to a real PartitionFunctionType enum constant. Old readers rely on the enum lookup + /// throwing IllegalArgumentException to fail fast and skip partition pruning, instead of silently treating the + /// column as un-partitioned. If anyone adds a "FunctionExpr" enum value, this test guards against the silent + /// regression in the cross-version fail-fast contract. @Test public void testFunctionExprSentinelIsNotARealPartitionFunctionType() { expectThrows(IllegalArgumentException.class, @@ -906,12 +904,10 @@ public void testFunctionExprSentinelIsNotARealPartitionFunctionType() { V1Constants.MetadataKeys.Column.PARTITION_FUNCTION_EXPR_SENTINEL)); } - /** - * Drift regression: {@code ColumnPartitionConfig.PARTITION_ID_NORMALIZER_*} string constants live in pinot-spi - * (which cannot reference {@link PartitionIntNormalizer} due to the module dependency direction). This test - * asserts the two stay in lockstep so adding a new normalizer to the enum without updating the validator string - * list trips a test failure rather than a silent runtime rejection. - */ + /// Drift regression: `ColumnPartitionConfig.PARTITION_ID_NORMALIZER_*` string constants live in pinot-spi + /// (which cannot reference [PartitionIntNormalizer] due to the module dependency direction). This test + /// asserts the two stay in lockstep so adding a new normalizer to the enum without updating the validator string + /// list trips a test failure rather than a silent runtime rejection. @Test public void testPartitionIdNormalizerConstantsMatchEnumValues() { Set enumNames = new HashSet<>(); diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ColumnPartitionConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ColumnPartitionConfig.java index b9e738a6f2b0..f0e1ef150927 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ColumnPartitionConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ColumnPartitionConfig.java @@ -27,23 +27,18 @@ import org.apache.pinot.spi.config.BaseJsonConfig; -/** - * Partition configuration for a single column. - * - *

    There are two mutually exclusive modes: - *

      - *
    • Name mode (legacy): configure {@code functionName} (and optionally {@code functionConfig}). - * Old and new nodes can all deserialize this format.
    • - *
    • Expression mode (new): configure {@code functionExpr} (and optionally - * {@code partitionIdNormalizer}). This mode produces JSON that omits {@code functionName}. - * Known limitation: expression-mode configs must only be written after all - * broker/server/controller nodes have been upgraded to a version that supports this feature. - * A node pre-dating expression mode will fail to deserialize such a config and will exclude - * the affected table from partition-aware routing. No mixed-version safety gate is enforced - * at the controller write path; operators are responsible for ensuring the cluster is fully - * upgraded before enabling expression-mode partitioning.
    • - *
    - */ +/// Partition configuration for a single column. +/// +/// There are two mutually exclusive modes: +/// +/// - **Name mode** (legacy): configure `functionName` (and optionally `functionConfig`). +/// Old and new nodes can all deserialize this format. +/// - **Expression mode** (new): configure `functionExpr` (and optionally `partitionIdNormalizer`). This mode produces +/// JSON that omits `functionName`. **Known limitation:** expression-mode configs must only be written after *all* +/// broker/server/controller nodes have been upgraded to a version that supports this feature. A node pre-dating +/// expression mode will fail to deserialize such a config and will exclude the affected table from partition-aware +/// routing. No mixed-version safety gate is enforced at the controller write path; operators are responsible for +/// ensuring the cluster is fully upgraded before enabling expression-mode partitioning. public class ColumnPartitionConfig extends BaseJsonConfig { public static final String PARTITION_ID_NORMALIZER_POSITIVE_MODULO = "POSITIVE_MODULO"; public static final String PARTITION_ID_NORMALIZER_ABS = "ABS"; @@ -107,17 +102,13 @@ public String getFunctionName() { return _functionName; } - /** - * Returns the function expression for expression-mode partitioning. - */ + /// Returns the function expression for expression-mode partitioning. @Nullable public String getFunctionExpr() { return _functionExpr; } - /** - * Returns the partition-id normalizer for expression-mode partitioning. - */ + /// Returns the partition-id normalizer for expression-mode partitioning. @Nullable public String getPartitionIdNormalizer() { return _partitionIdNormalizer; diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentPartitionConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentPartitionConfig.java index cb01103a3eb9..624aba428d37 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentPartitionConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/SegmentPartitionConfig.java @@ -61,18 +61,6 @@ public int getNumPartitions(String column) { return (config != null) ? config.getNumPartitions() : INVALID_NUM_PARTITIONS; } - @Nullable - public String getFunctionExpr(String column) { - ColumnPartitionConfig config = _columnPartitionMap.get(column); - return (config != null) ? config.getFunctionExpr() : null; - } - - @Nullable - public String getPartitionIdNormalizer(String column) { - ColumnPartitionConfig config = _columnPartitionMap.get(column); - return (config != null) ? config.getPartitionIdNormalizer() : null; - } - // TODO: remove this deprecated getter after release 1.7.0 once internal callers have migrated to // getColumnPartitionConfig(column).getFunctionConfig(). @Deprecated diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/FunctionNameUtils.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/FunctionNameUtils.java index 2504ff96a8e6..c4b49f07aeb2 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/FunctionNameUtils.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/FunctionNameUtils.java @@ -22,22 +22,18 @@ import org.apache.commons.lang3.StringUtils; -/** - * Utilities for normalizing function names across Pinot. - * - *

    This is the single authoritative implementation used by {@code FunctionRegistry}, - * {@code ScalarFunctionUtils}, and other function-resolution paths throughout the codebase. - */ +/// Utilities for normalizing function names across Pinot. +/// +/// This is the single authoritative implementation used by `FunctionRegistry`, +/// `ScalarFunctionUtils`, and other function-resolution paths throughout the codebase. public final class FunctionNameUtils { private FunctionNameUtils() { } - /** - * Returns the canonical form of a function name: underscores stripped, lower-cased. - * - *

    Both scalar functions and aggregate functions use this normalization so that - * {@code fnv1a_32}, {@code FNV1A32}, and {@code fnv1a32} all resolve to the same canonical name. - */ + /// Returns the canonical form of a function name: underscores stripped, lower-cased. + /// + /// Both scalar functions and aggregate functions use this normalization so that + /// `fnv1a_32`, `FNV1A32`, and `fnv1a32` all resolve to the same canonical name. public static String canonicalize(String name) { return StringUtils.remove(name, '_').toLowerCase(Locale.ROOT); } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/ScalarFunctionUtils.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/ScalarFunctionUtils.java index f7fd1d5ba2d1..e7da35a76d48 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/ScalarFunctionUtils.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/ScalarFunctionUtils.java @@ -25,9 +25,7 @@ import org.apache.pinot.spi.annotations.ScalarFunction; -/** - * Helper methods for discovering and registering scalar functions. - */ +/// Helper methods for discovering and registering scalar functions. public final class ScalarFunctionUtils { private ScalarFunctionUtils() { } @@ -55,11 +53,9 @@ public static List getScalarFunctionNames(ScalarFunction scalarFunction, return List.copyOf(canonicalNames); } - /** - * Returns the canonical form of a function name: underscores stripped, lower-cased. - * - * @see FunctionNameUtils#canonicalize(String) - */ + /// Returns the canonical form of a function name: underscores stripped, lower-cased. + /// + /// @see FunctionNameUtils#canonicalize(String) public static String canonicalize(String name) { return FunctionNameUtils.canonicalize(name); } 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 f3292d096060..29dd9afa918d 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 @@ -28,6 +28,7 @@ import org.testng.annotations.Test; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.expectThrows; @@ -107,8 +108,10 @@ public void testSegmentPartitionConfigWithFunctionExpr() SegmentPartitionConfig actualPartitionConfig = actualIndexingConfig.getSegmentPartitionConfig(); assertEquals(actualPartitionConfig.getNumPartitions("raw_key"), 64); - assertEquals(actualPartitionConfig.getFunctionExpr("raw_key"), "fnv1a_32(md5(raw_key))"); - assertEquals(actualPartitionConfig.getPartitionIdNormalizer("raw_key"), "MASK"); + ColumnPartitionConfig actualColumnPartitionConfig = actualPartitionConfig.getColumnPartitionConfig("raw_key"); + assertNotNull(actualColumnPartitionConfig); + assertEquals(actualColumnPartitionConfig.getFunctionExpr(), "fnv1a_32(md5(raw_key))"); + assertEquals(actualColumnPartitionConfig.getPartitionIdNormalizer(), "MASK"); assertNull(actualPartitionConfig.getFunctionName("raw_key")); } @@ -126,8 +129,10 @@ public void testSegmentPartitionConfigWithFunctionExprAbsNormalizer() SegmentPartitionConfig actualPartitionConfig = actualIndexingConfig.getSegmentPartitionConfig(); assertEquals(actualPartitionConfig.getNumPartitions("raw_key"), 64); - assertEquals(actualPartitionConfig.getFunctionExpr("raw_key"), "murmur2(raw_key)"); - assertEquals(actualPartitionConfig.getPartitionIdNormalizer("raw_key"), "ABS"); + ColumnPartitionConfig actualColumnPartitionConfig = actualPartitionConfig.getColumnPartitionConfig("raw_key"); + assertNotNull(actualColumnPartitionConfig); + assertEquals(actualColumnPartitionConfig.getFunctionExpr(), "murmur2(raw_key)"); + assertEquals(actualColumnPartitionConfig.getPartitionIdNormalizer(), "ABS"); assertNull(actualPartitionConfig.getFunctionName("raw_key")); } From 29451401aaebecd648c162011c5f4893b93e8498 Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Fri, 8 May 2026 12:11:29 -0700 Subject: [PATCH 26/27] Drop unrelated rebase drift: restore AWS SDK 2.44.4 and PrometheusTemplateRegexpTest MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Two pre-existing master changes were accidentally reverted by an earlier rebase resolution and never re-restored: 1. pom.xml: aws.sdk.version had drifted back to 2.44.3 vs master's 2.44.4 (upstream PR #18445). Restore the master version so this branch does not silently downgrade the AWS SDK. 2. pinot-common/src/test/.../PrometheusTemplateRegexpTest.java was accidentally deleted (302-line test file unrelated to this PR's scope — partition function expressions). Restore from master so we don't drop unrelated test coverage. Both are scope-hygiene fixes (per C8.4) and have no behavioral impact on the chained partition function feature. Co-Authored-By: Claude Opus 4.7 (1M context) --- .../PrometheusTemplateRegexpTest.java | 302 ++++++++++++++++++ pom.xml | 2 +- 2 files changed, 303 insertions(+), 1 deletion(-) create mode 100644 pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/PrometheusTemplateRegexpTest.java diff --git a/pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/PrometheusTemplateRegexpTest.java b/pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/PrometheusTemplateRegexpTest.java new file mode 100644 index 000000000000..c7ad6eff399c --- /dev/null +++ b/pinot-common/src/test/java/org/apache/pinot/common/metrics/prometheus/PrometheusTemplateRegexpTest.java @@ -0,0 +1,302 @@ +/** + * 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.common.metrics.prometheus; + +import java.io.FileReader; +import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.regex.PatternSyntaxException; +import java.util.stream.Collectors; +import org.testng.Assert; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; +import org.yaml.snakeyaml.Yaml; + + +/** + * Verifies that the Prometheus JMX template regexp patterns defined in the docker config YAML files + * are valid Java regexps and match expected JMX metric name strings with correct capture groups. + * + * Config files under test: docker/images/pinot/etc/jmx_prometheus_javaagent/configs/ + * + * @see Issue #13588 + */ +public class PrometheusTemplateRegexpTest { + + private static final String CONFIG_BASE_PATH = + "../docker/images/pinot/etc/jmx_prometheus_javaagent/configs"; + + @DataProvider(name = "configFiles") + public Object[][] configFiles() { + return new Object[][]{ + {"broker.yml"}, + {"server.yml"}, + {"controller.yml"}, + {"minion.yml"}, + {"pinot.yml"} + }; + } + + /** + * Verifies every pattern in each YAML config file compiles as a valid Java regexp. + */ + @Test(dataProvider = "configFiles") + public void testAllPatternsAreValidRegexp(String configFile) + throws Exception { + List patterns = extractPatterns(CONFIG_BASE_PATH + "/" + configFile); + Assert.assertFalse(patterns.isEmpty(), + "Expected at least one rule pattern in " + configFile); + for (String patternStr : patterns) { + try { + Pattern.compile(patternStr); + } catch (PatternSyntaxException e) { + Assert.fail( + "Invalid regexp in " + configFile + ": [" + patternStr + "] - " + e.getDescription()); + } + } + } + + // ---- Broker patterns ---- + + /** + * broker.yml: meters/timers scoped to tableNameWithType. + * e.g. pinot.broker.myTable_REALTIME.queries + */ + @Test + public void testBrokerTableWithTypeMeterPattern() + throws Exception { + String pattern = loadPatternByName("broker.yml", "pinot_$1_$6_$7"); + Matcher m = Pattern.compile(pattern).matcher( + "\"org.apache.pinot.common.metrics\"<>Count"); + Assert.assertTrue(m.matches(), "Pattern should match broker table-scoped meter"); + Assert.assertEquals(m.group(1), "broker"); + Assert.assertEquals(m.group(4), "myTable"); + Assert.assertEquals(m.group(5), "REALTIME"); + Assert.assertEquals(m.group(6), "queries"); + Assert.assertEquals(m.group(7), "Count"); + } + + /** + * broker.yml: meters/timers scoped to tableNameWithType with database prefix. + * e.g. pinot.broker.myDb.myTable_OFFLINE.queries + */ + @Test + public void testBrokerTableWithTypeMeterPatternWithDatabase() + throws Exception { + String pattern = loadPatternByName("broker.yml", "pinot_$1_$6_$7"); + Matcher m = Pattern.compile(pattern).matcher( + "\"org.apache.pinot.common.metrics\"<>Count"); + Assert.assertTrue(m.matches(), "Pattern should match broker table-scoped meter with database prefix"); + Assert.assertEquals(m.group(1), "broker"); + Assert.assertEquals(m.group(3), "myDb"); + Assert.assertEquals(m.group(4), "myTable"); + Assert.assertEquals(m.group(5), "OFFLINE"); + Assert.assertEquals(m.group(6), "queries"); + Assert.assertEquals(m.group(7), "Count"); + } + + /** + * broker.yml: meters/timers scoped to rawTableName. + * e.g. pinot.broker.myTable.queries + */ + @Test + public void testBrokerRawTableNameMeterPattern() + throws Exception { + String pattern = loadPatternByName("broker.yml", "pinot_$1_$5_$6"); + Matcher m = Pattern.compile(pattern).matcher( + "\"org.apache.pinot.common.metrics\"<>Count"); + Assert.assertTrue(m.matches(), "Pattern should match broker raw-table-name meter"); + Assert.assertEquals(m.group(1), "broker"); + Assert.assertEquals(m.group(4), "myTable"); + Assert.assertEquals(m.group(5), "queries"); + Assert.assertEquals(m.group(6), "Count"); + } + + /** + * broker.yml: global gauge/meter/timer (no table scope). + * e.g. pinot.broker.totalDocuments + */ + @Test + public void testBrokerGlobalMeterPattern() + throws Exception { + String pattern = loadPatternByName("broker.yml", "pinot_broker_$1_$2"); + Matcher m = Pattern.compile(pattern).matcher( + "\"org.apache.pinot.common.metrics\"<>Value"); + Assert.assertTrue(m.matches(), "Pattern should match global broker gauge"); + Assert.assertEquals(m.group(1), "totalDocuments"); + Assert.assertEquals(m.group(2), "Value"); + } + + // ---- Server patterns ---- + + /** + * server.yml: meters/timers scoped to tableNameWithType. + * e.g. pinot.server.myTable_OFFLINE.segmentUploadFailure + */ + @Test + public void testServerTableWithTypeMeterPattern() + throws Exception { + String pattern = loadPatternByName("server.yml", "pinot_server_$5_$6"); + Matcher m = Pattern.compile(pattern).matcher( + "\"org.apache.pinot.common.metrics\"<>Count"); + Assert.assertTrue(m.matches(), "Pattern should match server table-scoped meter"); + Assert.assertEquals(m.group(3), "myTable"); + Assert.assertEquals(m.group(4), "OFFLINE"); + Assert.assertEquals(m.group(5), "segmentUploadFailure"); + Assert.assertEquals(m.group(6), "Count"); + } + + /** + * server.yml: gauge scoped to tableNameWithType with partition. + * e.g. pinot.server.queries.myTable_REALTIME.3 + */ + @Test + public void testServerTableWithTypeAndPartitionGaugePattern() + throws Exception { + String pattern = loadPatternByName("server.yml", "pinot_server_$1_$7"); + Matcher m = Pattern.compile(pattern).matcher( + "\"org.apache.pinot.common.metrics\"<>Value"); + Assert.assertTrue(m.matches(), "Pattern should match server table-scoped gauge with partition"); + Assert.assertEquals(m.group(1), "queries"); + Assert.assertEquals(m.group(4), "myTable"); + Assert.assertEquals(m.group(5), "REALTIME"); + Assert.assertEquals(m.group(6), "3"); + Assert.assertEquals(m.group(7), "Value"); + } + + // ---- Controller patterns ---- + + /** + * controller.yml: minion task-type gauge. + * e.g. pinot.controller.numMinionTasksInProgress.SegmentGenerationAndPush + */ + @Test + public void testControllerTaskTypeGaugePattern() + throws Exception { + String pattern = loadPatternByName("controller.yml", "pinot_controller_$1_$3"); + Matcher m = Pattern.compile(pattern).matcher( + "\"org.apache.pinot.common.metrics\"<>Value"); + Assert.assertTrue(m.matches(), "Pattern should match controller task-type gauge"); + Assert.assertEquals(m.group(1), "numMinionTasksInProgress"); + Assert.assertEquals(m.group(2), "SegmentGenerationAndPush"); + Assert.assertEquals(m.group(3), "Value"); + } + + /** + * controller.yml: meters/timers scoped to tableNameWithType. + * e.g. pinot.controller.myTable_OFFLINE.segmentUploadFailure + */ + @Test + public void testControllerTableWithTypeMeterPattern() + throws Exception { + String pattern = loadPatternByName("controller.yml", "pinot_$1_$6_$7"); + Matcher m = Pattern.compile(pattern).matcher( + "\"org.apache.pinot.common.metrics\"<>Count"); + Assert.assertTrue(m.matches(), "Pattern should match controller table-scoped meter"); + Assert.assertEquals(m.group(1), "controller"); + Assert.assertEquals(m.group(4), "myTable"); + Assert.assertEquals(m.group(5), "OFFLINE"); + Assert.assertEquals(m.group(6), "segmentUploadFailure"); + Assert.assertEquals(m.group(7), "Count"); + } + + // ---- Minion patterns ---- + + /** + * minion.yml: meters/timers scoped to tableNameWithType and taskType. + * e.g. pinot.minion.myTable_REALTIME.SegmentGenerationAndPush.segmentUploadFailure + */ + @Test + public void testMinionTableWithTypeAndTaskTypeMeterPattern() + throws Exception { + String pattern = loadPatternByName("minion.yml", "pinot_minion_$6_$7"); + Matcher m = Pattern.compile(pattern).matcher( + "\"org.apache.pinot.common.metrics\"<>Count"); + Assert.assertTrue(m.matches(), "Pattern should match minion table + taskType scoped meter"); + Assert.assertEquals(m.group(3), "myTable"); + Assert.assertEquals(m.group(4), "REALTIME"); + Assert.assertEquals(m.group(5), "SegmentGenerationAndPush"); + Assert.assertEquals(m.group(6), "segmentUploadFailure"); + Assert.assertEquals(m.group(7), "Count"); + } + + /** + * minion.yml: meters/timers accepting either rawTableName or tableNameWithType. + * e.g. pinot.minion.myTable.queries + */ + @Test + public void testMinionTableOrIdScopedMeterPattern() + throws Exception { + String pattern = loadPatternByName("minion.yml", "pinot_minion_$2_$3"); + Matcher m = Pattern.compile(pattern).matcher( + "\"org.apache.pinot.common.metrics\"<>Value"); + Assert.assertTrue(m.matches(), "Pattern should match minion table/id scoped meter"); + Assert.assertEquals(m.group(1), "myTable"); + Assert.assertEquals(m.group(2), "numberOfSegmentsQueued"); + Assert.assertEquals(m.group(3), "Value"); + } + + /** + * Returns the pattern string for the rule whose {@code name} field equals {@code ruleName}. + * Keying off the rule name survives YAML rule reorderings — inserting or moving a rule in + * the config file will not silently shift the index and cause this test to assert against + * the wrong pattern. + */ + @SuppressWarnings("unchecked") + private String loadPatternByName(String configFile, String ruleName) + throws Exception { + Yaml yaml = new Yaml(); + try (FileReader reader = new FileReader(CONFIG_BASE_PATH + "/" + configFile)) { + Map config = yaml.load(reader); + List> rules = (List>) config.get("rules"); + return rules.stream() + .filter(rule -> ruleName.equals(rule.get("name"))) + .map(rule -> (String) rule.get("pattern")) + .findFirst() + .orElseThrow(() -> new IllegalArgumentException( + "No rule with name '" + ruleName + "' found in " + configFile)); + } + } + + @SuppressWarnings("unchecked") + private List extractPatterns(String filePath) + throws Exception { + Yaml yaml = new Yaml(); + try (FileReader reader = new FileReader(filePath)) { + Map config = yaml.load(reader); + List> rules = (List>) config.get("rules"); + return rules.stream() + .filter(rule -> rule.containsKey("pattern")) + .map(rule -> (String) rule.get("pattern")) + .collect(Collectors.toList()); + } + } +} diff --git a/pom.xml b/pom.xml index a4b26f005915..a51aeb6c18ec 100644 --- a/pom.xml +++ b/pom.xml @@ -192,7 +192,7 @@ 0.15.1 0.4.7 4.3.0 - 2.44.3 + 2.44.4 1.3.6 1.24.1 2.14.2 From f2421466bcb6dffd62adb6155eb7619c3b487062 Mon Sep 17 00:00:00 2001 From: Xiang Fu Date: Fri, 8 May 2026 12:19:04 -0700 Subject: [PATCH 27/27] Emit PARTITION_PRUNER_FAIL_OPEN metric on broker pruner exception Reviewer flagged that the broker pruner's RuntimeException catch logs at ERROR but otherwise silently degrades the table to scatter-gather, with no visible metric to surface the regression in dashboards/alerts. Add a new BrokerMeter PARTITION_PRUNER_FAIL_OPEN and increment it from both SinglePartitionColumnSegmentPruner and MultiPartitionColumnsSegmentPruner when partition function evaluation throws. The metric is keyed by tableNameWithType so operators can spot which table's expression is buggy without trawling logs. Co-Authored-By: Claude Opus 4.7 (1M context) --- .../MultiPartitionColumnsSegmentPruner.java | 6 ++++++ .../SinglePartitionColumnSegmentPruner.java | 9 ++++++++- .../org/apache/pinot/common/metrics/BrokerMeter.java | 8 ++++++++ 3 files changed, 22 insertions(+), 1 deletion(-) diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/MultiPartitionColumnsSegmentPruner.java b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/MultiPartitionColumnsSegmentPruner.java index ec1c0e37b999..6475991ef199 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/MultiPartitionColumnsSegmentPruner.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/MultiPartitionColumnsSegmentPruner.java @@ -30,6 +30,8 @@ import org.apache.helix.zookeeper.datamodel.ZNRecord; import org.apache.pinot.broker.routing.segmentpartition.SegmentPartitionInfo; import org.apache.pinot.broker.routing.segmentpartition.SegmentPartitionUtils; +import org.apache.pinot.common.metrics.BrokerMeter; +import org.apache.pinot.common.metrics.BrokerMetrics; import org.apache.pinot.common.request.BrokerRequest; import org.apache.pinot.common.request.Expression; import org.apache.pinot.common.request.Function; @@ -177,6 +179,10 @@ private boolean isPartitionMatch(SegmentPartitionInfo partitionInfo, String valu try { return partitionInfo.getPartitions().contains(partitionInfo.getPartitionFunction().getPartition(value)); } catch (RuntimeException e) { + BrokerMetrics brokerMetrics = BrokerMetrics.get(); + if (brokerMetrics != null) { + brokerMetrics.addMeteredTableValue(_tableNameWithType, BrokerMeter.PARTITION_PRUNER_FAIL_OPEN, 1); + } // Fail-open: a buggy partition function/expression must not drop user query results. Log at ERROR (not WARN) // so this surfaces in alerting — silent fail-open hides table-config bugs that should be fixed. LOGGER.error("Failed to evaluate partition function for table: {}, partition column: {}; falling back to " diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/SinglePartitionColumnSegmentPruner.java b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/SinglePartitionColumnSegmentPruner.java index 818fc6e6e35a..0b85f4b00b10 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/SinglePartitionColumnSegmentPruner.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpruner/SinglePartitionColumnSegmentPruner.java @@ -29,6 +29,8 @@ import org.apache.helix.zookeeper.datamodel.ZNRecord; import org.apache.pinot.broker.routing.segmentpartition.SegmentPartitionInfo; import org.apache.pinot.broker.routing.segmentpartition.SegmentPartitionUtils; +import org.apache.pinot.common.metrics.BrokerMeter; +import org.apache.pinot.common.metrics.BrokerMetrics; import org.apache.pinot.common.request.BrokerRequest; import org.apache.pinot.common.request.Expression; import org.apache.pinot.common.request.Function; @@ -162,10 +164,15 @@ private boolean isPartitionMatch(SegmentPartitionInfo partitionInfo, String valu return partitionInfo.getPartitions().contains(partitionInfo.getPartitionFunction().getPartition(value)); } catch (RuntimeException e) { // Fail-open: a buggy partition function/expression must not drop user query results. Log at ERROR (not WARN) - // so this surfaces in alerting — silent fail-open hides table-config bugs that should be fixed. + // and emit a meter so this surfaces in alerting — silent fail-open hides table-config bugs that should be + // fixed. LOGGER.error("Failed to evaluate partition function for table: {}, partition column: {}; falling back to " + "scatter-gather (no pruning) for this query. Fix the partition expression to avoid query-time fail-open.", _tableNameWithType, _partitionColumn, e); + BrokerMetrics brokerMetrics = BrokerMetrics.get(); + if (brokerMetrics != null) { + brokerMetrics.addMeteredTableValue(_tableNameWithType, BrokerMeter.PARTITION_PRUNER_FAIL_OPEN, 1); + } return true; } } diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metrics/BrokerMeter.java b/pinot-common/src/main/java/org/apache/pinot/common/metrics/BrokerMeter.java index 3970452d1e47..a948ccf75cd8 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/metrics/BrokerMeter.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/metrics/BrokerMeter.java @@ -253,6 +253,14 @@ public class BrokerMeter implements AbstractMetrics.Meter { public static final BrokerMeter INVALID_SEGMENT_PARTITION_IN_QUERY = create("INVALID_SEGMENT_PARTITION_IN_QUERY", "queries", false); + /** + * Number of times a broker partition pruner caught a {@link RuntimeException} from + * {@code PartitionFunction.getPartition(value)} and fell back to scatter-gather (no pruning) for the query. + * Surfaces silently degraded routing — if non-zero for a table, the partition expression should be reviewed. + */ + public static final BrokerMeter PARTITION_PRUNER_FAIL_OPEN = create("PARTITION_PRUNER_FAIL_OPEN", + "queries", false); + /** * Number of queries executed with cursors. This count includes queries that use SSE and MSE */