Versions Compared

Key

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

...

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. 

...

Code Block
languagejava
themeEclipse
titlePublic Constructor
linenumberstrue
	/**
     * 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.
     * Initialized the internally mocked wall-clock time with {@link System#currentTimeMillis() current system time}.
     *
     * @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.

...