You are viewing a plain text version of this content. The canonical link for it is here.
Posted to yarn-commits@hadoop.apache.org by ka...@apache.org on 2014/06/07 02:51:51 UTC

svn commit: r1601046 - in /hadoop/common/branches/branch-2/hadoop-yarn-project: ./ hadoop-yarn/dev-support/ hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/

Author: kasha
Date: Sat Jun  7 00:51:50 2014
New Revision: 1601046

URL: http://svn.apache.org/r1601046
Log:
YARN-2122. In AllocationFileLoaderService, the reloadThread should be created in init() and started in start(). (Robert Kanter via kasha)

Modified:
    hadoop/common/branches/branch-2/hadoop-yarn-project/CHANGES.txt
    hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
    hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/CHANGES.txt?rev=1601046&r1=1601045&r2=1601046&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/CHANGES.txt (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/CHANGES.txt Sat Jun  7 00:51:50 2014
@@ -124,6 +124,9 @@ Release 2.5.0 - UNRELEASED
 
     YARN-1977. Add tests on getApplicationRequest with filtering start time range. (junping_du)
 
+    YARN-2122. In AllocationFileLoaderService, the reloadThread should be created
+    in init() and started in start(). (Robert Kanter via kasha)
+
   OPTIMIZATIONS
 
   BUG FIXES 
@@ -150,6 +153,7 @@ Release 2.5.0 - UNRELEASED
 
     YARN-2011. Fix typo and warning in TestLeafQueue (Chen He via junping_du)
 
+
     YARN-2042. String shouldn't be compared using == in
     QueuePlacementRule#NestedUserQueue#getQueueForApp (Chen He via Sandy Ryza)
 

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml?rev=1601046&r1=1601045&r2=1601046&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml Sat Jun  7 00:51:50 2014
@@ -142,6 +142,11 @@
     <Class name="org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.LogAggregationService" />
     <Bug pattern="IS2_INCONSISTENT_SYNC" />
   </Match>
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.AllocationFileLoaderService" />
+    <Field name="allocFile" />
+    <Bug pattern="IS2_INCONSISTENT_SYNC" />
+  </Match>
   <!-- Inconsistent sync warning - minimumAllocation is only initialized once and never changed -->
   <Match>
     <Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler" />

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java?rev=1601046&r1=1601045&r2=1601046&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java Sat Jun  7 00:51:50 2014
@@ -98,55 +98,59 @@ public class AllocationFileLoaderService
   }
   
   @Override
-  public void init(Configuration conf) {
+  public void serviceInit(Configuration conf) throws Exception {
     this.allocFile = getAllocationFile(conf);
-    super.init(conf);
-  }
-  
-  @Override
-  public void start() {
-    if (allocFile == null) {
-      return;
-    }
-    reloadThread = new Thread() {
-      public void run() {
-        while (running) {
-          long time = clock.getTime();
-          long lastModified = allocFile.lastModified();
-          if (lastModified > lastSuccessfulReload &&
-              time > lastModified + ALLOC_RELOAD_WAIT_MS) {
-            try {
-              reloadAllocations();
-            } catch (Exception ex) {
+    if (allocFile != null) {
+      reloadThread = new Thread() {
+        @Override
+        public void run() {
+          while (running) {
+            long time = clock.getTime();
+            long lastModified = allocFile.lastModified();
+            if (lastModified > lastSuccessfulReload &&
+                time > lastModified + ALLOC_RELOAD_WAIT_MS) {
+              try {
+                reloadAllocations();
+              } catch (Exception ex) {
+                if (!lastReloadAttemptFailed) {
+                  LOG.error("Failed to reload fair scheduler config file - " +
+                      "will use existing allocations.", ex);
+                }
+                lastReloadAttemptFailed = true;
+              }
+            } else if (lastModified == 0l) {
               if (!lastReloadAttemptFailed) {
-                LOG.error("Failed to reload fair scheduler config file - " +
-                    "will use existing allocations.", ex);
+                LOG.warn("Failed to reload fair scheduler config file because" +
+                    " last modified returned 0. File exists: "
+                    + allocFile.exists());
               }
               lastReloadAttemptFailed = true;
             }
-          } else if (lastModified == 0l) {
-            if (!lastReloadAttemptFailed) {
-              LOG.warn("Failed to reload fair scheduler config file because" +
-                  " last modified returned 0. File exists: " + allocFile.exists());
+            try {
+              Thread.sleep(reloadIntervalMs);
+            } catch (InterruptedException ex) {
+              LOG.info(
+                  "Interrupted while waiting to reload alloc configuration");
             }
-            lastReloadAttemptFailed = true;
-          }
-          try {
-            Thread.sleep(reloadIntervalMs);
-          } catch (InterruptedException ex) {
-            LOG.info("Interrupted while waiting to reload alloc configuration");
           }
         }
-      }
-    };
-    reloadThread.setName("AllocationFileReloader");
-    reloadThread.setDaemon(true);
-    reloadThread.start();
-    super.start();
+      };
+      reloadThread.setName("AllocationFileReloader");
+      reloadThread.setDaemon(true);
+    }
+    super.serviceInit(conf);
+  }
+  
+  @Override
+  public void serviceStart() throws Exception {
+    if (reloadThread != null) {
+      reloadThread.start();
+    }
+    super.serviceStart();
   }
   
   @Override
-  public void stop() {
+  public void serviceStop() throws Exception {
     running = false;
     if (reloadThread != null) {
       reloadThread.interrupt();
@@ -156,7 +160,7 @@ public class AllocationFileLoaderService
         LOG.warn("reloadThread fails to join.");
       }
     }
-    super.stop();
+    super.serviceStop();
   }
   
   /**