Versions Compared

Key

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

Table of Contents

This page is meant as a template for writing a KIP. To create a KIP choose Tools->Copy on this page and modify with your content and replace the heading with the next KIP number and a description of your issue. Replace anything in italics with your own description.

Status

Current state: KIP DRAFT [One of "Under Discussion", "Accepted", "Rejected"] Discarded in favor of KIP-470: TopologyTestDriver test input and output usability improvements

Discussion thread: here [Change the link from the KIP proposal email archive to your own email thread]
JIRA: here [Change the link from KAFKA-1 to your own ticket] 

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

...

You easily end up writing helper methods in your test classedclasses, but this can be avoided when adding generic input and output topic classes wrapping existing functionality.

Public Interfaces



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

public class TestInputTopic<K, V> {
    //Timestamp handling 
    //Record timestamp can be provided when piping input or use internally tracked time initialized with constructors:
    //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.

    //Constructors with serializers
    TestInputTopic(final TopologyTestDriver driver, final String topicName, final Serializer<K> keySerializer, final Serializer<V> valueSerializer);
    TestInputTopic(final TopologyTestDriver driver, final String topicName, final Serializer<K> keySerializer, final Serializer<V> valueSerializer, final long startTimestampMs);
    TestInputTopic(final TopologyTestDriver driver, final String topicName, final Serializer<K> keySerializer, final Serializer<V> valueSerializer, final long startTimestampMs, final long autoAdvanceMs);
    //Constructors with serdes
    TestInputTopic(final TopologyTestDriver driver, final String topicName, final Serde<K> keySerde, final Serde<V> valueSerde);
    TestInputTopic(final TopologyTestDriver driver, final String topicName, final Serde<K> keySerde, final Serde<V> valueSerde, final long startTimestampMs);
    TestInputTopic(final TopologyTestDriver driver, final String topicName, final Serde<K> keySerde, final Serde<V> valueSerde, 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 key, final V value);
    void pipeInput(final V value, final long timestampMs);
    void pipeInput(final K key, final V value, final long timestampMs);
    void pipeInput(final K key, final V value, final Headers headers);
    void pipeInput(final K key, final V value, final Headers headers, final long timestampMs);

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



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

public class TestOutputTopic<K, V> {
	//Constructor with serializers
    TestOutputTopic(final TopologyTestDriver driver, final String topic, final Serde<K> keySerde, final Serde<V> valueSerde);
    //Constructor with serdes
    TestOutputTopic(final TopologyTestDriver driver, final String topic, final Deserializer<K> keyDeserializer, final Deserializer<V> valueDeserializer);

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

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


Proposed Changes

This improvement adds TestInputTopic class which wraps 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.

Public Interfaces

Briefly list any new interfaces that will be introduced as part of this proposal or any existing interfaces that will be removed or changed. The purpose of this section is to concisely call out the public contract that will come along with this feature.

A public interface is any change to the following:

  • Binary log format

  • The network protocol and api behavior

  • Any class in the public packages under clientsConfiguration, especially client configuration

    • org/apache/kafka/common/serialization

    • org/apache/kafka/common

    • org/apache/kafka/common/errors

    • org/apache/kafka/clients/producer

    • org/apache/kafka/clients/consumer (eventually, once stable)

  • Monitoring

  • Command line tools and arguments

  • Anything else that will likely break existing users in some way when they upgrade

Proposed Changes

...

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 = new TestInputTopic<>(testDriver, TestStream.INPUT_TOPIC, new Serdes.StringSerde(), new Serdes.StringSerde());
    outputTopic = new TestOutputTopic<>(testDriver, TestStream.OUTPUT_TOPIC, new Serdes.StringSerde(), new Serdes.StringSerde());
  }

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




Compatibility, Deprecation, and Migration Plan

  • What impact (if any) will there be on existing users?
  • If we are changing behavior how will we phase out the older behavior?
  • If we need special migration tools, describe them here.
  • When will we remove the existing behavior?

Rejected Alternatives

...

This is only adding new classes. The tests utilizing directly TopologyTestDriver can still be used. There are no compatibility issues.

Rejected Alternatives