ソースを参照

YARN-2939. Fix new findbugs warnings in hadoop-yarn-common. (Li Lu via junping_du)

Junping Du 10 年 前
コミット
a696fbb001
15 ファイル変更79 行追加56 行削除
  1. 2 0
      hadoop-yarn-project/CHANGES.txt
  2. 11 0
      hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
  3. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PriorityPBImpl.java
  4. 5 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
  5. 5 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
  6. 6 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
  7. 2 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelsStore.java
  8. 8 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/Graph.java
  9. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/VisualizeStateMachine.java
  10. 12 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/LinuxResourceCalculatorPlugin.java
  11. 17 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java
  12. 0 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java
  13. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsBasedProcessTree.java
  14. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java
  15. 5 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java

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

@@ -261,6 +261,8 @@ Release 2.7.0 - UNRELEASED
     YARN-2977. Fixed intermittent TestNMClient failure.
     (Junping Du via ozawa)
 
+    YARN-2939. Fix new findbugs warnings in hadoop-yarn-common. (Li Lu via junping_du)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

+ 11 - 0
hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml

@@ -265,6 +265,11 @@
     <Class name="org.apache.hadoop.yarn.YarnUncaughtExceptionHandler"/>
     <Bug pattern="DM_EXIT"/>
   </Match>
+  <Match>
+    <Class name="org.apache.hadoop.yarn.event.AsyncDispatcher$2"/>
+    <Method name="run" />
+    <Bug pattern="DM_EXIT"/>
+  </Match>
 
   <!-- AsyncDispatcher will kill the process if there is an error dispatching -->
   <Match>
@@ -391,4 +396,10 @@
     <Bug pattern="UI_INHERITANCE_UNSAFE_GETRESOURCE"/>
   </Match>
 
+  <!-- Ignore the false alarms on DM_DEFAULT_ENCODING (encoding already set) -->
+  <Match>
+    <Class name="org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat$LogReader" />
+    <Bug pattern="DM_DEFAULT_ENCODING" />
+  </Match>
+
 </FindBugsFilter>

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PriorityPBImpl.java

@@ -68,7 +68,7 @@ public class PriorityPBImpl extends Priority {
   
   @Override
   public String toString() {
-    return Integer.valueOf(getPriority()).toString();
+    return Integer.toString(getPriority());
   }
 
 }  

+ 5 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java

@@ -136,9 +136,13 @@ public class TimelineClientImpl extends TimelineClient {
 
     // Indicates if retries happened last time. Only tests should read it.
     // In unit tests, retryOn() calls should _not_ be concurrent.
+    private boolean retried = false;
+
     @Private
     @VisibleForTesting
-    public boolean retried = false;
+    boolean getRetired() {
+      return retried;
+    }
 
     // Constructor with default retry settings
     public TimelineClientConnectionRetry(Configuration conf) {

+ 5 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java

@@ -30,6 +30,7 @@ import java.io.InputStreamReader;
 import java.io.OutputStream;
 import java.io.PrintStream;
 import java.io.Writer;
+import java.nio.charset.Charset;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -263,7 +264,7 @@ public class AggregatedLogFormat {
           this.uploadedFiles.add(logFile);
         } catch (IOException e) {
           String message = logErrorMessage(logFile, e);
-          out.write(message.getBytes());
+          out.write(message.getBytes(Charset.forName("UTF-8")));
         } finally {
           IOUtils.cleanup(LOG, in);
         }
@@ -651,7 +652,7 @@ public class AggregatedLogFormat {
       OutputStream os = null;
       PrintStream ps = null;
       try {
-        os = new WriterOutputStream(writer);
+        os = new WriterOutputStream(writer, Charset.forName("UTF-8"));
         ps = new PrintStream(os);
         while (true) {
           try {
@@ -781,7 +782,8 @@ public class AggregatedLogFormat {
         currentLogData =
             new BoundedInputStream(valueStream, currentLogLength);
         currentLogData.setPropagateClose(false);
-        currentLogISR = new InputStreamReader(currentLogData);
+        currentLogISR = new InputStreamReader(currentLogData,
+            Charset.forName("UTF-8"));
         currentLogType = logType;
       } catch (EOFException e) {
       }

+ 6 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java

@@ -83,11 +83,16 @@ public class CommonNodeLabelsManager extends AbstractService {
   protected NodeLabelsStore store;
 
   protected static class Label {
-    public Resource resource;
+    private Resource resource;
 
     protected Label() {
       this.resource = Resource.newInstance(0, 0);
     }
+
+    public Resource getResource() {
+      return this.resource;
+    }
+
   }
 
   /**

+ 2 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelsStore.java

@@ -29,8 +29,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 
 public abstract class NodeLabelsStore implements Closeable {
   protected final CommonNodeLabelsManager mgr;
-  protected Configuration conf;
-  
+
   public NodeLabelsStore(CommonNodeLabelsManager mgr) {
     this.mgr = mgr;
   }
@@ -59,9 +58,7 @@ public abstract class NodeLabelsStore implements Closeable {
    */
   public abstract void recover() throws IOException;
   
-  public void init(Configuration conf) throws Exception {
-    this.conf = conf;
-  }
+  public void init(Configuration conf) throws Exception {}
   
   public CommonNodeLabelsManager getNodeLabelsManager() {
     return mgr;

+ 8 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/Graph.java

@@ -17,8 +17,10 @@
  */
 package org.apache.hadoop.yarn.state;
 
-import java.io.FileWriter;
+import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
@@ -149,7 +151,7 @@ public class Graph {
     StringBuilder sb = new StringBuilder();
     if (this.parent == null) {
       sb.append("digraph " + name + " {\n");
-      sb.append(String.format("graph [ label=%s, fontsize=24, fontname=Helvetica];\n",
+      sb.append(String.format("graph [ label=%s, fontsize=24, fontname=Helvetica];%n",
           wrapSafeString(name)));
       sb.append("node [fontsize=12, fontname=Helvetica];\n");
       sb.append("edge [fontsize=9, fontcolor=blue, fontname=Arial];\n");
@@ -163,14 +165,14 @@ public class Graph {
     }
     for (Node n : nodes) {
       sb.append(String.format(
-          "%s%s [ label = %s ];\n",
+          "%s%s [ label = %s ];%n",
           indent,
           wrapSafeString(n.getUniqueId()),
           n.id));
       List<Edge> combinedOuts = combineEdges(n.outs);
       for (Edge e : combinedOuts) {
         sb.append(String.format(
-            "%s%s -> %s [ label = %s ];\n",
+            "%s%s -> %s [ label = %s ];%n",
             indent,
             wrapSafeString(e.from.getUniqueId()),
             wrapSafeString(e.to.getUniqueId()),
@@ -186,7 +188,8 @@ public class Graph {
   }
 
   public void save(String filepath) throws IOException {
-    FileWriter fout = new FileWriter(filepath);
+    OutputStreamWriter fout = new OutputStreamWriter(
+        new FileOutputStream(filepath), Charset.forName("UTF-8"));
     fout.write(generateGraphViz());
     fout.close();
   }

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/VisualizeStateMachine.java

@@ -56,7 +56,7 @@ public class VisualizeStateMachine {
 
   public static void main(String [] args) throws Exception {
     if (args.length < 3) {
-      System.err.printf("Usage: %s <GraphName> <class[,class[,...]]> <OutputFile>\n",
+      System.err.printf("Usage: %s <GraphName> <class[,class[,...]]> <OutputFile>%n",
           VisualizeStateMachine.class.getName());
       System.exit(1);
     }

+ 12 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/LinuxResourceCalculatorPlugin.java

@@ -19,9 +19,11 @@
 package org.apache.hadoop.yarn.util;
 
 import java.io.BufferedReader;
+import java.io.FileInputStream;
 import java.io.FileNotFoundException;
-import java.io.FileReader;
+import java.io.InputStreamReader;
 import java.io.IOException;
+import java.nio.charset.Charset;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -147,9 +149,10 @@ public class LinuxResourceCalculatorPlugin extends ResourceCalculatorPlugin {
 
     // Read "/proc/memInfo" file
     BufferedReader in = null;
-    FileReader fReader = null;
+    InputStreamReader fReader = null;
     try {
-      fReader = new FileReader(procfsMemFile);
+      fReader = new InputStreamReader(
+          new FileInputStream(procfsMemFile), Charset.forName("UTF-8"));
       in = new BufferedReader(fReader);
     } catch (FileNotFoundException f) {
       // shouldn't happen....
@@ -206,9 +209,10 @@ public class LinuxResourceCalculatorPlugin extends ResourceCalculatorPlugin {
     }
     // Read "/proc/cpuinfo" file
     BufferedReader in = null;
-    FileReader fReader = null;
+    InputStreamReader fReader = null;
     try {
-      fReader = new FileReader(procfsCpuFile);
+      fReader = new InputStreamReader(
+          new FileInputStream(procfsCpuFile), Charset.forName("UTF-8"));
       in = new BufferedReader(fReader);
     } catch (FileNotFoundException f) {
       // shouldn't happen....
@@ -253,9 +257,10 @@ public class LinuxResourceCalculatorPlugin extends ResourceCalculatorPlugin {
   private void readProcStatFile() {
     // Read "/proc/stat" file
     BufferedReader in = null;
-    FileReader fReader = null;
+    InputStreamReader fReader = null;
     try {
-      fReader = new FileReader(procfsStatFile);
+      fReader = new InputStreamReader(
+          new FileInputStream(procfsStatFile), Charset.forName("UTF-8"));
       in = new BufferedReader(fReader);
     } catch (FileNotFoundException f) {
       // shouldn't happen....

+ 17 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java

@@ -20,10 +20,12 @@ package org.apache.hadoop.yarn.util;
 
 import java.io.BufferedReader;
 import java.io.File;
+import java.io.FileInputStream;
 import java.io.FileNotFoundException;
-import java.io.FileReader;
+import java.io.InputStreamReader;
 import java.io.IOException;
 import java.math.BigInteger;
+import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.LinkedList;
@@ -297,7 +299,7 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
   }
 
   private static final String PROCESSTREE_DUMP_FORMAT =
-      "\t|- %s %s %d %d %s %d %d %d %d %s\n";
+      "\t|- %s %s %d %d %s %d %d %d %d %s%n";
 
   public List<String> getCurrentProcessIDs() {
     List<String> currentPIDs = new ArrayList<String>();
@@ -317,7 +319,7 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
     // The header.
     ret.append(String.format("\t|- PID PPID PGRPID SESSID CMD_NAME "
         + "USER_MODE_TIME(MILLIS) SYSTEM_TIME(MILLIS) VMEM_USAGE(BYTES) "
-        + "RSSMEM_USAGE(PAGES) FULL_CMD_LINE\n"));
+        + "RSSMEM_USAGE(PAGES) FULL_CMD_LINE%n"));
     for (ProcessInfo p : processTree.values()) {
       if (p != null) {
         ret.append(String.format(PROCESSTREE_DUMP_FORMAT, p.getPid(), p
@@ -489,10 +491,12 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
     ProcessInfo ret = null;
     // Read "procfsDir/<pid>/stat" file - typically /proc/<pid>/stat
     BufferedReader in = null;
-    FileReader fReader = null;
+    InputStreamReader fReader = null;
     try {
       File pidDir = new File(procfsDir, pinfo.getPid());
-      fReader = new FileReader(new File(pidDir, PROCFS_STAT_FILE));
+      fReader = new InputStreamReader(
+          new FileInputStream(
+              new File(pidDir, PROCFS_STAT_FILE)), Charset.forName("UTF-8"));
       in = new BufferedReader(fReader);
     } catch (FileNotFoundException f) {
       // The process vanished in the interim!
@@ -671,11 +675,12 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
         return ret;
       }
       BufferedReader in = null;
-      FileReader fReader = null;
+      InputStreamReader fReader = null;
       try {
-        fReader =
-            new FileReader(new File(new File(procfsDir, pid.toString()),
-                PROCFS_CMDLINE_FILE));
+        fReader = new InputStreamReader(
+            new FileInputStream(
+                new File(new File(procfsDir, pid.toString()), PROCFS_CMDLINE_FILE)),
+                Charset.forName("UTF-8"));
       } catch (FileNotFoundException f) {
         // The process vanished in the interim!
         return ret;
@@ -725,14 +730,15 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
   private static void constructProcessSMAPInfo(ProcessTreeSmapMemInfo pInfo,
       String procfsDir) {
     BufferedReader in = null;
-    FileReader fReader = null;
+    InputStreamReader fReader = null;
     try {
       File pidDir = new File(procfsDir, pInfo.getPid());
       File file = new File(pidDir, SMAPS);
       if (!file.exists()) {
         return;
       }
-      fReader = new FileReader(file);
+      fReader = new InputStreamReader(
+          new FileInputStream(file), Charset.forName("UTF-8"));
       in = new BufferedReader(fReader);
       ProcessSmapMemoryInfo memoryMappingInfo = null;
       List<String> lines = IOUtils.readLines(in);

+ 0 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java

@@ -32,18 +32,6 @@ import org.apache.hadoop.util.Shell;
 @InterfaceAudience.LimitedPrivate({"YARN", "MAPREDUCE"})
 @InterfaceStability.Unstable
 public abstract class ResourceCalculatorPlugin extends Configured {
-  
-  protected String processPid = null;
-
-  /**
-   * set the pid of the process for which <code>getProcResourceValues</code>
-   * will be invoked
-   * 
-   * @param pid
-   */
-  public void setProcessPid(String pid) {
-    processPid = pid;
-  }
 
   /**
    * Obtain the total size of the virtual memory present in the system.

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsBasedProcessTree.java

@@ -162,10 +162,10 @@ public class WindowsBasedProcessTree extends ResourceCalculatorProcessTree {
     StringBuilder ret = new StringBuilder();
     // The header.
     ret.append(String.format("\t|- PID " + "CPU_TIME(MILLIS) "
-        + "VMEM(BYTES) WORKING_SET(BYTES)\n"));
+        + "VMEM(BYTES) WORKING_SET(BYTES)%n"));
     for (ProcessInfo p : processTree.values()) {
       if (p != null) {
-        ret.append(String.format("\t|- %s %d %d %d\n", p.pid,
+        ret.append(String.format("\t|- %s %d %d %d%n", p.pid,
             p.cpuTimeMs, p.vmem, p.workingSet));
       }
     }

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java

@@ -197,7 +197,7 @@ public class TestTimelineClient {
         ce.getMessage().contains("Connection retries limit exceeded"));
       // we would expect this exception here, check if the client has retried
       Assert.assertTrue("Retry filter didn't perform any retries! ", client
-        .connectionRetry.retried);
+        .connectionRetry.getRetired());
     }
   }
 
@@ -272,7 +272,7 @@ public class TestTimelineClient {
             .getMessage().contains("Connection retries limit exceeded"));
     // we would expect this exception here, check if the client has retried
     Assert.assertTrue("Retry filter didn't perform any retries! ",
-        client.connectionRetry.retried);
+        client.connectionRetry.getRetired());
   }
 
   private static ClientResponse mockEntityClientResponse(

+ 5 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java

@@ -351,7 +351,7 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
         if (oldLabels.isEmpty()) {
           // update labels
           Label label = labelCollections.get(NO_LABEL);
-          Resources.subtractFrom(label.resource, oldNM.resource);
+          Resources.subtractFrom(label.getResource(), oldNM.resource);
 
           // update queues, all queue can access this node
           for (Queue q : queueCollections.values()) {
@@ -364,7 +364,7 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
             if (null == label) {
               continue;
             }
-            Resources.subtractFrom(label.resource, oldNM.resource);
+            Resources.subtractFrom(label.getResource(), oldNM.resource);
           }
 
           // update queues, only queue can access this node will be subtract
@@ -383,7 +383,7 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
         if (newLabels.isEmpty()) {
           // update labels
           Label label = labelCollections.get(NO_LABEL);
-          Resources.addTo(label.resource, newNM.resource);
+          Resources.addTo(label.getResource(), newNM.resource);
 
           // update queues, all queue can access this node
           for (Queue q : queueCollections.values()) {
@@ -393,7 +393,7 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
           // update labels
           for (String labelName : newLabels) {
             Label label = labelCollections.get(labelName);
-            Resources.addTo(label.resource, newNM.resource);
+            Resources.addTo(label.getResource(), newNM.resource);
           }
 
           // update queues, only queue can access this node will be subtract
@@ -414,7 +414,7 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
       if (null == labelCollections.get(label)) {
         return Resources.none();
       }
-      return labelCollections.get(label).resource;
+      return labelCollections.get(label).getResource();
     } finally {
       readLock.unlock();
     }