KAFKA-9273: Extract testShouldAutoShutdownOnIncompleteMetadata from S…#9108
KAFKA-9273: Extract testShouldAutoShutdownOnIncompleteMetadata from S…#9108bbejeck merged 4 commits intoapache:trunkfrom
Conversation
|
retest this please |
…treamTableJoinIntegrationTest into its own test The main goal is to remove usage of embedded broker (EmbeddedKafkaCluster) in AbstractJoinIntegrationTest and its subclasses. This is because the tests under this class are no longer using the embedded broker, except for two. testShouldAutoShutdownOnIncompleteMetadata is one of such tests. Furthermore, this test does not actually perfom stream-table join; it is testing an edge case of joining with a non-existent topic, so it should be in a separate test.
48e3f83 to
1669509
Compare
|
Hi, there's another unit test that need to be moved out ( (Btw, I force-push updated the commit with the right user email) |
|
@albert02lowis I see, let's try to do all of them in one PR then. |
…amStreamJoinIntegrationTest into its own test. shouldNotAccessJoinStoresWhenGivingName is the last unit test that has a dependency to embedded broker in AbstractJoinIntegrationTest. This test does not actually test the behaviour of stream-stream join; It is testing an edge case of accessing the store of a join, so it should be in a separate test.
…egrationTest. The tests under AbstractJoinIntegrationTest previously requires embedded broker, but it is better to use TopologyTestDriver instead. Existing stream-stream join tests that still make use of embedded broker is moved to use TopologyTestDriver, by specifying it under BOOTSTRAP_SERVERS_CONFIG. And then the old EmbeddedKafkaCluster in AbstractJoinIntegrationTest is safely removed.
79f4c72 to
865379e
Compare
|
Hi @abbccdda sure thing, I have added 2 more commits to this PR and the unit + integration tests also looks good in my local |
|
Ok to test |
bbejeck
left a comment
There was a problem hiding this comment.
@albert02lowis thanks for the contribution. Overall the PR looks good to me, after the one comment is addressed we'll probably get this merged.
| public static void setupConfigsAndUtils() { | ||
|
|
||
| STREAMS_CONFIG.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); | ||
| STREAMS_CONFIG.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); |
There was a problem hiding this comment.
@albert02lowis I've confirmed locally that the test failures are related. The TopologyTestDriver still needs a bootstrap servers config value.
Adding something like STREAMS_CONFIG.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "test:0000"); into the AbstractJoinIntegrationTest.setupConfigsAndUtils() gets all tests passing.
I'm not sure how the JDK 14 tests passed as locally I got the same errors as the JDK 8 and 11 build locally.
There was a problem hiding this comment.
I see, let me add back this BOOTSTRAP_SERVERS_CONFIG inside setupConfigsAndUtils then 👍🏻
There was a problem hiding this comment.
Hi @bbejeck I have done the required change in a new commit. I have also extracted out redundant calls to set BOOTSTRAP_SERVERS_CONFIG in the subclasses' test methods.
…egrationTest. Previously the setting of BOOTSTRAP_SERVERS_CONFIG is added on each test method, but this can actually be extracted to the BeforeClass method which is setupConfigsAndUtils in AbstractJoinIntegrationTest.
|
retest this please |
|
Retest this please. |
|
Ok to test |
|
retest this please |
|
ok to test |
bbejeck
left a comment
There was a problem hiding this comment.
Thanks for the update @albert02lowis, this LGTM. The streams build passed for me locally. Once we can get a build running here, I'll merge it.
|
retest this please |
|
Java 8 failed on an unrelated test retest this please |
|
Java 8 passed retest this please |
|
Java 11 passed retest this please |
|
merged #9108 into trunk Thanks for the contribution @albert02lowis! |
apache#9108) The main goal is to remove usage of embedded broker (EmbeddedKafkaCluster) in AbstractJoinIntegrationTest and its subclasses. This is because the tests under this class are no longer using the embedded broker, except for two. testShouldAutoShutdownOnIncompleteMetadata is one of such tests. Furthermore, this test does not actually perfom stream-table join; it is testing an edge case of joining with a non-existent topic, so it should be in a separate test. Testing strategy: run existing unit and integration test Reviewers: Boyang Chen <boyang@confluent.io>, Bill Bejeck <bbejeck@apache.org>
apache#9108) The main goal is to remove usage of embedded broker (EmbeddedKafkaCluster) in AbstractJoinIntegrationTest and its subclasses. This is because the tests under this class are no longer using the embedded broker, except for two. testShouldAutoShutdownOnIncompleteMetadata is one of such tests. Furthermore, this test does not actually perfom stream-table join; it is testing an edge case of joining with a non-existent topic, so it should be in a separate test. Testing strategy: run existing unit and integration test Reviewers: Boyang Chen <boyang@confluent.io>, Bill Bejeck <bbejeck@apache.org>
…treamTableJoinIntegrationTest into its own test
The main goal is to remove usage of embedded broker (EmbeddedKafkaCluster) in AbstractJoinIntegrationTest and its subclasses.
This is because the tests under this class are no longer using the embedded broker, except for two.
testShouldAutoShutdownOnIncompleteMetadata is one of such tests.
Furthermore, this test does not actually perfom stream-table join; it is testing an edge case of joining with a non-existent topic, so it should be in a separate test.
Testing strategy: run existing unit and integration test
@bbejeck
Committer Checklist (excluded from commit message)