Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-7136: Avoid deadlocks in synchronized metrics reporters #5341

Merged

Conversation

rajinisivaram
Copy link
Contributor

We need to use the same lock for metric update and read to avoid NPE and concurrent modification exceptions. Sensor add/remove/update are synchronized on Sensor since they access lists and maps that are not thread-safe. Reporters are notified of metrics add/remove while holding (Sensor, Metrics) locks and reporters may synchronize on the reporter lock. Metric read may be invoked by metrics reporters while holding a reporter lock. So read/update cannot be synchronized using Sensor since that could lead to deadlock. This PR introduces a new lock in Sensor for update/read.
Locking order:

- Sensor#add: Sensor -> Metrics -> MetricsReporter
- Metrics#removeSensor: Sensor -> Metrics -> MetricsReporter
- KafkaMetric#metricValue: MetricsReporter -> Sensor#metricLock
- Sensor#record: Sensor -> Sensor#metricLock

Committer Checklist (excluded from commit message)

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

Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

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

@rajinisivaram : Thanks for the patch. LGTM. Just a couple of minor comments below.

op.run();
}
} catch (Throwable t) {
log.error("Metric {} failed with exception", opName, t);
Copy link
Contributor

Choose a reason for hiding this comment

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

If we get an exception, should we fail the test?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The test fails if there is an exception since it verifies that the task is still running.

* </p><p>
* Locking order (assume all MetricsReporter methods may be synchronized):
* <ul>
* <li>Sensor#add: Sensor -> Metrics -> MetricsReporter</li>
Copy link
Contributor

Choose a reason for hiding this comment

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

It would be useful to document that the Sensor object itself is used as a lock to protect the access to stats and metrics.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@junrao Thanks for the review, I have added that to the comment.

Copy link
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.

One nit comment, otherwise LGTM.

*/
private Object metricLock(Stat stat) {
return this;
private Object metricLock() {
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: do we still need this function? Could we just reference the metricLock object directly? Since it is private my understanding is that it was not intended to be used outside this class.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@guozhangwang Thanks for the review. I left the method in since it gives a good place to document :-) I think the method would get optimized away at runtime anyway. I don't mind changing it if you think it may be confusing.

Copy link
Contributor

Choose a reason for hiding this comment

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

Ah I see. MVN then :)

Copy link
Contributor

Choose a reason for hiding this comment

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

We can document the field too, right? If there's no reason to have a method, I'd remove it.

@guozhangwang guozhangwang merged commit 1f8527b into apache:trunk Jul 6, 2018
guozhangwang pushed a commit that referenced this pull request Jul 6, 2018
We need to use the same lock for metric update and read to avoid NPE and concurrent modification exceptions. Sensor add/remove/update are synchronized on Sensor since they access lists and maps that are not thread-safe. Reporters are notified of metrics add/remove while holding (Sensor, Metrics) locks and reporters may synchronize on the reporter lock. Metric read may be invoked by metrics reporters while holding a reporter lock. So read/update cannot be synchronized using Sensor since that could lead to deadlock. This PR introduces a new lock in Sensor for update/read.
Locking order:

- Sensor#add: Sensor -> Metrics -> MetricsReporter
- Metrics#removeSensor: Sensor -> Metrics -> MetricsReporter
- KafkaMetric#metricValue: MetricsReporter -> Sensor#metricLock
- Sensor#record: Sensor -> Sensor#metricLock

Reviewers: Jun Rao <junrao@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
guozhangwang pushed a commit that referenced this pull request Jul 6, 2018
We need to use the same lock for metric update and read to avoid NPE and concurrent modification exceptions. Sensor add/remove/update are synchronized on Sensor since they access lists and maps that are not thread-safe. Reporters are notified of metrics add/remove while holding (Sensor, Metrics) locks and reporters may synchronize on the reporter lock. Metric read may be invoked by metrics reporters while holding a reporter lock. So read/update cannot be synchronized using Sensor since that could lead to deadlock. This PR introduces a new lock in Sensor for update/read.
Locking order:

- Sensor#add: Sensor -> Metrics -> MetricsReporter
- Metrics#removeSensor: Sensor -> Metrics -> MetricsReporter
- KafkaMetric#metricValue: MetricsReporter -> Sensor#metricLock
- Sensor#record: Sensor -> Sensor#metricLock


Reviewers: Jun Rao <junrao@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
Copy link
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.

Cherry-picked to 1.1 and 2.0.

hachikuji added a commit that referenced this pull request Jul 6, 2018
This was broken when picking #5341 into 1.1.

Reviewers: Guozhang Wang <wangguoz@gmail.com>
allenxwang pushed a commit to allenxwang/kafka that referenced this pull request Aug 24, 2018
…:1.1.1-sync to 1.1-nflx

* commit '9611672e287c1a7933a78590e3f381da2ae7d136': (57 commits)
  MINOR: increase dev version from 1.1.1-SNAPSHOT to 1.1.2-SNAPSHOT (apache#5409)
  MINOR: Add thread dumps if broker node cannot be stopped (apache#5373)
  MINOR: update release.py
  MINOR: fix upgrade docs for Streams (apache#5392)
  MINOR: improve docs version numbers (apache#5372)
  Update version on the branch to 1.1.2-SNAPSHOT
  KAFKA-6292; Improve FileLogInputStream batch position checks to avoid type overflow (apache#4928)
  HOTFIX: Fix checkstyle errors in MetricsTest (apache#5345)
  KAFKA-7136: Avoid deadlocks in synchronized metrics reporters (apache#5341)
  MINOR: Close timing window in SimpleAclAuthorizer startup (apache#5318)
  MINOR: Use kill_java_processes when killing ConsoleConsumer in system tests (apache#5297)
  KAFKA-7104: More consistent leader's state in fetch response (apache#5305)
  Revert "MINOR: Avoid coarse lock in Pool#getAndMaybePut (apache#5258)"
  MINOR: Avoid coarse lock in Pool#getAndMaybePut (apache#5258)
  MINOR: bugfix streams total metrics (apache#5277)
  KAFKA-7082: Concurrent create topics may throw NodeExistsException (apache#5259)
  MINOR: Upgrade to Gradle 4.8.1
  KAFKA-7012: Don't process SSL channels without data to process (apache#5237)
  KAFKA-7058: Comparing schema default values using Objects#deepEquals()
  KAFKA-7047: Added SimpleHeaderConverter to plugin isolation whitelist
  ...
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
4 participants