Skip to content

KAFKA-10564: only process non-empty task directories when internally cleaning obsolete state stores#9373

Merged
guozhangwang merged 3 commits intoapache:trunkfrom
mikebin:KAFKA-10564
Oct 7, 2020
Merged

KAFKA-10564: only process non-empty task directories when internally cleaning obsolete state stores#9373
guozhangwang merged 3 commits intoapache:trunkfrom
mikebin:KAFKA-10564

Conversation

@mikebin
Copy link
Copy Markdown
Contributor

@mikebin mikebin commented Oct 5, 2020

Avoid continuous repeated logging by not trying to clean empty task directories, which are longer fully deleted during internal cleanup as of https://issues.apache.org/jira/browse/KAFKA-6647.

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

Copy link
Copy Markdown
Member

@ableegoldman ableegoldman left a comment

Choose a reason for hiding this comment

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

Thanks for the quick patch! I love a good one-liner 😄

Can you include a test in StateDirectoryTest as well? I think it would be sufficient to start out with a single non-empty task directory and then verify that the first time cleanRemovedTasksCalledByCleanerThread() is called it will log this message + clear out everything except the lock file, but if it's invoked a second time, nothing is logged.

@ableegoldman
Copy link
Copy Markdown
Member

ableegoldman commented Oct 6, 2020

By the way, John was so kind as to cherrypick the trunk PR that rearranged things in this method back to the 2.6 branch -- so we should be able to cherrypick this PR smoothly, no need for a separate one for 2.6

Looks like just a handful of unrelated flaky test failures in the builds, hopefully we'll have better luck on the next run

Build / JDK 11 / kafka.api.MetricsTest.testMetrics
Build / JDK 8 / kafka.network.ConnectionQuotasTest.testNoConnectionLimitsByDefault
Build / JDK 15 / kafka.api.ConsumerBounceTest.testRollingBrokerRestartsWithSmallerMaxGroupSizeConfigDisruptsBigGroup
Build / JDK 15 / kafka.integration.MetricsDuringTopicCreationDeletionTest.testMetricsDuringTopicCreateDelete
Build / JDK 15 / org.apache.kafka.streams.integration.KTableKTableForeignKeyInnerJoinMultiIntegrationTest.shouldInnerJoinMultiPartitionQueryable

@mikebin
Copy link
Copy Markdown
Contributor Author

mikebin commented Oct 6, 2020

Thanks for the review @ableegoldman! Added a unit test. And thanks @vvcephei for making it easier to cherry pick this fix back to 2.6.

Copy link
Copy Markdown
Member

@ableegoldman ableegoldman left a comment

Choose a reason for hiding this comment

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

LGTM!

Only known flaky tests in the build failure:

Build / JDK 11 / org.apache.kafka.streams.integration.EosBetaUpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosBeta[true]
Build / JDK 11 / org.apache.kafka.streams.integration.EosIntegrationTest.shouldNotViolateEosIfOneTaskFailsWithState[exactly_once_beta]
Build / JDK 15 / org.apache.kafka.streams.integration.EosBetaUpgradeIntegrationTest.shouldUpgradeFromEosAlphaToEosBeta[true]

@ableegoldman
Copy link
Copy Markdown
Member

FYI this should be cherrypicked back to 2.6 once merged

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!

@guozhangwang guozhangwang merged commit 250c71b into apache:trunk Oct 7, 2020
@guozhangwang
Copy link
Copy Markdown
Contributor

Cherry-picked to 2.6.

guozhangwang pushed a commit that referenced this pull request Oct 7, 2020
…cleaning obsolete state stores (#9373)

Avoid continuous repeated logging by not trying to clean empty task directories, which are longer fully deleted during internal cleanup as of https://issues.apache.org/jira/browse/KAFKA-6647.

Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
@mikebin
Copy link
Copy Markdown
Contributor Author

mikebin commented Oct 7, 2020

Thank you all!

javierfreire pushed a commit to javierfreire/kafka that referenced this pull request Oct 8, 2020
…cleaning obsolete state stores (apache#9373)

Avoid continuous repeated logging by not trying to clean empty task directories, which are longer fully deleted during internal cleanup as of https://issues.apache.org/jira/browse/KAFKA-6647.

Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
ijuma added a commit to confluentinc/kafka that referenced this pull request Oct 8, 2020
* commit '2804257fe221f37e5098bd': (67 commits)
  KAFKA-10562: Properly invoke new StateStoreContext init (apache#9388)
  MINOR: trivial cleanups, javadoc errors, omitted StateStore tests, etc. (apache#8130)
  KAFKA-10564: only process non-empty task directories when internally cleaning obsolete state stores (apache#9373)
  KAFKA-9274: fix incorrect default value for `task.timeout.ms` config (apache#9385)
  KAFKA-10362: When resuming Streams active task with EOS, the checkpoint file is deleted (apache#9247)
  KAFKA-10028: Implement write path for feature versioning system (KIP-584) (apache#9001)
  KAFKA-10402: Upgrade system tests to python3 (apache#9196)
  KAFKA-10186; Abort transaction with pending data with TransactionAbortedException (apache#9280)
  MINOR: Remove `TargetVoters` from `DescribeQuorum` (apache#9376)
  Revert "KAFKA-10469: Resolve logger levels hierarchically (apache#9266)"
  MINOR: Don't publish javadocs for raft module (apache#9336)
  KAFKA-9929: fix: add missing default implementations (apache#9321)
  KAFKA-10188: Prevent SinkTask::preCommit from being called after SinkTask::stop (apache#8910)
  KAFKA-10338; Support PEM format for SSL key and trust stores (KIP-651) (apache#9345)
  KAFKA-10527; Voters should not reinitialize as leader in same epoch (apache#9348)
  MINOR: Refactor unit tests around RocksDBConfigSetter (apache#9358)
  KAFKA-6733: Printing additional ConsumerRecord fields in DefaultMessageFormatter (apache#9099)
  MINOR: Annotate test BlockingConnectorTest as integration test (apache#9379)
  MINOR: Fix failing test due to KAFKA-10556 PR (apache#9372)
  KAFKA-10439: Connect's Values to parse BigInteger as Decimal with zero scale. (apache#9320)
  ...
@chia7712
Copy link
Copy Markdown
Member

@mikebin Thanks for this patch.

I noticed test shouldCleanupObsoleteStateDirectoriesOnlyOnce is flaky and the root cause may be similar to acfa079

Could you take a look? thanks!

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