noob-se7en commented on code in PR #17587:
URL: https://github.com/apache/pinot/pull/17587#discussion_r2751175185
##########
pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaStreamMetadataProvider.java:
##########
@@ -57,23 +63,47 @@ public class KafkaStreamMetadataProvider extends
KafkaPartitionLevelConnectionHa
implements StreamMetadataProvider {
private static final Logger LOGGER =
LoggerFactory.getLogger(KafkaStreamMetadataProvider.class);
+ private final boolean _partialPartitions;
+ private final List<Integer> _partitionIdSubset;
public KafkaStreamMetadataProvider(String clientId, StreamConfig
streamConfig) {
this(clientId, streamConfig, Integer.MIN_VALUE);
}
public KafkaStreamMetadataProvider(String clientId, StreamConfig
streamConfig, int partition) {
super(clientId, streamConfig, partition);
+ List<Integer> subset =
+
KafkaPartitionSubsetUtils.getPartitionIdsFromConfig(_config.getStreamConfigMap());
+ if (subset != null) {
+ _partialPartitions = true;
+ _partitionIdSubset = Collections.unmodifiableList(subset);
+ } else {
+ _partialPartitions = false;
+ _partitionIdSubset = Collections.emptyList();
+ }
}
@Override
public int fetchPartitionCount(long timeoutMillis) {
try {
List<PartitionInfo> partitionInfos = _consumer.partitionsFor(_topic,
Duration.ofMillis(timeoutMillis));
- if (CollectionUtils.isNotEmpty(partitionInfos)) {
- return partitionInfos.size();
+ if (CollectionUtils.isEmpty(partitionInfos)) {
+ throw new RuntimeException(String.format("Failed to fetch partition
information for topic: %s", _topic));
Review Comment:
I think we want to avoid String.format:
https://github.com/apache/pinot/issues/14404
##########
pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaStreamMetadataProvider.java:
##########
@@ -96,6 +139,55 @@ public Set<Integer> fetchPartitionIds(long timeoutMillis) {
}
}
+ @Override
+ public List<PartitionGroupMetadata> computePartitionGroupMetadata(String
clientId, StreamConfig streamConfig,
+ List<PartitionGroupConsumptionStatus> partitionGroupConsumptionStatuses,
int timeoutMillis)
+ throws IOException, java.util.concurrent.TimeoutException {
+ if (!_partialPartitions) {
+ return
StreamMetadataProvider.super.computePartitionGroupMetadata(clientId,
streamConfig,
+ partitionGroupConsumptionStatuses, timeoutMillis);
+ }
+ List<Integer> subset = _partitionIdSubset;
+ Set<Integer> topicIds = fetchPartitionIds(timeoutMillis);
+ Map<Integer, StreamPartitionMsgOffset> consumptionByPartition = new
HashMap<>();
+ for (PartitionGroupConsumptionStatus s :
partitionGroupConsumptionStatuses) {
+ consumptionByPartition.put(s.getStreamPartitionGroupId(),
s.getEndOffset());
+ }
+ StreamConsumerFactory streamConsumerFactory =
StreamConsumerFactoryProvider.create(streamConfig);
+ List<PartitionGroupMetadata> result = new ArrayList<>(subset.size());
+ for (Integer partitionId : subset) {
+ if (!topicIds.contains(partitionId)) {
+ LOGGER.warn(
+ "Configured partition id {} does not exist in topic {} when
computing partition group metadata. "
+ + "This indicates that topic partitions may have changed
between validation and metadata "
+ + "computation. Skipping this partition. Current topic
partitions: {}",
+ partitionId, _topic, topicIds);
+ continue;
+ }
+ StreamPartitionMsgOffset startOffset =
consumptionByPartition.get(partitionId);
+ if (startOffset == null) {
+ try (StreamMetadataProvider partitionMetadataProvider =
+ streamConsumerFactory.createPartitionMetadataProvider(
+ StreamConsumerFactory.getUniqueClientId(clientId),
partitionId)) {
+ startOffset = partitionMetadataProvider.fetchStreamPartitionOffset(
+ streamConfig.getOffsetCriteria(), timeoutMillis);
+ }
+ }
+ result.add(new PartitionGroupMetadata(partitionId, startOffset));
+ }
+ return result;
+ }
+
+ @Override
+ public List<PartitionGroupMetadata> computePartitionGroupMetadata(String
clientId, StreamConfig streamConfig,
+ List<PartitionGroupConsumptionStatus> partitionGroupConsumptionStatuses,
int timeoutMillis,
+ boolean forceGetOffsetFromStream) throws IOException,
java.util.concurrent.TimeoutException {
+ if (forceGetOffsetFromStream) {
+ return computePartitionGroupMetadata(clientId, streamConfig,
Collections.emptyList(), timeoutMillis);
+ }
+ return computePartitionGroupMetadata(clientId, streamConfig,
partitionGroupConsumptionStatuses, timeoutMillis);
+ }
Review Comment:
Is this method required?
##########
pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaStreamMetadataProvider.java:
##########
@@ -57,23 +63,47 @@ public class KafkaStreamMetadataProvider extends
KafkaPartitionLevelConnectionHa
implements StreamMetadataProvider {
private static final Logger LOGGER =
LoggerFactory.getLogger(KafkaStreamMetadataProvider.class);
+ private final boolean _partialPartitions;
+ private final List<Integer> _partitionIdSubset;
public KafkaStreamMetadataProvider(String clientId, StreamConfig
streamConfig) {
this(clientId, streamConfig, Integer.MIN_VALUE);
}
public KafkaStreamMetadataProvider(String clientId, StreamConfig
streamConfig, int partition) {
super(clientId, streamConfig, partition);
+ List<Integer> subset =
+
KafkaPartitionSubsetUtils.getPartitionIdsFromConfig(_config.getStreamConfigMap());
+ if (subset != null) {
+ _partialPartitions = true;
+ _partitionIdSubset = Collections.unmodifiableList(subset);
+ } else {
+ _partialPartitions = false;
+ _partitionIdSubset = Collections.emptyList();
+ }
}
@Override
public int fetchPartitionCount(long timeoutMillis) {
try {
List<PartitionInfo> partitionInfos = _consumer.partitionsFor(_topic,
Duration.ofMillis(timeoutMillis));
- if (CollectionUtils.isNotEmpty(partitionInfos)) {
- return partitionInfos.size();
+ if (CollectionUtils.isEmpty(partitionInfos)) {
+ throw new RuntimeException(String.format("Failed to fetch partition
information for topic: %s", _topic));
+ }
+ if (_partialPartitions) {
+ Set<Integer> topicIds = new HashSet<>();
+ for (PartitionInfo p : partitionInfos) {
+ topicIds.add(p.partition());
Review Comment:
We are just returning count here, so I think we can skip adding to Set
##########
pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaStreamMetadataProvider.java:
##########
@@ -57,23 +63,47 @@ public class KafkaStreamMetadataProvider extends
KafkaPartitionLevelConnectionHa
implements StreamMetadataProvider {
private static final Logger LOGGER =
LoggerFactory.getLogger(KafkaStreamMetadataProvider.class);
+ private final boolean _partialPartitions;
+ private final List<Integer> _partitionIdSubset;
public KafkaStreamMetadataProvider(String clientId, StreamConfig
streamConfig) {
this(clientId, streamConfig, Integer.MIN_VALUE);
}
public KafkaStreamMetadataProvider(String clientId, StreamConfig
streamConfig, int partition) {
super(clientId, streamConfig, partition);
+ List<Integer> subset =
+
KafkaPartitionSubsetUtils.getPartitionIdsFromConfig(_config.getStreamConfigMap());
+ if (subset != null) {
+ _partialPartitions = true;
+ _partitionIdSubset = Collections.unmodifiableList(subset);
+ } else {
+ _partialPartitions = false;
+ _partitionIdSubset = Collections.emptyList();
+ }
}
@Override
public int fetchPartitionCount(long timeoutMillis) {
try {
List<PartitionInfo> partitionInfos = _consumer.partitionsFor(_topic,
Duration.ofMillis(timeoutMillis));
- if (CollectionUtils.isNotEmpty(partitionInfos)) {
- return partitionInfos.size();
+ if (CollectionUtils.isEmpty(partitionInfos)) {
+ throw new RuntimeException(String.format("Failed to fetch partition
information for topic: %s", _topic));
Review Comment:
Applicable for other places as well
##########
pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaStreamMetadataProvider.java:
##########
@@ -96,6 +139,55 @@ public Set<Integer> fetchPartitionIds(long timeoutMillis) {
}
}
+ @Override
+ public List<PartitionGroupMetadata> computePartitionGroupMetadata(String
clientId, StreamConfig streamConfig,
+ List<PartitionGroupConsumptionStatus> partitionGroupConsumptionStatuses,
int timeoutMillis)
+ throws IOException, java.util.concurrent.TimeoutException {
+ if (!_partialPartitions) {
+ return
StreamMetadataProvider.super.computePartitionGroupMetadata(clientId,
streamConfig,
+ partitionGroupConsumptionStatuses, timeoutMillis);
+ }
+ List<Integer> subset = _partitionIdSubset;
+ Set<Integer> topicIds = fetchPartitionIds(timeoutMillis);
+ Map<Integer, StreamPartitionMsgOffset> consumptionByPartition = new
HashMap<>();
Review Comment:
```suggestion
Map<Integer, StreamPartitionMsgOffset> partitionIdToEndOffset = new
HashMap<>(partitionGroupConsumptionStatuses.size());
```
##########
pinot-plugins/pinot-stream-ingestion/pinot-kafka-base/src/main/java/org/apache/pinot/plugin/stream/kafka/KafkaPartitionSubsetUtils.java:
##########
@@ -0,0 +1,79 @@
+/**
+ * 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.plugin.stream.kafka;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.commons.lang3.StringUtils;
+
+
+/**
+ * Utilities for parsing and validating Kafka partition subset configuration
+ * (stream.kafka.partition.ids) from stream config.
+ */
+public final class KafkaPartitionSubsetUtils {
+
+ private KafkaPartitionSubsetUtils() {
+ }
+
+ /**
+ * Reads the optional comma-separated partition ID list from the stream
config map.
+ * Returns a sorted, deduplicated list for stable ordering when used for
partition group metadata.
+ * Duplicate IDs in the config are silently removed.
+ *
+ * @param streamConfigMap table stream config map (e.g. from
+ * {@link
org.apache.pinot.spi.stream.StreamConfig#getStreamConfigsMap()})
+ * @return Sorted list of unique partition IDs when
stream.kafka.partition.ids is set and non-empty;
+ * null when not set or blank
+ * @throws IllegalArgumentException if the value contains invalid
(non-integer) entries
+ */
+ @Nullable
+ public static List<Integer> getPartitionIdsFromConfig(Map<String, String>
streamConfigMap) {
+ String key =
KafkaStreamConfigProperties.constructStreamProperty(KafkaStreamConfigProperties.PARTITION_IDS);
+ String value = streamConfigMap.get(key);
+ if (StringUtils.isBlank(value)) {
+ return null;
+ }
+ String[] parts = value.split(",");
+ Set<Integer> idSet = new LinkedHashSet<>(parts.length);
Review Comment:
nit, we are sorting the set anyways so we can just use the simple hashset.
##########
pinot-plugins/pinot-stream-ingestion/pinot-kafka-3.0/src/main/java/org/apache/pinot/plugin/stream/kafka30/KafkaStreamMetadataProvider.java:
##########
@@ -57,23 +63,47 @@ public class KafkaStreamMetadataProvider extends
KafkaPartitionLevelConnectionHa
implements StreamMetadataProvider {
private static final Logger LOGGER =
LoggerFactory.getLogger(KafkaStreamMetadataProvider.class);
+ private final boolean _partialPartitions;
+ private final List<Integer> _partitionIdSubset;
public KafkaStreamMetadataProvider(String clientId, StreamConfig
streamConfig) {
this(clientId, streamConfig, Integer.MIN_VALUE);
}
public KafkaStreamMetadataProvider(String clientId, StreamConfig
streamConfig, int partition) {
super(clientId, streamConfig, partition);
+ List<Integer> subset =
+
KafkaPartitionSubsetUtils.getPartitionIdsFromConfig(_config.getStreamConfigMap());
+ if (subset != null) {
+ _partialPartitions = true;
+ _partitionIdSubset = Collections.unmodifiableList(subset);
+ } else {
+ _partialPartitions = false;
+ _partitionIdSubset = Collections.emptyList();
+ }
}
@Override
public int fetchPartitionCount(long timeoutMillis) {
try {
List<PartitionInfo> partitionInfos = _consumer.partitionsFor(_topic,
Duration.ofMillis(timeoutMillis));
- if (CollectionUtils.isNotEmpty(partitionInfos)) {
- return partitionInfos.size();
+ if (CollectionUtils.isEmpty(partitionInfos)) {
+ throw new RuntimeException(String.format("Failed to fetch partition
information for topic: %s", _topic));
+ }
+ if (_partialPartitions) {
+ Set<Integer> topicIds = new HashSet<>();
+ for (PartitionInfo p : partitionInfos) {
+ topicIds.add(p.partition());
Review Comment:
Same comments of kafka20 class
##########
pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaStreamMetadataProvider.java:
##########
@@ -96,6 +139,55 @@ public Set<Integer> fetchPartitionIds(long timeoutMillis) {
}
}
+ @Override
+ public List<PartitionGroupMetadata> computePartitionGroupMetadata(String
clientId, StreamConfig streamConfig,
+ List<PartitionGroupConsumptionStatus> partitionGroupConsumptionStatuses,
int timeoutMillis)
+ throws IOException, java.util.concurrent.TimeoutException {
+ if (!_partialPartitions) {
+ return
StreamMetadataProvider.super.computePartitionGroupMetadata(clientId,
streamConfig,
+ partitionGroupConsumptionStatuses, timeoutMillis);
+ }
+ List<Integer> subset = _partitionIdSubset;
+ Set<Integer> topicIds = fetchPartitionIds(timeoutMillis);
Review Comment:
```suggestion
Set<Integer> partitionIds = fetchPartitionIds(timeoutMillis);
```
##########
pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaStreamMetadataProvider.java:
##########
@@ -86,6 +116,19 @@ public Set<Integer> fetchPartitionIds(long timeoutMillis) {
if (CollectionUtils.isEmpty(partitionInfos)) {
throw new RuntimeException(String.format("Failed to fetch partition
information for topic: %s", _topic));
}
+ if (_partialPartitions) {
+ Set<Integer> topicIds =
Sets.newHashSetWithExpectedSize(partitionInfos.size());
Review Comment:
nit
```suggestion
Set<Integer> partitionIds =
Sets.newHashSetWithExpectedSize(partitionInfos.size());
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]