ソースを参照

YARN-2914. [YARN-1492] Potential race condition in Singleton implementation of SharedCacheUploaderMetrics, CleanerMetrics, ClientSCMMetrics. (Varun Saxena via kasha)

Karthik Kambatla 10 年 前
コミット
e597249d36

+ 3 - 0
hadoop-yarn-project/CHANGES.txt

@@ -225,6 +225,9 @@ Release 2.7.0 - UNRELEASED
     YARN-2356. yarn status command for non-existent application/application 
     attempt/container is too verbose. (Sunil G via devaraj)
 
+    YARN-2914. [YARN-1492] Potential race condition in Singleton implementation of 
+    SharedCacheUploaderMetrics, CleanerMetrics, ClientSCMMetrics. (Varun Saxena via kasha)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/CleanerService.java

@@ -91,7 +91,7 @@ public class CleanerService extends CompositeService {
           "It appears there is another CleanerService running in the cluster");
     }
 
-    this.metrics = CleanerMetrics.initSingleton(conf);
+    this.metrics = CleanerMetrics.getInstance();
 
     // Start dependent services (i.e. AppChecker)
     super.serviceStart();

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/ClientProtocolService.java

@@ -94,7 +94,7 @@ public class ClientProtocolService extends AbstractService implements
   @Override
   protected void serviceStart() throws Exception {
     Configuration conf = getConfig();
-    this.metrics = ClientSCMMetrics.initSingleton(conf);
+    this.metrics = ClientSCMMetrics.getInstance();
 
     YarnRPC rpc = YarnRPC.create(conf);
     this.server =

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/SharedCacheUploaderService.java

@@ -72,7 +72,7 @@ public class SharedCacheUploaderService extends AbstractService
   @Override
   protected void serviceStart() throws Exception {
     Configuration conf = getConfig();
-    this.metrics = SharedCacheUploaderMetrics.initSingleton(conf);
+    this.metrics = SharedCacheUploaderMetrics.getInstance();
 
     YarnRPC rpc = YarnRPC.create(conf);
     this.server =

+ 4 - 26
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/metrics/CleanerMetrics.java

@@ -21,7 +21,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.metrics2.MetricsSource;
 import org.apache.hadoop.metrics2.MetricsSystem;
 import org.apache.hadoop.metrics2.annotation.Metric;
@@ -43,31 +42,10 @@ import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
 public class CleanerMetrics {
   public static final Log LOG = LogFactory.getLog(CleanerMetrics.class);
   private final MetricsRegistry registry = new MetricsRegistry("cleaner");
-
-  enum Singleton {
-    INSTANCE;
-
-    CleanerMetrics impl;
-
-    synchronized CleanerMetrics init(Configuration conf) {
-      if (impl == null) {
-        impl = create(conf);
-      }
-      return impl;
-    }
-  }
-
-  public static CleanerMetrics initSingleton(Configuration conf) {
-    return Singleton.INSTANCE.init(conf);
-  }
-
+  private final static CleanerMetrics INSTANCE = create();
+  
   public static CleanerMetrics getInstance() {
-    CleanerMetrics topMetrics = Singleton.INSTANCE.impl;
-    if (topMetrics == null)
-      throw new IllegalStateException(
-          "The CleanerMetics singlton instance is not initialized."
-              + " Have you called init first?");
-    return topMetrics;
+    return INSTANCE;
   }
 
   @Metric("number of deleted files over all runs")
@@ -120,7 +98,7 @@ public class CleanerMetrics {
    */
   MetricsSource metricSource;
 
-  static CleanerMetrics create(Configuration conf) {
+  static CleanerMetrics create() {
     MetricsSystem ms = DefaultMetricsSystem.instance();
 
     CleanerMetrics metricObject = new CleanerMetrics();

+ 4 - 27
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/metrics/ClientSCMMetrics.java

@@ -21,7 +21,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.metrics2.MetricsSystem;
 import org.apache.hadoop.metrics2.annotation.Metric;
 import org.apache.hadoop.metrics2.annotation.Metrics;
@@ -40,37 +39,15 @@ public class ClientSCMMetrics {
 
   private static final Log LOG = LogFactory.getLog(ClientSCMMetrics.class);
   final MetricsRegistry registry;
+  private final static ClientSCMMetrics INSTANCE = create();
 
-  ClientSCMMetrics() {
+  private ClientSCMMetrics() {
     registry = new MetricsRegistry("clientRequests");
     LOG.debug("Initialized " + registry);
   }
-
-  enum Singleton {
-    INSTANCE;
-
-    ClientSCMMetrics impl;
-
-    synchronized ClientSCMMetrics init(Configuration conf) {
-      if (impl == null) {
-        impl = create();
-      }
-      return impl;
-    }
-  }
-
-  public static ClientSCMMetrics initSingleton(Configuration conf) {
-    return Singleton.INSTANCE.init(conf);
-  }
-
+  
   public static ClientSCMMetrics getInstance() {
-    ClientSCMMetrics topMetrics = Singleton.INSTANCE.impl;
-    if (topMetrics == null) {
-      throw new IllegalStateException(
-          "The ClientSCMMetrics singleton instance is not initialized."
-          + " Have you called init first?");
-    }
-    return topMetrics;
+    return INSTANCE;
   }
 
   static ClientSCMMetrics create() {

+ 3 - 26
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/metrics/SharedCacheUploaderMetrics.java

@@ -21,7 +21,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.metrics2.MetricsSystem;
 import org.apache.hadoop.metrics2.annotation.Metric;
 import org.apache.hadoop.metrics2.annotation.Metrics;
@@ -41,37 +40,15 @@ public class SharedCacheUploaderMetrics {
   static final Log LOG =
       LogFactory.getLog(SharedCacheUploaderMetrics.class);
   final MetricsRegistry registry;
+  private final static SharedCacheUploaderMetrics INSTANCE = create();
 
-  SharedCacheUploaderMetrics() {
+  private SharedCacheUploaderMetrics() {
     registry = new MetricsRegistry("SharedCacheUploaderRequests");
     LOG.debug("Initialized "+ registry);
   }
 
-  enum Singleton {
-    INSTANCE;
-
-    SharedCacheUploaderMetrics impl;
-
-    synchronized SharedCacheUploaderMetrics init(Configuration conf) {
-      if (impl == null) {
-        impl = create();
-      }
-      return impl;
-    }
-  }
-
-  public static SharedCacheUploaderMetrics
-      initSingleton(Configuration conf) {
-    return Singleton.INSTANCE.init(conf);
-  }
-
   public static SharedCacheUploaderMetrics getInstance() {
-    SharedCacheUploaderMetrics topMetrics = Singleton.INSTANCE.impl;
-    if (topMetrics == null)
-      throw new IllegalStateException(
-          "The SharedCacheUploaderMetrics singleton instance is not"
-          + "initialized. Have you called init first?");
-    return topMetrics;
+    return INSTANCE;
   }
 
   static SharedCacheUploaderMetrics create() {

+ 0 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/test/java/org/apache/hadoop/yarn/server/sharedcachemanager/metrics/TestCleanerMetrics.java

@@ -30,7 +30,6 @@ public class TestCleanerMetrics {
 
   @Before
   public void init() {
-    CleanerMetrics.initSingleton(conf);
     cleanerMetrics = CleanerMetrics.getInstance();
   }