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

KAFKA-4461 Added support to ProcessorTopologyTestDriver for internal topics. #2499

Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@ public class ProcessorTopologyTest {
private static final String INPUT_TOPIC_2 = "input-topic-2";
private static final String OUTPUT_TOPIC_1 = "output-topic-1";
private static final String OUTPUT_TOPIC_2 = "output-topic-2";
private static final String THROUGH_TOPIC_1 = "through-topic-1";

private static long timestamp = 1000L;
private final TopologyBuilder builder = new TopologyBuilder();
Expand Down Expand Up @@ -234,6 +235,17 @@ public void testDrivingSimpleMultiSourceTopology() {
assertNoOutputRecord(OUTPUT_TOPIC_1);
}

@Test
public void testDrivingInternalRepartitioningTopology() {
driver = new ProcessorTopologyTestDriver(config, createInternalRepartitioningTopology());
driver.process(INPUT_TOPIC_1, "key1", "value1", STRING_SERIALIZER, STRING_SERIALIZER);
driver.process(INPUT_TOPIC_1, "key2", "value2", STRING_SERIALIZER, STRING_SERIALIZER);
driver.process(INPUT_TOPIC_1, "key3", "value3", STRING_SERIALIZER, STRING_SERIALIZER);
assertNextOutputRecord(OUTPUT_TOPIC_1, "key1", "value1");
assertNextOutputRecord(OUTPUT_TOPIC_1, "key2", "value2");
assertNextOutputRecord(OUTPUT_TOPIC_1, "key3", "value3");
}

@Test
public void shouldCreateStringWithSourceAndTopics() throws Exception {
builder.addSource("source", "topic1", "topic2");
Expand Down Expand Up @@ -337,6 +349,13 @@ private TopologyBuilder createStatefulTopology(String storeName) {
.addSink("counts", OUTPUT_TOPIC_1, "processor");
}

private TopologyBuilder createInternalRepartitioningTopology() {
return builder.addSource("source", INPUT_TOPIC_1)
.addInternalTopic(THROUGH_TOPIC_1)
.addSink("sink0", THROUGH_TOPIC_1, "source")
.addSource("source1", THROUGH_TOPIC_1)
.addSink("sink1", OUTPUT_TOPIC_1, "source1");
}

private TopologyBuilder createSimpleMultiSourceTopology(int partition) {
return builder.addSource("source-1", STRING_DESERIALIZER, STRING_DESERIALIZER, INPUT_TOPIC_1)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,12 @@
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.MockConsumer;
Expand Down Expand Up @@ -149,6 +151,7 @@ public class ProcessorTopologyTestDriver {
private final Map<String, TopicPartition> partitionsByTopic = new HashMap<>();
private final Map<TopicPartition, AtomicLong> offsetsByTopicPartition = new HashMap<>();
private final Map<String, Queue<ProducerRecord<byte[], byte[]>>> outputRecordsByTopic = new HashMap<>();
private final Set<String> internalTopics = new HashSet<>();
private final ProcessorTopology globalTopology;
private final Map<String, TopicPartition> globalPartitionsByTopic = new HashMap<>();
private StreamTask task;
Expand All @@ -175,15 +178,18 @@ public List<PartitionInfo> partitionsFor(String topic) {
};
restoreStateConsumer = createRestoreConsumer(id, storeNames);

// Identify internal topics for forwarding in process ...
for (TopologyBuilder.TopicsInfo topicsInfo : builder.topicGroups().values()) {
internalTopics.addAll(topicsInfo.repartitionSourceTopics.keySet());
}

// Set up all of the topic+partition information and subscribe the consumer to each ...
for (String topic : topology.sourceTopics()) {
TopicPartition tp = new TopicPartition(topic, 1);
partitionsByTopic.put(topic, tp);
offsetsByTopicPartition.put(tp, new AtomicLong());
}



consumer.assign(offsetsByTopicPartition.keySet());

final StateDirectory stateDirectory = new StateDirectory(applicationId, TestUtils.tempDirectory().getPath());
Expand Down Expand Up @@ -249,6 +255,11 @@ public void process(String topicName, byte[] key, byte[] value) {
outputRecordsByTopic.put(record.topic(), outputRecords);
}
outputRecords.add(record);

// Forward back into the topology if the produced record is to an internal topic ...
if (internalTopics.contains(record.topic())) {
process(record.topic(), record.key(), record.value());
}
}
} else {
final TopicPartition global = globalPartitionsByTopic.get(topicName);
Expand Down