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..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 @@ -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,9 @@ 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.data.FieldSpec; import org.apache.pinot.spi.utils.CommonConstants.Helix.StateModel.SegmentStateModel; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -61,7 +65,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 +75,33 @@ 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 [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) { + this(tableNameWithType, partitionColumn, + new ColumnPartitionConfig(partitionFunctionName, numPartitions), null); + } + + public SegmentPartitionMetadataManager(String tableNameWithType, String partitionColumn, + ColumnPartitionConfig columnPartitionConfig) { + this(tableNameWithType, partitionColumn, columnPartitionConfig, null); + } + + /// 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; _partitionColumn = partitionColumn; - _partitionFunctionName = partitionFunctionName; - _numPartitions = numPartitions; + _partitionFunction = fieldSpec != null + ? PartitionFunctionFactory.getPartitionFunction(partitionColumn, columnPartitionConfig, fieldSpec) + : PartitionFunctionFactory.getPartitionFunction(partitionColumn, columnPartitionConfig); + _numPartitions = _partitionFunction.getNumPartitions(); } @Override @@ -103,7 +128,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 +141,41 @@ 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) { + Map functionConfig = partitionFunction.getFunctionConfig(); + if (functionConfig != null && !functionConfig.isEmpty() + && !Objects.equals(_partitionFunction.getFunctionConfig(), functionConfig)) { + return false; + } + + String partitionIdNormalizer = partitionFunction.getPartitionIdNormalizer(); + if (partitionIdNormalizer != null && !partitionIdNormalizer.isEmpty() + && !equalsIgnoreCase(_partitionFunction.getPartitionIdNormalizer(), partitionIdNormalizer)) { + return false; + } + + String functionName = partitionFunction.getName(); + if (functionName != null) { + String configuredFunctionName = _partitionFunction.getName(); + return configuredFunctionName != null && configuredFunctionName.equalsIgnoreCase(functionName); + } + + 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) { if (znRecord == null) { return INVALID_CREATION_TIME_MS; @@ -306,7 +366,7 @@ private void computeTablePartitionReplicatedServersInfo() { } } _tablePartitionReplicatedServersInfo = - new TablePartitionReplicatedServersInfo(_tableNameWithType, _partitionColumn, _partitionFunctionName, + new TablePartitionReplicatedServersInfo(_tableNameWithType, _partitionColumn, getPartitionFunctionDescription(), _numPartitions, partitionInfoMap, segmentsWithInvalidPartition); } @@ -337,7 +397,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..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,12 +30,16 @@ 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; 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 +47,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 +146,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 +161,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 +174,21 @@ 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) { + 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 " + + "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 7016484e48c4..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,12 +29,16 @@ 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; 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 +46,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 +135,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 +145,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 +158,22 @@ 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) { + // Fail-open: a buggy partition function/expression must not drop user query results. Log at ERROR (not WARN) + // 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-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..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 @@ -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,169 @@ 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()); + } + + @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); + 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 through the FunctionExpr sentinel"); + 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); + } + + 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) { + 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, + functionConfig, 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..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 @@ -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,68 @@ 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. - *

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
  • - *
- */ -public class InbuiltFunctionEvaluator implements FunctionEvaluator { - // Root of the execution tree - private final ExecutableNode _rootNode; - private final List _arguments; - private final String _functionExpression; +/// 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) { - _functionExpression = functionExpression; - _arguments = new ArrayList<>(); - _rootNode = planExecution(RequestContextUtils.getExpression(functionExpression)); + this(functionExpression, new ArrayList<>()); + } + + /// 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); } - 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,183 +107,42 @@ 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 [FunctionInvoker], with null propagation and + /// type conversion. Allocates a fresh argument scratch array per `execute` call, so multiple threads + /// and re-entrant invocations of the same node instance do not share mutable state. private static class FunctionExecutionNode implements ExecutableNode { final FunctionInvoker _functionInvoker; final FunctionInfo _functionInfo; final ExecutableNode[] _argumentNodes; - final Object[] _arguments; FunctionExecutionNode(FunctionInfo functionInfo, ExecutableNode[] argumentNodes) { _functionInvoker = new FunctionInvoker(functionInfo); _functionInfo = functionInfo; _argumentNodes = argumentNodes; - _arguments = new Object[_argumentNodes.length]; } @Override public Object execute(GenericRow row) { try { int numArguments = _argumentNodes.length; + Object[] arguments = new Object[numArguments]; 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); } @@ -292,23 +152,24 @@ public Object execute(GenericRow row) { public Object execute(Object[] values) { try { int numArguments = _argumentNodes.length; + Object[] arguments = new Object[numArguments]; 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); } @@ -319,75 +180,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/evaluator/InbuiltPartitionEvaluatorFactory.java b/pinot-common/src/main/java/org/apache/pinot/common/evaluator/InbuiltPartitionEvaluatorFactory.java new file mode 100644 index 000000000000..c0c7239a7510 --- /dev/null +++ b/pinot-common/src/main/java/org/apache/pinot/common/evaluator/InbuiltPartitionEvaluatorFactory.java @@ -0,0 +1,91 @@ +/** + * 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; + + +/// [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 + 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..bac3fe1efadd --- /dev/null +++ b/pinot-common/src/main/java/org/apache/pinot/common/evaluator/PartitionFunctionEvaluator.java @@ -0,0 +1,214 @@ +/** + * 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 [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: `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); + } + + 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 [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; + private final ExecutableNode[] _argumentNodes; + + PartitionFunctionExecutionNode(FunctionInfo functionInfo, ExecutableNode[] argumentNodes) { + _functionInvoker = new FunctionInvoker(functionInfo); + _functionInfo = functionInfo; + _argumentNodes = argumentNodes; + } + + @Override + public Object execute(GenericRow row) { + try { + int numArguments = _argumentNodes.length; + Object[] arguments = new Object[numArguments]; + 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 { + int numArguments = _argumentNodes.length; + Object[] arguments = new Object[numArguments]; + 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 `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(); + 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(argument); + 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/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..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 @@ -29,6 +29,7 @@ import org.apache.calcite.sql.type.SqlTypeName; 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 { @@ -105,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) { @@ -216,4 +217,11 @@ public static boolean isAssertEnabled() { //CHECKSTYLE:ON return assertEnabled; } + + /// 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 ada184185d5c..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,23 +108,29 @@ 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 + // "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); } - /** - * Computes 32-bit MurmurHash2 of the given string. - * - * @param input the byte array to hash - * @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)); @@ -163,6 +169,12 @@ 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,17 +236,16 @@ public static byte[] murmurHash3X64Bit128(byte[] input, int seed) { * @param input the byte array to hash * @return 32-bit hash */ - @ScalarFunction + // "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); } - /** - * Computes 32-bit FNV-1 hash of the given string using UTF-8 bytes. - * - * @param input the string to hash - * @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)); @@ -246,7 +257,8 @@ public static int fnv1Hash32UTF8(String input) { * @param input the byte array to hash * @return 32-bit hash */ - @ScalarFunction + // "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); } @@ -268,7 +280,8 @@ public static int fnv1aHash32UTF8(String input) { * @param input the byte array to hash * @return 64-bit hash */ - @ScalarFunction + // "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); } @@ -290,7 +303,8 @@ public static long fnv1Hash64UTF8(String input) { * @param input the byte array to hash * @return 64-bit hash */ - @ScalarFunction + // "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/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/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 */ diff --git a/pinot-common/src/main/resources/META-INF/services/org.apache.pinot.segment.spi.partition.pipeline.PartitionEvaluatorFactory b/pinot-common/src/main/resources/META-INF/services/org.apache.pinot.segment.spi.partition.pipeline.PartitionEvaluatorFactory new file mode 100644 index 000000000000..d243ff1abe1f --- /dev/null +++ b/pinot-common/src/main/resources/META-INF/services/org.apache.pinot.segment.spi.partition.pipeline.PartitionEvaluatorFactory @@ -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.evaluator.InbuiltPartitionEvaluatorFactory 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..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 @@ -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,32 @@ 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 `@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 + // 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]); + } + } } 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..8f722539883e --- /dev/null +++ b/pinot-common/src/test/java/org/apache/pinot/common/function/scalar/PartitionFunctionExprCommonScalarFunctionTest.java @@ -0,0 +1,146 @@ +/** + * 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.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.assertTrue; +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"); + } + + /// 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 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^54 — clearly past the boundary + IllegalStateException error = expectThrows(IllegalStateException.class, + () -> 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 `< 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" + /// 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); + 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/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/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 new file mode 100644 index 000000000000..90568c58c320 --- /dev/null +++ b/pinot-common/src/test/java/org/apache/pinot/segment/spi/function/scalar/PartitionFunctionExprTestFunctions.java @@ -0,0 +1,55 @@ +/** + * 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.math.BigDecimal; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.pinot.spi.annotations.ScalarFunction; + + +/// 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() { + } + + @ScalarFunction + public static int identity(int value) { + return value; + } + + @ScalarFunction + 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-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..fb5e38506f76 --- /dev/null +++ b/pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionExprIntegrationTest.java @@ -0,0 +1,63 @@ +/** + * 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 `pinot-common` on the classpath +/// (for [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-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 new file mode 100644 index 000000000000..793b3d806c73 --- /dev/null +++ b/pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/metadata/ColumnPartitionMetadataTest.java @@ -0,0 +1,112 @@ +/** + * 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.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; + +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.getFunctionName(), metadata.getFunctionName()); + assertEquals(roundTripped.getNumPartitions(), metadata.getNumPartitions()); + assertEquals(roundTripped.getPartitions(), metadata.getPartitions()); + assertNull(roundTripped.getFunctionExpr()); + assertNull(roundTripped.getPartitionIdNormalizer()); + assertEquals(roundTripped, metadata); + } + + @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.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()); + 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-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 new file mode 100644 index 000000000000..16cf0dccc300 --- /dev/null +++ b/pinot-common/src/test/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompilerTest.java @@ -0,0 +1,223 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pinot.segment.spi.partition.pipeline; + +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.List; +import java.util.function.ToIntFunction; +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 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 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))"); + 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(), null); + 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)")); + assertTrue(error.getMessage().contains("must reference exactly that column"), + "Unexpected error: " + error.getMessage()); + } + + @Test + public void testRejectsNonIntPartitionFunctionOutput() { + // 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 + 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"); + } + + 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-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..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 @@ -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; @@ -120,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; @@ -1145,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 @@ -1168,10 +1173,18 @@ SegmentPartitionMetadata getPartitionMetadataFromTableConfig(TableConfig tableCo + "The stream partition count is used. Please update the table config accordingly.", perStreamNumPartitions, columnPartitionConfig.getNumPartitions()); } + 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(columnPartitionConfig.getFunctionName(), perStreamNumPartitions, - Collections.singleton(streamPartitionId), columnPartitionConfig.getFunctionConfig()); - return new SegmentPartitionMetadata(Collections.singletonMap(entry.getKey(), columnPartitionMetadata)); + new ColumnPartitionMetadata(partitionFunction, Collections.singleton(streamPartitionId)); + 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-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..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 @@ -104,6 +104,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.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; @@ -1571,6 +1573,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 +1581,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(); @@ -1792,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(); @@ -1800,13 +1813,44 @@ 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()) { - LOGGER.debug("tableNameWithType: {}, columnName: {},, segmentName: {}, change: num partitions", + 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); + } + // 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); + } + // 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); 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..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 @@ -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 @@ -2045,9 +2044,12 @@ 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(partitionFunctionName, numPartitions, - columnPartitionConfig.getFunctionConfig())); + PartitionFunctionFactory.getPartitionFunction(partitionColumn, columnPartitionConfig, numPartitions, + _schema.getFieldSpecFor(partitionColumn))); 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..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 @@ -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,12 @@ 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 +86,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 +100,18 @@ 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..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 @@ -18,10 +18,13 @@ */ 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.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; import org.apache.pinot.spi.data.readers.GenericRow; @@ -29,17 +32,31 @@ * 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. + 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)))); + return computePartition(genericRow.getValue(_column)); } @Override @@ -53,6 +70,22 @@ 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]))); + 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(partitionId); } } 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-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..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 @@ -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,93 @@ 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 +284,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..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 @@ -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 @@ -438,9 +437,12 @@ 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(partitionFunctionName, numPartitions, - columnPartitionConfig.getFunctionConfig())); + PartitionFunctionFactory.getPartitionFunction(partitionColumn, columnPartitionConfig, numPartitions, + _schema.getFieldSpecFor(partitionColumn))); 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..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; @@ -617,27 +619,27 @@ public static void addColumnMetadataInfo(PropertiesConfiguration properties, Str properties.setProperty(getKeyFor(column, IS_AUTO_GENERATED), "true"); } - // Min/max value - if (fieldSpec.getFieldType() != FieldType.COMPLEX) { - // Regular (non-complex) field - if (totalDocs > 0) { - Object min = columnStatistics.getMinValue(); - Object max = columnStatistics.getMaxValue(); - // NOTE: - // Min/max could be null for real-time aggregate metrics. We don't directly call addColumnMinMaxValueInfo() to - // avoid setting MIN_MAX_VALUE_INVALID flag, which will prevent ColumnMinMaxValueGenerator from generating them - // when loading the segment. - if (min != null && max != null) { - addColumnMinMaxValueInfo(properties, column, min, max, storedType); - } - } - } - - // Partition function PartitionFunction partitionFunction = columnStatistics.getPartitionFunction(); if (partitionFunction != null) { + // 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()); + } + if (partitionFunction.getPartitionIdNormalizer() != null) { + 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) { @@ -647,6 +649,29 @@ public static void addColumnMetadataInfo(PropertiesConfiguration properties, Str } } } + + FieldType fieldType = fieldSpec.getFieldType(); + // 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(); + Object max = columnStatistics.getMaxValue(); + // NOTE: + // Min/max could be null for real-time aggregate metrics. We don't directly call addColumnMinMaxValueInfo() to + // avoid setting MIN_MAX_VALUE_INVALID flag, which will prevent ColumnMinMaxValueGenerator from generating them + // when loading the segment. + if (min != null && max != null) { + addColumnMinMaxValueInfo(properties, column, min, max, storedType); + } + } + } } /** 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..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 @@ -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,6 +163,18 @@ 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); + } + } + + /// 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) { + _partitions.add(partitionId); + } } } 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..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,6 +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; @@ -1568,6 +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); + PartitionFunction partitionFunction = + PartitionFunctionFactory.getPartitionFunction(column, columnPartitionMap.get(column), fieldSpec); + if (partitionFunction instanceof PartitionPipelineFunction) { + ((PartitionPipelineFunction) partitionFunction).validateOutputType(); + } } } } 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..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 @@ -173,9 +173,26 @@ 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"; + /// 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. `"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 // Replaced by LENGTH_OF_LONGEST_ELEMENT 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..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 @@ -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; @@ -94,22 +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) { + ColumnPartitionConfig config = getColumnPartitionConfig(column); + return config != null ? config.getFunctionExpr() : null; } - @Deprecated @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) { + 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 new file mode 100644 index 000000000000..30537a7630c8 --- /dev/null +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/function/ExecutableFunctionEvaluator.java @@ -0,0 +1,298 @@ +/** + * 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 +/// [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. + 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. + /// + /// 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; + private final ExecutableNode[] _argumentNodes; + + public FunctionNode(String functionName, Invoker invoker, ExecutableNode[] argumentNodes) { + _functionName = functionName; + _invoker = invoker; + _argumentNodes = argumentNodes; + } + + @Override + public Object execute(GenericRow row) { + Object[] arguments = new Object[_argumentNodes.length]; + 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 = new Object[_argumentNodes.length]; + 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..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 @@ -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,26 @@ 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) { + // 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 { + 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..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 @@ -18,29 +18,56 @@ */ 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. - * That is, two invocations of {@code PartitionFunction.getPartition(value)} - * with the same value are expected to produce the same result. - */ +/// 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. - * - * @param value Value for which to determine the partition id. - * @return partition id for the value. - */ + /// 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 [#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)); + } + /** * Returns the name of the partition function. * @return Name of the partition function. @@ -57,4 +84,29 @@ public interface PartitionFunction extends Serializable { default Map getFunctionConfig() { return null; } + + @JsonIgnore + @Nullable + default String getPartitionColumn() { + return null; + } + + @JsonIgnore + @Nullable + 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() { + 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..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 @@ -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,199 @@ 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 [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"); + 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. + /// + /// @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"); + 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()); + } + + /// 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) { + 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 [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 `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 `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 + && fieldSpec.getDataType().getStoredType() == FieldSpec.DataType.BYTES) { + return PartitionFunctionExprCompiler.compilePartitionFunction(columnName, PartitionValueType.BYTES, + config.getFunctionExpr(), numPartitions, config.getPartitionIdNormalizer()); + } + return getPartitionFunction(columnName, config.getFunctionName(), numPartitions, config.getFunctionConfig(), + config.getFunctionExpr(), config.getPartitionIdNormalizer()); + } + + 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..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 @@ -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; @@ -34,6 +35,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,34 +53,96 @@ @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. `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. - */ + /// 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) { _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().isBytesInput() + ? PartitionValueType.BYTES.name() : null); + } + + @Nullable public String getFunctionName() { return _functionName; } + /// 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() { + 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 +158,19 @@ 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, _partitions, _functionConfig, _functionExpr, + _partitionIdNormalizer, _inputType); } /** @@ -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 : value; + } + /** * Custom deserializer for {@link ColumnPartitionMetadata}. *

@@ -150,6 +224,11 @@ 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"; + // 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 @@ -181,8 +260,27 @@ 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); + // 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), + numPartitionsNode.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/PartitionEvaluatorFactory.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionEvaluatorFactory.java new file mode 100644 index 000000000000..4e0b1ede409b --- /dev/null +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionEvaluatorFactory.java @@ -0,0 +1,43 @@ +/** + * 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 [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 [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 new file mode 100644 index 000000000000..51187cb781a3 --- /dev/null +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionFunctionExprCompiler.java @@ -0,0 +1,282 @@ +/** + * 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 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.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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/// 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 + // 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). + // + // 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. + // 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) + .expireAfterWrite(PIPELINE_CACHE_EXPIRE_HOURS, TimeUnit.HOURS) + .build(); + + private PartitionFunctionExprCompiler() { + } + + // Lazy holder: initialized on first use so that class-load of this utility does not fail when + // 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 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"); + // 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); + } + // 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().getName().equals(inbuiltFqn)) { + LOGGER.info("Found {} PartitionEvaluatorFactory implementations on the classpath: {}; preferring {}", + factories.size(), factories, f.getClass().getName()); + return f; + } + } + PartitionEvaluatorFactory chosen = factories.get(0); + LOGGER.info("Found {} PartitionEvaluatorFactory implementations on the classpath: {}; using first: {}", + factories.size(), factories, chosen.getClass().getName()); + return chosen; + } + } + + public static PartitionPipeline compile(String rawColumn, String functionExpr) { + return compile(rawColumn, PartitionValueType.STRING, functionExpr, null); + } + + 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 [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"); + Preconditions.checkArgument(inputType != null, "Input type 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); + + String canonicalExpr = canonicalize(functionExpr); + boolean isBytesInput = inputType == PartitionValueType.BYTES; + PipelineCacheKey cacheKey = new PipelineCacheKey(rawColumn, isBytesInput, canonicalExpr, partitionIdNormalizer); + 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, + 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 [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 + ? PartitionIntNormalizer.fromConfigString(partitionIdNormalizer) + : PartitionIntNormalizer.fromConfigString(ColumnPartitionConfig.PARTITION_ID_NORMALIZER_POSITIVE_MODULO); + PartitionPipeline pipeline = compile(rawColumn, inputType, functionExpr, normalizer); + return new PartitionPipelineFunction(pipeline, numPartitions); + } + + /// 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()); + 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(' '); + } + } + + 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); + } + } +} 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..f8524ddc0257 --- /dev/null +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionIntNormalizer.java @@ -0,0 +1,91 @@ +/** + * 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 `+ 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..2e093f561a6a --- /dev/null +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipeline.java @@ -0,0 +1,119 @@ +/** + * 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.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, 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; + private final String _canonicalFunctionExpr; + @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) { + Preconditions.checkNotNull(rawColumn, "Raw column must be configured"); + Preconditions.checkNotNull(canonicalFunctionExpr, "Canonical function expression must be configured"); + Preconditions.checkNotNull(evaluator, "Function evaluator must be configured"); + _rawColumn = rawColumn; + _isBytesInput = isBytesInput; + _canonicalFunctionExpr = canonicalFunctionExpr; + _intNormalizer = intNormalizer; + _evaluator = evaluator; + _arguments = Collections.singletonList(rawColumn); + } + + public String getRawColumn() { + return _rawColumn; + } + + public boolean isBytesInput() { + return _isBytesInput; + } + + public String getCanonicalFunctionExpr() { + return _canonicalFunctionExpr; + } + + @Nullable + public PartitionIntNormalizer getIntNormalizer() { + return _intNormalizer; + } + + @Override + public List getArguments() { + return _arguments; + } + + @Override + public Object evaluate(GenericRow genericRow) { + Object inputValue = genericRow.getValue(_rawColumn); + if (inputValue == null) { + return null; + } + 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))}); + } + // 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 + 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; + } + if (_isBytesInput) { + if (inputValue instanceof byte[]) { + return _evaluator.evaluate(new Object[]{inputValue}); + } + return _evaluator.evaluate(new Object[]{BytesUtils.toBytes( + FieldSpec.getStringValue(inputValue))}); + } + 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 new file mode 100644 index 000000000000..d6c959ecc731 --- /dev/null +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionPipelineFunction.java @@ -0,0 +1,299 @@ +/** + * 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.math.BigDecimal; +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; +import org.apache.pinot.spi.function.FunctionEvaluator; +import org.apache.pinot.spi.utils.BytesUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/// [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 [#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. + 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; + // 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"); + 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; + _intNormalizer = pipeline.getIntNormalizer(); + } + + 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 + /// [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 + // 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 anyNonNullProbeSucceeded = false; + RuntimeException lastFailure = null; + for (Object sample : samples) { + Object probe; + try { + probe = _pipeline.evaluate(new Object[]{sample}); + } catch (RuntimeException e) { + lastFailure = e; + continue; + } + if (sample != null) { + anyNonNullProbeSucceeded = true; + } + if (probe == null) { + continue; + } + 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 (!anyNonNullProbeSucceeded) { + throw new IllegalArgumentException( + "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")); + } + } + + 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 broker routing), convert the hex string back to raw bytes so the partition computation matches ingestion. + if (_pipeline.isBytesInput()) { + return getPartition(BytesUtils.toBytes(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 [PartitionValueType#BYTES] input type, avoiding the hex-encoding round-trip. + @Override + public int getPartition(byte[] bytes) { + if (!_pipeline.isBytesInput()) { + return getPartition(BytesUtils.toHexString(bytes)); + } + return normalizeResult(_pipeline.evaluate(new Object[]{bytes})); + } + + private int normalizeResult(Object result) { + 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()); + 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 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, + "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()); + } + // _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); + } + if (num instanceof BigDecimal) { + try { + return normalizeBigIntegerResult(((BigDecimal) num).toBigIntegerExact(), result); + } 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); + } + 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) { + 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 + public String getName() { + return NAME; + } + + @Override + public int getNumPartitions() { + return _numPartitions; + } + + @Override + @JsonIgnore + public String getPartitionColumn() { + return _pipeline.getRawColumn(); + } + + @Override + @JsonIgnore(false) // Override @JsonIgnore on PartitionFunction#getFunctionExpr to expose this in serialization. + @JsonProperty("functionExpr") + public String getFunctionExpr() { + return _pipeline.getCanonicalFunctionExpr(); + } + + @Override + @JsonIgnore(false) // Override @JsonIgnore on PartitionFunction#getPartitionIdNormalizer. + @JsonProperty("partitionIdNormalizer") + public String getPartitionIdNormalizer() { + PartitionIntNormalizer intNormalizer = _pipeline.getIntNormalizer(); + return intNormalizer != null ? intNormalizer.name() : null; + } + + // FunctionEvaluator implementation + + @Override + public List getArguments() { + return _pipeline.getArguments(); + } + + @Override + public Object evaluate(GenericRow genericRow) { + // 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; + } + + @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; + } + int partitionId = (inputValue instanceof byte[] && _pipeline.isBytesInput()) + ? getPartition((byte[]) inputValue) + : getPartition(FieldSpec.getStringValue(inputValue)); + return partitionId == NULL_RESULT_PARTITION_ID ? null : partitionId; + } + + @Override + public String toString() { + return NAME; + } +} 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..81927dcdad1e --- /dev/null +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/partition/pipeline/PartitionValueType.java @@ -0,0 +1,60 @@ +/** + * 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/partition/PartitionFunctionTest.java b/pinot-segment-spi/src/test/java/org/apache/pinot/segment/spi/partition/PartitionFunctionTest.java index aa4537a23ac4..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 @@ -20,8 +20,15 @@ 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; import org.apache.pinot.spi.utils.JsonUtils; import org.apache.pinot.spi.utils.hash.FnvHashFunctions; @@ -30,6 +37,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 +441,60 @@ 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); + } + private void testBasicProperties(PartitionFunction partitionFunction, String functionName, int numPartitions) { testBasicProperties(partitionFunction, functionName, numPartitions, null); } @@ -829,4 +891,36 @@ 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( + V1Constants.MetadataKeys.Column.PARTITION_FUNCTION_EXPR_SENTINEL)); + } + + /// 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<>(); + 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/ColumnPartitionConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ColumnPartitionConfig.java index 2e3518a4c9d2..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 @@ -21,27 +21,73 @@ 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; +/// 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"; + 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; + // 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; } @@ -51,10 +97,23 @@ 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 +132,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..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,6 +61,8 @@ public int getNumPartitions(String column) { return (config != null) ? config.getNumPartitions() : INVALID_NUM_PARTITIONS; } + // 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) { 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..c4b49f07aeb2 --- /dev/null +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/FunctionNameUtils.java @@ -0,0 +1,40 @@ +/** + * 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 `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 + /// `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 new file mode 100644 index 000000000000..e7da35a76d48 --- /dev/null +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/ScalarFunctionUtils.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.spi.utils; + +import java.lang.reflect.Method; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Set; +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 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 fe293525780c..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,8 +28,10 @@ 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; public class IndexingConfigTest { @@ -92,6 +94,62 @@ 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); + 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")); + } + + @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); + ColumnPartitionConfig actualColumnPartitionConfig = actualPartitionConfig.getColumnPartitionConfig("raw_key"); + assertNotNull(actualColumnPartitionConfig); + assertEquals(actualColumnPartitionConfig.getFunctionExpr(), "murmur2(raw_key)"); + assertEquals(actualColumnPartitionConfig.getPartitionIdNormalizer(), "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..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,11 +226,8 @@ private boolean isSegmentMatchPartition(SegmentGenerationJobSpec spec, String se } // Compute partition ID for the specified partition column value - int partitionId = PartitionFunctionFactory.getPartitionFunction( - columnMetadata.getFunctionName(), - columnMetadata.getNumPartitions(), - columnMetadata.getFunctionConfig() - ).getPartition(_partitionColumnValue); + int partitionId = PartitionFunctionFactory.getPartitionFunction(_partitionColumn, columnMetadata) + .getPartition(_partitionColumnValue); // Return true if segment contains the computed partition return columnMetadata.getPartitions().contains(partitionId); 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