KAFKA-7944: Improve Suppress test coverage#6382
KAFKA-7944: Improve Suppress test coverage#6382guozhangwang merged 6 commits intoapache:trunkfrom vvcephei:KAFKA-7944-suppress-window-test
Conversation
* add a normal windowed suppress with short windows and a short grace period * improve the smoke test so that it actually verifies the indended conditions
vvcephei
left a comment
There was a problem hiding this comment.
@guozhangwang @mjsax @bbejeck @ableegoldman ,
In response to a mailing list message that they still saw a problem with Suppress after the fix, I bumped up the priority of KAFKA-7944. (this was similar to the reporter's topology)
I also took the time to double check we weren't getting a false negative (test passing without the intended verification), and learned that actually we could get false negatives for a number of reasons:
- we could process the entire data set before the first bounce.
- sometimes, the node that we bounce hadn't actually processed anything (it was all assigned to the other node).
I also wanted to verify the smoke test app with EOS.
Since I took the time to refactor the smoke_test system test to achieve these goals, and there was a large overlap with the bounce test, I also unified the tests by adding the crash parameter to the smoke test.
I requested a review from you all so that we can get as much scrutiny on this system test as possible. Trying not to create a new flaky test here...
| final ArrayList<SmokeTestClient> clients = new ArrayList<>(); | ||
|
|
||
| CLUSTER.createTopics(SmokeTestDriver.topics()); | ||
| IntegrationTestUtils.cleanStateBeforeTest(CLUSTER, SmokeTestDriver.topics()); |
There was a problem hiding this comment.
More stable and future-proof test setup code. I needed this because I added a second test here, which I've since removed.
Leaving this change, though, so we won't have to waste time debugging again next time we try to add a test.
| public static Collection<Object[]> parameters() { | ||
| return Arrays.asList(new Object[] {false}, new Object[] {true}); | ||
| } | ||
|
|
| final KafkaStreams streamsClient = new KafkaStreams(build, getStreamsConfig(props)); | ||
| streamsClient.setStateListener((newState, oldState) -> { | ||
| System.out.printf("%s: %s -> %s%n", name, oldState, newState); | ||
| System.out.printf("%s %s: %s -> %s%n", name, Instant.now(), oldState, newState); |
There was a problem hiding this comment.
I added this while debugging the smoke test. Printing the time allows us to correlate events across test nodes and with the test logs.
|
|
||
| streamify(smallWindowSum, "sws-raw"); | ||
| streamify(smallWindowSum.suppress(untilWindowCloses(BufferConfig.unbounded())), "sws-suppressed"); | ||
|
|
There was a problem hiding this comment.
Added a "small window sum" stream to the topology, which does a "normal" windowed computation. The exact data production timing is non-deterministic, so we can't verify the results, but we can verify that there is exactly one result per windowed key in the suppressed stream
|
|
||
| public class SmokeTestDriver extends SmokeTestUtil { | ||
| private static final String[] TOPICS = new String[] { | ||
| private static final String[] TOPICS = { |
There was a problem hiding this comment.
Apparently this is unnecessary now?
| processor2.start() | ||
| monitor3.wait_until('REBALANCING -> RUNNING', | ||
| timeout_sec=120, | ||
| err_msg="Never saw 'REBALANCING -> RUNNING' message " + str(processor2.node.account) |
There was a problem hiding this comment.
Again, wait for the node to start. This isn't strictly necessary, since we're just going to wait for "processed" next, but I thought it made the tests more readable and debuggable. For example, when you get a test failure, you'll know whether the node failed to start or whether it started, but didn't process anything.
There was a problem hiding this comment.
Not a comment: I actually think it is indeed necessary to avoid flakiness, remember @bbejeck once talked about it in an older PR?
There was a problem hiding this comment.
I think that was a different situation, like maybe we stopped and then re-started a node? In that case, you need to grab a new monitor after you stop and before you re-start, to be sure your grep won't match the "started" message from the first time it was running.
Since we're querying the same monitor here in both blocks, we're grepping over the same range of the file. The monitor pins the start-point for grepping when you create it. For example, since we create this monitor before we start Streams, it uses "byte 1" as the starting points for all greps for both REBALANCING -> RUNNING and processed.
| ) | ||
|
|
||
| # make sure we're not already done processing (which would invalidate the test) | ||
| self.driver.node.account.ssh("! grep 'Result Verification' %s" % self.driver.STDOUT_FILE, allow_fail=False) |
There was a problem hiding this comment.
Again, before bouncing the node, make sure the verification isn't already done, which would be pointless.
| self.processor2.stop() | ||
| self.processor3.stop() | ||
| self.processor4.stop() | ||
| processor3.stop() |
There was a problem hiding this comment.
The test is already over. We can stop the app gracefully.
| node = self.driver.node | ||
| node.account.ssh("grep SUCCESS %s" % self.driver.STDOUT_FILE, allow_fail=False) | ||
| if crash and not eos: | ||
| self.driver.node.account.ssh("grep -E 'SUCCESS|PROCESSED-MORE-THAN-GENERATED' %s" % self.driver.STDOUT_FILE, allow_fail=False) |
There was a problem hiding this comment.
If we crash without EOS, we might process some duplicates (this is the whole point of EOS), so Streams is operating properly if it either does exactly the right thing or processes too many records.
| if crash and not eos: | ||
| self.driver.node.account.ssh("grep -E 'SUCCESS|PROCESSED-MORE-THAN-GENERATED' %s" % self.driver.STDOUT_FILE, allow_fail=False) | ||
| else: | ||
| self.driver.node.account.ssh("grep SUCCESS %s" % self.driver.STDOUT_FILE, allow_fail=False) |
There was a problem hiding this comment.
In all other cases, we expect to get exactly the right result.
|
System tests:
|
|
\cc @ableegoldman for review |
| public static Map<String, Set<Integer>> generate(final String kafka, | ||
| final int numKeys, | ||
| final int maxRecordsPerKey, | ||
| final Duration timeToSpend, |
|
|
||
| class StreamsSmokeTestEOSJobRunnerService(StreamsSmokeTestBaseService): | ||
| def __init__(self, test_context, kafka): | ||
| super(StreamsSmokeTestEOSJobRunnerService, self).__init__(test_context, kafka, "process-eos") |
There was a problem hiding this comment.
I thought there were already existingEOS tests that use the StreamsSmokeTest?
|
|
||
| processor2.stop_nodes(not crash) | ||
|
|
||
| processor3.start() |
There was a problem hiding this comment.
Don't we want to at least processor3 starts successfully here?
There was a problem hiding this comment.
We could wait on that, it might make the test more readable. But it's not strictly necessary, since if it doesn't start, then we won't finish processing.
I'll add it for readability.
There was a problem hiding this comment.
Done, it occurred to me during the change that it also has the side benefit of making sure that the third processor does process some events.
|
|
| final KafkaStreams streamsClient = new KafkaStreams(build, getStreamsConfig(props)); | ||
| streamsClient.setStateListener((newState, oldState) -> { | ||
| System.out.printf("%s: %s -> %s%n", name, oldState, newState); | ||
| System.out.printf("%s %s: %s -> %s%n", name, Instant.now(), oldState, newState); |
| } | ||
|
|
||
|
|
||
| @SuppressWarnings("DynamicRegexReplaceableByCompiledPattern") |
There was a problem hiding this comment.
There was a problem hiding this comment.
Ah, oops. I'll remove it.
| public static Map<String, Set<Integer>> generate(final String kafka, | ||
| final int numKeys, | ||
| final int maxRecordsPerKey, | ||
| final Duration timeToSpend, |
There was a problem hiding this comment.
hey @vvcephei I'm wondering if we can simplify the code further: with timeToSpend now can we just remove autoTerminate?
-
SmokeTestDriverIntegrationTest.java: we are sending 1000 * 10 = 10K records only, so likely that will stop even before the second instance can be started, similar to the issue you observed in system test. I'd suggest we just set
timeToSpendto 10 seconds so we have some enough time to start up to 10 / 1 (sleep time) = 10 instances. -
As for StreamsSmokeTest itself, we only disableAutoTerminate in three of StreamsUpgradeTest cases, since we do not need to verify the number of records sent / received but only check upgrade completed. We can also use 30 seconds (not sure if it is sufficient, but we can get on average how much time those three tests will take from our nightlies), and even if the test completes before that it will call
driver.stopto force-stop the driver anyways.
| public static Map<String, Set<Integer>> generate(final String kafka, | ||
| final int numKeys, | ||
| final int maxRecordsPerKey, | ||
| final Duration timeToSpend, |
There was a problem hiding this comment.
Of course we can then remove the whole disableAutoTerminate thing from StreamsSmokeTest.
| import time | ||
|
|
||
|
|
||
| class StreamsBounceTest(KafkaTest): |
| // this starts the stream processing app | ||
| new SmokeTestClient(UUID.randomUUID().toString()).start(streamsProperties); | ||
| break; | ||
| case "process-eos": |
There was a problem hiding this comment.
Can we use StreamsEosTest instead? See my other comment below.
|
|
||
| class StreamsSmokeTestEOSJobRunnerService(StreamsSmokeTestBaseService): | ||
| def __init__(self, test_context, kafka): | ||
| super(StreamsSmokeTestEOSJobRunnerService, self).__init__(test_context, kafka, "process-eos") |
There was a problem hiding this comment.
To my other comment above: what's the difference of running StreamsSmokeTestBaseService#process-eos v.s. StreamsEosTestBaseService#process? The former uses StreamsSmokeTest while the latter use StreamsEosTest client. Can we just consolidate them?
| node = self.driver.node | ||
| node.account.ssh("grep SUCCESS %s" % self.driver.STDOUT_FILE, allow_fail=False) | ||
| if crash and not eos: | ||
| self.driver.node.account.ssh("grep -E 'SUCCESS|PROCESSED-MORE-THAN-GENERATED' %s" % self.driver.STDOUT_FILE, allow_fail=False) |
| processor2.start() | ||
| monitor3.wait_until('REBALANCING -> RUNNING', | ||
| timeout_sec=120, | ||
| err_msg="Never saw 'REBALANCING -> RUNNING' message " + str(processor2.node.account) |
There was a problem hiding this comment.
Not a comment: I actually think it is indeed necessary to avoid flakiness, remember @bbejeck once talked about it in an older PR?
|
Both Java 8 and Java 11 failed but build results already cleaned up. retest this please |
|
Jenkins threw some crazy exception during the system tests. Re-running: System tests passed: http://confluent-kafka-branch-builder-system-test-results.s3-us-west-2.amazonaws.com/2019-03-07--001.1552001126--vvcephei--KAFKA-7944-suppress-window-test--48dabcd/report.html |
|
Reported unrelated failure: https://issues.apache.org/jira/browse/KAFKA-7965 Retest this, please. |
|
test results are unavailable. Retest this, please. |
|
Thanks for the response, @guozhangwang . I've created https://issues.apache.org/jira/browse/KAFKA-8080 to follow up. |
|
test results are unavailable. Retest this, please. |
|
Thanks, @guozhangwang ! |
* warn-apache-kafka/trunk: (41 commits) MINOR: Avoid double null check in KStream#transform() (apache#6429) KAFKA-7944: Improve Suppress test coverage (apache#6382) KAFKA-3522: add missing guards for TimestampedXxxStore (apache#6356) MINOR: Change Trogdor agent's cleanup executor to a cached thread pool (apache#6309) KAFKA-7976; Update config before notifying controller of unclean leader update (apache#6426) KAFKA-7801: TopicCommand should not be able to alter transaction topic partition count KAFKA-8091; Wait for processor shutdown before testing removed listeners (apache#6425) MINOR: Update delete topics zk path in assertion error messages KAFKA-7939: Fix timing issue in KafkaAdminClientTest.testCreateTopicsRetryBackoff KAFKA-7922: Return authorized operations in Metadata request response (KIP-430 Part-2) MINOR: Print usage when parse fails during console producer MINOR: fix Scala compiler warning (apache#6417) KAFKA-7288; Fix check in SelectorTest to wait for no buffered bytes (apache#6415) KAFKA-8065: restore original input record timestamp in forward() (apache#6393) MINOR: cleanup deprectaion annotations (apache#6290) KAFKA-3522: Add TimestampedWindowStore builder/runtime classes (apache#6173) KAFKA-8069; Fix early expiration of offsets due to invalid loading of expire timestamp (apache#6401) KAFKA-8070: Increase consumer startup timeout in system tests (apache#6405) KAFKA-8040: Streams handle initTransactions timeout (apache#6372) KAFKA-7980 - Fix timing issue in SocketServerTest.testConnectionRateLimit (apache#6391) ...
* add a normal windowed suppress with short windows and a short grace period * improve the smoke test so that it actually verifies the intended conditions See https://issues.apache.org/jira/browse/KAFKA-7944 Reviewers: Bill Bejeck <bill@confluent.io>, Guozhang Wang <guozhang@confluent.io>
After apache#6382, the system test streams_eos_test.py is redundant. As in apache#20718, the verification logic has already been migrated, so we only need to delete the related system tests Reviewers: Matthias J. Sax <matthias@confluent.io>
period
conditions
See https://issues.apache.org/jira/browse/KAFKA-7944
Committer Checklist (excluded from commit message)