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

Compare with Current View Page History

« Previous Version 7 Current »

Status

Current state: Under Discussion

Discussion thread: here 

JIRA: https://issues.apache.org/jira/browse/KAFKA-10629


Motivation

For many use cases of TopologyTestDriver, we don't need to specify properties parameter. As of https://github.com/apache/kafka/pull/9477, many TopologyTestDriver usages will have no configurations at all to specify, so we should provide a constructor that doesn't take a Properties argument. Right now, such configuration-free usages have to provide an empty Properties object.

There can be other use cases where we would like to specify initial clock time without specifying properties. So we can have another constructor which takes in parameter of initial clock time along with topology parameter.

Application-id set by default should be randomized so that no two tests can interfere with each other. 

Public Interfaces

Public Constructor
	/**
     * Create a new test diver instance.
     * Default test properties are used to initialize the driver instance
     *
     * @param topology the topology to be tested
     */
    public TopologyTestDriver(final Topology topology) {
        this(topology, new Properties());
    }

	/**
     * Create a new test diver instance.
     *
     * @param topology the topology to be tested
     * @param initialWallClockTimeMs the initial value of internally mocked wall-clock time
     */
	 public TopologyTestDriver(final Topology topology,
                           final Instant initialWallClockTimeMs) {
     	this(topology, new Properties(), initialWallClockTimeMs);
 	 }

Proposed Changes

We propose to add new constructor to TopologyTestDriver class. This constructor will have only Topology as parameter. Kafka streams config has APPLICATION_ID_CONFIG and BOOTSTRAP_SERVERS_CONFIG as required parameters. Those values will be provided in the private constructor of TopologyTestDriver. We want to set randomized application id to avoid conflicts with tests running in parallel. Proposing to change private constructor of TopologyTestDriver in following way.

 private TopologyTestDriver(final InternalTopologyBuilder builder,
                               final Properties config,
                               final long initialWallClockTimeMs) {
        final Properties configCopy = new Properties();
        configCopy.putAll(config);
        if (!configCopy.containsKey(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG)) {
            configCopy.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "dummy-bootstrap-host:0");
        }
        if (!configCopy.containsKey(StreamsConfig.APPLICATION_ID_CONFIG)) {
            // provide randomized dummy app-id if it's not specified
            configCopy.setProperty(StreamsConfig.APPLICATION_ID_CONFIG,
                    "dummy-topology-test-driver-app-id-" + ThreadLocalRandom.current().nextInt());
        }
		.
		.
		.
		.
}



Classes which will have usage for this constructor are 

  1. StreamsBuilderTest.java
  2. TopologyTest.java 
  3. AbstractStreamTest.java
  4. ProcessorNodeTest.java 
  5. TopologyTestDriver.java
  6. TestTopicsTest.java 
  7. TopologyTestDriverTest.java

Compatibility, Deprecation, and Migration Plan

The proposal is backward-compatible because it only adds a new constructor and does not change any existing constructors/methods. All changes will affect test classes and will get rid of redundant properties parameter.

Rejected Alternatives

If there are alternative ways of accomplishing the same thing, what were they? The purpose of this section is to motivate why the design is the way it is and not some other way.

  • No labels