Pārlūkot izejas kodu

Revert ca3381de61ff53604d9389694049a1a4dedc7571 and 20674255ad891bb693ecb2faa5bdb37477073dac as missing JIRA number in commit message

Junping Du 10 gadi atpakaļ
vecāks
revīzija
856b46cf47
13 mainītis faili ar 39 papildinājumiem un 71 dzēšanām
  1. 0 3
      hadoop-yarn-project/CHANGES.txt
  2. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
  3. 4 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/main/java/org/apache/hadoop/yarn/applications/unmanagedamlauncher/UnmanagedAMLauncher.java
  4. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
  5. 6 14
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
  6. 2 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java
  7. 1 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/QueueCLI.java
  8. 1 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/CuratorService.java
  9. 10 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java
  10. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/RegistryAdminService.java
  11. 3 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/lib/ZKClient.java
  12. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
  13. 2 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java

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

@@ -275,9 +275,6 @@ Release 2.7.0 - UNRELEASED
     YARN-2340. Fixed NPE when queue is stopped during RM restart.
     (Rohith Sharmaks via jianhe)
 
-    YARN-2940. Fix new findbugs warnings in rest of the hadoop-yarn components. (Li Lu 
-    via junping_du)
-
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java

@@ -463,11 +463,11 @@ public class ApplicationMaster {
       scriptPath = envs.get(DSConstants.DISTRIBUTEDSHELLSCRIPTLOCATION);
 
       if (envs.containsKey(DSConstants.DISTRIBUTEDSHELLSCRIPTTIMESTAMP)) {
-        shellScriptPathTimestamp = Long.parseLong(envs
+        shellScriptPathTimestamp = Long.valueOf(envs
             .get(DSConstants.DISTRIBUTEDSHELLSCRIPTTIMESTAMP));
       }
       if (envs.containsKey(DSConstants.DISTRIBUTEDSHELLSCRIPTLEN)) {
-        shellScriptPathLen = Long.parseLong(envs
+        shellScriptPathLen = Long.valueOf(envs
             .get(DSConstants.DISTRIBUTEDSHELLSCRIPTLEN));
       }
       if (!scriptPath.isEmpty()

+ 4 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/main/java/org/apache/hadoop/yarn/applications/unmanagedamlauncher/UnmanagedAMLauncher.java

@@ -25,7 +25,6 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.net.InetAddress;
-import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.Map;
@@ -233,11 +232,11 @@ public class UnmanagedAMLauncher {
     Process amProc = Runtime.getRuntime().exec(amCmd, envAMList.toArray(envAM));
 
     final BufferedReader errReader = 
-        new BufferedReader(new InputStreamReader(
-            amProc.getErrorStream(), Charset.forName("UTF-8")));
+        new BufferedReader(new InputStreamReader(amProc
+                                                 .getErrorStream()));
     final BufferedReader inReader = 
-        new BufferedReader(new InputStreamReader(
-            amProc.getInputStream(), Charset.forName("UTF-8")));
+        new BufferedReader(new InputStreamReader(amProc
+                                                 .getInputStream()));
     
     // read error and input streams as this would free up the buffers
     // free the error stream buffer

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java

@@ -260,9 +260,10 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
         blacklistToAdd.addAll(blacklistAdditions);
         blacklistToRemove.addAll(blacklistRemovals);
         
-        ResourceBlacklistRequest blacklistRequest =
+        ResourceBlacklistRequest blacklistRequest = 
+            (blacklistToAdd != null) || (blacklistToRemove != null) ? 
             ResourceBlacklistRequest.newInstance(blacklistToAdd,
-                blacklistToRemove);
+                blacklistToRemove) : null;
         
         allocateRequest =
             AllocateRequest.newInstance(lastResponseId, progressIndicator,

+ 6 - 14
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java

@@ -19,9 +19,7 @@ package org.apache.hadoop.yarn.client.cli;
 
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
-import java.io.OutputStreamWriter;
 import java.io.PrintWriter;
-import java.nio.charset.Charset;
 import java.text.DecimalFormat;
 import java.util.EnumSet;
 import java.util.HashSet;
@@ -278,8 +276,7 @@ public class ApplicationCLI extends YarnCLI {
     }
     // Use PrintWriter.println, which uses correct platform line ending.
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    PrintWriter appAttemptReportStr = new PrintWriter(
-        new OutputStreamWriter(baos, Charset.forName("UTF-8")));
+    PrintWriter appAttemptReportStr = new PrintWriter(baos);
     if (appAttemptReport != null) {
       appAttemptReportStr.println("Application Attempt Report : ");
       appAttemptReportStr.print("\tApplicationAttempt-Id : ");
@@ -338,8 +335,7 @@ public class ApplicationCLI extends YarnCLI {
     }
     // Use PrintWriter.println, which uses correct platform line ending.
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    PrintWriter containerReportStr = new PrintWriter(
-        new OutputStreamWriter(baos, Charset.forName("UTF-8")));
+    PrintWriter containerReportStr = new PrintWriter(baos);
     if (containerReport != null) {
       containerReportStr.println("Container Report : ");
       containerReportStr.print("\tContainer-Id : ");
@@ -380,8 +376,7 @@ public class ApplicationCLI extends YarnCLI {
   private void listApplications(Set<String> appTypes,
       EnumSet<YarnApplicationState> appStates) throws YarnException,
       IOException {
-    PrintWriter writer = new PrintWriter(
-        new OutputStreamWriter(sysout, Charset.forName("UTF-8")));
+    PrintWriter writer = new PrintWriter(sysout);
     if (allAppStates) {
       for (YarnApplicationState appState : YarnApplicationState.values()) {
         appStates.add(appState);
@@ -483,8 +478,7 @@ public class ApplicationCLI extends YarnCLI {
     }
     // Use PrintWriter.println, which uses correct platform line ending.
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    PrintWriter appReportStr = new PrintWriter(
-        new OutputStreamWriter(baos, Charset.forName("UTF-8")));
+    PrintWriter appReportStr = new PrintWriter(baos);
     if (appReport != null) {
       appReportStr.println("Application Report : ");
       appReportStr.print("\tApplication-Id : ");
@@ -560,8 +554,7 @@ public class ApplicationCLI extends YarnCLI {
    */
   private void listApplicationAttempts(String applicationId) throws YarnException,
       IOException {
-    PrintWriter writer = new PrintWriter(
-        new OutputStreamWriter(sysout, Charset.forName("UTF-8")));
+    PrintWriter writer = new PrintWriter(sysout);
 
     List<ApplicationAttemptReport> appAttemptsReport = client
         .getApplicationAttempts(ConverterUtils.toApplicationId(applicationId));
@@ -587,8 +580,7 @@ public class ApplicationCLI extends YarnCLI {
    */
   private void listContainers(String appAttemptId) throws YarnException,
       IOException {
-    PrintWriter writer = new PrintWriter(
-        new OutputStreamWriter(sysout, Charset.forName("UTF-8")));
+    PrintWriter writer = new PrintWriter(sysout);
 
     List<ContainerReport> appsReport = client
         .getContainers(ConverterUtils.toApplicationAttemptId(appAttemptId));

+ 2 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java

@@ -19,9 +19,7 @@ package org.apache.hadoop.yarn.client.cli;
 
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
-import java.io.OutputStreamWriter;
 import java.io.PrintWriter;
-import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Date;
@@ -143,8 +141,7 @@ public class NodeCLI extends YarnCLI {
    */
   private void listClusterNodes(Set<NodeState> nodeStates) 
             throws YarnException, IOException {
-    PrintWriter writer = new PrintWriter(
-        new OutputStreamWriter(sysout, Charset.forName("UTF-8")));
+    PrintWriter writer = new PrintWriter(sysout);
     List<NodeReport> nodesReport = client.getNodeReports(
                                        nodeStates.toArray(new NodeState[0]));
     writer.println("Total Nodes:" + nodesReport.size());
@@ -170,8 +167,7 @@ public class NodeCLI extends YarnCLI {
     List<NodeReport> nodesReport = client.getNodeReports();
     // Use PrintWriter.println, which uses correct platform line ending.
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    PrintWriter nodeReportStr = new PrintWriter(
-        new OutputStreamWriter(baos, Charset.forName("UTF-8")));
+    PrintWriter nodeReportStr = new PrintWriter(baos);
     NodeReport nodeReport = null;
     for (NodeReport report : nodesReport) {
       if (!report.getNodeId().equals(nodeId)) {

+ 1 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/QueueCLI.java

@@ -18,9 +18,7 @@
 package org.apache.hadoop.yarn.client.cli;
 
 import java.io.IOException;
-import java.io.OutputStreamWriter;
 import java.io.PrintWriter;
-import java.nio.charset.Charset;
 import java.text.DecimalFormat;
 import java.util.Set;
 
@@ -104,8 +102,7 @@ public class QueueCLI extends YarnCLI {
    */
   private int listQueue(String queueName) throws YarnException, IOException {
     int rc;
-    PrintWriter writer = new PrintWriter(
-        new OutputStreamWriter(sysout, Charset.forName("UTF-8")));
+    PrintWriter writer = new PrintWriter(sysout);
 
     QueueInfo queueInfo = client.getQueueInfo(queueName);
     if (queueInfo != null) {

+ 1 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/CuratorService.java

@@ -491,10 +491,7 @@ public class CuratorService extends CompositeService
   public boolean zkPathExists(String path) throws IOException {
     checkServiceLive();
     try {
-      // if zkStat(path) returns without throwing an exception, the return value
-      // is guaranteed to be not null
-      zkStat(path);
-      return true;
+      return zkStat(path) != null;
     } catch (PathNotFoundException e) {
       return false;
     } catch (IOException e) {

+ 10 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java

@@ -592,17 +592,17 @@ public class RegistrySecurity extends AbstractService {
    * Note the semicolon on the last entry
    */
   private static final String JAAS_ENTRY =
-      "%s { %n"
-      + " %s required%n"
+      "%s { \n"
+      + " %s required\n"
       // kerberos module
-      + " keyTab=\"%s\"%n"
-      + " debug=true%n"
-      + " principal=\"%s\"%n"
-      + " useKeyTab=true%n"
-      + " useTicketCache=false%n"
-      + " doNotPrompt=true%n"
-      + " storeKey=true;%n"
-      + "}; %n"
+      + " keyTab=\"%s\"\n"
+      + " debug=true\n"
+      + " principal=\"%s\"\n"
+      + " useKeyTab=true\n"
+      + " useTicketCache=false\n"
+      + " doNotPrompt=true\n"
+      + " storeKey=true;\n"
+      + "}; \n"
       ;
 
   /**

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/RegistryAdminService.java

@@ -217,9 +217,9 @@ public class RegistryAdminService extends RegistryOperationsService {
 
       String message = String.format(Locale.ENGLISH,
           "Failed to create root paths {%s};" +
-          "%ndiagnostics={%s}" +
-          "%ncurrent registry is:" +
-          "%n{%s}",
+          "\ndiagnostics={%s}" +
+          "\ncurrent registry is:" +
+          "\n{%s}",
           e,
           bindingDiagnosticDetails(),
           dumpRegistryRobustly(true));

+ 3 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/lib/ZKClient.java

@@ -19,7 +19,6 @@
 package org.apache.hadoop.yarn.lib;
 
 import java.io.IOException;
-import java.nio.charset.Charset;
 import java.util.List;
 
 import org.apache.zookeeper.CreateMode;
@@ -56,8 +55,8 @@ public class ZKClient {
   public void registerService(String path, String data) throws
     IOException, InterruptedException {
     try {
-      zkClient.create(path, data.getBytes(Charset.forName("UTF-8")),
-          ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
+      zkClient.create(path, data.getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, 
+          CreateMode.EPHEMERAL);
     } catch(KeeperException ke) {
       throw new IOException(ke);
     }
@@ -110,7 +109,7 @@ public class ZKClient {
     try {
       Stat stat = new Stat();
       byte[] byteData = zkClient.getData(path, false, stat);
-      data = new String(byteData, Charset.forName("UTF-8"));
+      data = new String(byteData);
     } catch(KeeperException ke) {
       throw new IOException(ke);
     }

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java

@@ -118,7 +118,7 @@ public class BuilderUtils {
   public static ApplicationId newApplicationId(RecordFactory recordFactory,
       long clustertimestamp, CharSequence id) {
     return ApplicationId.newInstance(clustertimestamp,
-        Integer.parseInt(id.toString()));
+        Integer.valueOf(id.toString()));
   }
 
   public static ApplicationId newApplicationId(RecordFactory recordFactory,
@@ -137,7 +137,7 @@ public class BuilderUtils {
 
   public static ApplicationId convert(long clustertimestamp, CharSequence id) {
     return ApplicationId.newInstance(clustertimestamp,
-        Integer.parseInt(id.toString()));
+        Integer.valueOf(id.toString()));
   }
 
   public static ContainerId newContainerId(ApplicationAttemptId appAttemptId,

+ 2 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java

@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.PrintWriter;
-import java.io.ObjectInputStream;
 import java.net.InetAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
@@ -69,9 +68,9 @@ public class WebAppProxyServlet extends HttpServlet {
   
   public static final String PROXY_USER_COOKIE_NAME = "proxy-user";
 
-  private transient List<TrackingUriPlugin> trackingUriPlugins;
+  private final List<TrackingUriPlugin> trackingUriPlugins;
   private final String rmAppPageUrlBase;
-  private transient YarnConfiguration conf;
+  private final transient YarnConfiguration conf;
 
   private static class _ implements Hamlet._ {
     //Empty
@@ -351,13 +350,4 @@ public class WebAppProxyServlet extends HttpServlet {
       throw new IOException(e);
     }
   }
-
-  private void readObject(ObjectInputStream input)
-      throws IOException, ClassNotFoundException {
-    input.defaultReadObject();
-    conf = new YarnConfiguration();
-    this.trackingUriPlugins =
-        conf.getInstances(YarnConfiguration.YARN_TRACKING_URL_GENERATOR,
-            TrackingUriPlugin.class);
-  }
 }