浏览代码

Merge r1555021 through r1557038 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-5535@1557039 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 11 年之前
父节点
当前提交
1c7aa44d62
共有 19 个文件被更改,包括 298 次插入51 次删除
  1. 11 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 5 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
  3. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java
  4. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
  5. 40 5
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/metrics/RpcMetrics.java
  6. 0 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java
  7. 40 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
  8. 1 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
  9. 5 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  10. 8 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  11. 22 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  12. 19 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
  13. 41 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
  14. 61 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
  15. 8 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java
  16. 5 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/ssl/TestEncryptedShuffle.java
  17. 5 0
      hadoop-yarn-project/CHANGES.txt
  18. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto
  19. 23 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml

+ 11 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -412,6 +412,12 @@ Release 2.4.0 - UNRELEASED
     HADOOP-10198. DomainSocket: add support for socketpair.
     (Colin Patrick McCabe via wang)
 
+    HADOOP-10208. Remove duplicate initialization in StringUtils.getStringCollection.
+    (Benoy Antony via jing9)
+
+    HADOOP-9420. Add percentile or max metric for rpcQueueTime, processing time.
+    (Liang Xie via wang)
+
   OPTIMIZATIONS
 
     HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)
@@ -500,6 +506,11 @@ Release 2.4.0 - UNRELEASED
     HADOOP-10147  HDFS-5678 Upgrade to commons-logging 1.1.3 to avoid potential
     deadlock in MiniDFSCluster (stevel)
 
+    HADOOP-10207. TestUserGroupInformation#testLogin is flaky (jxiang via cmccabe)
+
+    HADOOP-10214. Fix multithreaded correctness warnings in ActiveStandbyElector
+    (Liang Xie via kasha)
+
 Release 2.3.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 5 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java

@@ -242,4 +242,9 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
 
   public static final String HADOOP_USER_GROUP_METRICS_PERCENTILES_INTERVALS =
     "hadoop.user.group.metrics.percentiles.intervals";
+
+  public static final String RPC_METRICS_QUANTILE_ENABLE =
+      "rpc.metrics.quantile.enable";
+  public static final String  RPC_METRICS_PERCENTILES_INTERVALS_KEY =
+      "rpc.metrics.percentiles.intervals";
 }

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java

@@ -768,7 +768,7 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
   }
 
   @InterfaceAudience.Private
-  public void terminateConnection() {
+  public synchronized void terminateConnection() {
     if (zkClient == null) {
       return;
     }

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java

@@ -2193,7 +2193,7 @@ public abstract class Server {
     listener = new Listener();
     this.port = listener.getAddress().getPort();    
     connectionManager = new ConnectionManager();
-    this.rpcMetrics = RpcMetrics.create(this);
+    this.rpcMetrics = RpcMetrics.create(this, conf);
     this.rpcDetailedMetrics = RpcDetailedMetrics.create(this.port);
     this.tcpNoDelay = conf.getBoolean(
         CommonConfigurationKeysPublic.IPC_SERVER_TCPNODELAY_KEY,

+ 40 - 5
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/metrics/RpcMetrics.java

@@ -19,14 +19,17 @@ package org.apache.hadoop.ipc.metrics;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.metrics2.annotation.Metric;
 import org.apache.hadoop.metrics2.annotation.Metrics;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.lib.MetricsRegistry;
 import org.apache.hadoop.metrics2.lib.MutableCounterInt;
 import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableQuantiles;
 import org.apache.hadoop.metrics2.lib.MutableRate;
 
 /**
@@ -41,26 +44,48 @@ public class RpcMetrics {
   final Server server;
   final MetricsRegistry registry;
   final String name;
+  final boolean rpcQuantileEnable;
   
-  RpcMetrics(Server server) {
+  RpcMetrics(Server server, Configuration conf) {
     String port = String.valueOf(server.getListenerAddress().getPort());
-    name = "RpcActivityForPort"+ port;
+    name = "RpcActivityForPort" + port;
     this.server = server;
     registry = new MetricsRegistry("rpc").tag("port", "RPC port", port);
-    LOG.debug("Initialized "+ registry);
+    int[] intervals = conf.getInts(
+        CommonConfigurationKeys.RPC_METRICS_PERCENTILES_INTERVALS_KEY);
+    rpcQuantileEnable = (intervals.length > 0) && conf.getBoolean(
+        CommonConfigurationKeys.RPC_METRICS_QUANTILE_ENABLE, false);
+    if (rpcQuantileEnable) {
+      rpcQueueTimeMillisQuantiles =
+          new MutableQuantiles[intervals.length];
+      rpcProcessingTimeMillisQuantiles =
+          new MutableQuantiles[intervals.length];
+      for (int i = 0; i < intervals.length; i++) {
+        int interval = intervals[i];
+        rpcQueueTimeMillisQuantiles[i] = registry.newQuantiles("rpcQueueTime"
+            + interval + "s", "rpc queue time in milli second", "ops",
+            "latency", interval);
+        rpcProcessingTimeMillisQuantiles[i] = registry.newQuantiles(
+            "rpcProcessingTime" + interval + "s",
+            "rpc processing time in milli second", "ops", "latency", interval);
+      }
+    }
+    LOG.debug("Initialized " + registry);
   }
 
   public String name() { return name; }
 
-  public static RpcMetrics create(Server server) {
-    RpcMetrics m = new RpcMetrics(server);
+  public static RpcMetrics create(Server server, Configuration conf) {
+    RpcMetrics m = new RpcMetrics(server, conf);
     return DefaultMetricsSystem.instance().register(m.name, null, m);
   }
 
   @Metric("Number of received bytes") MutableCounterLong receivedBytes;
   @Metric("Number of sent bytes") MutableCounterLong sentBytes;
   @Metric("Queue time") MutableRate rpcQueueTime;
+  MutableQuantiles[] rpcQueueTimeMillisQuantiles;
   @Metric("Processsing time") MutableRate rpcProcessingTime;
+  MutableQuantiles[] rpcProcessingTimeMillisQuantiles;
   @Metric("Number of authentication failures")
   MutableCounterInt rpcAuthenticationFailures;
   @Metric("Number of authentication successes")
@@ -146,6 +171,11 @@ public class RpcMetrics {
   //@Override
   public void addRpcQueueTime(int qTime) {
     rpcQueueTime.add(qTime);
+    if (rpcQuantileEnable) {
+      for (MutableQuantiles q : rpcQueueTimeMillisQuantiles) {
+        q.add(qTime);
+      }
+    }
   }
 
   /**
@@ -155,5 +185,10 @@ public class RpcMetrics {
   //@Override
   public void addRpcProcessingTime(int processingTime) {
     rpcProcessingTime.add(processingTime);
+    if (rpcQuantileEnable) {
+      for (MutableQuantiles q : rpcProcessingTimeMillisQuantiles) {
+        q.add(processingTime);
+      }
+    }
   }
 }

+ 0 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java

@@ -343,7 +343,6 @@ public class StringUtils {
     if (str == null)
       return values;
     StringTokenizer tokenizer = new StringTokenizer(str, delim);
-    values = new ArrayList<String>();
     while (tokenizer.hasMoreTokens()) {
       values.add(tokenizer.nextToken());
     }

+ 40 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.ipc;
 
 import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
 import static org.apache.hadoop.test.MetricsAsserts.assertCounterGt;
+import static org.apache.hadoop.test.MetricsAsserts.getLongCounter;
 import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotSame;
@@ -67,6 +68,7 @@ import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.security.authorize.Service;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.test.MetricsAsserts;
 import org.apache.hadoop.test.MockitoUtil;
 import org.junit.Before;
 import org.junit.Test;
@@ -961,6 +963,44 @@ public class TestRPC {
     }
   }
 
+  @Test
+  public void testRpcMetrics() throws Exception {
+    Configuration configuration = new Configuration();
+    final int interval = 1;
+    configuration.setBoolean(CommonConfigurationKeys.
+        RPC_METRICS_QUANTILE_ENABLE, true);
+    configuration.set(CommonConfigurationKeys.
+        RPC_METRICS_PERCENTILES_INTERVALS_KEY, "" + interval);
+    final Server server = new RPC.Builder(configuration)
+        .setProtocol(TestProtocol.class).setInstance(new TestImpl())
+        .setBindAddress(ADDRESS).setPort(0).setNumHandlers(5).setVerbose(true)
+        .build();
+    server.start();
+    final TestProtocol proxy = RPC.getProxy(TestProtocol.class,
+        TestProtocol.versionID, server.getListenerAddress(), configuration);
+    try {
+      for (int i=0; i<1000; i++) {
+        proxy.ping();
+        proxy.echo("" + i);
+      }
+      MetricsRecordBuilder rpcMetrics =
+          getMetrics(server.getRpcMetrics().name());
+      assertTrue("Expected non-zero rpc queue time",
+          getLongCounter("RpcQueueTimeNumOps", rpcMetrics) > 0);
+      assertTrue("Expected non-zero rpc processing time",
+          getLongCounter("RpcProcessingTimeNumOps", rpcMetrics) > 0);
+      MetricsAsserts.assertQuantileGauges("RpcQueueTime" + interval + "s",
+          rpcMetrics);
+      MetricsAsserts.assertQuantileGauges("RpcProcessingTime" + interval + "s",
+          rpcMetrics);
+    } finally {
+      if (proxy != null) {
+        RPC.stopProxy(proxy);
+      }
+      server.stop();
+    }
+  }
+
   public static void main(String[] args) throws IOException {
     new TestRPC().testCallsInternal(conf);
 

+ 1 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java

@@ -738,7 +738,7 @@ public class TestUserGroupInformation {
       long groups) throws InterruptedException {
     MetricsRecordBuilder rb = getMetrics("UgiMetrics");
     if (groups > 0) {
-      assertCounter("GetGroupsNumOps", groups, rb);
+      assertCounterGt("GetGroupsNumOps", groups-1, rb);
       double avg = getDoubleGauge("GetGroupsAvgTime", rb);
       assertTrue(avg >= 0.0);
 

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -245,6 +245,9 @@ Trunk (Unreleased)
     HDFS-5715. Use Snapshot ID to indicate the corresponding Snapshot for a
     FileDiff/DirectoryDiff. (jing9)
 
+    HDFS-5721. sharedEditsImage in Namenode#initializeSharedEdits() should be 
+    closed before method returns. (Ted Yu via junping_du)
+
   OPTIMIZATIONS
 
     HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only. (cmccabe)
@@ -736,6 +739,8 @@ Release 2.4.0 - UNRELEASED
     HDFS-5690. DataNode fails to start in secure mode when dfs.http.policy equals to 
     HTTP_ONLY. (Haohui Mai via jing9)
 
+    HDFS-5449. WebHdfs compatibility broken between 2.2 and 1.x / 23.x (kihwal)
+
   BREAKDOWN OF HDFS-2832 SUBTASKS AND RELATED JIRAS
 
     HDFS-4985. Add storage type to the protocol and expose it in block report

+ 8 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -622,8 +622,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
     long loadStart = now();
     String nameserviceId = DFSUtil.getNamenodeNameServiceId(conf);
-    namesystem.loadFSImage(startOpt, fsImage,
-      HAUtil.isHAEnabled(conf, nameserviceId));
+    try {
+      namesystem.loadFSImage(startOpt, fsImage,
+        HAUtil.isHAEnabled(conf, nameserviceId));
+    } catch (IOException ioe) {
+      LOG.warn("Encountered exception loading fsimage", ioe);
+      fsImage.close();
+      throw ioe;
+    }
     long timeTakenToLoadFSImage = now() - loadStart;
     LOG.info("Finished loading FSImage in " + timeTakenToLoadFSImage + " msecs");
     NameNodeMetrics nnMetrics = NameNode.getNameNodeMetrics();

+ 22 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -816,14 +816,20 @@ public class NameNode implements NameNodeStatusMXBean {
     System.out.println("Formatting using clusterid: " + clusterId);
     
     FSImage fsImage = new FSImage(conf, nameDirsToFormat, editDirsToFormat);
-    FSNamesystem fsn = new FSNamesystem(conf, fsImage);
-    fsImage.getEditLog().initJournalsForWrite();
-    
-    if (!fsImage.confirmFormat(force, isInteractive)) {
-      return true; // aborted
+    try {
+      FSNamesystem fsn = new FSNamesystem(conf, fsImage);
+      fsImage.getEditLog().initJournalsForWrite();
+
+      if (!fsImage.confirmFormat(force, isInteractive)) {
+        return true; // aborted
+      }
+
+      fsImage.format(fsn, clusterId);
+    } catch (IOException ioe) {
+      LOG.warn("Encountered exception during format: ", ioe);
+      fsImage.close();
+      throw ioe;
     }
-    
-    fsImage.format(fsn, clusterId);
     return false;
   }
 
@@ -897,6 +903,7 @@ public class NameNode implements NameNodeStatusMXBean {
     }
 
     NNStorage existingStorage = null;
+    FSImage sharedEditsImage = null;
     try {
       FSNamesystem fsns =
           FSNamesystem.loadFromDisk(getConfigurationWithoutSharedEdits(conf));
@@ -906,7 +913,7 @@ public class NameNode implements NameNodeStatusMXBean {
       
       List<URI> sharedEditsDirs = FSNamesystem.getSharedEditsDirs(conf);
       
-      FSImage sharedEditsImage = new FSImage(conf,
+      sharedEditsImage = new FSImage(conf,
           Lists.<URI>newArrayList(),
           sharedEditsDirs);
       sharedEditsImage.getEditLog().initJournalsForWrite();
@@ -934,6 +941,13 @@ public class NameNode implements NameNodeStatusMXBean {
       LOG.error("Could not initialize shared edits dir", ioe);
       return true; // aborted
     } finally {
+      if (sharedEditsImage != null) {
+        try {
+          sharedEditsImage.close();
+        }  catch (IOException ioe) {
+          LOG.warn("Could not close sharedEditsImage", ioe);
+        }
+      }
       // Have to unlock storage explicitly for the case when we're running in a
       // unit test, which runs in the same JVM as NNs.
       if (existingStorage != null) {

+ 19 - 14
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java

@@ -190,24 +190,29 @@ public class BootstrapStandby implements Tool, Configurable {
     // Load the newly formatted image, using all of the directories (including shared
     // edits)
     FSImage image = new FSImage(conf);
-    image.getStorage().setStorageInfo(storage);
-    image.initEditLog();
-    assert image.getEditLog().isOpenForRead() :
+    try {
+      image.getStorage().setStorageInfo(storage);
+      image.initEditLog();
+      assert image.getEditLog().isOpenForRead() :
         "Expected edit log to be open for read";
-    
-    // Ensure that we have enough edits already in the shared directory to
-    // start up from the last checkpoint on the active.
-    if (!checkLogsAvailableForRead(image, imageTxId, curTxId)) {
-      return ERR_CODE_LOGS_UNAVAILABLE;
-    }
-    
-    image.getStorage().writeTransactionIdFileToStorage(curTxId);
 
-    // Download that checkpoint into our storage directories.
-    MD5Hash hash = TransferFsImage.downloadImageToStorage(
+      // Ensure that we have enough edits already in the shared directory to
+      // start up from the last checkpoint on the active.
+      if (!checkLogsAvailableForRead(image, imageTxId, curTxId)) {
+        return ERR_CODE_LOGS_UNAVAILABLE;
+      }
+
+      image.getStorage().writeTransactionIdFileToStorage(curTxId);
+
+      // Download that checkpoint into our storage directories.
+      MD5Hash hash = TransferFsImage.downloadImageToStorage(
         otherHttpAddr, imageTxId,
         storage, true);
-    image.saveDigestAndRenameCheckpointImage(imageTxId, hash);
+      image.saveDigestAndRenameCheckpointImage(imageTxId, hash);
+    } catch (IOException ioe) {
+      image.close();
+      throw ioe;
+    }
     return 0;
   }
 

+ 41 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java

@@ -271,7 +271,7 @@ public class JsonUtil {
   }
   
   /** Convert a DatanodeInfo to a Json map. */
-  private static Map<String, Object> toJsonMap(final DatanodeInfo datanodeinfo) {
+  static Map<String, Object> toJsonMap(final DatanodeInfo datanodeinfo) {
     if (datanodeinfo == null) {
       return null;
     }
@@ -279,6 +279,9 @@ public class JsonUtil {
     // TODO: Fix storageID
     final Map<String, Object> m = new TreeMap<String, Object>();
     m.put("ipAddr", datanodeinfo.getIpAddr());
+    // 'name' is equivalent to ipAddr:xferPort. Older clients (1.x, 0.23.x) 
+    // expects this instead of the two fields.
+    m.put("name", datanodeinfo.getXferAddr());
     m.put("hostName", datanodeinfo.getHostName());
     m.put("storageID", datanodeinfo.getDatanodeUuid());
     m.put("xferPort", datanodeinfo.getXferPort());
@@ -325,17 +328,49 @@ public class JsonUtil {
   }
 
   /** Convert a Json map to an DatanodeInfo object. */
-  static DatanodeInfo toDatanodeInfo(final Map<?, ?> m) {
+  static DatanodeInfo toDatanodeInfo(final Map<?, ?> m) 
+    throws IOException {
     if (m == null) {
       return null;
     }
 
+    // ipAddr and xferPort are the critical fields for accessing data.
+    // If any one of the two is missing, an exception needs to be thrown.
+
+    // Handle the case of old servers (1.x, 0.23.x) sending 'name' instead
+    //  of ipAddr and xferPort.
+    String ipAddr = getString(m, "ipAddr", null);
+    int xferPort = getInt(m, "xferPort", -1);
+    if (ipAddr == null) {
+      String name = getString(m, "name", null);
+      if (name != null) {
+        int colonIdx = name.indexOf(':');
+        if (colonIdx > 0) {
+          ipAddr = name.substring(0, colonIdx);
+          xferPort = Integer.parseInt(name.substring(colonIdx +1));
+        } else {
+          throw new IOException(
+              "Invalid value in server response: name=[" + name + "]");
+        }
+      } else {
+        throw new IOException(
+            "Missing both 'ipAddr' and 'name' in server response.");
+      }
+      // ipAddr is non-null & non-empty string at this point.
+    }
+
+    // Check the validity of xferPort.
+    if (xferPort == -1) {
+      throw new IOException(
+          "Invalid or missing 'xferPort' in server response.");
+    }
+
     // TODO: Fix storageID
     return new DatanodeInfo(
-        (String)m.get("ipAddr"),
+        ipAddr,
         (String)m.get("hostName"),
         (String)m.get("storageID"),
-        (int)(long)(Long)m.get("xferPort"),
+        xferPort,
         (int)(long)(Long)m.get("infoPort"),
         getInt(m, "infoSecurePort", 0),
         (int)(long)(Long)m.get("ipcPort"),
@@ -368,7 +403,8 @@ public class JsonUtil {
   }
 
   /** Convert an Object[] to a DatanodeInfo[]. */
-  private static DatanodeInfo[] toDatanodeInfoArray(final Object[] objects) {
+  private static DatanodeInfo[] toDatanodeInfoArray(final Object[] objects) 
+      throws IOException {
     if (objects == null) {
       return null;
     } else if (objects.length == 0) {

+ 61 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java

@@ -24,6 +24,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.util.Time;
@@ -61,7 +62,7 @@ public class TestJsonUtil {
   }
   
   @Test
-  public void testToDatanodeInfoWithoutSecurePort() {
+  public void testToDatanodeInfoWithoutSecurePort() throws Exception {
     Map<String, Object> response = new HashMap<String, Object>();
     
     response.put("ipAddr", "127.0.0.1");
@@ -84,4 +85,63 @@ public class TestJsonUtil {
     
     JsonUtil.toDatanodeInfo(response);
   }
+
+  @Test
+  public void testToDatanodeInfoWithName() throws Exception {
+    Map<String, Object> response = new HashMap<String, Object>();
+
+    // Older servers (1.x, 0.23, etc.) sends 'name' instead of ipAddr
+    // and xferPort.
+    String name = "127.0.0.1:1004";
+    response.put("name", name);
+    response.put("hostName", "localhost");
+    response.put("storageID", "fake-id");
+    response.put("infoPort", 1338l);
+    response.put("ipcPort", 1339l);
+    response.put("capacity", 1024l);
+    response.put("dfsUsed", 512l);
+    response.put("remaining", 512l);
+    response.put("blockPoolUsed", 512l);
+    response.put("lastUpdate", 0l);
+    response.put("xceiverCount", 4096l);
+    response.put("networkLocation", "foo.bar.baz");
+    response.put("adminState", "NORMAL");
+    response.put("cacheCapacity", 123l);
+    response.put("cacheUsed", 321l);
+
+    DatanodeInfo di = JsonUtil.toDatanodeInfo(response);
+    Assert.assertEquals(name, di.getXferAddr());
+
+    // The encoded result should contain name, ipAddr and xferPort.
+    Map<String, Object> r = JsonUtil.toJsonMap(di);
+    Assert.assertEquals(name, (String)r.get("name"));
+    Assert.assertEquals("127.0.0.1", (String)r.get("ipAddr"));
+    // In this test, it is Integer instead of Long since json was not actually
+    // involved in constructing the map.
+    Assert.assertEquals(1004, (int)(Integer)r.get("xferPort"));
+
+    // Invalid names
+    String[] badNames = {"127.0.0.1", "127.0.0.1:", ":", "127.0.0.1:sweet", ":123"};
+    for (String badName : badNames) {
+      response.put("name", badName);
+      checkDecodeFailure(response);
+    }
+
+    // Missing both name and ipAddr
+    response.remove("name");
+    checkDecodeFailure(response);
+
+    // Only missing xferPort
+    response.put("ipAddr", "127.0.0.1");
+    checkDecodeFailure(response);
+  }
+
+  private void checkDecodeFailure(Map<String, Object> map) {
+    try {
+      JsonUtil.toDatanodeInfo(map);
+      Assert.fail("Exception not thrown against bad input.");
+    } catch (Exception e) {
+      // expected
+    }
+  }
 }

+ 8 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java

@@ -182,9 +182,10 @@ public class TestMRApps {
     MRApps.setClasspath(environment, job.getConfiguration());
     assertTrue(environment.get("CLASSPATH").startsWith(
       ApplicationConstants.Environment.PWD.$() + File.pathSeparator));
-    String yarnAppClasspath = 
-        job.getConfiguration().get(
-            YarnConfiguration.YARN_APPLICATION_CLASSPATH);
+    String yarnAppClasspath = job.getConfiguration().get(
+        YarnConfiguration.YARN_APPLICATION_CLASSPATH,
+        StringUtils.join(",",
+            YarnConfiguration.DEFAULT_YARN_APPLICATION_CLASSPATH));
     if (yarnAppClasspath != null) {
       yarnAppClasspath = yarnAppClasspath.replaceAll(",\\s*", File.pathSeparator)
         .trim();
@@ -217,7 +218,10 @@ public class TestMRApps {
     MRApps.setClasspath(environment, conf);
     assertTrue(environment.get("CLASSPATH").startsWith(
       ApplicationConstants.Environment.PWD.$() + File.pathSeparator));
-    String confClasspath = job.getConfiguration().get(YarnConfiguration.YARN_APPLICATION_CLASSPATH);
+    String confClasspath = job.getConfiguration().get(
+        YarnConfiguration.YARN_APPLICATION_CLASSPATH,
+        StringUtils.join(",",
+            YarnConfiguration.DEFAULT_YARN_APPLICATION_CLASSPATH));
     if (confClasspath != null) {
       confClasspath = confClasspath.replaceAll(",\\s*", File.pathSeparator)
         .trim();

+ 5 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/ssl/TestEncryptedShuffle.java

@@ -31,6 +31,7 @@ import org.apache.hadoop.mapred.RunningJob;
 
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -86,8 +87,10 @@ public class TestEncryptedShuffle {
     conf.set("dfs.block.access.token.enable", "false");
     conf.set("dfs.permissions", "true");
     conf.set("hadoop.security.authentication", "simple");
-    String cp = conf.get(YarnConfiguration.YARN_APPLICATION_CLASSPATH) +
-      File.pathSeparator + classpathDir;
+    String cp = conf.get(YarnConfiguration.YARN_APPLICATION_CLASSPATH,
+        StringUtils.join(",",
+            YarnConfiguration.DEFAULT_YARN_APPLICATION_CLASSPATH))
+        + File.pathSeparator + classpathDir;
     conf.set(YarnConfiguration.YARN_APPLICATION_CLASSPATH, cp);
     dfsCluster = new MiniDFSCluster(conf, 1, true, null);
     FileSystem fileSystem = dfsCluster.getFileSystem();

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

@@ -205,6 +205,8 @@ Release 2.4.0 - UNRELEASED
 
     YARN-1568. Rename clusterid to clusterId in ActiveRMInfoProto (kasha)
 
+    YARN-1579. ActiveRMInfoProto fields should be optional (kasha)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -395,6 +397,9 @@ Release 2.3.0 - UNRELEASED
     YARN-1438. Ensure container diagnostics includes exception from container
     launch. (stevel via acmurthy)
 
+    YARN-1138. yarn.application.classpath is set to point to $HADOOP_CONF_DIR
+    etc., which does not work on Windows. (Chuan Liu via cnauroth)
+
 Release 2.2.0 - 2013-10-13
 
   INCOMPATIBLE CHANGES

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto

@@ -138,6 +138,6 @@ message RMStateVersionProto {
 ///////////// RM Failover related records ////////////////////////
 //////////////////////////////////////////////////////////////////
 message ActiveRMInfoProto {
-  required string clusterId = 1;
-  required string rmId = 2;
+  optional string clusterId = 1;
+  optional string rmId = 2;
 }

+ 23 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml

@@ -1016,10 +1016,29 @@
   <!-- Applications' Configuration-->
   
   <property>
-    <description>CLASSPATH for YARN applications. A comma-separated list
-    of CLASSPATH entries</description>
-     <name>yarn.application.classpath</name>
-     <value>$HADOOP_CONF_DIR,$HADOOP_COMMON_HOME/share/hadoop/common/*,$HADOOP_COMMON_HOME/share/hadoop/common/lib/*,$HADOOP_HDFS_HOME/share/hadoop/hdfs/*,$HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/*,$HADOOP_YARN_HOME/share/hadoop/yarn/*,$HADOOP_YARN_HOME/share/hadoop/yarn/lib/*</value>
+    <description>
+      CLASSPATH for YARN applications. A comma-separated list
+      of CLASSPATH entries. When this value is empty, the following default
+      CLASSPATH for YARN applications would be used. 
+      For Linux:
+      $HADOOP_CONF_DIR,
+      $HADOOP_COMMON_HOME/share/hadoop/common/*,
+      $HADOOP_COMMON_HOME/share/hadoop/common/lib/*,
+      $HADOOP_HDFS_HOME/share/hadoop/hdfs/*,
+      $HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/*,
+      $HADOOP_YARN_HOME/share/hadoop/yarn/*,
+      $HADOOP_YARN_HOME/share/hadoop/yarn/lib/*
+      For Windows:
+      %HADOOP_CONF_DIR%,
+      %HADOOP_COMMON_HOME%/share/hadoop/common/*,
+      %HADOOP_COMMON_HOME%/share/hadoop/common/lib/*,
+      %HADOOP_HDFS_HOME%/share/hadoop/hdfs/*,
+      %HADOOP_HDFS_HOME%/share/hadoop/hdfs/lib/*,
+      %HADOOP_YARN_HOME%/share/hadoop/yarn/*,
+      %HADOOP_YARN_HOME%/share/hadoop/yarn/lib/*
+    </description>
+    <name>yarn.application.classpath</name>
+    <value></value>
   </property>
 
   <!-- Other configuration -->