Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

Table of Contents

Status

Current state: Under Discussion'Accepted'

Discussion thread: here

JIRA:

Jira
serverASF JIRA
columnskey,summary,type,created,updated,due,assignee,reporter,priority,status,resolution
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyKAFKA-8233

...

Code Block
languagejava
titleKafkaClientTopologyTestDriver
package org.apache.kafka.clientsstreams;

//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> {public class TopologyTestDriver {
    public TopologyTestDriver(Topology topology, Properties config); // existing constructor
    @Deprecate public TopologyTestDriver(Topology topology, Properties config, long initialWallClockTimeMs);
    Stringpublic topicTopologyTestDriver();
Topology topology, Properties config, HeadersInstant headers(initialWallClockTime);

    @Deprecate Kpublic void keyadvanceWallClockTime(long advanceMs);
 // can  V value();
    Long timestamp();
}
Code Block
titleProducerRecord
package org.apache.kafka.clients.producer;
public class ProducerRecord<K, V> implements ClientRecord<K, V> {
...
Code Block
languagejava
titleTopologyTestDriver
package org.apache.kafka.streams;

public class TopologyTestDriver {
	public TopologyTestDriver(Topology topology, Properties configtrigger wall-clock-time punctuation
	public void advanceWallClockTime(Duration advance); // can trigger wall-clock-time punctuation

    //Deprecate old pipe and read methods
    @Deprecate public void pipeInput(ConsumerRecord<byte[], byte[]> record); // initializedcan WallClockMockTimeMstrigger with System.currentTimeMillis()

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

    //Deprecate old pipe and read methodsevent-time punctuation
    @Deprecate public void pipeInput(List<ConsumerRecord<byte[], byte[]>> records); // can trigger event-time punctuation
    @Deprecate public void pipeInput(ConsumerRecord<byteProducerRecord<byte[], byte[]> recordreadOutput(String topic); // can trigger event-time punctuation
    @Deprecate public <K, void pipeInput(List<ConsumerRecord<byte[], byte[]>> records); // can trigger event-time punctuation
    @Deprecate public ProducerRecord<byte[], byte[]> readOutput(String topic);
    @Deprecate publicV> ProducerRecord<K, V> readOutput(String topic, Deserializer<K> keyDeserializer, Deserializer<V> valueDeserializer);

	// methods for TestTopic object creation
	public final <K, V> ProducerRecord<KTestOutputTopic<K, V> readOutputcreateOutputTopic(final String topictopicName, Deserializer<K>final keyDeserializerSerializer<K> keySerializer, final Deserializer<V>Serializer<V> valueDeserializervalueSerializer);

	// event- Uses current system time isas automaticallystart 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();
}
timestamp. Auto-advance is disabled.
	public final <K, V> TestInputTopic<K, V> createInputTopic(final String topicName, final Deserializer<K> keyDeserializer, final Deserializer<V> valueDeserializer);
    //Uses provided startTimestamp and autoAdvance duration for timestamp generation
	public final <K, V> TestInputTopic<K, V> createInputTopic(final String topicName, final Deserializer<K> keyDeserializer, final Deserializer<V> valueDeserializer, final Instant startTimestamp, final Duration autoAdvance);

    ...
}



Code Block
languagejava
titleTestInputTopic
package org.apache.kafka.streams;

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 configured with parameters:
    //startTimestamp the initial timestamp for generated records, if not provided uses current system time as start timestamp.
    //autoAdvance the time increment per generated record, if not provided auto-advance is disabled.

    //Advances the internally tracked time.
    void advanceTime(final Duration advance);

	//Methods to pipe single record
    void pipeInput(final V value);
    void pipeInput(final K key, final V value);

    // Use provided timestamp, does not auto advance internally tracked time.
    void pipeInput(final V value, final Instant timestamp);
    void pipeInput(final K key, final V value, final Instant timestamp);

    // Method with long provided to support easier migration of old tests
   void pipeInput(final K key, final V value, final long timestampMs);

    // If record timestamp set, does not auto advance internally tracked time.
Code Block
languagejava
titleTestInputTopic
package org.apache.kafka.streams;

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.

	//Reinitialize timestamp and advanceMs
    public void configureTiming(final long startTimestampMs);
    public 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 K keyTestRecord<K, finalV> V valuerecord);

    	//Methods Useto pipe providedlist timestamp,of doesrecords
 not auto advance internallyvoid tracked time.
    void pipeInput(final V value, final long timestampMspipeValueList(final List<V> values);
    void pipeInputpipeKeyValueList(final K keyList<KeyValue<K, final V value, final long timestampMs);

    // Use record timestamp, does not auto advance internally tracked time.
    void pipeInput(final ClientRecord<K, V> record);

	//Methods to pipe list of records
    void pipeValueList(final List<V> values);
    void pipeKeyValueList(final List<KeyValue<K, V>> keyValues);

    // Use provided timestamp, does not auto advance internally tracked time.
    void pipeValueList(final List<V> values, final long startTimestamp, final long advanceMs);
    void pipeKeyValueList(final List<KeyValue<K, V>> keyValues, final long startTimestamp, final long advanceMs);

    // Use record timestamp, does not auto advance internally tracked time.
    void pipeRecordList(final List<? extends ClientRecord<K, V>> records);
}
Code Block
languagejava
titleTestOutputTopic
package org.apache.kafka.streams;

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, throw NoSuchElement if no record in queue
    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>> readRecordsToList();
}
Code Block
languagejava
titleOutputVerifier
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;
 
}
Code Block
languagejava
titleConsumerRecordFactory
package org.apache.kafka.streams.test;

@Deprecated
public class ConsumerRecordFactory<K, V> {
...
Code Block
languagejava
titleTestRecordFactory
package org.apache.kafka.streams.test;
	
//Mainly used by TestInputTopic
public class TestRecordFactory<K, V> {
    // default
    public TestRecordFactory(final String topicName) // initialized startTimestampMs with System.currentTimeMillis() and autoAdvanceMs with zero
    public TestRecordFactory(final String topicName, long startTimestampMs);
    public TestRecordFactory(final String topicName, long startTimestampMs, long autoAdvanceMs);
	
	//Reinitialize timestamp and advanceMs
    public void configureTiming(final long startTimestampMs);
    public void configureTiming(final long startTimestampMs, final long autoAdvanceMs);

	//Advance current timestamp
    public void advanceTimeMs(final long advanceMs);
	
	// create single records with default topic name and given timestamp
    public TestRecord<K, V> create(K key, V value, long timestampMs);
    public TestRecord<K, V> create(V value, long timestampMs);
    public TestRecord<K, V> create(K key, V value, Headers headers, long timestampMs);
    public TestRecord<K, V> create(V value, Headers headers, long timestampMs);

    // create single records with default topic name and generated timestamp
	public TestRecord<K, V> create(K key, V value);
    public TestRecord<K, V> create(V value); 
    public TestRecord<K, V> create(K key, V value, Headers headers);
    public TestRecord<K, V> create(V value, Headers headers);

 
    // create list of records with default topic name internally tracked time
    public List<TestRecord<K, V>> create(final List<KeyValue<K, V>> keyValues);
	// does not auto advance internally tracked time.
    public List<TestRecord<K, V>> create(final List<KeyValue<K, V>> keyValues, long startTimestamp, long advanceMs);
    public List<TestRecord<K, V>> create(final List<KeyValue<K, V>> keyValues, long startTimestamp); //advanceMs =1

}
V>> keyValues);

    // Use provided timestamp, does not auto advance internally tracked time.
    void pipeValueList(final List<V> values, final Instant startTimestamp, final Duration advanceMs);
    void pipeKeyValueList(final List<KeyValue<K, V>> keyValues, final Instant startTimestamp, final Duration advanceMs);

    // If record timestamp set, does not auto advance internally tracked time.
    void pipeRecordList(final List<? extends TestRecord<K, V>> records);
}


Code Block
languagejava
titleTestOutputTopic
package org.apache.kafka.streams;

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, throw NoSuchElement if no record in queue
    V readValue();
    KeyValue<K, V> readKeyValue();
    TestRecord<K, V> readRecord();

    //Output as collection
    List<V> readValuesToList();
    List<KeyValue<K, V>> readKeyValuesToList();
    Map<K, V> readKeyValuesToMap();
    List<TestRecord<K, V>> readRecordsToList();
}


Code Block
languagejava
titleTestRecord
package org.apache.kafka.streams.test;
public class TestRecord<K, V> {
	//Constructors
    public TestRecord(final V value);
    public TestRecord(final K key, final V value);
    public TestRecord(final K key, final V value, final Headers headers);
    public TestRecord(final K key, final V value, final Instant recordTime);
    public TestRecord(final K key, final V value, final Headers headers, final Instant recordTime);
    public TestRecord(final K key, final V value, final Headers headers, final Long timestamp);
	
	//Constructor based on existing record
    public TestRecord(final ConsumerRecord<K, V> record
Code Block
languagejava
titleTestRecord
package org.apache.kafka.streams.test;
public class TestRecord<K, V> implements org.apache.kafka.clients.ClientRecord<K, V> {
	//Constructors
    public TestRecord(String topic, Long timestamp, K key, V value, Headers headers);
    public TestRecord(Stringfinal topicProducerRecord<K, Long timestamp, K key, V valueV> record);
	
	// Methods like in publicProducerRecord TestRecord(String topic, K key, V value,/ ConsumerRecord
    public Headers headers();
    public K TestRecord(String topic, K key,key();
    public V value();
    public TestRecord(String topic, V valueLong timestamp();
	
	//Constructor Getters
  by based onpublic existing recordHeaders getHeaders();
    public TestRecord(ConsumerRecord<K, V> recordK getKey();
    public TestRecord(ClientRecord<K, V> recordV getValue();
    public Instant getRecordTime();
	
	// ClientRecord interfaceOverrides
    public String toString();
    public Stringboolean topicequals(Object o);
    public Headersint headershashCode();
}


Code Block
languagejava
titleOutputVerifier
package org.apache.kafka.streams.test;

//Recommended to use normal publicassertion library methods 
@Deprecated
public class OutputVerifier {
...


Code Block
languagejava
titleConsumerRecordFactory
package org.apache.kafka.streams.test;

//Similar functionality now in TestInputTopic
@Deprecated
public class ConsumerRecordFactory<K, V> {
...K key();
    public V value();
    public Long timestamp();
	
	//Overrides
    public String toString();
    public boolean equals(Object o);
    public int hashCode();
}


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.

Code Block
languagejava
titleExample
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.StringSerdeStringDeserializer(), new Serdes.StringSerdeStringDeserializer());
    outputTopic = testDriver.createOutputTopic(TestStream.OUTPUT_TOPIC, new Serdes.StringSerdeStringSerializer(), new Serdes.LongSerdeLongSerializer());
  }

  @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();
  }
}

...

The tests utilizing old TopologyTestDriver can still use deprecated methods.Migration plan ???

Rejected Alternatives

  • This is will replace replacing KIP-456: Helper classes to make it simpler to write test logic with TopologyTestDriver if accepted
  • Deprecate current TestTopologyDriver and move new to test package. This would have enabled to keep also TestInputTopic and TestOutputTopic classes in test package, not in very crowded streams root package.
  • Add ClientRecord interface to client package and modifiy ProducerRecord (and / or ConsumerRecord) to implement it, to be to utilize OutputVerifier with ProducerRecord and TestRecord