mirror of https://github.com/apache/kafka.git
				
				
				
			KAFKA-4828: ProcessorTopologyTestDriver does not work when using through
This resolves the following issues in the ProcessorTopologyTestDriver: - It should not create an internal changelog topic when using `through()` and `table()` - It should forward the produced record back into the topology if it is to a source topic Jira ticket: https://issues.apache.org/jira/browse/KAFKA-4828 The contribution is my original work and I license the work to the project under the project's open source license. Author: Hamidreza Afzali <hrafzali@gmail.com> Reviewers: Matthias J. Sax, Guozhang Wang Closes #2629 from hrafzali/KAFKA-4828_ProcessorTopologyTestDriver_through
This commit is contained in:
		
							parent
							
								
									b1272500bd
								
							
						
					
					
						commit
						5b013d9cd2
					
				| 
						 | 
				
			
			@ -629,7 +629,7 @@ public class TopologyBuilderTest {
 | 
			
		|||
                    goodNodeName)
 | 
			
		||||
                .addProcessor(badNodeName, new LocalMockProcessorSupplier(), sourceNodeName);
 | 
			
		||||
 | 
			
		||||
            final ProcessorTopologyTestDriver driver = new ProcessorTopologyTestDriver(streamsConfig, builder, LocalMockProcessorSupplier.STORE_NAME);
 | 
			
		||||
            final ProcessorTopologyTestDriver driver = new ProcessorTopologyTestDriver(streamsConfig, builder);
 | 
			
		||||
            driver.process("topic", null, null);
 | 
			
		||||
        } catch (final StreamsException e) {
 | 
			
		||||
            final Throwable cause = e.getCause();
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -189,7 +189,7 @@ public class ProcessorTopologyTest {
 | 
			
		|||
    @Test
 | 
			
		||||
    public void testDrivingStatefulTopology() {
 | 
			
		||||
        String storeName = "entries";
 | 
			
		||||
        driver = new ProcessorTopologyTestDriver(config, createStatefulTopology(storeName), storeName);
 | 
			
		||||
        driver = new ProcessorTopologyTestDriver(config, createStatefulTopology(storeName));
 | 
			
		||||
        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);
 | 
			
		||||
| 
						 | 
				
			
			@ -214,7 +214,7 @@ public class ProcessorTopologyTest {
 | 
			
		|||
        final TopologyBuilder topologyBuilder = this.builder
 | 
			
		||||
                .addGlobalStore(globalStore, global, STRING_DESERIALIZER, STRING_DESERIALIZER, topic, "processor", define(new StatefulProcessor("my-store")));
 | 
			
		||||
 | 
			
		||||
        driver = new ProcessorTopologyTestDriver(config, topologyBuilder, "my-store");
 | 
			
		||||
        driver = new ProcessorTopologyTestDriver(config, topologyBuilder);
 | 
			
		||||
        driver.process(topic, "key1", "value1", STRING_SERIALIZER, STRING_SERIALIZER);
 | 
			
		||||
        driver.process(topic, "key2", "value2", STRING_SERIALIZER, STRING_SERIALIZER);
 | 
			
		||||
        assertEquals("value1", globalStore.get("key1"));
 | 
			
		||||
| 
						 | 
				
			
			@ -235,6 +235,17 @@ public class ProcessorTopologyTest {
 | 
			
		|||
        assertNoOutputRecord(OUTPUT_TOPIC_1);
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    @Test
 | 
			
		||||
    public void testDrivingForwardToSourceTopology() {
 | 
			
		||||
        driver = new ProcessorTopologyTestDriver(config, createForwardToSourceTopology());
 | 
			
		||||
        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_2, "key1", "value1");
 | 
			
		||||
        assertNextOutputRecord(OUTPUT_TOPIC_2, "key2", "value2");
 | 
			
		||||
        assertNextOutputRecord(OUTPUT_TOPIC_2, "key3", "value3");
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    @Test
 | 
			
		||||
    public void testDrivingInternalRepartitioningTopology() {
 | 
			
		||||
        driver = new ProcessorTopologyTestDriver(config, createInternalRepartitioningTopology());
 | 
			
		||||
| 
						 | 
				
			
			@ -380,6 +391,13 @@ public class ProcessorTopologyTest {
 | 
			
		|||
                .addSink("sink1", OUTPUT_TOPIC_1, "source1");
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    private TopologyBuilder createForwardToSourceTopology() {
 | 
			
		||||
        return builder.addSource("source-1", INPUT_TOPIC_1)
 | 
			
		||||
                .addSink("sink-1", OUTPUT_TOPIC_1, "source-1")
 | 
			
		||||
                .addSource("source-2", OUTPUT_TOPIC_1)
 | 
			
		||||
                .addSink("sink-2", OUTPUT_TOPIC_2, "source-2");
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    private TopologyBuilder createSimpleMultiSourceTopology(int partition) {
 | 
			
		||||
        return builder.addSource("source-1", STRING_DESERIALIZER, STRING_DESERIALIZER, INPUT_TOPIC_1)
 | 
			
		||||
                .addProcessor("processor-1", define(new ForwardingProcessor()), "source-1")
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -52,7 +52,6 @@ import org.apache.kafka.streams.processor.internals.GlobalStateUpdateTask;
 | 
			
		|||
import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
 | 
			
		||||
import org.apache.kafka.streams.processor.internals.ProcessorContextImpl;
 | 
			
		||||
import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
 | 
			
		||||
import org.apache.kafka.streams.processor.internals.ProcessorStateManager;
 | 
			
		||||
import org.apache.kafka.streams.processor.internals.ProcessorTopology;
 | 
			
		||||
import org.apache.kafka.streams.processor.internals.RecordCollectorImpl;
 | 
			
		||||
import org.apache.kafka.streams.processor.internals.StateDirectory;
 | 
			
		||||
| 
						 | 
				
			
			@ -143,9 +142,10 @@ public class ProcessorTopologyTestDriver {
 | 
			
		|||
 | 
			
		||||
    private final Serializer<byte[]> bytesSerializer = new ByteArraySerializer();
 | 
			
		||||
 | 
			
		||||
    private final String applicationId = "test-driver-application";
 | 
			
		||||
    private final static String APPLICATION_ID = "test-driver-application";
 | 
			
		||||
    private final static int PARTITION_ID = 0;
 | 
			
		||||
    private final static TaskId TASK_ID = new TaskId(0, PARTITION_ID);
 | 
			
		||||
 | 
			
		||||
    private final TaskId id;
 | 
			
		||||
    private final ProcessorTopology topology;
 | 
			
		||||
    private final MockConsumer<byte[], byte[]> consumer;
 | 
			
		||||
    private final MockProducer<byte[], byte[]> producer;
 | 
			
		||||
| 
						 | 
				
			
			@ -163,11 +163,9 @@ public class ProcessorTopologyTestDriver {
 | 
			
		|||
     * Create a new test driver instance.
 | 
			
		||||
     * @param config the stream configuration for the topology
 | 
			
		||||
     * @param builder the topology builder that will be used to create the topology instance
 | 
			
		||||
     * @param storeNames the optional names of the state stores that are used by the topology
 | 
			
		||||
     */
 | 
			
		||||
    public ProcessorTopologyTestDriver(StreamsConfig config, TopologyBuilder builder, String... storeNames) {
 | 
			
		||||
        id = new TaskId(0, 0);
 | 
			
		||||
        topology = builder.setApplicationId(applicationId).build(null);
 | 
			
		||||
    public ProcessorTopologyTestDriver(StreamsConfig config, TopologyBuilder builder) {
 | 
			
		||||
        topology = builder.setApplicationId(APPLICATION_ID).build(null);
 | 
			
		||||
        globalTopology  = builder.buildGlobalStateTopology();
 | 
			
		||||
 | 
			
		||||
        // Set up the consumer and producer ...
 | 
			
		||||
| 
						 | 
				
			
			@ -175,10 +173,10 @@ public class ProcessorTopologyTestDriver {
 | 
			
		|||
        producer = new MockProducer<byte[], byte[]>(true, bytesSerializer, bytesSerializer) {
 | 
			
		||||
            @Override
 | 
			
		||||
            public List<PartitionInfo> partitionsFor(String topic) {
 | 
			
		||||
                return Collections.singletonList(new PartitionInfo(topic, 0, null, null, null));
 | 
			
		||||
                return Collections.singletonList(new PartitionInfo(topic, PARTITION_ID, null, null, null));
 | 
			
		||||
            }
 | 
			
		||||
        };
 | 
			
		||||
        restoreStateConsumer = createRestoreConsumer(id, storeNames);
 | 
			
		||||
        restoreStateConsumer = createRestoreConsumer(TASK_ID, topology.storeToChangelogTopic());
 | 
			
		||||
 | 
			
		||||
        // Identify internal topics for forwarding in process ...
 | 
			
		||||
        for (TopologyBuilder.TopicsInfo topicsInfo : builder.topicGroups().values()) {
 | 
			
		||||
| 
						 | 
				
			
			@ -187,14 +185,14 @@ public class ProcessorTopologyTestDriver {
 | 
			
		|||
 | 
			
		||||
        // 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);
 | 
			
		||||
            TopicPartition tp = new TopicPartition(topic, PARTITION_ID);
 | 
			
		||||
            partitionsByTopic.put(topic, tp);
 | 
			
		||||
            offsetsByTopicPartition.put(tp, new AtomicLong());
 | 
			
		||||
        }
 | 
			
		||||
 | 
			
		||||
        consumer.assign(offsetsByTopicPartition.keySet());
 | 
			
		||||
 | 
			
		||||
        final StateDirectory stateDirectory = new StateDirectory(applicationId, TestUtils.tempDirectory().getPath(), Time.SYSTEM);
 | 
			
		||||
        final StateDirectory stateDirectory = new StateDirectory(APPLICATION_ID, TestUtils.tempDirectory().getPath(), Time.SYSTEM);
 | 
			
		||||
        final StreamsMetrics streamsMetrics = new MockStreamsMetrics(new Metrics());
 | 
			
		||||
        final ThreadCache cache = new ThreadCache("mock", 1024 * 1024, streamsMetrics);
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -218,8 +216,8 @@ public class ProcessorTopologyTestDriver {
 | 
			
		|||
        }
 | 
			
		||||
 | 
			
		||||
        if (!partitionsByTopic.isEmpty()) {
 | 
			
		||||
            task = new StreamTask(id,
 | 
			
		||||
                                  applicationId,
 | 
			
		||||
            task = new StreamTask(TASK_ID,
 | 
			
		||||
                                  APPLICATION_ID,
 | 
			
		||||
                                  partitionsByTopic.values(),
 | 
			
		||||
                                  topology,
 | 
			
		||||
                                  consumer,
 | 
			
		||||
| 
						 | 
				
			
			@ -263,8 +261,8 @@ public class ProcessorTopologyTestDriver {
 | 
			
		|||
                }
 | 
			
		||||
                outputRecords.add(record);
 | 
			
		||||
 | 
			
		||||
                // Forward back into the topology if the produced record is to an internal topic ...
 | 
			
		||||
                if (internalTopics.contains(record.topic())) {
 | 
			
		||||
                // Forward back into the topology if the produced record is to an internal or a source topic ...
 | 
			
		||||
                if (internalTopics.contains(record.topic()) || topology.sourceTopics().contains(record.topic())) {
 | 
			
		||||
                    process(record.topic(), record.key(), record.value(), record.timestamp());
 | 
			
		||||
                }
 | 
			
		||||
            }
 | 
			
		||||
| 
						 | 
				
			
			@ -339,7 +337,7 @@ public class ProcessorTopologyTestDriver {
 | 
			
		|||
 | 
			
		||||
    /**
 | 
			
		||||
     * Get the {@link StateStore} with the given name. The name should have been supplied via
 | 
			
		||||
     * {@link #ProcessorTopologyTestDriver(StreamsConfig, TopologyBuilder, String...) this object's constructor}, and is
 | 
			
		||||
     * {@link #ProcessorTopologyTestDriver(StreamsConfig, TopologyBuilder) this object's constructor}, and is
 | 
			
		||||
     * presumed to be used by a Processor within the topology.
 | 
			
		||||
     * <p>
 | 
			
		||||
     * This is often useful in test cases to pre-populate the store before the test case instructs the topology to
 | 
			
		||||
| 
						 | 
				
			
			@ -355,7 +353,7 @@ public class ProcessorTopologyTestDriver {
 | 
			
		|||
 | 
			
		||||
    /**
 | 
			
		||||
     * Get the {@link KeyValueStore} with the given name. The name should have been supplied via
 | 
			
		||||
     * {@link #ProcessorTopologyTestDriver(StreamsConfig, TopologyBuilder, String...) this object's constructor}, and is
 | 
			
		||||
     * {@link #ProcessorTopologyTestDriver(StreamsConfig, TopologyBuilder) this object's constructor}, and is
 | 
			
		||||
     * presumed to be used by a Processor within the topology.
 | 
			
		||||
     * <p>
 | 
			
		||||
     * This is often useful in test cases to pre-populate the store before the test case instructs the topology to
 | 
			
		||||
| 
						 | 
				
			
			@ -393,10 +391,10 @@ public class ProcessorTopologyTestDriver {
 | 
			
		|||
     * driver object unless this method is overwritten with a functional consumer.
 | 
			
		||||
     *
 | 
			
		||||
     * @param id the ID of the stream task
 | 
			
		||||
     * @param storeNames the names of the stores that this
 | 
			
		||||
     * @param storeToChangelogTopic the map of the names of the stores to the changelog topics
 | 
			
		||||
     * @return the mock consumer; never null
 | 
			
		||||
     */
 | 
			
		||||
    protected MockConsumer<byte[], byte[]> createRestoreConsumer(TaskId id, String... storeNames) {
 | 
			
		||||
    protected MockConsumer<byte[], byte[]> createRestoreConsumer(TaskId id, Map<String, String> storeToChangelogTopic) {
 | 
			
		||||
        MockConsumer<byte[], byte[]> consumer = new MockConsumer<byte[], byte[]>(OffsetResetStrategy.LATEST) {
 | 
			
		||||
            @Override
 | 
			
		||||
            public synchronized void seekToEnd(Collection<TopicPartition> partitions) {
 | 
			
		||||
| 
						 | 
				
			
			@ -414,16 +412,16 @@ public class ProcessorTopologyTestDriver {
 | 
			
		|||
                return 0L;
 | 
			
		||||
            }
 | 
			
		||||
        };
 | 
			
		||||
        // For each store name ...
 | 
			
		||||
        for (String storeName : storeNames) {
 | 
			
		||||
            String topicName = ProcessorStateManager.storeChangelogTopic(applicationId, storeName);
 | 
			
		||||
        // For each store ...
 | 
			
		||||
        for (Map.Entry<String, String> storeAndTopic: storeToChangelogTopic.entrySet()) {
 | 
			
		||||
            String topicName = storeAndTopic.getValue();
 | 
			
		||||
            // Set up the restore-state topic ...
 | 
			
		||||
            // consumer.subscribe(new TopicPartition(topicName, 1));
 | 
			
		||||
            // Set up the partition that matches the ID (which is what ProcessorStateManager expects) ...
 | 
			
		||||
            List<PartitionInfo> partitionInfos = new ArrayList<>();
 | 
			
		||||
            partitionInfos.add(new PartitionInfo(topicName, id.partition, null, null, null));
 | 
			
		||||
            partitionInfos.add(new PartitionInfo(topicName, PARTITION_ID, null, null, null));
 | 
			
		||||
            consumer.updatePartitions(topicName, partitionInfos);
 | 
			
		||||
            consumer.updateEndOffsets(Collections.singletonMap(new TopicPartition(topicName, id.partition), 0L));
 | 
			
		||||
            consumer.updateEndOffsets(Collections.singletonMap(new TopicPartition(topicName, PARTITION_ID), 0L));
 | 
			
		||||
        }
 | 
			
		||||
        return consumer;
 | 
			
		||||
    }
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
		Loading…
	
		Reference in New Issue