You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 3 Next »

Status

Current state: Draft

Discussion thread:

JIRA: Unable to render Jira issues macro, execution error.

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

THIS IS STILL WORK IN PROGRESS.

This KIP is inspired by the Discussion in KIP-456: Helper classes to make it simpler to write test logic with TopologyTestDriver

The proposal in KIP-456 was to add alternate way to input and output topic, but this KIP enhance those classes and deprecate old functionality to make clear interface for test writer to use.

When using the old TopologyTestDriver you need to call ConsumerRecordFactory to create ConsumerRecord passed into pipeInput method to write to topic. Also when calling readOutput to consume from topic, you need to provide correct Deserializers each time.

You easily end up writing helper methods in your test classes, but this can be avoided when adding generic input and output topic classes to implement the needed functionality.

Also the logic of the old TopologyTestDriver is confusing, when you need to pipe ConsumerRecords to produce record to input topic and receive ProducerRecords when consuming from output topic.

Non-existing topic and no record in the queue scenarious are modified to throw Exception instead of returning null.

Public Interfaces


KafkaClient
package org.apache.kafka.clients;

//New Interface to make possible to use ProducereRecord in TopologyTestDriver
//ConsumerRecord not modified to implement this because timestamp() return long, not Long
public interface ClientRecord<K, V> {
    String topic();
    Headers headers();
    K key();
    V value();
    Long timestamp();
}
ProducerRecord
package org.apache.kafka.clients.producer;
public class ProducerRecord<K, V> implements ClientRecord<K, V> {
...



TopologyTestDriver
package org.apache.kafka.streams;
 
@Deprecated
public class TopologyTestDriver {
	...
 
    @Deprecate public void pipeInput(ConsumerRecord<byte[], byte[]> record); // can trigger event-time punctuation
    @Deprecate public void pipeInput(List<ConsumerRecord<byte[], byte[]>> records); // can trigger event-time punctuation
 
    // methods for result verification
  
    @Deprecate public ProducerRecord<byte[], byte[]> readOutput(String topic);
    @Deprecate public <K, V> ProducerRecord<K, V> readOutput(String topic, Deserializer<K> keyDeserializer, Deserializer<V> valueDeserializer);
 
    ...
}


TopologyTestDriver (new)
package org.apache.kafka.streams.test;

public class TopologyTestDriver {
	public TopologyTestDriver(Topology topology, Properties config); // initialized WallClockMockTimeMs with System.currentTimeMillis()

	public TopologyTestDriver(Topology topology, Properties config, long initialWallClockTimeMs);

	// event-time is automatically advanced based on the provided input records, and thus event-time punctuation are triggered automatically
	// wall-clock time is mocked and not advanced automatically; user can advance wall-clock time manually, and thus, trigger wall-clock time punctuations manually
	public void advanceWallClockTime(long advanceMs); // can trigger wall-clock-time punctuation

	// methods for TestTopic object creation
	//TODO investigate, is there way to find out serde based on topology
	public final <K, V> TestInputTopic<K, V> createInputTopic(final String topicName, final Serde<K> keySerde, final Serde<V> valueSerde);
	public final <K, V> TestOutputTopic<K, V> createOutputTopic(final String topicName, final Serde<K> keySerde, final Serde<V> valueSerde);

	public Map<String, StateStore> getAllStateStores()
	public StateStore getStateStore(String name);
	public <K, V> KeyValueStore<K, V> getKeyValueStore(String name);
	public <K, V> WindowStore<K, V> getWindowStore(String name);
	public <K, V> SessionStore<K, V> getSessionStore(String name);

	public void close();
}


TestInputTopic
package org.apache.kafka.streams.test;

public class TestInputTopic<K, V> {
    //Create by TopologyTestDriver, Constructors are package private

    //Timestamp handling 
    //Record timestamp can be provided when piping input or use internally tracked time initialized configured with configureTiming:
    //startTimestampMs the initial timestamp for generated records, if not provided uses current system time as start timestamp.
    //autoAdvanceMs the time increment per generated record, if not provided auto-advance is disabled.

    void configureTiming(final long startTimestampMs, final long autoAdvanceMs);
    //Advances the internally tracked time.
    void advanceTimeMs(final long advanceMs);

	//Methods to pipe single record
    void pipeInput(final V value);
    void pipeInput(final V value, final long timestampMs);

    void pipeInput(final K key, final V value);
    void pipeInput(final K key, final V value, final long timestampMs);

    void pipeInput(final ClientRecord<K, V> record);
    void pipeInput(final ClientRecord<K, V> record, final long timestampMs);

	//Methods to pipe list of records
    void pipeValueList(final List<V> values);
    void pipeValueList(final List<V> values, final long startTimestamp, final long advanceMs);

    void pipeKeyValueList(final List<KeyValue<K, V>> keyValues);
    void pipeKeyValueList(final List<KeyValue<K, V>> keyValues, final long startTimestamp, final long advanceMs);

    void pipeRecordList(final List<ClientRecord<K, V>> records);
    void pipeRecordList(final List<ClientRecord<K, V>> records, final long startTimestamp, final long advanceMs);
}
TestOutputTopic
package org.apache.kafka.streams.test;

public class TestOutputTopic<K, V> {
    //Create by TopologyTestDriver, Constructors are package private

	//Method to check queue size
    final long getQueueSize();
    final boolean isEmpty();

	//Methods to readOutput
    V readValue();
    KeyValue<K, V> readKeyValue();
    ClientRecord<K, V> readRecord();

    //Output as collection
    List<V> readValuesToList();
    Map<K, V> readKeyValuesToMap();
    List<KeyValue<K, V>> readKeyValuesToList();
    List<ClientRecord<K, V>> readRecordToList();
}
OutputVerifier
package org.apache.kafka.streams.test;
 
//OutputVerifier modified to accept ClientRecord instead of ProducerRecord
public class OutputVerifier {
 
    public static <K,V> void compareValue(ClientRecord<K, V> record, V expectedValue) throws AssertionError;
    public static <K,V> void compareValue(ClientRecord<K, V> record, ClientRecord<K, V> expectedRecord) throws AssertionError;
 
    public static <K,V> void compareKeyValue(ClientRecord<K, V> record, K expectedKey, V expectedValue) throws AssertionError;
    public static <K,V> void compareKeyValue(ClientRecord<K, V> record, ClientRecord<K, V> expectedRecord) throws AssertionError;
 
    public static <K,V> void compareValueTimestamp(ClientRecord<K, V> record, V expectedValue, long expectedTimestamp) throws AssertionError;
    public static <K,V> void compareValueTimestamp(ClientRecord<K, V> record, ClientRecord<K, V> expectedRecord) throws AssertionError;
 
    public static <K,V> void compareKeyValueTimestamp(ClientRecord<K, V> record, K expectedKey, V expectedValue, long expectedTimestamp) throws AssertionError;
    public static <K,V> void compareKeyValueTimestamp(ClientRecord<K, V> record, ClientRecord<K, V> expectedRecord) throws AssertionError;
 
}


ConsumerRecordFactory
package org.apache.kafka.streams.test;

@Deprecated
public class ConsumerRecordFactory<K, V> {


Proposed Changes

This improvement adds TestInputTopic class which replaces TopologyTestDriver  and ConsumerRecordFactory methods as one class to be used to write to Input Topics and TestOutputTopic class which collects TopologyTestDriver  reading methods and provide typesafe read methods.

Example
public class SimpleTopicTest {

  private TopologyTestDriver testDriver;
  private TestInputTopic<String, String> inputTopic;
  private TestOutputTopic<String, String> outputTopic;

  @Before
  public void setup() {
    testDriver = new TopologyTestDriver(TestStream.getTopology(), TestStream.getConfig());
	inputTopic = testDriver.createInputTopic(TestStream.INPUT_TOPIC, new Serdes.StringSerde(), new Serdes.StringSerde());
    outputTopic = testDriver.createOutputTopic(TestStream.OUTPUT_TOPIC, new Serdes.StringSerde(), new Serdes.LongSerde());
  }

  @After
  public void tearDown() {
      testDriver.close();
  }

  @Test
  public void testOneWord() {
    //Feed word "Hello" to inputTopic and no kafka key, timestamp is irrelevant in this case
    inputTopic.pipeInput("Hello");
    assertThat(outputTopic.readValue()).isEqualTo("Hello");
    //No more output in topic
    assertThat(outputTopic.isEmpty()).isTrue();
  }
}


  • New Example utilizing new classes test added to streams/examples/src/test/java/org/apache/kafka/streams/examples/wordcount/WordCountDemoTest.java


Compatibility, Deprecation, and Migration Plan

There are no compatibility issues.

The tests utilizing old TopologyTestDriver can still use deprecated methods.

Migration plan ???

Rejected Alternatives

  • No labels