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

Compare with Current View Page History

« Previous Version 4 Next »

Status

Current state: Under Discussion

Discussion thread: here

JIRA: here

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

Motivation

When using the TopologyTestDriver to assert topologies are working as expected, it can be useful to know the full set of topics the topology produces to during a test run - including internal changelog and repartition topics.  This information allows the full set of records produced during a test run to be captured without needing prior knowledge of all the the topic names.

Users wanting to ensure changes they make to code or Kafka Streams version can use this full set of produced records to compare the output before and after the changes.  If the full set of produced records remain consistent, then the user can be confident that their changes are compatible with the previous version.

Public Interfaces

Addition of a new method to TopologyTestDriver called getOutputTopicNames, that will return the name of all topic names with which it would be possible to call the existing createOutputTopic and get back a topic with data in it, i.e. it returns the names of the topics that topology has output records to.  This set includes internal and sink topics. 

The method name deliberately aligns with the `createOutputTopic` method, even though some may say that using 'output topic' is misleading as the method also returns the names of internal topics, e.g. repartition and changelog topics. As createOutputTopic already allows access to internal topics, the name getOutputTopicNames seems consistent with the existing method.  Using a different name would lead to a less cohesive API. 

/**
 * Get all the names of all the topics to which records have been output.
 * <p>
 * Call this method after piping the input into the test driver to retrieve the full set of topics the topology
 * produced records to.
 * <p>
 * The returned set of topic names includes changelog, repartition and sink topic names.
 *
 * @return the set of output topic names.
 */
public final Set<String> getOutputTopicNames(){...}

Proposed Changes

Phase 1:

public final Set<String> getOutputTopicNames() {
    return Collections.unmodifiableSet(outputRecordsByTopic.keySet());
}

Phase 2:

The implementation will be changed such that the key set of outputRecordsByTopic is initialized with all known output, i.e. internal and sink, topic names upon construction of the TopologyTestDriver from the TopologyDescription.  This second phase will mean getOutputTopicNames() will return most topic names even before any input is piped through the topology. Topics that use a non-static TopicNameExtractor will not  be populated on construction.

Compatibility, Deprecation, and Migration Plan

None

Rejected Alternatives

None

  • No labels