noob-se7en commented on code in PR #17587:
URL: https://github.com/apache/pinot/pull/17587#discussion_r2760387499
##########
pinot-plugins/pinot-stream-ingestion/pinot-kafka-3.0/src/main/java/org/apache/pinot/plugin/stream/kafka30/KafkaStreamMetadataProvider.java:
##########
@@ -82,20 +114,46 @@ public int fetchPartitionCount(long timeoutMillis) {
@Override
public Set<Integer> fetchPartitionIds(long timeoutMillis) {
try {
- List<PartitionInfo> partitionInfos = _consumer.partitionsFor(_topic,
Duration.ofMillis(timeoutMillis));
- if (CollectionUtils.isEmpty(partitionInfos)) {
- throw new RuntimeException(String.format("Failed to fetch partition
information for topic: %s", _topic));
- }
- Set<Integer> partitionIds =
Sets.newHashSetWithExpectedSize(partitionInfos.size());
- for (PartitionInfo partitionInfo : partitionInfos) {
- partitionIds.add(partitionInfo.partition());
+ if (_partialPartitions) {
+ return new HashSet<>(_partitionIdSubset);
}
- return partitionIds;
+ return toPartitionIdSet(getPartitionInfos(timeoutMillis));
} catch (TimeoutException e) {
throw new TransientConsumerException(e);
}
}
+ @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;
+ Map<Integer, StreamPartitionMsgOffset> partitionIdToEndOffset =
+ new HashMap<>(partitionGroupConsumptionStatuses.size());
+ for (PartitionGroupConsumptionStatus s :
partitionGroupConsumptionStatuses) {
+ partitionIdToEndOffset.put(s.getStreamPartitionGroupId(),
s.getEndOffset());
+ }
+ StreamConsumerFactory streamConsumerFactory =
StreamConsumerFactoryProvider.create(streamConfig);
+ List<PartitionGroupMetadata> result = new ArrayList<>(subset.size());
+ for (Integer partitionId : subset) {
+ StreamPartitionMsgOffset startOffset =
partitionIdToEndOffset.get(partitionId);
Review Comment:
nit:
```suggestion
StreamPartitionMsgOffset endOffset =
partitionIdToEndOffset.get(partitionId);
```
##########
pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaStreamMetadataProvider.java:
##########
@@ -57,23 +63,49 @@ 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();
+ }
+ }
+
+ private List<PartitionInfo> getPartitionInfos(long timeoutMillis) {
+ List<PartitionInfo> partitionInfos = _consumer.partitionsFor(_topic,
Duration.ofMillis(timeoutMillis));
+ if (CollectionUtils.isEmpty(partitionInfos)) {
+ throw new RuntimeException("Failed to fetch partition information for
topic: " + _topic);
+ }
+ return partitionInfos;
+ }
+
+ private Set<Integer> toPartitionIdSet(List<PartitionInfo> partitionInfos) {
+ Set<Integer> partitionIds =
Sets.newHashSetWithExpectedSize(partitionInfos.size());
+ for (PartitionInfo partitionInfo : partitionInfos) {
+ partitionIds.add(partitionInfo.partition());
+ }
+ return partitionIds;
}
@Override
public int fetchPartitionCount(long timeoutMillis) {
try {
- List<PartitionInfo> partitionInfos = _consumer.partitionsFor(_topic,
Duration.ofMillis(timeoutMillis));
- if (CollectionUtils.isNotEmpty(partitionInfos)) {
- return partitionInfos.size();
+ if (_partialPartitions) {
+ return _partitionIdSubset.size();
Review Comment:
So the partial subset for table will never change?
##########
pinot-plugins/pinot-stream-ingestion/pinot-kafka-3.0/src/main/java/org/apache/pinot/plugin/stream/kafka30/KafkaStreamMetadataProvider.java:
##########
@@ -82,20 +114,46 @@ public int fetchPartitionCount(long timeoutMillis) {
@Override
public Set<Integer> fetchPartitionIds(long timeoutMillis) {
try {
- List<PartitionInfo> partitionInfos = _consumer.partitionsFor(_topic,
Duration.ofMillis(timeoutMillis));
- if (CollectionUtils.isEmpty(partitionInfos)) {
- throw new RuntimeException(String.format("Failed to fetch partition
information for topic: %s", _topic));
- }
- Set<Integer> partitionIds =
Sets.newHashSetWithExpectedSize(partitionInfos.size());
- for (PartitionInfo partitionInfo : partitionInfos) {
- partitionIds.add(partitionInfo.partition());
+ if (_partialPartitions) {
+ return new HashSet<>(_partitionIdSubset);
}
- return partitionIds;
+ return toPartitionIdSet(getPartitionInfos(timeoutMillis));
} catch (TimeoutException e) {
throw new TransientConsumerException(e);
}
}
+ @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;
+ Map<Integer, StreamPartitionMsgOffset> partitionIdToEndOffset =
+ new HashMap<>(partitionGroupConsumptionStatuses.size());
+ for (PartitionGroupConsumptionStatus s :
partitionGroupConsumptionStatuses) {
+ partitionIdToEndOffset.put(s.getStreamPartitionGroupId(),
s.getEndOffset());
+ }
+ StreamConsumerFactory streamConsumerFactory =
StreamConsumerFactoryProvider.create(streamConfig);
+ List<PartitionGroupMetadata> result = new ArrayList<>(subset.size());
+ for (Integer partitionId : subset) {
+ StreamPartitionMsgOffset startOffset =
partitionIdToEndOffset.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));
Review Comment:
So `fetchPartitionIds` returns the same subset created in constructor
everytime.
But in this method we add new partitionIds, so behaviour seems inconsistent
across these methods.
--
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]