Skip to content

HOTFIX: don't try to remove uninitialized changelogs from assignment & don't prematurely mark task closed#8140

Merged
guozhangwang merged 4 commits intoapache:trunkfrom
ableegoldman:HOTFIX-dont-re-close-closed-task
Feb 20, 2020
Merged

HOTFIX: don't try to remove uninitialized changelogs from assignment & don't prematurely mark task closed#8140
guozhangwang merged 4 commits intoapache:trunkfrom
ableegoldman:HOTFIX-dont-re-close-closed-task

Conversation

@ableegoldman
Copy link
Copy Markdown
Member

@ableegoldman ableegoldman commented Feb 20, 2020

This fixes two issues which together caused the soak to crash/some test to fail occasionally.

What happened was:
In the main StreamThread loop we initialized a new task in TaskManager#checkForCompletedRestoration which includes registering, but not initializing, its changelogs. We then complete the loop and call poll, which resulted in a rebalance that revoked the newly-initialized task. In TaskManager#handleAssignment we then closed the task cleanly and go to remove the changelogs from the StoreChangelogReader only to get an IllegalStateException because the changelog partitions were not in the restore consumer's assignment (due to being uninitialized).

This by itself should^ be a recoverable error, as we catch exceptions here and retry closing the task as unclean. Of course the task actually was successfully closed (clean) so we now get an unexpected exception Illegal state CLOSED while closing active task

The fix(es) I'd propose are:

  1. Keep the restore consumer's assignment in sync with the registered changelogs, ie the set ChangelogReader#changelogs but pause them until they are initialized edit: since the consumer does still perform some actions (gg fetches) on paused partitions, we should avoid adding uninitialized changelogs to the restore consumer's assignment. Instead, we should just skip them when removing.
  2. Move the StoreChangelogReader#remove call to before the task.closeClean so that the task is only marked as closed if everything was successful. We should do so regardless, as we should (attempt to) remove the changelogs even if the clean close failed and we must do unclean.

@ableegoldman
Copy link
Copy Markdown
Member Author

@guozhangwang @vvcephei


transitionTo(State.SUSPENDED);
log.info("Suspended running");
log.info("Suspended active");
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Technically it may be running or restoring here


// we can always let changelog reader try restoring in order to initialize the changelogs;
// if there's no active restoring or standby updating it would not try to fetch any data
changelogReader.restore();
Copy link
Copy Markdown
Member Author

@ableegoldman ableegoldman Feb 20, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This isn't a necessary part of the fix, but I feel it makes more sense to move this here: the restore consumer's can only make progress on tasks that have been initialized, which only happens in taskManager.tryToCompleteRestoration

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree. Originally I put it in front intentionally since initializeIfNeeded is called at task creation not here, and we would only try to initialize the changelogs in restore and hence only check if the restoration can be completed.

Now that we've delayed the initializeIfNeeded in tryToComplete we should reorder the restore call as well.

* @throws StreamsException if the store's change log does not contain the partition
*/
boolean checkForCompletedRestoration() {
boolean tryToCompleteRestoration() {
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also not necessary, but I feel it better suggests/reminds that it is taking some action here and not just checking a condition

Copy link
Copy Markdown
Contributor

@vvcephei vvcephei left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks, @ableegoldman !

Looks good to me. There's one logging mistake (below) that needs to be corrected, but otherwise I think it's ready to merge.

@vvcephei
Copy link
Copy Markdown
Contributor

Ok to test.

Copy link
Copy Markdown
Contributor

@guozhangwang guozhangwang left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @ableegoldman Thanks for the hot-fix! I agree with you about fix 2) here, for fix 1) I have some alternative ideas since it is risky to add the partitions without setting their positions (we discussed in another PR @vvcephei and decide we may want to set the reset-policy to always none for restore consumer as well, which would cause NoOffsetsForPartition in the immediate poll call.

", this should not happen: " + changelogs);
}

addChangelogsToRestoreConsumer(Collections.singleton(partition));
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The reason I did not do this is that in consumer, even if a partition is paused, we would still try to fetch is fetching position in the poll call if it was not available. And that's why I have to defer this after we've found the starting position and initialize them with seek calls.

So I'm thinking maybe we would just remove the illegal-state check from the remoreFromRestoreConsumer call and just log it as a debug?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah, I didn't realize the consumer would do anything with paused partitions. I'm not sure we want to remove the verification entirely, but we can check any "missing" partitions against the changelogs map instead.


// we can always let changelog reader try restoring in order to initialize the changelogs;
// if there's no active restoring or standby updating it would not try to fetch any data
changelogReader.restore();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree. Originally I put it in front intentionally since initializeIfNeeded is called at task creation not here, and we would only try to initialize the changelogs in restore and hence only check if the restoration can be completed.

Now that we've delayed the initializeIfNeeded in tryToComplete we should reorder the restore call as well.

try {
task.closeClean();
changelogReader.remove(task.changelogPartitions());
task.closeClean();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yup! I also found this bug and fixed in another PR, but great to see it confirmed in yours as well :)

ableegoldman and others added 2 commits February 20, 2020 12:13
Co-Authored-By: John Roesler <vvcephei@users.noreply.github.com>
@ableegoldman ableegoldman changed the title HOTFIX: keep restore consumer assignment in sync & don't prematurely mark task closed HOTFIX: don't try to remove uninitialized changelogs from assignment & don't prematurely mark task closed Feb 20, 2020
@ableegoldman
Copy link
Copy Markdown
Member Author

Ran the tests locally twice: one unrelated failure in shouldWorkWithRebalance (that fix coming next) but otherwise all green. Prior to this I saw 1-2 failures per run due to the issue(s) fixed here

Copy link
Copy Markdown
Contributor

@guozhangwang guozhangwang left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM! Merging to trunk now.

@guozhangwang guozhangwang merged commit 0d16c26 into apache:trunk Feb 20, 2020
ijuma added a commit to confluentinc/kafka that referenced this pull request Feb 24, 2020
* apache-github/trunk: (23 commits)
  KAFKA-9530; Fix flaky test `testDescribeGroupWithShortInitializationTimeout` (apache#8154)
  HOTFIX: fix NPE in Kafka Streams IQ (apache#8158)
  MINOR: set scala version automatically based on gradle.properties
  KAFKA-9577; SaslClientAuthenticator incorrectly negotiates SASL_HANDSHAKE version (apache#8142)
  KAFKA-9441: Add internal TransactionManager (apache#8105)
  MINOR: Document endpoints for connector topic tracking (KIP-558)
  MINOR: Standby task commit needed when offsets updated (apache#8146)
  KAFKA-9206; Throw KafkaException on CORRUPT_MESSAGE error in Fetch response (apache#8111)
  MINOR: Remove unwanted regexReplace on tests/kafkatest/__init__.py
  KAFKA-9586: Fix errored json filename in ops documentation
  KAFKA-9575: Mention ZooKeeper 3.5.7 upgrade
  KAFKA-9481: Graceful handling TaskMigrated and TaskCorrupted (apache#8058)
  HOTFIX: don't try to remove uninitialized changelogs from assignment & don't prematurely mark task closed (apache#8140)
  MINOR: Fix javadoc at org.apache.kafka.clients.producer.KafkaProducer.InterceptorCallback#onCompletion (apache#7337)
  MINOR: Improve EOS example exception handling (apache#8052)
  MINOR: Fix a number of warnings in clients test (apache#8073)
  MINOR: Update shell scripts to support z/OS system (apache#7913)
  MINOR: Wording fix in Streams DSL docs (apache#5692)
  MINOR: Add missing @test annotation to MetadataTest#testMetadataMerge (apache#8141)
  KAFKA-9533: ValueTransform forwards `null` values (apache#8108)
  ...
ijuma added a commit to confluentinc/kafka that referenced this pull request Feb 24, 2020
…etrics-common

* confluent/master: (76 commits)
  KAFKA-9530; Fix flaky test `testDescribeGroupWithShortInitializationTimeout` (apache#8154)
  HOTFIX: fix NPE in Kafka Streams IQ (apache#8158)
  MINOR: set scala version automatically based on gradle.properties
  KAFKA-9577; SaslClientAuthenticator incorrectly negotiates SASL_HANDSHAKE version (apache#8142)
  KAFKA-9441: Add internal TransactionManager (apache#8105)
  MINOR: Document endpoints for connector topic tracking (KIP-558)
  MINOR: Standby task commit needed when offsets updated (apache#8146)
  Changes to migrate to Artifactory (#263)
  KAFKA-9206; Throw KafkaException on CORRUPT_MESSAGE error in Fetch response (apache#8111)
  MINOR: Remove unwanted regexReplace on tests/kafkatest/__init__.py
  KAFKA-9586: Fix errored json filename in ops documentation
  KAFKA-9575: Mention ZooKeeper 3.5.7 upgrade
  KAFKA-9481: Graceful handling TaskMigrated and TaskCorrupted (apache#8058)
  HOTFIX: don't try to remove uninitialized changelogs from assignment & don't prematurely mark task closed (apache#8140)
  MINOR: Fix javadoc at org.apache.kafka.clients.producer.KafkaProducer.InterceptorCallback#onCompletion (apache#7337)
  MINOR: Improve EOS example exception handling (apache#8052)
  MINOR: Fix a number of warnings in clients test (apache#8073)
  MINOR: Update shell scripts to support z/OS system (apache#7913)
  MINOR: Wording fix in Streams DSL docs (apache#5692)
  MINOR: Add missing @test annotation to MetadataTest#testMetadataMerge (apache#8141)
  ...
@apurvam
Copy link
Copy Markdown
Contributor

apurvam commented Feb 25, 2020

Why are there no new test cases for this bug in this PR?

@ableegoldman
Copy link
Copy Markdown
Member Author

I'll make a note to add test cases for this in the next PR

@ableegoldman ableegoldman deleted the HOTFIX-dont-re-close-closed-task branch June 26, 2020 22:37
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants