Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FEATURE] Add getTopics API to StreamMetadataProvider for Topic Enumeration in Streaming Connectors #14654

Merged
merged 5 commits into from
Dec 13, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.commons.collections4.CollectionUtils;
import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
import org.apache.kafka.common.PartitionInfo;
Expand Down Expand Up @@ -166,6 +167,31 @@ public Map<String, PartitionLagState> getCurrentPartitionLagState(
return perPartitionLag;
}

@Override
public List<TopicMetadata> getTopics() {
Map<String, List<PartitionInfo>> namePartitionsMap = _consumer.listTopics();
if (namePartitionsMap == null) {
return Collections.emptyList();
}
return namePartitionsMap.keySet()
.stream()
.map(topic -> new KafkaTopicMetadata().setName(topic))
.collect(Collectors.toList());
}

public static class KafkaTopicMetadata implements TopicMetadata {
private String _name;

public String getName() {
return _name;
}

public KafkaTopicMetadata setName(String name) {
_name = name;
return this;
}
}

@Override
public void close()
throws IOException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.commons.collections4.CollectionUtils;
import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
import org.apache.kafka.common.PartitionInfo;
Expand Down Expand Up @@ -166,6 +167,30 @@ public Map<String, PartitionLagState> getCurrentPartitionLagState(
return perPartitionLag;
}

@Override
public List<TopicMetadata> getTopics() {
Map<String, List<PartitionInfo>> namePartitionsMap = _consumer.listTopics();
if (namePartitionsMap == null) {
return Collections.emptyList();
}
return namePartitionsMap.keySet()
.stream()
.map(topic -> new KafkaTopicMetadata().setName(topic))
.collect(Collectors.toList());
}

public static class KafkaTopicMetadata implements TopicMetadata {
private String _name;

public String getName() {
return _name;
}

public KafkaTopicMetadata setName(String name) {
_name = name;
return this;
}
}
@Override
public void close()
throws IOException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,9 +20,11 @@

import java.time.Instant;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
Expand All @@ -39,6 +41,7 @@
import org.apache.pinot.spi.stream.StreamConsumerFactoryProvider;
import org.apache.pinot.spi.stream.StreamMessage;
import org.apache.pinot.spi.stream.StreamMessageMetadata;
import org.apache.pinot.spi.stream.StreamMetadataProvider;
import org.apache.pinot.spi.stream.StreamPartitionMsgOffset;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
Expand Down Expand Up @@ -399,4 +402,29 @@ public void testOffsetsExpired()
}
assertEquals(messageBatch.getOffsetOfNextBatch().toString(), "700");
}

@Test
public void testListTopics() {
String streamType = "kafka";
String streamKafkaBrokerList = _kafkaBrokerAddress;
String streamKafkaConsumerType = "simple";
String clientId = "clientId";
String tableNameWithType = "tableName_REALTIME";

Map<String, String> streamConfigMap = new HashMap<>();
streamConfigMap.put("streamType", streamType);
streamConfigMap.put("stream.kafka.topic.name", "NON_EXISTING_TOPIC");
streamConfigMap.put("stream.kafka.broker.list", streamKafkaBrokerList);
streamConfigMap.put("stream.kafka.consumer.type", streamKafkaConsumerType);
streamConfigMap.put("stream.kafka.consumer.factory.class.name", getKafkaConsumerFactoryName());
streamConfigMap.put("stream.kafka.decoder.class.name", "decoderClass");
StreamConfig streamConfig = new StreamConfig(tableNameWithType, streamConfigMap);

KafkaStreamMetadataProvider streamMetadataProvider = new KafkaStreamMetadataProvider(clientId, streamConfig);
List<StreamMetadataProvider.TopicMetadata> topics = streamMetadataProvider.getTopics();
List<String> topicNames = topics.stream()
.map(StreamMetadataProvider.TopicMetadata::getName)
.collect(Collectors.toList());
assertTrue(topicNames.containsAll(List.of(TEST_TOPIC_1, TEST_TOPIC_2, TEST_TOPIC_3)));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -106,6 +106,22 @@ default Map<String, PartitionLagState> getCurrentPartitionLagState(
return result;
}

/**
* Fetches the list of available topics/streams
*
* @return List of topics
*/
default List<TopicMetadata> getTopics() {
throw new UnsupportedOperationException();
}

/**
* Represents the metadata of a topic. This can be used to represent the topic name and other metadata in the future.
*/
interface TopicMetadata {
String getName();
}

class UnknownLagState extends PartitionLagState {
}
}
Loading