Currently when we identify version probing we return early from onAssignment and never get to updating the TaskManager and general state with the new assignment. Since we do actually give out "real" assignments even during version probing, a StreamThread should take real ownership of its tasks/partitions including cleaning them up in onPartitionsRevoked which gets invoked when we call onLeavePrepare as part of triggering the follow-up rebalance.
Every member will always get an assignment encoded with the lowest common version, so there should be no problem decoding a VP assignment. We should just allow onAssignment to proceed as usual so that the TaskManager is in a consistent state, and knows what all its tasks/partitions are when the first rebalance completes and the next one is triggered.
Reviewers: Boyang Chen <boyang@confluent.io>, Matthias J. Sax <mjsax@apache.org>, Guozhang Wang <wangguoz@gmail.com>
We should not use StreamsMetricsImpl. threadLevelSensor directly which would only retrieve the sensor but would not add any metrics to the sensor. Generally speaking we should always use the corresponding-level Metrics class (e.g. ThreadMetrics) to get the sensors which are populated with metrics.
Reviewers: Guozhang Wang <wangguoz@gmail.com>, John Roesler <john@confluent.io>, Bill Bejeck <bbejeck@gmail.com>, Matthias J. Sax <matthias@confluent.io>
Rather than maintain hand coded protocol serialization code, Streams could use the same code-generation framework as Clients/Core.
There isn't a perfect match, since the code generation framework includes an assumption that you're generating "protocol messages", rather than just arbitrary blobs, but I think it's close enough to justify using it, and improving it over time.
Using the code generation allows us to drop a lot of detail-oriented, brittle, and hard-to-maintain serialization logic in favor of a schema spec.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Boyang Chen <boyang@confluent.io>, Bill Bejeck <bill@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Refactors metrics according to KIP-444
Introduces StateStoreMetrics as a central provider for state store metrics
Adds metric scope (a.k.a. store type) to the in-memory suppression buffer
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Bill Bejeck <bbejeck@gmail.com>
Third bugfix for the failing broker bounce system test with cooperative rebalancing:
tl;dr We need to remove everything associated with a task when it is closed, but in some cases (eg AssignedTasks#commit) on a TaskMigratedExceptionwe would close it as a zombie and then (only) remove the taskId from therunning` map. This left its partitions, restorers, state stores, etc around and in an undefined state, causing exceptions when closing and/or opening the stores again.
Longer explanation:
In AssignedTasks (the abstract class from which the standby and active task variations extend) a commit failure (even due to broker down/unavailable) is treated as a TaskMigratedException after which the failed task is closed as a zombie and removed from running -- the remaining tasks (ie those still in running are then also closed as zombies in the subsequent onPartitionsLost
However we do not remove the closed task from runningByPartition nor do we remove the corresponding changelogs, if restoring, from the StoreChangelogReader since that applies only to active tasks, and AssignedTasks is generic/abstract. The changelog reader then retains a mapping from the closed task's changelog partition to its CompositeRestoreListener (and does not replace this when the new one comes along after the rebalance). The restore listener has a reference to a specific RocksDBStore instance, one which was closed when the task was closed as a zombie, so it accidentally tries to restore to the "old" RocksDBStore instance rather than the new one that was just opened.
Although technically this bug existed before KIP-429, it was only uncovered now that we remove tasks and clear their state/partitions/etc one at a time. We don't technically need to cherrypick the fix back earlier as before we just blindly clear all data structures entirely during an eager rebalance.
Reviewers: Boyang Chen <boyang@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Inside onLeavePrepare we would look into the assignment and try to revoke the owned tasks and notify users via RebalanceListener#onPartitionsRevoked, and then clear the assignment.
However, the subscription's assignment is already cleared in this.subscriptions.unsubscribe(); which means user's rebalance listener would never be triggered. In other words, from consumer client's pov nothing is owned after unsubscribe, but from the user caller's pov the partitions are not revoked yet. For callers like Kafka Streams which rely on the rebalance listener to maintain their internal state, this leads to inconsistent state management and failure cases.
Before KIP-429 this issue is hidden away since every time the consumer re-joins the group later, it would still revoke everything anyways regardless of the passed-in parameters of the rebalance listener; with KIP-429 this is easier to reproduce now.
Our fixes are following:
• Inside unsubscribe, first do onLeavePrepare / maybeLeaveGroup and then subscription.unsubscribe. This we we are guaranteed that the streams' tasks are all closed as revoked by then.
• [Optimization] If the generation is reset due to fatal error from join / hb response etc, then we know that all partitions are lost, and we should not trigger onPartitionRevoked, but instead just onPartitionsLost inside onLeavePrepare. This is because we don't want to commit for lost tracks during rebalance which is doomed to fail as we don't have any generation info.
Reviewers: Matthias J. Sax <matthias@confluent.io>, A. Sophie Blee-Goldman <sophie@confluent.io>, Bill Bejeck <bill@confluent.io>, Guozhang Wang <guozhang@confluent.io>
Introduces TaskMetrics class
Introduces dropped-records
Replaces skipped-records with dropped-records with latest built-in
metrics version
Does not add standby-process-ratio and active-process-ratio
Does not refactor parent sensors for processor node metrics
Reviewers: Guozhang Wang <wangguoz@gmail.com>, John Roesler <john@confluent.io>, Bill Bejeck <bbejeck@gmail.com>
In the case of unclean close we still need to make sure all the stores are flushed before closing any.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Boyang Chen <boyang@confluent.io>, Bill Bejeck <bill@confluent.io>, Guozhang Wang <guozhang@confluent.io>
This is not guaranteed to actually fix queryOnRebalance, since the
failure could never be reproduced locally. I did not bump timeouts
because it looks like that has been done in the past for this test
without success. Instead this change makes the following improvements:
It waits for the application to be in a RUNNING state before
proceeding with the test.
It waits for the remaining instance to return to RUNNING state
within a timeout after rebalance. I observed once that we were able to
do the KV queries but the instance was still in REBALANCING, so this
should reduce some opportunity for flakiness.
The meat of this change: we now iterate over all keys in one shot
(vs. one at a time with a timeout) and collect various failures, all of
which are reported at the end. This should help us to narrow down the
cause of flakiness if it shows up again.
Reviewers: Bill Bejeck <bill@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
For scenarios where the incoming traffic of all input partitions are small, there's a pitfall that the enforced processing timer is not reset after we have enforce processed ALL records. The fix itself is pretty simple: we just reset the timer when there's no buffered records.
Reviewers: Javier Holguera <javier.holguera@gmail.com>, Boyang Chen <boyang@confluent.io>, Bill Bejeck <bill@confluent.io>
After many runs of reproducing the failure (on my local MP5 it takes about 100 - 200 run to get one) I think it is more likely a flaky one and not exposing a real bug in rebalance protocol.
What I've observed is that, when the verifying consumer is trying to fetch from the output topics (there are 11 of them), it poll(1sec) each time, and retries 30 times if there's no more data to fetch and stop. It means that if there are no data fetched from the output topics for 30 * 1 = 30 seconds then the verification would stop (potentially too early). And for the failure cases, we observe consistent rebalancing among the closing / newly created clients since the closing is async, i.e. while new clients are added it is possible that closing clients triggered rebalance are not completed yet (note that each instance is configured with 3 threads, and in the worst case there are 6 instances running / pending shutdown at the same time, so a group fo 3 * 6 = 18 members is possible).
However, there's still a possible bug that in KIP-429, somehow the rebalance can never stabilize and members keep re-rejoining and hence cause it to fail. We have another unit test that have bumped up to 3 rebalance by @ableegoldman and if that failed again then it may be a better confirmation such bug may exist.
So what I've done so far for this test:
1. When closing a client, wait for it to complete closure before moving on to the next iteration and starting a new instance to reduce the rebalance churns.
2. Poll for 5 seconds instead of 1 to wait for longer time: 5 * 30 = 150 seconds, and locally my laptop finished this test in about 50 seconds.
3. Minor debug logging improvement; in fact some of them is to reduce redundant debug logging since it is too long and sometimes hides the key information.
Reviewers: Bill Bejeck <bill@confluent.io>, John Roesler <john@confluent.io>
* Made commit-over-tasks sensor and skipped-records sensor optional since they are removed in the latest version
* Refactored methods for sensor creation
* Adapted unit and integration tests
Reviewers: Guozhang Wang <wangguoz@gmail.com>
Migrate this integration test to use TopologyTestDriver instead of running 3 Streams instances.
Dropped one test that was attempting to produce specific interleavings. If anything, these should be verified deterministically by unit testing.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Also put in some additional logging that makes sense to add, and proved helpful in debugging this particular issue.
Unit tests verifying the encoded supported version were added.
This should get cherry-picked back to 2.1
Reviewers: Bill Bejeck <bill@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Fix the formatting and wording of the foreign-key join javadoc
Optimize handling of null extracted foreign keys
Reviewers: Sophie Blee-Goldman <sophie@confluent.io>, Bill Bejeck <bbejeck@gmail.com>
In a KTable context, null record values have a special "tombstone" significance. We should always bypass the serdes for such tombstones, since otherwise the serde could violate Streams' table semantics.
Added test coverage for this case and fixed the code accordingly.
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>, Bill Bejeck <bill@confluent.io>
I looked into the logs of the above tickets, and I think for a couple fo them it is due to the fact that the threads takes time to restore, or just stabilize the rebalance since there are multi-threads. Adding the hook to wait for state to transit to RUNNING upon starting.
Reviewers: Chris Pettitt <cpettitt@confluent.io>, Matthias J. Sax <matthias@confluent.io>
Subtopologies are currently ordered alphabetically by source node, which prior to KIP-307 happened to always result in the "correct" (ie topological) order. Now that users may name their nodes anything they want, we must explicitly order them so that upstream node groups/subtopologies come first and the downstream ones come after.
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Bill Bejeck <bbejeck@gmail.com>
All four flavors of the repartition/optimization tests have been reported as flaky and failed in one place or another:
* RepartitionOptimizingIntegrationTest.shouldSendCorrectRecords_OPTIMIZED
* RepartitionOptimizingIntegrationTest.shouldSendCorrectRecords_NO_OPTIMIZATION
* RepartitionWithMergeOptimizingIntegrationTest.shouldSendCorrectRecords_OPTIMIZED
* RepartitionWithMergeOptimizingIntegrationTest.shouldSendCorrectRecords_NO_OPTIMIZATION
They're pretty similar so it makes sense to knock them all out at once. This PR does three things:
* Switch to in-memory stores wherever possible
* Name all operators and update the Topology accordingly (not really a flaky test fix, but had to update the topology names anyway because of the IM stores so figured might as well)
* Port to TopologyTestDriver -- this is the "real" fix, should make a big difference as these repartition tests required multiple roundtrips with the Kafka cluster (while using only the default timeout)
Reviewers: Bill Bejeck <bill@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
After merged #7412 we realized it does not necessarily need that long time: instead of putting 2 million records, we can just have a single put followed by a flush, to make sure that rocksDB file exists locally (verified that after flush the sst file always exist).
Now the RocksDBStoreTest takes about 2.5 seconds, and removing the integration annotation from it.
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Bill Bejeck <bill@confluent.io>
* Renamed tag client-id to thread-id for thread-level metrics and below
* Corrected metrics tag keys for state store that had suffix "-id" instead of "state-id"
Reviewers: John Roesler <john@confluent.io>, Bill Bejeck <bill@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Key improvements with this PR:
* tasks will remain available for IQ during a rebalance (but not during restore)
* continue restoring and processing standby tasks during a rebalance
* continue processing active tasks during rebalance until the RecordQueue is empty*
* only revoked tasks must suspended/closed
* StreamsPartitionAssignor tries to return tasks to their previous consumers within a client
* but do not try to commit, for now (pending KAFKA-7312)
Reviewers: John Roesler <john@confluent.io>, Boyang Chen <boyang@confluent.io>, Guozhang Wang <wangguoz@gmail.com>