فهرست منبع

YARN-10680. Revisit try blocks without catch blocks but having finally blocks. Contributed by Susheel Gupta

Szilard Nemeth 2 سال پیش
والد
کامیت
b0d5182c31

+ 9 - 1
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java

@@ -36,6 +36,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceCommitRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.sls.SLSRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @Private
 @Unstable
@@ -45,6 +47,7 @@ public class SLSCapacityScheduler extends CapacityScheduler implements
   private final SLSSchedulerCommons schedulerCommons;
   private Configuration conf;
   private SLSRunner runner;
+  private static final Logger LOG = LoggerFactory.getLogger(SLSCapacityScheduler.class);
 
   public SLSCapacityScheduler() {
     schedulerCommons = new SLSSchedulerCommons(this);
@@ -105,7 +108,12 @@ public class SLSCapacityScheduler extends CapacityScheduler implements
 
   @Override
   public void handle(SchedulerEvent schedulerEvent) {
-    schedulerCommons.handle(schedulerEvent);
+    try {
+      schedulerCommons.handle(schedulerEvent);
+    } catch(Exception e) {
+      LOG.error("Caught exception while handling scheduler event", e);
+      throw e;
+    }
   }
 
   @Override

+ 9 - 1
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSFairScheduler.java

@@ -31,6 +31,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
 import org.apache.hadoop.yarn.sls.SLSRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.List;
 
@@ -40,6 +42,7 @@ public class SLSFairScheduler extends FairScheduler
     implements SchedulerWrapper, Configurable {
   private final SLSSchedulerCommons schedulerCommons;
   private SLSRunner runner;
+  private static final Logger LOG = LoggerFactory.getLogger(SLSFairScheduler.class);
 
   public SLSFairScheduler() {
     schedulerCommons = new SLSSchedulerCommons(this);
@@ -63,7 +66,12 @@ public class SLSFairScheduler extends FairScheduler
 
   @Override
   public void handle(SchedulerEvent schedulerEvent) {
-    schedulerCommons.handle(schedulerEvent);
+    try {
+      schedulerCommons.handle(schedulerEvent);
+    } catch (Exception e){
+      LOG.error("Caught exception while handling scheduler event", e);
+      throw e;
+    }
   }
 
   @Override

+ 7 - 1
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerMetrics.java

@@ -178,7 +178,7 @@ public abstract class SchedulerMetrics {
     pool.scheduleAtFixedRate(new HistogramsRunnable(), 0, 1000,
         TimeUnit.MILLISECONDS);
 
-    // a thread to output metrics for real-tiem tracking
+    // a thread to output metrics for real-time tracking
     pool.scheduleAtFixedRate(new MetricsLogRunnable(), 0, 1000,
         TimeUnit.MILLISECONDS);
 
@@ -467,6 +467,9 @@ public abstract class SchedulerMetrics {
         schedulerHistogramList.add(histogram);
         histogramTimerMap.put(histogram, schedulerHandleTimerMap.get(e));
       }
+    } catch (Exception e) {
+      LOG.error("Caught exception while registering scheduler metrics", e);
+      throw e;
     } finally {
       samplerLock.unlock();
     }
@@ -510,6 +513,9 @@ public abstract class SchedulerMetrics {
             }
         );
       }
+    } catch (Exception e) {
+      LOG.error("Caught exception while registering nodes usage metrics", e);
+      throw e;
     } finally {
       samplerLock.unlock();
     }