[
https://issues.apache.org/jira/browse/HADOOP-12482?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14983509#comment-14983509
]
Lei (Eddy) Xu commented on HADOOP-12482:
----------------------------------------
Thanks a lot for working on this, [~twu]. A few comments
{code}
private static class readThread
{code}
class name should be {{CamelCase}}.
{code}
class TsSource implements MetricsSource
{code}
Can it be {{private static class}}, it is better to be named as
{{MatricsSource}} or similar.
{code}
for (MBeanAttributeInfo mBeanAttributeInfo : info.getAttributes()) {
found = mBeanAttributeInfo.getName().equals(key);
}
assertTrue("Key " + key + " not found", found);
{code}
It is not clear to me that, 1) are you suggesting they are all {{true}}, or at
least one of them are true ? 2) is there only one {{mBeamAttributeInfo}}?
Instead of sleeping in {{RepeatingTesetThread}}, can we use
{{ScheduledExecutorService#scheduleAtFixedRate}}?
Finally, it might be nice to have some documents in
{{TestMetricsSourceAdapter}} to describe how the Race condition might happen.
> Race condition in JMX cache update
> ----------------------------------
>
> Key: HADOOP-12482
> URL: https://issues.apache.org/jira/browse/HADOOP-12482
> Project: Hadoop Common
> Issue Type: Bug
> Affects Versions: 2.7.1
> Reporter: Tony Wu
> Assignee: Tony Wu
> Attachments: HADOOP-12482.001.patch, HADOOP-12482.002.patch,
> HADOOP-12482.003.patch
>
>
> updateJmxCache() was updated in HADOOP-11301. However the patch introduced a
> race condition. In updateJmxCache() function in MetricsSourceAdapter.java:
> {code:java}
> private void updateJmxCache() {
> boolean getAllMetrics = false;
> synchronized (this) {
> if (Time.now() - jmxCacheTS >= jmxCacheTTL) {
> // temporarilly advance the expiry while updating the cache
> jmxCacheTS = Time.now() + jmxCacheTTL;
> if (lastRecs == null) {
> getAllMetrics = true;
> }
> } else {
> return;
> }
> if (getAllMetrics) {
> MetricsCollectorImpl builder = new MetricsCollectorImpl();
> getMetrics(builder, true);
> }
> updateAttrCache();
> if (getAllMetrics) {
> updateInfoCache();
> }
> jmxCacheTS = Time.now();
> lastRecs = null; // in case regular interval update is not running
> }
> }
> {code}
> Notice that getAllMetrics is set to true when:
> # jmxCacheTTL has passed
> # lastRecs == null
> lastRecs is set to null in the same function, but gets reassigned by
> getMetrics().
> However getMetrics() can be called from a different thread:
> # MetricsSystemImpl.onTimerEvent()
> # MetricsSystemImpl.publishMetricsNow()
> Consider the following sequence:
> # updateJmxCache() is called by getMBeanInfo() from a thread getting cached
> info.
> ** lastRecs is set to null.
> # metrics sources is updated with new value/field.
> # getMetrics() is called by publishMetricsNow() or onTimerEvent() from a
> different thread getting the latest metrics.
> ** lastRecs is updated (!= null).
> # jmxCacheTTL passed.
> # updateJmxCache() is called again via getMBeanInfo().
> ** However because lastRecs is already updated (!= null), getAllMetrics will
> not be set to true. So updateInfoCache() is not called and getMBeanInfo()
> returns the old cached info.
> We ran into this issue on a cluster where a new metric did not get published
> until much later.
> The case can be made worse by a periodic call to getMetrics() (driven by an
> external program or script). In such case getMBeanInfo() may never be able to
> retrieve the new record.
> The desired behavior should be that updateJmxCache() will guarantee to call
> updateInfoCache() once after jmxCacheTTL, if lastRecs has been set to null by
> updateJmxCache() itself.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)