Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.
Comment: TestRecord and TestRecordFactory

Table of Contents

Status

Current state:  DraftUnder Discussion

Discussion thread:

JIRA:

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

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 stream application code is very compact and the test code is a lot of bigger code base than actual implementation of the application, that's why it would be good to get test code easily readable and understandable and that way also maintainable.

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 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.

...

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 VK valuekey, final longV timestampMsvalue);

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

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

	//Methods to pipe list of// records
Use record timestamp, does void pipeValueList(final List<V> values);not auto advance internally tracked time.
    void pipeValueListpipeInput(final List<V>ClientRecord<K, values, final long startTimestamp, final long advanceMs);

V> record);

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

 final long startTimestamp, final// long advanceMs);

    void pipeRecordList(final List<ClientRecord<K, V>> records);Use provided timestamp, does not auto advance internally tracked time.
    void pipeRecordListpipeValueList(final List<ClientRecord<K, V>> recordsList<V> values, final long startTimestamp, final long advanceMs);
}
Code Block
languagejava
titleTestOutputTopic
package org.apache.kafka.streams;

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

    //Create Use byrecord TopologyTestDrivertimestamp, Constructorsdoes arenot package private

	//Method to check queue sizeauto advance internally tracked time.
    finalvoid pipeRecordList(final List<? extends ClientRecord<K, V>> records);
}



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

public class TestOutputTopic<K, V> {long getQueueSize();
    final boolean isEmpty();

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

    //Output as collection
    List<V> readValuesToList//Create by TopologyTestDriver, Constructors are package private

	//Method to check queue size
    final long getQueueSize();
    Map<K,final V>boolean readKeyValuesToMapisEmpty();

	//Methods to readOutput, throw NoSuchElement if no record in queue
   List<KeyValue<K, V>>V readKeyValuesToListreadValue();
    List<ClientRecord<KKeyValue<K, V>>V> readRecordToListreadKeyValue();
}
Code Block
languagejava
titleOutputVerifier
package org.apache.kafka.streams.test;
 
//OutputVerifier modified toClientRecord<K, accept ClientRecord instead of ProducerRecord
public class OutputVerifier {
 
    public static <K,V> void compareValue(ClientRecord<K, V> record, V expectedValue) throws AssertionErrorV> readRecord();

    //Output as collection
    List<V> readValuesToList();
    Map<K, V> readKeyValuesToMap();
    public static <KList<KeyValue<K,V> voidV>> readKeyValuesToList();
    List<ClientRecord<K, V>> readRecordToList();
}


Code Block
languagejava
titleOutputVerifier
package org.apache.kafka.streams.testcompareValue(ClientRecord<K, V> record, ClientRecord<K, V> expectedRecord) throws AssertionError;
 
//OutputVerifier modified to accept ClientRecord instead of ProducerRecord
public class OutputVerifier {
 
    public static <K,V> void compareKeyValuecompareValue(ClientRecord<K, V> record, K expectedKey, V expectedValueV expectedValue) throws AssertionError;
    public static <K,V> void compareKeyValuecompareValue(ClientRecord<K, V> record, ClientRecord<K, V> expectedRecord) throws AssertionError;
 
    public static <K,V> void compareValueTimestampcompareKeyValue(ClientRecord<K, V> record, VK expectedValueexpectedKey, longV expectedTimestampexpectedValue) throws AssertionError;
    public static <K,V> void compareValueTimestampcompareKeyValue(ClientRecord<K, V> record, ClientRecord<K, V> expectedRecord) throws AssertionError;
 
    public static <K,V> void compareKeyValueTimestampcompareValueTimestamp(ClientRecord<K, V> record, K expectedKey, V expectedValue, long expectedTimestamp) throws AssertionError;
    public static <K,V> void compareKeyValueTimestampcompareValueTimestamp(ClientRecord<K, V> record, ClientRecord<K, V> expectedRecord) throws AssertionError;
 
}
Code Block
languagejava
titleConsumerRecordFactory
package    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

}


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(String topic, Long timestamp, K key, V value);
    public TestRecord(String topic, K key, V value, Headers headers);
    public TestRecord(String topic, K key, V value);
    public TestRecord(String topic, V value);
	
	//Constructor by based on existing record
    public TestRecord(ConsumerRecord<K, V> record);
    public TestRecord(ClientRecord<K, V> record);
	
	// ClientRecord interface
    public String topic();
    public Headers headers();
    public K key();
    public V value();
    public Long timestamp();
	
	//Overrides
    public String toString();
    public boolean equals(Object o);
    public int hashCode();
}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.

...