Bläddra i källkod

Merge branch 'trunk' into HDFS-7240

 Conflicts:
	hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/util/MBeans.java
	hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/util/DummyMXBean.java
Anu Engineer 7 år sedan
förälder
incheckning
7de7daa5bb
100 ändrade filer med 3911 tillägg och 1139 borttagningar
  1. 2 1
      BUILDING.txt
  2. 1 1
      dev-support/bin/create-release
  3. 21 22
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
  4. 0 137
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/DataOutputByteBuffer.java
  5. 10 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MultipleIOException.java
  6. 3 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
  7. 11 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/util/MBeans.java
  8. 80 10
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java
  9. 40 0
      hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
  10. 3 0
      hadoop-common-project/hadoop-common/src/site/markdown/GroupsMapping.md
  11. 241 0
      hadoop-common-project/hadoop-common/src/site/markdown/release/3.0.1/CHANGES.3.0.1.md
  12. 54 0
      hadoop-common-project/hadoop-common/src/site/markdown/release/3.0.1/RELEASENOTES.3.0.1.md
  13. 7 27
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
  14. 61 135
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestDataByteBuffers.java
  15. 1 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/util/DummyMXBean.java
  16. 31 7
      hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
  17. 1 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/pom.xml
  18. 1 1
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMetrics.java
  19. 65 3
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java
  20. 3 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MultipleDestinationMountTableResolver.java
  21. 179 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/order/AvailableSpaceResolver.java
  22. 2 1
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/order/DestinationOrder.java
  23. 34 113
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/order/LocalResolver.java
  24. 160 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/order/RouterResolver.java
  25. 13 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
  26. 4 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterHttpServer.java
  27. 9 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterMetricsService.java
  28. 15 26
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
  29. 655 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterWebHdfsMethods.java
  30. 2 1
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MountTable.java
  31. 4 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MountTablePBImpl.java
  32. 1 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/proto/FederationProtocol.proto
  33. 21 21
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/webapps/router/federationhealth.html
  34. 23 12
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/webapps/router/federationhealth.js
  35. 138 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/webapps/static/rbf.css
  36. 37 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/xsl/configuration.xsl
  37. 129 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/web/RouterWebHDFSContract.java
  38. 45 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/web/TestRouterWebHDFSContractAppend.java
  39. 52 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/web/TestRouterWebHDFSContractConcat.java
  40. 49 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/web/TestRouterWebHDFSContractCreate.java
  41. 49 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/web/TestRouterWebHDFSContractDelete.java
  42. 48 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/web/TestRouterWebHDFSContractMkdir.java
  43. 63 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/web/TestRouterWebHDFSContractOpen.java
  44. 49 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/web/TestRouterWebHDFSContractRename.java
  45. 64 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/web/TestRouterWebHDFSContractRootDirectory.java
  46. 63 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/web/TestRouterWebHDFSContractSeek.java
  47. 22 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/web/package-info.java
  48. 5 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MiniRouterDFSCluster.java
  49. 232 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/order/TestAvailableSpaceResolver.java
  50. 17 5
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestConnectionManager.java
  51. 9 1
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAllResolver.java
  52. 39 2
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java
  53. 26 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/resources/contract/webhdfs.xml
  54. 11 0
      hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_3.0.1.xml
  55. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclStorage.java
  56. 66 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
  57. 9 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
  58. 5 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
  59. 25 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
  60. 24 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
  61. 15 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
  62. 4 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/XAttrStorage.java
  63. 52 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
  64. 29 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
  65. 15 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/TestRefreshCallQueue.java
  66. 13 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
  67. 6 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalDistributedCacheManager.java
  68. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalJobRunner.java
  69. 71 24
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapred/TestLocalDistributedCacheManager.java
  70. 13 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestTaskID.java
  71. 1 1
      hadoop-project-dist/pom.xml
  72. 1 1
      hadoop-project/pom.xml
  73. 1 2
      hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunCredentialsProvider.java
  74. 206 0
      hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSBlockOutputStream.java
  75. 24 10
      hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
  76. 90 83
      hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java
  77. 0 111
      hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSOutputStream.java
  78. 87 28
      hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
  79. 15 7
      hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
  80. 28 4
      hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSBlockOutputStream.java
  81. 5 5
      hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSInputStream.java
  82. 1 1
      hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractDistCp.java
  83. 1 1
      hadoop-tools/hadoop-azure-datalake/pom.xml
  84. 0 40
      hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
  85. 0 46
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
  86. 0 141
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemBlockLocations.java
  87. 4 4
      hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/state/StatePool.java
  88. 2 1
      hadoop-tools/hadoop-rumen/src/test/java/org/apache/hadoop/tools/rumen/TestHistograms.java
  89. 2 0
      hadoop-tools/hadoop-sls/src/main/bin/slsrun.sh
  90. 5 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java
  91. 23 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  92. 2 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
  93. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
  94. 15 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
  95. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/DistributedShellTimelinePlugin.java
  96. 9 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
  97. 18 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/ApiServiceClient.java
  98. 24 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
  99. 79 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Simplified-V1-API-Layer-For-Services.yaml
  100. 12 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ClientAMProtocol.java

+ 2 - 1
BUILDING.txt

@@ -9,6 +9,7 @@ Requirements:
 * ProtocolBuffer 2.5.0
 * CMake 3.1 or newer (if compiling native code)
 * Zlib devel (if compiling native code)
+* Cyrus SASL devel (if compiling native code)
 * openssl devel (if compiling native hadoop-pipes and to get the best HDFS encryption performance)
 * Linux FUSE (Filesystem in Userspace) version 2.6 or above (if compiling fuse_dfs)
 * Jansson C XML parsing library ( if compiling libwebhdfs )
@@ -67,7 +68,7 @@ Installing required packages for clean install of Ubuntu 14.04 LTS Desktop:
 * Maven
   $ sudo apt-get -y install maven
 * Native libraries
-  $ sudo apt-get -y install build-essential autoconf automake libtool cmake zlib1g-dev pkg-config libssl-dev
+  $ sudo apt-get -y install build-essential autoconf automake libtool cmake zlib1g-dev pkg-config libssl-dev libsasl2-dev
 * ProtocolBuffer 2.5.0 (required)
   $ sudo apt-get -y install protobuf-compiler
 

+ 1 - 1
dev-support/bin/create-release

@@ -389,7 +389,7 @@ function option_parse
     hadoop_error "ERROR: Cannot set --mvncache and --dockercache simultaneously."
     exit 1
   else
-    MVNCACHE=${MVNCACHE:-"${HOME}/.m2"}
+    MVNCACHE=${MVNCACHE:-"${HOME}/.m2/repository"}
   fi
 
   if [[ "${ASFRELEASE}" = true ]]; then

+ 21 - 22
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java

@@ -229,7 +229,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
   private static final Logger LOG_DEPRECATION =
       LoggerFactory.getLogger(
           "org.apache.hadoop.conf.Configuration.deprecation");
-  private static final Set<String> TAGS = new HashSet<>();
+  private static final Set<String> TAGS = ConcurrentHashMap.newKeySet();
 
   private boolean quietmode = true;
 
@@ -2335,7 +2335,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
    * @return password or null if not found
    * @throws IOException
    */
-  protected char[] getPasswordFromCredentialProviders(String name)
+  public char[] getPasswordFromCredentialProviders(String name)
       throws IOException {
     char[] pass = null;
     try {
@@ -2935,7 +2935,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
         resources.set(i, ret);
       }
     }
-    this.removeUndeclaredTags(properties);
+    this.addTags(properties);
   }
   
   private Resource loadResource(Properties properties,
@@ -3183,29 +3183,28 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
   }
 
   /**
-   * Removes undeclared tags and related properties from propertyTagsMap.
-   * Its required because ordering of properties in xml config files is not
-   * guaranteed.
+   * Add tags defined in HADOOP_SYSTEM_TAGS, HADOOP_CUSTOM_TAGS.
    * @param prop
    */
-  private void removeUndeclaredTags(Properties prop) {
+  public void addTags(Properties prop) {
     // Get all system tags
-    if (prop.containsKey(CommonConfigurationKeys.HADOOP_SYSTEM_TAGS)){
-      String systemTags = prop.getProperty(CommonConfigurationKeys
-              .HADOOP_SYSTEM_TAGS);
-      Arrays.stream(systemTags.split(",")).forEach(tag -> TAGS.add(tag));
-    }
-    // Get all custom tags
-    if (prop.containsKey(CommonConfigurationKeys.HADOOP_CUSTOM_TAGS)) {
-      String customTags = prop.getProperty(CommonConfigurationKeys
-          .HADOOP_CUSTOM_TAGS);
-      Arrays.stream(customTags.split(",")).forEach(tag -> TAGS.add(tag));
-    }
+    try {
+      if (prop.containsKey(CommonConfigurationKeys.HADOOP_SYSTEM_TAGS)) {
+        String systemTags = prop.getProperty(CommonConfigurationKeys
+            .HADOOP_SYSTEM_TAGS);
+        Arrays.stream(systemTags.split(",")).forEach(tag -> TAGS.add(tag));
+      }
+      // Get all custom tags
+      if (prop.containsKey(CommonConfigurationKeys.HADOOP_CUSTOM_TAGS)) {
+        String customTags = prop.getProperty(CommonConfigurationKeys
+            .HADOOP_CUSTOM_TAGS);
+        Arrays.stream(customTags.split(",")).forEach(tag -> TAGS.add(tag));
+      }
 
-    Set undeclaredTags = propertyTagsMap.keySet();
-    if (undeclaredTags.retainAll(TAGS)) {
-      LOG.info("Removed undeclared tags:");
+    } catch (Exception ex) {
+      LOG.trace("Error adding tags in configuration", ex);
     }
+
   }
 
   /**
@@ -3219,8 +3218,8 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
   private void readTagFromConfig(String attributeValue, String confName, String
       confValue, List<String> confSource) {
     for (String tagStr : attributeValue.split(",")) {
-      tagStr = tagStr.trim();
       try {
+        tagStr = tagStr.trim();
         // Handle property with no/null value
         if (confValue == null) {
           confValue = "";

+ 0 - 137
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/DataOutputByteBuffer.java

@@ -1,137 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.io;
-
-import java.io.DataOutputStream;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-import java.util.List;
-import java.util.ArrayList;
-import java.util.LinkedList;
-
-public class DataOutputByteBuffer extends DataOutputStream {
-
-   static class Buffer extends OutputStream {
-
-    final byte[] b = new byte[1];
-    final boolean direct;
-    final List<ByteBuffer> active = new ArrayList<ByteBuffer>();
-    final List<ByteBuffer> inactive = new LinkedList<ByteBuffer>();
-    int size;
-    int length;
-    ByteBuffer current;
-
-    Buffer(int size, boolean direct) {
-      this.direct = direct;
-      this.size = size;
-      current = direct
-          ? ByteBuffer.allocateDirect(size)
-          : ByteBuffer.allocate(size);
-    }
-    @Override
-    public void write(int b) {
-      this.b[0] = (byte)(b & 0xFF);
-      write(this.b);
-    }
-    @Override
-    public void write(byte[] b) {
-      write(b, 0, b.length);
-    }
-    @Override
-    public void write(byte[] b, int off, int len) {
-      int rem = current.remaining();
-      while (len > rem) {
-        current.put(b, off, rem);
-        length += rem;
-        current.flip();
-        active.add(current);
-        off += rem;
-        len -= rem;
-        rem = getBuffer(len);
-      }
-      current.put(b, off, len);
-      length += len;
-    }
-    int getBuffer(int newsize) {
-      if (inactive.isEmpty()) {
-        size = Math.max(size << 1, newsize);
-        current = direct
-            ? ByteBuffer.allocateDirect(size)
-            : ByteBuffer.allocate(size);
-      } else {
-        current = inactive.remove(0);
-      }
-      return current.remaining();
-    }
-    ByteBuffer[] getData() {
-      ByteBuffer[] ret = active.toArray(new ByteBuffer[active.size() + 1]);
-      ByteBuffer tmp = current.duplicate();
-      tmp.flip();
-      ret[ret.length - 1] = tmp.slice();
-      return ret;
-    }
-    int getLength() {
-      return length;
-    }
-    void reset() {
-      length = 0;
-      current.rewind();
-      inactive.add(0, current);
-      for (int i = active.size() - 1; i >= 0; --i) {
-        ByteBuffer b = active.remove(i);
-        b.rewind();
-        inactive.add(0, b);
-      }
-      current = inactive.remove(0);
-    }
-  }
-
-  private final Buffer buffers;
-
-  public DataOutputByteBuffer() {
-    this(32);
-  }
-
-  public DataOutputByteBuffer(int size) {
-    this(size, false);
-  }
-
-  public DataOutputByteBuffer(int size, boolean direct) {
-    this(new Buffer(size, direct));
-  }
-
-  private DataOutputByteBuffer(Buffer buffers) {
-    super(buffers);
-    this.buffers = buffers;
-  }
-
-  public ByteBuffer[] getData() {
-    return buffers.getData();
-  }
-
-  public int getLength() {
-    return buffers.getLength();
-  }
-
-  public void reset() {
-    this.written = 0;
-    buffers.reset();
-  }
-
-}

+ 10 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MultipleIOException.java

@@ -76,5 +76,15 @@ public class MultipleIOException extends IOException {
     public IOException build() {
       return createIOException(exceptions);
     }
+
+    /**
+     * @return whether any exception was added.
+     */
+    public boolean isEmpty() {
+      if (exceptions == null) {
+        return true;
+      }
+      return exceptions.isEmpty();
+    }
   }
 }

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

@@ -661,6 +661,9 @@ public abstract class Server {
   public synchronized void refreshCallQueue(Configuration conf) {
     // Create the next queue
     String prefix = getQueueClassPrefix();
+    this.maxQueueSize = handlerCount * conf.getInt(
+        CommonConfigurationKeys.IPC_SERVER_HANDLER_QUEUE_SIZE_KEY,
+        CommonConfigurationKeys.IPC_SERVER_HANDLER_QUEUE_SIZE_DEFAULT);
     callQueue.swapQueue(getSchedulerClass(prefix, conf),
         getQueueClass(prefix, conf), maxQueueSize, prefix, conf);
   }

+ 11 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/util/MBeans.java

@@ -32,17 +32,19 @@ import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+
+import com.google.common.base.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
  * This util class provides a method to register an MBean using
  * our standard naming convention as described in the doc
- *  for {link {@link #register(String, String, Object)}
+ *  for {link {@link #register(String, String, Object)}.
  */
 @InterfaceAudience.Public
 @InterfaceStability.Stable
-public class MBeans {
+public final class MBeans {
   private static final Logger LOG = LoggerFactory.getLogger(MBeans.class);
   private static final String DOMAIN_PREFIX = "Hadoop:";
   private static final String SERVICE_PREFIX = "service=";
@@ -52,10 +54,13 @@ public class MBeans {
       "^" + DOMAIN_PREFIX + SERVICE_PREFIX + "([^,]+)," +
       NAME_PREFIX + "(.+)$");
 
+  private MBeans() {
+  }
+
   /**
    * Register the MBean using our standard MBeanName format
    * "hadoop:service=<serviceName>,name=<nameName>"
-   * Where the <serviceName> and <nameName> are the supplied parameters
+   * Where the <serviceName> and <nameName> are the supplied parameters.
    *
    * @param serviceName
    * @param nameName
@@ -84,6 +89,9 @@ public class MBeans {
                                     Map<String, String> properties,
                                     Object theMbean) {
     final MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+    Preconditions.checkNotNull(properties,
+        "JMX bean properties should not be null for "
+            + "bean registration.");
     ObjectName name = getMBeanName(serviceName, nameName, properties);
     if (name != null) {
       try {

+ 80 - 10
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java

@@ -109,6 +109,27 @@ public class LdapGroupsMapping
   public static final String LDAP_KEYSTORE_PASSWORD_FILE_KEY = LDAP_KEYSTORE_PASSWORD_KEY + ".file";
   public static final String LDAP_KEYSTORE_PASSWORD_FILE_DEFAULT = "";
 
+
+  /**
+   * File path to the location of the SSL truststore to use
+   */
+  public static final String LDAP_TRUSTSTORE_KEY = LDAP_CONFIG_PREFIX +
+      ".ssl.truststore";
+
+  /**
+   * The key of the credential entry containing the password for
+   * the LDAP SSL truststore
+   */
+  public static final String LDAP_TRUSTSTORE_PASSWORD_KEY =
+      LDAP_CONFIG_PREFIX +".ssl.truststore.password";
+
+  /**
+   * The path to a file containing the password for
+   * the LDAP SSL truststore
+   */
+  public static final String LDAP_TRUSTSTORE_PASSWORD_FILE_KEY =
+      LDAP_TRUSTSTORE_PASSWORD_KEY + ".file";
+
   /*
    * User to bind to the LDAP server with
    */
@@ -226,6 +247,8 @@ public class LdapGroupsMapping
   private boolean useSsl;
   private String keystore;
   private String keystorePass;
+  private String truststore;
+  private String truststorePass;
   private String bindUser;
   private String bindPassword;
   private String userbaseDN;
@@ -526,8 +549,19 @@ public class LdapGroupsMapping
       // Set up SSL security, if necessary
       if (useSsl) {
         env.put(Context.SECURITY_PROTOCOL, "ssl");
-        System.setProperty("javax.net.ssl.keyStore", keystore);
-        System.setProperty("javax.net.ssl.keyStorePassword", keystorePass);
+        if (!keystore.isEmpty()) {
+          System.setProperty("javax.net.ssl.keyStore", keystore);
+        }
+        if (!keystorePass.isEmpty()) {
+          System.setProperty("javax.net.ssl.keyStorePassword", keystorePass);
+        }
+        if (!truststore.isEmpty()) {
+          System.setProperty("javax.net.ssl.trustStore", truststore);
+        }
+        if (!truststorePass.isEmpty()) {
+          System.setProperty("javax.net.ssl.trustStorePassword",
+              truststorePass);
+        }
       }
 
       env.put(Context.SECURITY_PRINCIPAL, bindUser);
@@ -572,15 +606,10 @@ public class LdapGroupsMapping
     if (ldapUrl == null || ldapUrl.isEmpty()) {
       throw new RuntimeException("LDAP URL is not configured");
     }
-    
+
     useSsl = conf.getBoolean(LDAP_USE_SSL_KEY, LDAP_USE_SSL_DEFAULT);
-    keystore = conf.get(LDAP_KEYSTORE_KEY, LDAP_KEYSTORE_DEFAULT);
-    
-    keystorePass = getPassword(conf, LDAP_KEYSTORE_PASSWORD_KEY,
-        LDAP_KEYSTORE_PASSWORD_DEFAULT);
-    if (keystorePass.isEmpty()) {
-      keystorePass = extractPassword(conf.get(LDAP_KEYSTORE_PASSWORD_FILE_KEY,
-          LDAP_KEYSTORE_PASSWORD_FILE_DEFAULT));
+    if (useSsl) {
+      loadSslConf(conf);
     }
     
     bindUser = conf.get(BIND_USER_KEY, BIND_USER_DEFAULT);
@@ -643,6 +672,47 @@ public class LdapGroupsMapping
     this.conf = conf;
   }
 
+  private void loadSslConf(Configuration sslConf) {
+    keystore = sslConf.get(LDAP_KEYSTORE_KEY, LDAP_KEYSTORE_DEFAULT);
+    keystorePass = getPassword(sslConf, LDAP_KEYSTORE_PASSWORD_KEY,
+        LDAP_KEYSTORE_PASSWORD_DEFAULT);
+    if (keystorePass.isEmpty()) {
+      keystorePass = extractPassword(sslConf.get(
+          LDAP_KEYSTORE_PASSWORD_FILE_KEY,
+          LDAP_KEYSTORE_PASSWORD_FILE_DEFAULT));
+    }
+
+    truststore = sslConf.get(LDAP_TRUSTSTORE_KEY, "");
+    truststorePass = getPasswordFromCredentialProviders(
+        sslConf, LDAP_TRUSTSTORE_PASSWORD_KEY, "");
+    if (truststorePass.isEmpty()) {
+      truststorePass = extractPassword(
+          sslConf.get(LDAP_TRUSTSTORE_PASSWORD_FILE_KEY, ""));
+    }
+  }
+
+  String getPasswordFromCredentialProviders(
+      Configuration conf, String alias, String defaultPass) {
+    String password = defaultPass;
+    try {
+      char[] passchars = conf.getPasswordFromCredentialProviders(alias);
+      if (passchars != null) {
+        password = new String(passchars);
+      }
+    } catch (IOException ioe) {
+      LOG.warn("Exception while trying to get password for alias {}: {}",
+          alias, ioe);
+    }
+    return password;
+  }
+
+  /**
+   * Passwords should not be stored in configuration. Use
+   * {@link #getPasswordFromCredentialProviders(
+   *            Configuration, String, String)}
+   * to avoid reading passwords from a configuration file.
+   */
+  @Deprecated
   String getPassword(Configuration conf, String alias, String defaultPass) {
     String password = defaultPass;
     try {

+ 40 - 0
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml

@@ -316,6 +316,27 @@
   </description>
 </property>
 
+<property>
+  <name>hadoop.security.group.mapping.ldap.ssl.truststore</name>
+  <value></value>
+  <description>
+    File path to the SSL truststore that contains the root certificate used to
+    sign the LDAP server's certificate. Specify this if the LDAP server's
+    certificate is not signed by a well known certificate authority.
+  </description>
+</property>
+
+<property>
+  <name>hadoop.security.group.mapping.ldap.ssl.truststore.password.file</name>
+  <value></value>
+  <description>
+    The path to a file containing the password of the LDAP SSL truststore.
+
+    IMPORTANT: This file should be readable only by the Unix user running
+    the daemons.
+  </description>
+</property>
+
 <property>
   <name>hadoop.security.group.mapping.ldap.bind.user</name>
   <value></value>
@@ -2507,6 +2528,25 @@
   </description>
 </property>
 
+<property>
+  <name>hadoop.security.key.default.bitlength</name>
+  <value>128</value>
+  <description>
+    The length (bits) of keys we want the KeyProvider to produce. Key length
+    defines the upper-bound on an algorithm's security, ideally, it would
+    coincide with the lower-bound on an algorithm's security.
+  </description>
+</property>
+
+<property>
+  <name>hadoop.security.key.default.cipher</name>
+  <value>AES/CTR/NoPadding</value>
+  <description>
+    This indicates the algorithm that be used by KeyProvider for generating
+    key, and will be converted to CipherSuite when creating encryption zone.
+  </description>
+</property>
+
 <property>
   <name>fs.har.impl.disable.cache</name>
   <value>true</value>

+ 3 - 0
hadoop-common-project/hadoop-common/src/site/markdown/GroupsMapping.md

@@ -112,6 +112,9 @@ For some LDAP servers, such as Active Directory, the user object returned in the
 Therefore, it is possible to infer the user's groups from the first query without sending the second one, and it may reduce group name resolution latency incurred by the second query. If it fails to get group names, it will fall back to the typical two-query scenario and send the second query to get group names.
 To enable this feature, set `hadoop.security.group.mapping.ldap.search.attr.memberof` to `memberOf`, and Hadoop will resolve group names using this attribute in the user object.
 
+If the LDAP server's certificate is not signed by a well known certificate authority, specify the path to the truststore in `hadoop.security.group.mapping.ldap.ssl.truststore`.
+Similar to keystore, specify the truststore password file in `hadoop.security.group.mapping.ldap.ssl.truststore.password.file`.
+
 Composite Groups Mapping
 --------
 `CompositeGroupsMapping` works by enumerating a list of service providers in `hadoop.security.group.mapping.providers`.

+ 241 - 0
hadoop-common-project/hadoop-common/src/site/markdown/release/3.0.1/CHANGES.3.0.1.md

@@ -0,0 +1,241 @@
+
+<!---
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+-->
+# Apache Hadoop Changelog
+
+## Release 3.0.1 - 2018-03-16
+
+### INCOMPATIBLE CHANGES:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [HDFS-12990](https://issues.apache.org/jira/browse/HDFS-12990) | Change default NameNode RPC port back to 8020 |  Critical | namenode | Xiao Chen | Xiao Chen |
+
+
+### IMPORTANT ISSUES:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [HDFS-13083](https://issues.apache.org/jira/browse/HDFS-13083) | RBF: Fix doc error setting up client |  Major | federation | tartarus | tartarus |
+
+
+### IMPROVEMENTS:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [HADOOP-14872](https://issues.apache.org/jira/browse/HADOOP-14872) | CryptoInputStream should implement unbuffer |  Major | fs, security | John Zhuge | John Zhuge |
+| [YARN-7414](https://issues.apache.org/jira/browse/YARN-7414) | FairScheduler#getAppWeight() should be moved into FSAppAttempt#getWeight() |  Minor | fairscheduler | Daniel Templeton | Soumabrata Chakraborty |
+| [HADOOP-15023](https://issues.apache.org/jira/browse/HADOOP-15023) | ValueQueue should also validate (lowWatermark \* numValues) \> 0 on construction |  Minor | . | Xiao Chen | Xiao Chen |
+| [HDFS-12814](https://issues.apache.org/jira/browse/HDFS-12814) | Add blockId when warning slow mirror/disk in BlockReceiver |  Trivial | hdfs | Jiandan Yang | Jiandan Yang |
+| [YARN-7524](https://issues.apache.org/jira/browse/YARN-7524) | Remove unused FairSchedulerEventLog |  Major | fairscheduler | Wilfred Spiegelenburg | Wilfred Spiegelenburg |
+| [YARN-7495](https://issues.apache.org/jira/browse/YARN-7495) | Improve robustness of the AggregatedLogDeletionService |  Major | log-aggregation | Jonathan Eagles | Jonathan Eagles |
+| [YARN-7611](https://issues.apache.org/jira/browse/YARN-7611) | Node manager web UI should display container type in containers page |  Major | nodemanager, webapp | Weiwei Yang | Weiwei Yang |
+| [YARN-6483](https://issues.apache.org/jira/browse/YARN-6483) | Add nodes transitioning to DECOMMISSIONING state to the list of updated nodes returned to the AM |  Major | resourcemanager | Juan Rodríguez Hortalá | Juan Rodríguez Hortalá |
+| [HADOOP-15056](https://issues.apache.org/jira/browse/HADOOP-15056) | Fix TestUnbuffer#testUnbufferException failure |  Minor | test | Jack Bearden | Jack Bearden |
+| [HADOOP-15012](https://issues.apache.org/jira/browse/HADOOP-15012) | Add readahead, dropbehind, and unbuffer to StreamCapabilities |  Major | fs | John Zhuge | John Zhuge |
+| [HADOOP-15104](https://issues.apache.org/jira/browse/HADOOP-15104) | AliyunOSS: change the default value of max error retry |  Major | fs/oss | wujinhu | wujinhu |
+| [HDFS-12910](https://issues.apache.org/jira/browse/HDFS-12910) | Secure Datanode Starter should log the port when it fails to bind |  Minor | datanode | Stephen O'Donnell | Stephen O'Donnell |
+| [HDFS-12819](https://issues.apache.org/jira/browse/HDFS-12819) | Setting/Unsetting EC policy shows warning if the directory is not empty |  Minor | erasure-coding | Lei (Eddy) Xu | Lei (Eddy) Xu |
+| [HDFS-12927](https://issues.apache.org/jira/browse/HDFS-12927) | Update erasure coding doc to address unsupported APIs |  Major | erasure-coding | Lei (Eddy) Xu | Lei (Eddy) Xu |
+| [YARN-5418](https://issues.apache.org/jira/browse/YARN-5418) | When partial log aggregation is enabled, display the list of aggregated files on the container log page |  Major | . | Siddharth Seth | Xuan Gong |
+| [HDFS-12818](https://issues.apache.org/jira/browse/HDFS-12818) | Support multiple storages in DataNodeCluster / SimulatedFSDataset |  Minor | datanode, test | Erik Krogen | Erik Krogen |
+| [HDFS-9023](https://issues.apache.org/jira/browse/HDFS-9023) | When NN is not able to identify DN for replication, reason behind it can be logged |  Critical | hdfs-client, namenode | Surendra Singh Lilhore | Xiao Chen |
+| [HDFS-11847](https://issues.apache.org/jira/browse/HDFS-11847) | Enhance dfsadmin listOpenFiles command to list files blocking datanode decommissioning |  Major | hdfs | Manoj Govindassamy | Manoj Govindassamy |
+| [YARN-7678](https://issues.apache.org/jira/browse/YARN-7678) | Ability to enable logging of container memory stats |  Major | nodemanager | Jim Brennan | Jim Brennan |
+| [HDFS-11848](https://issues.apache.org/jira/browse/HDFS-11848) | Enhance dfsadmin listOpenFiles command to list files under a given path |  Major | . | Manoj Govindassamy | Yiqun Lin |
+| [HDFS-12945](https://issues.apache.org/jira/browse/HDFS-12945) | Switch to ClientProtocol instead of NamenodeProtocols in NamenodeWebHdfsMethods |  Minor | . | Wei Yan | Wei Yan |
+| [YARN-7590](https://issues.apache.org/jira/browse/YARN-7590) | Improve container-executor validation check |  Major | security, yarn | Eric Yang | Eric Yang |
+| [MAPREDUCE-6984](https://issues.apache.org/jira/browse/MAPREDUCE-6984) | MR AM to clean up temporary files from previous attempt in case of no recovery |  Major | applicationmaster | Gergo Repas | Gergo Repas |
+| [HADOOP-15185](https://issues.apache.org/jira/browse/HADOOP-15185) | Update adls connector to use the current version of ADLS SDK |  Major | fs/adl | Atul Sikaria | Atul Sikaria |
+| [HADOOP-15189](https://issues.apache.org/jira/browse/HADOOP-15189) | backport HADOOP-15039 to branch-2 and branch-3 |  Blocker | . | Genmao Yu | Genmao Yu |
+| [HADOOP-15186](https://issues.apache.org/jira/browse/HADOOP-15186) | Allow Azure Data Lake SDK dependency version to be set on the command line |  Major | build, fs/adl | Vishwajeet Dusane | Vishwajeet Dusane |
+| [HDFS-13092](https://issues.apache.org/jira/browse/HDFS-13092) | Reduce verbosity for ThrottledAsyncChecker.java:schedule |  Minor | datanode | Mukul Kumar Singh | Mukul Kumar Singh |
+| [HDFS-13062](https://issues.apache.org/jira/browse/HDFS-13062) | Provide support for JN to use separate journal disk per namespace |  Major | . | Bharat Viswanadham | Bharat Viswanadham |
+| [HADOOP-15212](https://issues.apache.org/jira/browse/HADOOP-15212) | Add independent secret manager method for logging expired tokens |  Major | security | Daryn Sharp | Daryn Sharp |
+| [YARN-7728](https://issues.apache.org/jira/browse/YARN-7728) | Expose container preemptions related information in Capacity Scheduler queue metrics |  Major | . | Eric Payne | Eric Payne |
+| [MAPREDUCE-7048](https://issues.apache.org/jira/browse/MAPREDUCE-7048) | Uber AM can crash due to unknown task in statusUpdate |  Major | mr-am | Peter Bacsko | Peter Bacsko |
+| [HADOOP-15204](https://issues.apache.org/jira/browse/HADOOP-15204) | Add Configuration API for parsing storage sizes |  Minor | conf | Anu Engineer | Anu Engineer |
+
+
+### BUG FIXES:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [YARN-7361](https://issues.apache.org/jira/browse/YARN-7361) | Improve the docker container runtime documentation |  Major | . | Shane Kumpf | Shane Kumpf |
+| [YARN-7489](https://issues.apache.org/jira/browse/YARN-7489) | ConcurrentModificationException in RMAppImpl#getRMAppMetrics |  Major | capacityscheduler | Tao Yang | Tao Yang |
+| [YARN-7525](https://issues.apache.org/jira/browse/YARN-7525) | Incorrect query parameters in cluster nodes REST API document |  Minor | documentation | Tao Yang | Tao Yang |
+| [HADOOP-15046](https://issues.apache.org/jira/browse/HADOOP-15046) | Document Apache Hadoop does not support Java 9 in BUILDING.txt |  Major | documentation | Akira Ajisaka | Hanisha Koneru |
+| [YARN-7531](https://issues.apache.org/jira/browse/YARN-7531) | ResourceRequest.equal does not check ExecutionTypeRequest.enforceExecutionType() |  Major | api | Haibo Chen | Haibo Chen |
+| [YARN-7513](https://issues.apache.org/jira/browse/YARN-7513) | Remove scheduler lock in FSAppAttempt.getWeight() |  Minor | fairscheduler | Wilfred Spiegelenburg | Wilfred Spiegelenburg |
+| [YARN-7390](https://issues.apache.org/jira/browse/YARN-7390) | All reservation related test cases failed when TestYarnClient runs against Fair Scheduler. |  Major | fairscheduler, reservation system | Yufei Gu | Yufei Gu |
+| [YARN-7363](https://issues.apache.org/jira/browse/YARN-7363) | ContainerLocalizer doesn't have a valid log4j config when using LinuxContainerExecutor |  Major | nodemanager | Yufei Gu | Yufei Gu |
+| [HDFS-12832](https://issues.apache.org/jira/browse/HDFS-12832) | INode.getFullPathName may throw ArrayIndexOutOfBoundsException lead to NameNode exit |  Critical | namenode | DENG FEI | Konstantin Shvachko |
+| [HADOOP-15042](https://issues.apache.org/jira/browse/HADOOP-15042) | Azure PageBlobInputStream.skip() can return negative value when numberOfPagesRemaining is 0 |  Minor | fs/azure | Rajesh Balamohan | Rajesh Balamohan |
+| [YARN-7558](https://issues.apache.org/jira/browse/YARN-7558) | "yarn logs" command fails to get logs for running containers if UI authentication is enabled. |  Critical | . | Namit Maheshwari | Xuan Gong |
+| [HDFS-12638](https://issues.apache.org/jira/browse/HDFS-12638) | Delete copy-on-truncate block along with the original block, when deleting a file being truncated |  Blocker | hdfs | Jiandan Yang | Konstantin Shvachko |
+| [HDFS-12836](https://issues.apache.org/jira/browse/HDFS-12836) | startTxId could be greater than endTxId when tailing in-progress edit log |  Major | hdfs | Chao Sun | Chao Sun |
+| [MAPREDUCE-5124](https://issues.apache.org/jira/browse/MAPREDUCE-5124) | AM lacks flow control for task events |  Major | mr-am | Jason Lowe | Peter Bacsko |
+| [YARN-7589](https://issues.apache.org/jira/browse/YARN-7589) | TestPBImplRecords fails with NullPointerException |  Major | . | Jason Lowe | Daniel Templeton |
+| [YARN-7455](https://issues.apache.org/jira/browse/YARN-7455) | quote\_and\_append\_arg can overflow buffer |  Major | nodemanager | Jason Lowe | Jim Brennan |
+| [HADOOP-15058](https://issues.apache.org/jira/browse/HADOOP-15058) | create-release site build outputs dummy shaded jars due to skipShade |  Blocker | . | Andrew Wang | Andrew Wang |
+| [HADOOP-14985](https://issues.apache.org/jira/browse/HADOOP-14985) | Remove subversion related code from VersionInfoMojo.java |  Minor | build | Akira Ajisaka | Ajay Kumar |
+| [HADOOP-15098](https://issues.apache.org/jira/browse/HADOOP-15098) | TestClusterTopology#testChooseRandom fails intermittently |  Major | test | Zsolt Venczel | Zsolt Venczel |
+| [HDFS-12891](https://issues.apache.org/jira/browse/HDFS-12891) | Do not invalidate blocks if toInvalidate is empty |  Major | . | Zsolt Venczel | Zsolt Venczel |
+| [YARN-7647](https://issues.apache.org/jira/browse/YARN-7647) | NM print inappropriate error log when node-labels is enabled |  Minor | . | Yang Wang | Yang Wang |
+| [HDFS-12907](https://issues.apache.org/jira/browse/HDFS-12907) | Allow read-only access to reserved raw for non-superusers |  Major | namenode | Daryn Sharp | Rushabh S Shah |
+| [HDFS-12881](https://issues.apache.org/jira/browse/HDFS-12881) | Output streams closed with IOUtils suppressing write errors |  Major | . | Jason Lowe | Ajay Kumar |
+| [YARN-7595](https://issues.apache.org/jira/browse/YARN-7595) | Container launching code suppresses close exceptions after writes |  Major | nodemanager | Jason Lowe | Jim Brennan |
+| [HADOOP-15085](https://issues.apache.org/jira/browse/HADOOP-15085) | Output streams closed with IOUtils suppressing write errors |  Major | . | Jason Lowe | Jim Brennan |
+| [YARN-7629](https://issues.apache.org/jira/browse/YARN-7629) | TestContainerLaunch# fails after YARN-7381 |  Major | . | Jason Lowe | Jason Lowe |
+| [YARN-7664](https://issues.apache.org/jira/browse/YARN-7664) | Several javadoc errors |  Blocker | . | Sean Mackrory | Sean Mackrory |
+| [HADOOP-15123](https://issues.apache.org/jira/browse/HADOOP-15123) | KDiag tries to load krb5.conf from KRB5CCNAME instead of KRB5\_CONFIG |  Minor | security | Vipin Rathor | Vipin Rathor |
+| [YARN-7661](https://issues.apache.org/jira/browse/YARN-7661) | NodeManager metrics return wrong value after update node resource |  Major | . | Yang Wang | Yang Wang |
+| [HDFS-12347](https://issues.apache.org/jira/browse/HDFS-12347) | TestBalancerRPCDelay#testBalancerRPCDelay fails very frequently |  Critical | test | Xiao Chen | Bharat Viswanadham |
+| [HDFS-12930](https://issues.apache.org/jira/browse/HDFS-12930) | Remove the extra space in HdfsImageViewer.md |  Trivial | documentation | Yiqun Lin | Rahul Pathak |
+| [YARN-7662](https://issues.apache.org/jira/browse/YARN-7662) | [Atsv2] Define new set of configurations for reader and collectors to bind. |  Major | . | Rohith Sharma K S | Rohith Sharma K S |
+| [HDFS-12845](https://issues.apache.org/jira/browse/HDFS-12845) | JournalNode Test failures |  Major | journal-node | Bharat Viswanadham | Bharat Viswanadham |
+| [YARN-7466](https://issues.apache.org/jira/browse/YARN-7466) | ResourceRequest has a different default for allocationRequestId than Container |  Major | . | Chandni Singh | Chandni Singh |
+| [YARN-7674](https://issues.apache.org/jira/browse/YARN-7674) | Update Timeline Reader web app address in UI2 |  Major | . | Rohith Sharma K S | Sunil G |
+| [HDFS-12938](https://issues.apache.org/jira/browse/HDFS-12938) | TestErasureCodigCLI testAll failing consistently. |  Major | erasure-coding, hdfs | Rushabh S Shah | Ajay Kumar |
+| [YARN-7542](https://issues.apache.org/jira/browse/YARN-7542) | Fix issue that causes some Running Opportunistic Containers to be recovered as PAUSED |  Major | . | Arun Suresh | Sampada Dehankar |
+| [HDFS-12915](https://issues.apache.org/jira/browse/HDFS-12915) | Fix findbugs warning in INodeFile$HeaderFormat.getBlockLayoutRedundancy |  Major | namenode | Wei-Chiu Chuang | Chris Douglas |
+| [HADOOP-15122](https://issues.apache.org/jira/browse/HADOOP-15122) | Lock down version of doxia-module-markdown plugin |  Blocker | . | Elek, Marton | Elek, Marton |
+| [HADOOP-15143](https://issues.apache.org/jira/browse/HADOOP-15143) | NPE due to Invalid KerberosTicket in UGI |  Major | . | Jitendra Nath Pandey | Mukul Kumar Singh |
+| [YARN-7692](https://issues.apache.org/jira/browse/YARN-7692) | Skip validating priority acls while recovering applications |  Blocker | resourcemanager | Charan Hebri | Sunil G |
+| [MAPREDUCE-7028](https://issues.apache.org/jira/browse/MAPREDUCE-7028) | Concurrent task progress updates causing NPE in Application Master |  Blocker | mr-am | Gergo Repas | Gergo Repas |
+| [YARN-7602](https://issues.apache.org/jira/browse/YARN-7602) | NM should reference the singleton JvmMetrics instance |  Major | nodemanager | Haibo Chen | Haibo Chen |
+| [HDFS-12913](https://issues.apache.org/jira/browse/HDFS-12913) | TestDNFencingWithReplication.testFencingStress fix mini cluster not yet active issue |  Major | . | Zsolt Venczel | Zsolt Venczel |
+| [HDFS-12860](https://issues.apache.org/jira/browse/HDFS-12860) | StripedBlockUtil#getRangesInternalBlocks throws exception for the block group size larger than 2GB |  Major | erasure-coding | Lei (Eddy) Xu | Lei (Eddy) Xu |
+| [YARN-7619](https://issues.apache.org/jira/browse/YARN-7619) | Max AM Resource value in Capacity Scheduler UI has to be refreshed for every user |  Major | capacity scheduler, yarn | Eric Payne | Eric Payne |
+| [YARN-7699](https://issues.apache.org/jira/browse/YARN-7699) | queueUsagePercentage is coming as INF for getApp REST api call |  Major | webapp | Sunil G | Sunil G |
+| [HDFS-12985](https://issues.apache.org/jira/browse/HDFS-12985) | NameNode crashes during restart after an OpenForWrite file present in the Snapshot got deleted |  Major | hdfs | Manoj Govindassamy | Manoj Govindassamy |
+| [YARN-7508](https://issues.apache.org/jira/browse/YARN-7508) | NPE in FiCaSchedulerApp when debug log enabled in async-scheduling mode |  Major | capacityscheduler | Tao Yang | Tao Yang |
+| [YARN-7663](https://issues.apache.org/jira/browse/YARN-7663) | RMAppImpl:Invalid event: START at KILLED |  Minor | resourcemanager | lujie | lujie |
+| [YARN-6948](https://issues.apache.org/jira/browse/YARN-6948) | Invalid event: ATTEMPT\_ADDED at FINAL\_SAVING |  Minor | yarn | lujie | lujie |
+| [HDFS-12994](https://issues.apache.org/jira/browse/HDFS-12994) | TestReconstructStripedFile.testNNSendsErasureCodingTasks fails due to socket timeout |  Major | erasure-coding | Lei (Eddy) Xu | Lei (Eddy) Xu |
+| [YARN-7665](https://issues.apache.org/jira/browse/YARN-7665) | Allow FS scheduler state dump to be turned on/off separately from FS debug log |  Major | . | Wilfred Spiegelenburg | Wilfred Spiegelenburg |
+| [HADOOP-15060](https://issues.apache.org/jira/browse/HADOOP-15060) | TestShellBasedUnixGroupsMapping.testFiniteGroupResolutionTime flaky |  Major | . | Miklos Szegedi | Miklos Szegedi |
+| [YARN-7735](https://issues.apache.org/jira/browse/YARN-7735) | Fix typo in YARN documentation |  Minor | documentation | Takanobu Asanuma | Takanobu Asanuma |
+| [YARN-7727](https://issues.apache.org/jira/browse/YARN-7727) | Incorrect log levels in few logs with QueuePriorityContainerCandidateSelector |  Minor | yarn | Prabhu Joseph | Prabhu Joseph |
+| [HDFS-11915](https://issues.apache.org/jira/browse/HDFS-11915) | Sync rbw dir on the first hsync() to avoid file lost on power failure |  Critical | . | Kanaka Kumar Avvaru | Vinayakumar B |
+| [YARN-7705](https://issues.apache.org/jira/browse/YARN-7705) | Create the container log directory with correct sticky bit in C code |  Major | nodemanager | Yufei Gu | Yufei Gu |
+| [YARN-7479](https://issues.apache.org/jira/browse/YARN-7479) | TestContainerManagerSecurity.testContainerManager[Simple] flaky in trunk |  Major | test | Botong Huang | Akira Ajisaka |
+| [HDFS-13004](https://issues.apache.org/jira/browse/HDFS-13004) | TestLeaseRecoveryStriped#testLeaseRecovery is failing when safeLength is 0MB or larger than the test file |  Major | hdfs | Zsolt Venczel | Zsolt Venczel |
+| [HDFS-9049](https://issues.apache.org/jira/browse/HDFS-9049) | Make Datanode Netty reverse proxy port to be configurable |  Major | datanode | Vinayakumar B | Vinayakumar B |
+| [YARN-7758](https://issues.apache.org/jira/browse/YARN-7758) | Add an additional check to the validity of container and application ids passed to container-executor |  Major | nodemanager | Miklos Szegedi | Yufei Gu |
+| [HADOOP-15150](https://issues.apache.org/jira/browse/HADOOP-15150) | in FsShell, UGI params should be overidden through env vars(-D arg) |  Major | . | Brahma Reddy Battula | Brahma Reddy Battula |
+| [HADOOP-15166](https://issues.apache.org/jira/browse/HADOOP-15166) | CLI MiniCluster fails with ClassNotFoundException o.a.h.yarn.server.timelineservice.collector.TimelineCollectorManager |  Major | . | Gera Shegalov | Gera Shegalov |
+| [HDFS-13039](https://issues.apache.org/jira/browse/HDFS-13039) | StripedBlockReader#createBlockReader leaks socket on IOException |  Critical | datanode, erasure-coding | Lei (Eddy) Xu | Lei (Eddy) Xu |
+| [HADOOP-15181](https://issues.apache.org/jira/browse/HADOOP-15181) | Typo in SecureMode.md |  Trivial | documentation | Masahiro Tanaka | Masahiro Tanaka |
+| [YARN-7796](https://issues.apache.org/jira/browse/YARN-7796) | Container-executor fails with segfault on certain OS configurations |  Major | nodemanager | Gergo Repas | Gergo Repas |
+| [YARN-7806](https://issues.apache.org/jira/browse/YARN-7806) | Distributed Shell should use timeline async api's |  Major | distributed-shell | Sumana Sathish | Rohith Sharma K S |
+| [MAPREDUCE-7015](https://issues.apache.org/jira/browse/MAPREDUCE-7015) | Possible race condition in JHS if the job is not loaded |  Major | jobhistoryserver | Peter Bacsko | Peter Bacsko |
+| [YARN-7737](https://issues.apache.org/jira/browse/YARN-7737) | prelaunch.err file not found exception on container failure |  Major | . | Jonathan Hung | Keqiu Hu |
+| [HDFS-13063](https://issues.apache.org/jira/browse/HDFS-13063) | Fix the incorrect spelling in HDFSHighAvailabilityWithQJM.md |  Trivial | documentation | Jianfei Jiang | Jianfei Jiang |
+| [YARN-7102](https://issues.apache.org/jira/browse/YARN-7102) | NM heartbeat stuck when responseId overflows MAX\_INT |  Critical | . | Botong Huang | Botong Huang |
+| [MAPREDUCE-7041](https://issues.apache.org/jira/browse/MAPREDUCE-7041) | MR should not try to clean up at first job attempt |  Major | . | Takanobu Asanuma | Gergo Repas |
+| [HDFS-13054](https://issues.apache.org/jira/browse/HDFS-13054) | Handling PathIsNotEmptyDirectoryException in DFSClient delete call |  Major | hdfs-client | Nanda kumar | Nanda kumar |
+| [MAPREDUCE-7020](https://issues.apache.org/jira/browse/MAPREDUCE-7020) | Task timeout in uber mode can crash AM |  Major | mr-am | Akira Ajisaka | Peter Bacsko |
+| [YARN-7765](https://issues.apache.org/jira/browse/YARN-7765) | [Atsv2] GSSException: No valid credentials provided - Failed to find any Kerberos tgt thrown by Timelinev2Client & HBaseClient in NM |  Blocker | . | Sumana Sathish | Rohith Sharma K S |
+| [HDFS-13065](https://issues.apache.org/jira/browse/HDFS-13065) | TestErasureCodingMultipleRacks#testSkewedRack3 is failing |  Major | hdfs | Gabor Bota | Gabor Bota |
+| [HDFS-12974](https://issues.apache.org/jira/browse/HDFS-12974) | Exception message is not printed when creating an encryption zone fails with AuthorizationException |  Minor | encryption | fang zhenyi | fang zhenyi |
+| [YARN-7698](https://issues.apache.org/jira/browse/YARN-7698) | A misleading variable's name in ApplicationAttemptEventDispatcher |  Minor | resourcemanager | Jinjiang Ling | Jinjiang Ling |
+| [YARN-7790](https://issues.apache.org/jira/browse/YARN-7790) | Improve Capacity Scheduler Async Scheduling to better handle node failures |  Critical | . | Sumana Sathish | Wangda Tan |
+| [HDFS-12528](https://issues.apache.org/jira/browse/HDFS-12528) | Add an option to not disable short-circuit reads on failures |  Major | hdfs-client, performance | Andre Araujo | Xiao Chen |
+| [HDFS-12897](https://issues.apache.org/jira/browse/HDFS-12897) | getErasureCodingPolicy should handle .snapshot dir better |  Major | erasure-coding, hdfs, snapshots | Harshakiran Reddy | LiXin Ge |
+| [MAPREDUCE-7033](https://issues.apache.org/jira/browse/MAPREDUCE-7033) | Map outputs implicitly rely on permissive umask for shuffle |  Critical | mrv2 | Jason Lowe | Jason Lowe |
+| [HDFS-12942](https://issues.apache.org/jira/browse/HDFS-12942) | Synchronization issue in FSDataSetImpl#moveBlock |  Major | . | Ajay Kumar | Ajay Kumar |
+| [HDFS-13048](https://issues.apache.org/jira/browse/HDFS-13048) | LowRedundancyReplicatedBlocks metric can be negative |  Major | metrics | Akira Ajisaka | Akira Ajisaka |
+| [HDFS-13100](https://issues.apache.org/jira/browse/HDFS-13100) | Handle IllegalArgumentException when GETSERVERDEFAULTS is not implemented in webhdfs. |  Critical | hdfs, webhdfs | Yongjun Zhang | Yongjun Zhang |
+| [YARN-7849](https://issues.apache.org/jira/browse/YARN-7849) | TestMiniYarnClusterNodeUtilization#testUpdateNodeUtilization fails due to heartbeat sync error |  Major | test | Jason Lowe | Botong Huang |
+| [YARN-7801](https://issues.apache.org/jira/browse/YARN-7801) | AmFilterInitializer should addFilter after fill all parameters |  Critical | . | Sumana Sathish | Wangda Tan |
+| [YARN-7890](https://issues.apache.org/jira/browse/YARN-7890) | NPE during container relaunch |  Major | . | Billie Rinaldi | Jason Lowe |
+| [YARN-7873](https://issues.apache.org/jira/browse/YARN-7873) | Revert YARN-6078 |  Blocker | . | Billie Rinaldi | Billie Rinaldi |
+| [HDFS-13115](https://issues.apache.org/jira/browse/HDFS-13115) | In getNumUnderConstructionBlocks(), ignore the inodeIds for which the inodes have been deleted |  Major | . | Yongjun Zhang | Yongjun Zhang |
+| [HDFS-12935](https://issues.apache.org/jira/browse/HDFS-12935) | Get ambiguous result for DFSAdmin command in HA mode when only one namenode is up |  Major | tools | Jianfei Jiang | Jianfei Jiang |
+| [HDFS-13120](https://issues.apache.org/jira/browse/HDFS-13120) | Snapshot diff could be corrupted after concat |  Major | namenode, snapshots | Xiaoyu Yao | Xiaoyu Yao |
+| [HDFS-10453](https://issues.apache.org/jira/browse/HDFS-10453) | ReplicationMonitor thread could stuck for long time due to the race between replication and delete of same file in a large cluster. |  Major | namenode | He Xiaoqiao | He Xiaoqiao |
+| [HDFS-8693](https://issues.apache.org/jira/browse/HDFS-8693) | refreshNamenodes does not support adding a new standby to a running DN |  Critical | datanode, ha | Jian Fang | Ajith S |
+| [MAPREDUCE-7052](https://issues.apache.org/jira/browse/MAPREDUCE-7052) | TestFixedLengthInputFormat#testFormatCompressedIn is flaky |  Major | client, test | Peter Bacsko | Peter Bacsko |
+| [HDFS-13112](https://issues.apache.org/jira/browse/HDFS-13112) | Token expiration edits may cause log corruption or deadlock |  Critical | namenode | Daryn Sharp | Daryn Sharp |
+| [YARN-7937](https://issues.apache.org/jira/browse/YARN-7937) | Fix http method name in Cluster Application Timeout Update API example request |  Minor | docs, documentation | Charan Hebri | Charan Hebri |
+| [HADOOP-10571](https://issues.apache.org/jira/browse/HADOOP-10571) | Use Log.\*(Object, Throwable) overload to log exceptions |  Major | . | Arpit Agarwal | Andras Bokor |
+| [HDFS-13164](https://issues.apache.org/jira/browse/HDFS-13164) | File not closed if streamer fail with DSQuotaExceededException |  Major | hdfs-client | Xiao Chen | Xiao Chen |
+| [HDFS-13244](https://issues.apache.org/jira/browse/HDFS-13244) | Add stack, conf, metrics links to utilities dropdown in NN webUI |  Major | . | Bharat Viswanadham | Bharat Viswanadham |
+| [YARN-8022](https://issues.apache.org/jira/browse/YARN-8022) | ResourceManager UI cluster/app/\<app-id\> page fails to render |  Blocker | webapp | Tarun Parimi | Tarun Parimi |
+
+
+### TESTS:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [MAPREDUCE-7011](https://issues.apache.org/jira/browse/MAPREDUCE-7011) | TestClientDistributedCacheManager::testDetermineCacheVisibilities assumes all parent dirs set other exec |  Trivial | . | Chris Douglas | Chris Douglas |
+
+
+### SUB-TASKS:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [HADOOP-14993](https://issues.apache.org/jira/browse/HADOOP-14993) | AliyunOSS: Override listFiles and listLocatedStatus |  Major | fs/oss | Genmao Yu | Genmao Yu |
+| [YARN-6953](https://issues.apache.org/jira/browse/YARN-6953) | Clean up ResourceUtils.setMinimumAllocationForMandatoryResources() and setMaximumAllocationForMandatoryResources() |  Minor | resourcemanager | Daniel Templeton | Manikandan R |
+| [HDFS-12801](https://issues.apache.org/jira/browse/HDFS-12801) | RBF: Set MountTableResolver as default file resolver |  Minor | . | Íñigo Goiri | Íñigo Goiri |
+| [HADOOP-15024](https://issues.apache.org/jira/browse/HADOOP-15024) | AliyunOSS: support user agent configuration and include that & Hadoop version information to oss server |  Major | fs, fs/oss | SammiChen | SammiChen |
+| [HDFS-12858](https://issues.apache.org/jira/browse/HDFS-12858) | RBF: Add router admin commands usage in HDFS commands reference doc |  Minor | documentation | Yiqun Lin | Yiqun Lin |
+| [HDFS-12835](https://issues.apache.org/jira/browse/HDFS-12835) | RBF: Fix Javadoc parameter errors |  Minor | . | Wei Yan | Wei Yan |
+| [YARN-6907](https://issues.apache.org/jira/browse/YARN-6907) | Node information page in the old web UI should report resource types |  Major | resourcemanager | Daniel Templeton | Gergely Novák |
+| [HDFS-12396](https://issues.apache.org/jira/browse/HDFS-12396) | Webhdfs file system should get delegation token from kms provider. |  Major | encryption, kms, webhdfs | Rushabh S Shah | Rushabh S Shah |
+| [YARN-7610](https://issues.apache.org/jira/browse/YARN-7610) | Extend Distributed Shell to support launching job with opportunistic containers |  Major | applications/distributed-shell | Weiwei Yang | Weiwei Yang |
+| [HDFS-12875](https://issues.apache.org/jira/browse/HDFS-12875) | RBF: Complete logic for -readonly option of dfsrouteradmin add command |  Major | . | Yiqun Lin | Íñigo Goiri |
+| [YARN-7383](https://issues.apache.org/jira/browse/YARN-7383) | Node resource is not parsed correctly for resource names containing dot |  Major | nodemanager, resourcemanager | Jonathan Hung | Gergely Novák |
+| [YARN-7119](https://issues.apache.org/jira/browse/YARN-7119) | Support multiple resource types in rmadmin updateNodeResource command |  Major | nodemanager, resourcemanager | Daniel Templeton | Manikandan R |
+| [YARN-7617](https://issues.apache.org/jira/browse/YARN-7617) | Add a flag in distributed shell to automatically PROMOTE opportunistic containers to guaranteed once they are started |  Minor | applications/distributed-shell | Weiwei Yang | Weiwei Yang |
+| [HDFS-12937](https://issues.apache.org/jira/browse/HDFS-12937) | RBF: Add more unit tests for router admin commands |  Major | test | Yiqun Lin | Yiqun Lin |
+| [YARN-7032](https://issues.apache.org/jira/browse/YARN-7032) | [ATSv2] NPE while starting hbase co-processor when HBase authorization is enabled. |  Critical | . | Rohith Sharma K S | Rohith Sharma K S |
+| [HADOOP-14965](https://issues.apache.org/jira/browse/HADOOP-14965) | s3a input stream "normal" fadvise mode to be adaptive |  Major | fs/s3 | Steve Loughran | Steve Loughran |
+| [HADOOP-15086](https://issues.apache.org/jira/browse/HADOOP-15086) | NativeAzureFileSystem file rename is not atomic |  Major | fs/azure | Shixiong Zhu | Thomas Marquardt |
+| [HDFS-12988](https://issues.apache.org/jira/browse/HDFS-12988) | RBF: Mount table entries not properly updated in the local cache |  Major | . | Íñigo Goiri | Íñigo Goiri |
+| [YARN-7716](https://issues.apache.org/jira/browse/YARN-7716) | metricsTimeStart and metricsTimeEnd should be all lower case in the doc |  Major | timelinereader | Haibo Chen | Haibo Chen |
+| [HDFS-12802](https://issues.apache.org/jira/browse/HDFS-12802) | RBF: Control MountTableResolver cache size |  Major | . | Íñigo Goiri | Íñigo Goiri |
+| [HDFS-12919](https://issues.apache.org/jira/browse/HDFS-12919) | RBF: Support erasure coding methods in RouterRpcServer |  Critical | . | Íñigo Goiri | Íñigo Goiri |
+| [YARN-6736](https://issues.apache.org/jira/browse/YARN-6736) | Consider writing to both ats v1 & v2 from RM for smoother upgrades |  Major | timelineserver | Vrushali C | Aaron Gresch |
+| [HADOOP-15027](https://issues.apache.org/jira/browse/HADOOP-15027) | AliyunOSS: Support multi-thread pre-read to improve sequential read from Hadoop to Aliyun OSS performance |  Major | fs/oss | wujinhu | wujinhu |
+| [HDFS-13028](https://issues.apache.org/jira/browse/HDFS-13028) | RBF: Fix spurious TestRouterRpc#testProxyGetStats |  Minor | . | Íñigo Goiri | Íñigo Goiri |
+| [YARN-5094](https://issues.apache.org/jira/browse/YARN-5094) | some YARN container events have timestamp of -1 |  Critical | timelineserver | Sangjin Lee | Haibo Chen |
+| [YARN-7782](https://issues.apache.org/jira/browse/YARN-7782) | Enable user re-mapping for Docker containers in yarn-default.xml |  Blocker | security, yarn | Eric Yang | Eric Yang |
+| [HDFS-12772](https://issues.apache.org/jira/browse/HDFS-12772) | RBF: Federation Router State State Store internal API |  Major | . | Íñigo Goiri | Íñigo Goiri |
+| [HDFS-13042](https://issues.apache.org/jira/browse/HDFS-13042) | RBF: Heartbeat Router State |  Major | . | Íñigo Goiri | Íñigo Goiri |
+| [HDFS-13049](https://issues.apache.org/jira/browse/HDFS-13049) | RBF: Inconsistent Router OPTS config in branch-2 and branch-3 |  Minor | . | Wei Yan | Wei Yan |
+| [HDFS-12574](https://issues.apache.org/jira/browse/HDFS-12574) | Add CryptoInputStream to WebHdfsFileSystem read call. |  Major | encryption, kms, webhdfs | Rushabh S Shah | Rushabh S Shah |
+| [HDFS-13044](https://issues.apache.org/jira/browse/HDFS-13044) | RBF: Add a safe mode for the Router |  Major | . | Íñigo Goiri | Íñigo Goiri |
+| [HDFS-13043](https://issues.apache.org/jira/browse/HDFS-13043) | RBF: Expose the state of the Routers in the federation |  Major | . | Íñigo Goiri | Íñigo Goiri |
+| [HDFS-12997](https://issues.apache.org/jira/browse/HDFS-12997) | Move logging to slf4j in BlockPoolSliceStorage and Storage |  Major | . | Ajay Kumar | Ajay Kumar |
+| [HDFS-13068](https://issues.apache.org/jira/browse/HDFS-13068) | RBF: Add router admin option to manage safe mode |  Major | . | Íñigo Goiri | Yiqun Lin |
+| [HADOOP-15247](https://issues.apache.org/jira/browse/HADOOP-15247) | Move commons-net up to 3.6 |  Minor | fs | Steve Loughran | Steve Loughran |
+| [HADOOP-15090](https://issues.apache.org/jira/browse/HADOOP-15090) | Add ADL troubleshooting doc |  Major | documentation, fs/adl | Steve Loughran | Steve Loughran |
+
+
+### OTHER:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [HADOOP-15149](https://issues.apache.org/jira/browse/HADOOP-15149) | CryptoOutputStream should implement StreamCapabilities |  Major | fs | Mike Drob | Xiao Chen |
+| [YARN-7691](https://issues.apache.org/jira/browse/YARN-7691) | Add Unit Tests for ContainersLauncher |  Major | . | Sampada Dehankar | Sampada Dehankar |
+| [HADOOP-15177](https://issues.apache.org/jira/browse/HADOOP-15177) | Update the release year to 2018 |  Blocker | build | Akira Ajisaka | Bharat Viswanadham |
+| [HADOOP-15197](https://issues.apache.org/jira/browse/HADOOP-15197) | Remove tomcat from the Hadoop-auth test bundle |  Major | . | Xiao Chen | Xiao Chen |
+
+

+ 54 - 0
hadoop-common-project/hadoop-common/src/site/markdown/release/3.0.1/RELEASENOTES.3.0.1.md

@@ -0,0 +1,54 @@
+
+<!---
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+-->
+# Apache Hadoop  3.0.1 Release Notes
+
+These release notes cover new developer and user-facing incompatibilities, important issues, features, and major improvements.
+
+
+---
+
+* [HADOOP-15027](https://issues.apache.org/jira/browse/HADOOP-15027) | *Major* | **AliyunOSS: Support multi-thread pre-read to improve sequential read from Hadoop to Aliyun OSS performance**
+
+Support multi-thread pre-read in AliyunOSSInputStream to improve the sequential read performance from Hadoop to Aliyun OSS.
+
+
+---
+
+* [HDFS-12528](https://issues.apache.org/jira/browse/HDFS-12528) | *Major* | **Add an option to not disable short-circuit reads on failures**
+
+Added an option to not disables short-circuit reads on failures, by setting dfs.domain.socket.disable.interval.seconds to 0.
+
+
+---
+
+* [HDFS-13083](https://issues.apache.org/jira/browse/HDFS-13083) | *Major* | **RBF: Fix doc error setting up client**
+
+Fix the document error of setting up HFDS Router Federation
+
+
+---
+
+* [HDFS-12990](https://issues.apache.org/jira/browse/HDFS-12990) | *Critical* | **Change default NameNode RPC port back to 8020**
+
+HDFS NameNode default RPC port is changed back to 8020. The only official release that has this differently is 3.0.0, which used 9820.
+
+It is recommended for 2.x users to upgrade to 3.0.1+, to reduce the burden on changing default NN RPC port.
+
+
+

+ 7 - 27
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java

@@ -27,7 +27,6 @@ import java.io.FileWriter;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStreamWriter;
-import java.io.PrintStream;
 import java.io.StringWriter;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
@@ -2392,33 +2391,14 @@ public class TestConfiguration {
 
   @Test
   public void testInvalidTags() throws Exception {
-    PrintStream output = System.out;
-    try {
-      ByteArrayOutputStream bytes = new ByteArrayOutputStream();
-      System.setOut(new PrintStream(bytes));
-
-      out = new BufferedWriter(new FileWriter(CONFIG));
-      startConfig();
-      appendPropertyByTag("dfs.cblock.trace.io", "false", "MYOWNTAG,TAG2");
-      endConfig();
-
-      Path fileResource = new Path(CONFIG);
-      conf.addResource(fileResource);
-      conf.getProps();
-
-      List<String> tagList = new ArrayList<>();
-      tagList.add("REQUIRED");
-      tagList.add("MYOWNTAG");
-      tagList.add("TAG2");
+    Path fileResource = new Path(CONFIG);
+    conf.addResource(fileResource);
+    conf.getProps();
 
-      Properties properties = conf.getAllPropertiesByTags(tagList);
-      assertEq(0, properties.size());
-      assertFalse(properties.containsKey("dfs.cblock.trace.io"));
-      assertFalse(bytes.toString().contains("Invalid tag "));
-      assertFalse(bytes.toString().contains("Tag"));
-    } finally {
-      System.setOut(output);
-    }
+    assertFalse(conf.isPropertyTag("BADTAG"));
+    assertFalse(conf.isPropertyTag("CUSTOM_TAG"));
+    assertTrue(conf.isPropertyTag("DEBUG"));
+    assertTrue(conf.isPropertyTag("HDFS"));
   }
 
   /**

+ 61 - 135
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestDataByteBuffers.java

@@ -22,7 +22,6 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.Arrays;
 import java.util.Random;
 
 import org.junit.Test;
@@ -30,53 +29,67 @@ import static org.junit.Assert.*;
 
 public class TestDataByteBuffers {
 
-  private static void readJunk(DataInput in, Random r, long seed, int iter) 
+  private static final Random RAND = new Random(31L);
+
+  private static void readJunk(DataInput in, int iter)
       throws IOException {
-    r.setSeed(seed);
+    RAND.setSeed(31L);
     for (int i = 0; i < iter; ++i) {
-      switch (r.nextInt(7)) {
-        case 0:
-          assertEquals((byte)(r.nextInt() & 0xFF), in.readByte()); break;
-        case 1:
-          assertEquals((short)(r.nextInt() & 0xFFFF), in.readShort()); break;
-        case 2:
-          assertEquals(r.nextInt(), in.readInt()); break;
-        case 3:
-          assertEquals(r.nextLong(), in.readLong()); break;
-        case 4:
-          assertEquals(Double.doubleToLongBits(r.nextDouble()),
-                       Double.doubleToLongBits(in.readDouble())); break;
-        case 5:
-          assertEquals(Float.floatToIntBits(r.nextFloat()),
-                       Float.floatToIntBits(in.readFloat())); break;
-        case 6:
-          int len = r.nextInt(1024);
-          byte[] vb = new byte[len];
-          r.nextBytes(vb);
-          byte[] b = new byte[len];
-          in.readFully(b, 0, len);
-          assertArrayEquals(vb, b);
-          break;
+      switch (RAND.nextInt(7)) {
+      case 0:
+        assertEquals((byte)(RAND.nextInt() & 0xFF), in.readByte()); break;
+      case 1:
+        assertEquals((short)(RAND.nextInt() & 0xFFFF), in.readShort()); break;
+      case 2:
+        assertEquals(RAND.nextInt(), in.readInt()); break;
+      case 3:
+        assertEquals(RAND.nextLong(), in.readLong()); break;
+      case 4:
+        assertEquals(Double.doubleToLongBits(RAND.nextDouble()),
+            Double.doubleToLongBits(in.readDouble()));
+        break;
+      case 5:
+        assertEquals(Float.floatToIntBits(RAND.nextFloat()),
+            Float.floatToIntBits(in.readFloat()));
+        break;
+      case 6:
+        int len = RAND.nextInt(1024);
+        byte[] vb = new byte[len];
+        RAND.nextBytes(vb);
+        byte[] b = new byte[len];
+        in.readFully(b, 0, len);
+        assertArrayEquals(vb, b);
+        break;
+      default:
+        throw new IOException();
       }
     }
   }
 
-  private static void writeJunk(DataOutput out, Random r, long seed, int iter)
-      throws IOException  {
-    r.setSeed(seed);
+  private static void writeJunk(DataOutput out, int iter)
+      throws IOException {
+    RAND.setSeed(31L);
     for (int i = 0; i < iter; ++i) {
-      switch (r.nextInt(7)) {
-        case 0: out.writeByte(r.nextInt()); break;
-        case 1: out.writeShort((short)(r.nextInt() & 0xFFFF)); break;
-        case 2: out.writeInt(r.nextInt()); break;
-        case 3: out.writeLong(r.nextLong()); break;
-        case 4: out.writeDouble(r.nextDouble()); break;
-        case 5: out.writeFloat(r.nextFloat()); break;
-        case 6:
-          byte[] b = new byte[r.nextInt(1024)];
-          r.nextBytes(b);
-          out.write(b);
-          break;
+      switch (RAND.nextInt(7)) {
+      case 0:
+        out.writeByte(RAND.nextInt()); break;
+      case 1:
+        out.writeShort((short)(RAND.nextInt() & 0xFFFF)); break;
+      case 2:
+        out.writeInt(RAND.nextInt()); break;
+      case 3:
+        out.writeLong(RAND.nextLong()); break;
+      case 4:
+        out.writeDouble(RAND.nextDouble()); break;
+      case 5:
+        out.writeFloat(RAND.nextFloat()); break;
+      case 6:
+        byte[] b = new byte[RAND.nextInt(1024)];
+        RAND.nextBytes(b);
+        out.write(b);
+        break;
+      default:
+        throw new IOException();
       }
     }
   }
@@ -84,113 +97,26 @@ public class TestDataByteBuffers {
   @Test
   public void testBaseBuffers() throws IOException {
     DataOutputBuffer dob = new DataOutputBuffer();
-    Random r = new Random();
-    long seed = r.nextLong();
-    r.setSeed(seed);
-    System.out.println("SEED: " + seed);
-    writeJunk(dob, r, seed, 1000);
+    writeJunk(dob, 1000);
     DataInputBuffer dib = new DataInputBuffer();
     dib.reset(dob.getData(), 0, dob.getLength());
-    readJunk(dib, r, seed, 1000);
+    readJunk(dib, 1000);
 
     dob.reset();
-    writeJunk(dob, r, seed, 1000);
+    writeJunk(dob, 1000);
     dib.reset(dob.getData(), 0, dob.getLength());
-    readJunk(dib, r, seed, 1000);
-  }
-
-  @Test
-  public void testByteBuffers() throws IOException {
-    DataOutputByteBuffer dob = new DataOutputByteBuffer();
-    Random r = new Random();
-    long seed = r.nextLong();
-    r.setSeed(seed);
-    System.out.println("SEED: " + seed);
-    writeJunk(dob, r, seed, 1000);
-    DataInputByteBuffer dib = new DataInputByteBuffer();
-    dib.reset(dob.getData());
-    readJunk(dib, r, seed, 1000);
-
-    dob.reset();
-    writeJunk(dob, r, seed, 1000);
-    dib.reset(dob.getData());
-    readJunk(dib, r, seed, 1000);
-  }
-
-  private static byte[] toBytes(ByteBuffer[] bufs, int len) {
-    byte[] ret = new byte[len];
-    int pos = 0;
-    for (int i = 0; i < bufs.length; ++i) {
-      int rem = bufs[i].remaining();
-      bufs[i].get(ret, pos, rem);
-      pos += rem;
-    }
-    return ret;
-  }
-
-  @Test
-  public void testDataOutputByteBufferCompatibility() throws IOException {
-    DataOutputBuffer dob = new DataOutputBuffer();
-    DataOutputByteBuffer dobb = new DataOutputByteBuffer();
-    Random r = new Random();
-    long seed = r.nextLong();
-    r.setSeed(seed);
-    System.out.println("SEED: " + seed);
-    writeJunk(dob, r, seed, 1000);
-    writeJunk(dobb, r, seed, 1000);
-    byte[] check = toBytes(dobb.getData(), dobb.getLength());
-    assertEquals(check.length, dob.getLength());
-    assertArrayEquals(check, Arrays.copyOf(dob.getData(), dob.getLength()));
-
-    dob.reset();
-    dobb.reset();
-    writeJunk(dob, r, seed, 3000);
-    writeJunk(dobb, r, seed, 3000);
-    check = toBytes(dobb.getData(), dobb.getLength());
-    assertEquals(check.length, dob.getLength());
-    assertArrayEquals(check, Arrays.copyOf(dob.getData(), dob.getLength()));
-
-    dob.reset();
-    dobb.reset();
-    writeJunk(dob, r, seed, 1000);
-    writeJunk(dobb, r, seed, 1000);
-    check = toBytes(dobb.getData(), dobb.getLength());
-    assertEquals("Failed Checking length = " + check.length,
-            check.length, dob.getLength());
-    assertArrayEquals(check, Arrays.copyOf(dob.getData(), dob.getLength()));
+    readJunk(dib, 1000);
   }
 
   @Test
-  public void TestDataInputByteBufferCompatibility() throws IOException {
+  public void testDataInputByteBufferCompatibility() throws IOException {
     DataOutputBuffer dob = new DataOutputBuffer();
-    Random r = new Random();
-    long seed = r.nextLong();
-    r.setSeed(seed);
-    System.out.println("SEED: " + seed);
-    writeJunk(dob, r, seed, 1000);
+    writeJunk(dob, 1000);
     ByteBuffer buf = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
 
     DataInputByteBuffer dib = new DataInputByteBuffer();
     dib.reset(buf);
-    readJunk(dib, r, seed, 1000);
-  }
-
-  @Test
-  public void TestDataOutputByteBufferCompatibility() throws IOException {
-    DataOutputByteBuffer dob = new DataOutputByteBuffer();
-    Random r = new Random();
-    long seed = r.nextLong();
-    r.setSeed(seed);
-    System.out.println("SEED: " + seed);
-    writeJunk(dob, r, seed, 1000);
-    ByteBuffer buf = ByteBuffer.allocate(dob.getLength());
-    for (ByteBuffer b : dob.getData()) {
-      buf.put(b);
-    }
-    buf.flip();
-    DataInputBuffer dib = new DataInputBuffer();
-    dib.reset(buf.array(), 0, buf.remaining());
-    readJunk(dib, r, seed, 1000);
+    readJunk(dib, 1000);
   }
 
 }

+ 1 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/util/DummyMXBean.java

@@ -18,7 +18,7 @@
 package org.apache.hadoop.metrics2.util;
 
 /**
- * Dummy MXBean class to test arbitrary key/values with MXBeans.
+ * Sample JMX Bean interface to test JMX registration.
  */
 public interface DummyMXBean {
 

+ 31 - 7
hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider;
 import org.apache.hadoop.crypto.key.kms.ValueQueue;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.MultipleIOException;
 import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.SecurityUtil;
@@ -84,6 +85,7 @@ import java.util.Collection;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
@@ -111,6 +113,10 @@ public class TestKMS {
 
   private SSLFactory sslFactory;
 
+  // Keep track of all key providers created during a test case, so they can be
+  // closed at test tearDown.
+  private List<KeyProvider> providersCreated = new LinkedList<>();
+
   @Rule
   public final Timeout testTimeout = new Timeout(180000);
 
@@ -144,13 +150,17 @@ public class TestKMS {
 
   protected KeyProvider createProvider(URI uri, Configuration conf)
       throws IOException {
-    return new LoadBalancingKMSClientProvider(
-        new KMSClientProvider[] { new KMSClientProvider(uri, conf) }, conf);
+    final KeyProvider ret = new LoadBalancingKMSClientProvider(
+        new KMSClientProvider[] {new KMSClientProvider(uri, conf)}, conf);
+    providersCreated.add(ret);
+    return ret;
   }
 
   private KMSClientProvider createKMSClientProvider(URI uri, Configuration conf)
       throws IOException {
-    return new KMSClientProvider(uri, conf);
+    final KMSClientProvider ret = new KMSClientProvider(uri, conf);
+    providersCreated.add(ret);
+    return ret;
   }
 
   protected <T> T runServer(String keystore, String password, File confDir,
@@ -311,13 +321,28 @@ public class TestKMS {
   }
 
   @After
-  public void tearDownMiniKdc() throws Exception {
+  public void tearDown() throws Exception {
     if (kdc != null) {
       kdc.stop();
       kdc = null;
     }
     UserGroupInformation.setShouldRenewImmediatelyForTests(false);
     UserGroupInformation.reset();
+    if (!providersCreated.isEmpty()) {
+      final MultipleIOException.Builder b = new MultipleIOException.Builder();
+      for (KeyProvider kp : providersCreated) {
+        try {
+          kp.close();
+        } catch (IOException e) {
+          LOG.error("Failed to close key provider.", e);
+          b.add(e);
+        }
+      }
+      providersCreated.clear();
+      if (!b.isEmpty()) {
+        throw b.build();
+      }
+    }
   }
 
   private <T> T doAs(String user, final PrivilegedExceptionAction<T> action)
@@ -449,6 +474,8 @@ public class TestKMS {
             }
           }
           Assert.assertTrue("Reloader is not alive", reloaderThread.isAlive());
+          // Explicitly close the provider so we can verify the internal thread
+          // is shutdown
           testKp.close();
           boolean reloaderStillAlive = true;
           for (int i = 0; i < 10; i++) {
@@ -476,7 +503,6 @@ public class TestKMS {
                     .addDelegationTokens("myuser", new Credentials());
                 Assert.assertEquals(1, tokens.length);
                 Assert.assertEquals("kms-dt", tokens[0].getKind().toString());
-                kp.close();
                 return null;
               }
             });
@@ -494,7 +520,6 @@ public class TestKMS {
               .addDelegationTokens("myuser", new Credentials());
           Assert.assertEquals(1, tokens.length);
           Assert.assertEquals("kms-dt", tokens[0].getKind().toString());
-          kp.close();
         }
         return null;
       }
@@ -2533,7 +2558,6 @@ public class TestKMS {
 
   @Test
   public void testTGTRenewal() throws Exception {
-    tearDownMiniKdc();
     Properties kdcConf = MiniKdc.createConf();
     kdcConf.setProperty(MiniKdc.MAX_TICKET_LIFETIME, "3");
     kdcConf.setProperty(MiniKdc.MIN_TICKET_LIFETIME, "3");

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/pom.xml

@@ -167,6 +167,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
             <configuration>
               <tasks>
                 <copy file="src/main/resources/hdfs-rbf-default.xml" todir="src/site/resources"/>
+                <copy file="src/main/xsl/configuration.xsl" todir="src/site/resources"/>
               </tasks>
             </configuration>
           </execution>

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMetrics.java

@@ -434,7 +434,7 @@ public class FederationMetrics implements FederationMBean {
     try {
       RouterRpcServer rpcServer = this.router.getRpcServer();
       DatanodeInfo[] live = rpcServer.getDatanodeReport(
-          DatanodeReportType.LIVE, TIME_OUT);
+          DatanodeReportType.LIVE, false, TIME_OUT);
 
       if (live.length > 0) {
         float totalDfsUsed = 0;

+ 65 - 3
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java

@@ -25,6 +25,8 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 
@@ -32,6 +34,7 @@ import javax.management.NotCompliantMBeanException;
 import javax.management.ObjectName;
 import javax.management.StandardMBean;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -39,6 +42,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamespaceInfo;
+import org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys;
 import org.apache.hadoop.hdfs.server.federation.router.Router;
 import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer;
 import org.apache.hadoop.hdfs.server.federation.store.MembershipStore;
@@ -56,6 +60,10 @@ import org.eclipse.jetty.util.ajax.JSON;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
 /**
  * Expose the Namenode metrics as the Router was one.
  */
@@ -65,6 +73,22 @@ public class NamenodeBeanMetrics
   private static final Logger LOG =
       LoggerFactory.getLogger(NamenodeBeanMetrics.class);
 
+  /** Prevent holding the page from loading too long. */
+  private static final String DN_REPORT_TIME_OUT =
+      RBFConfigKeys.FEDERATION_ROUTER_PREFIX + "dn-report.time-out";
+  /** We only wait for 1 second. */
+  private static final long DN_REPORT_TIME_OUT_DEFAULT =
+      TimeUnit.SECONDS.toMillis(1);
+
+  /** Time to cache the DN information. */
+  public static final String DN_REPORT_CACHE_EXPIRE =
+      RBFConfigKeys.FEDERATION_ROUTER_PREFIX + "dn-report.cache-expire";
+  /** We cache the DN information for 10 seconds by default. */
+  public static final long DN_REPORT_CACHE_EXPIRE_DEFAULT =
+      TimeUnit.SECONDS.toMillis(10);
+
+
+  /** Instance of the Router being monitored. */
   private final Router router;
 
   /** FSNamesystem bean. */
@@ -76,6 +100,11 @@ public class NamenodeBeanMetrics
   /** NameNodeStatus bean. */
   private ObjectName nnStatusBeanName;
 
+  /** Timeout to get the DN report. */
+  private final long dnReportTimeOut;
+  /** DN type -> full DN report in JSON. */
+  private final LoadingCache<DatanodeReportType, String> dnCache;
+
 
   public NamenodeBeanMetrics(Router router) {
     this.router = router;
@@ -114,6 +143,23 @@ public class NamenodeBeanMetrics
     } catch (NotCompliantMBeanException e) {
       throw new RuntimeException("Bad NameNodeStatus MBean setup", e);
     }
+
+    // Initialize the cache for the DN reports
+    Configuration conf = router.getConfig();
+    this.dnReportTimeOut = conf.getTimeDuration(
+        DN_REPORT_TIME_OUT, DN_REPORT_TIME_OUT_DEFAULT, TimeUnit.MILLISECONDS);
+    long dnCacheExpire = conf.getTimeDuration(
+        DN_REPORT_CACHE_EXPIRE,
+        DN_REPORT_CACHE_EXPIRE_DEFAULT, TimeUnit.MILLISECONDS);
+    this.dnCache = CacheBuilder.newBuilder()
+        .expireAfterWrite(dnCacheExpire, TimeUnit.MILLISECONDS)
+        .build(
+            new CacheLoader<DatanodeReportType, String>() {
+              @Override
+              public String load(DatanodeReportType type) throws Exception {
+                return getNodesImpl(type);
+              }
+            });
   }
 
   /**
@@ -298,17 +344,33 @@ public class NamenodeBeanMetrics
     return this.getNodes(DatanodeReportType.DECOMMISSIONING);
   }
 
+  /**
+   * Get all the nodes in the federation from a particular type. Getting this
+   * information is expensive and we use a cache.
+   * @param type Type of the datanodes to check.
+   * @return JSON with the nodes.
+   */
+  private String getNodes(final DatanodeReportType type) {
+    try {
+      return this.dnCache.get(type);
+    } catch (ExecutionException e) {
+      LOG.error("Cannot get the DN storage report for {}", type, e);
+    }
+    // If we cannot get the report, return empty JSON
+    return "{}";
+  }
+
   /**
    * Get all the nodes in the federation from a particular type.
-   * TODO this is expensive, we may want to cache it.
    * @param type Type of the datanodes to check.
    * @return JSON with the nodes.
    */
-  private String getNodes(DatanodeReportType type) {
+  private String getNodesImpl(final DatanodeReportType type) {
     final Map<String, Map<String, Object>> info = new HashMap<>();
     try {
       RouterRpcServer rpcServer = this.router.getRpcServer();
-      DatanodeInfo[] datanodes = rpcServer.getDatanodeReport(type);
+      DatanodeInfo[] datanodes =
+          rpcServer.getDatanodeReport(type, false, dnReportTimeOut);
       for (DatanodeInfo node : datanodes) {
         Map<String, Object> innerinfo = new HashMap<>();
         innerinfo.put("infoAddr", node.getInfoAddr());

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MultipleDestinationMountTableResolver.java

@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.EnumMap;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.federation.resolver.order.AvailableSpaceResolver;
 import org.apache.hadoop.hdfs.server.federation.resolver.order.DestinationOrder;
 import org.apache.hadoop.hdfs.server.federation.resolver.order.HashFirstResolver;
 import org.apache.hadoop.hdfs.server.federation.resolver.order.HashResolver;
@@ -77,6 +78,8 @@ public class MultipleDestinationMountTableResolver extends MountTableResolver {
     addResolver(DestinationOrder.LOCAL, new LocalResolver(conf, router));
     addResolver(DestinationOrder.RANDOM, new RandomResolver());
     addResolver(DestinationOrder.HASH_ALL, new HashResolver());
+    addResolver(DestinationOrder.SPACE,
+        new AvailableSpaceResolver(conf, router));
   }
 
   @Override

+ 179 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/order/AvailableSpaceResolver.java

@@ -0,0 +1,179 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.federation.resolver.order;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.federation.resolver.PathLocation;
+import org.apache.hadoop.hdfs.server.federation.resolver.order.AvailableSpaceResolver.SubclusterAvailableSpace;
+import org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.router.Router;
+import org.apache.hadoop.hdfs.server.federation.store.MembershipStore;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamenodeRegistrationsRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamenodeRegistrationsResponse;
+import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Order the destinations based on available space. This resolver uses a
+ * higher probability (instead of "always") to choose the cluster with higher
+ * available space.
+ */
+public class AvailableSpaceResolver
+    extends RouterResolver<String, SubclusterAvailableSpace> {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AvailableSpaceResolver.class);
+
+  /** Increases chance of files on subcluster with more available space. */
+  public static final String BALANCER_PREFERENCE_KEY =
+      RBFConfigKeys.FEDERATION_ROUTER_PREFIX
+      + "available-space-resolver.balanced-space-preference-fraction";
+  public static final float BALANCER_PREFERENCE_DEFAULT = 0.6f;
+
+  /** Random instance used in the subcluster comparison. */
+  private static final Random RAND = new Random();
+
+  /** Customized comparator for SubclusterAvailableSpace. */
+  private SubclusterSpaceComparator comparator;
+
+  public AvailableSpaceResolver(final Configuration conf,
+      final Router routerService) {
+    super(conf, routerService);
+    float balancedPreference = conf.getFloat(BALANCER_PREFERENCE_KEY,
+        BALANCER_PREFERENCE_DEFAULT);
+    if (balancedPreference < 0.5) {
+      LOG.warn("The balancer preference value is less than 0.5. That means more"
+          + " files will be allocated in cluster with lower available space.");
+    }
+
+    this.comparator = new SubclusterSpaceComparator(balancedPreference);
+  }
+
+  /**
+   * Get the mapping from NamespaceId to subcluster space info. It gets this
+   * mapping from the subclusters through expensive calls (e.g., RPC) and uses
+   * caching to avoid too many calls. The cache might be updated asynchronously
+   * to reduce latency.
+   *
+   * @return NamespaceId -> {@link SubclusterAvailableSpace}
+   */
+  @Override
+  protected Map<String, SubclusterAvailableSpace> getSubclusterInfo(
+      MembershipStore membershipStore) {
+    Map<String, SubclusterAvailableSpace> mapping = new HashMap<>();
+    try {
+      // Get the Namenode's available space info from the subclusters
+      // from the Membership store.
+      GetNamenodeRegistrationsRequest request = GetNamenodeRegistrationsRequest
+          .newInstance();
+      GetNamenodeRegistrationsResponse response = membershipStore
+          .getNamenodeRegistrations(request);
+      final List<MembershipState> nns = response.getNamenodeMemberships();
+      for (MembershipState nn : nns) {
+        try {
+          String nsId = nn.getNameserviceId();
+          long availableSpace = nn.getStats().getAvailableSpace();
+          mapping.put(nsId, new SubclusterAvailableSpace(nsId, availableSpace));
+        } catch (Exception e) {
+          LOG.error("Cannot get stats info for {}: {}.", nn, e.getMessage());
+        }
+      }
+    } catch (IOException ioe) {
+      LOG.error("Cannot get Namenodes from the State Store.", ioe);
+    }
+    return mapping;
+  }
+
+  @Override
+  protected String chooseFirstNamespace(String path, PathLocation loc) {
+    Map<String, SubclusterAvailableSpace> subclusterInfo =
+        getSubclusterMapping();
+    List<SubclusterAvailableSpace> subclusterList = new LinkedList<>(
+        subclusterInfo.values());
+    Collections.sort(subclusterList, comparator);
+
+    return subclusterList.size() > 0 ? subclusterList.get(0).getNameserviceId()
+        : null;
+  }
+
+  /**
+   * Inner class that stores cluster available space info.
+   */
+  static class SubclusterAvailableSpace {
+    private final String nsId;
+    private final long availableSpace;
+
+    SubclusterAvailableSpace(String nsId, long availableSpace) {
+      this.nsId = nsId;
+      this.availableSpace = availableSpace;
+    }
+
+    public String getNameserviceId() {
+      return this.nsId;
+    }
+
+    public long getAvailableSpace() {
+      return this.availableSpace;
+    }
+  }
+
+  /**
+   * Customized comparator for SubclusterAvailableSpace. If more available
+   * space the one cluster has, the higher priority it will have. But this
+   * is not absolute, there is a balanced preference to make this use a higher
+   * probability (instead of "always") to compare by this way.
+   */
+  static final class SubclusterSpaceComparator
+      implements Comparator<SubclusterAvailableSpace>, Serializable {
+    private int balancedPreference;
+
+    SubclusterSpaceComparator(float balancedPreference) {
+      Preconditions.checkArgument(
+          balancedPreference <= 1 && balancedPreference >= 0,
+          "The balancer preference value should be in the range 0.0 - 1.0");
+
+      this.balancedPreference = (int) (100 * balancedPreference);
+    }
+
+    @Override
+    public int compare(SubclusterAvailableSpace cluster1,
+        SubclusterAvailableSpace cluster2) {
+      int ret = cluster1.getAvailableSpace() > cluster2.getAvailableSpace() ? -1
+          : 1;
+
+      if (ret < 0) {
+        return (RAND.nextInt(100) < balancedPreference) ? -1 : 1;
+      } else {
+        return (RAND.nextInt(100) < balancedPreference) ? 1 : -1;
+      }
+    }
+  }
+}

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/order/DestinationOrder.java

@@ -26,5 +26,6 @@ public enum DestinationOrder {
   HASH, // Follow consistent hashing in the first folder level
   LOCAL, // Local first
   RANDOM, // Random order
-  HASH_ALL // Follow consistent hashing
+  HASH_ALL, // Follow consistent hashing
+  SPACE // Available space based order
 }

+ 34 - 113
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/order/LocalResolver.java

@@ -17,11 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.federation.resolver.order;
 
-import static org.apache.hadoop.util.Time.monotonicNow;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.net.HostAndPort;
-
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
@@ -30,17 +25,14 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.server.federation.resolver.PathLocation;
-import org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys;
 import org.apache.hadoop.hdfs.server.federation.router.Router;
 import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer;
 import org.apache.hadoop.hdfs.server.federation.store.MembershipStore;
-import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamenodeRegistrationsRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamenodeRegistrationsResponse;
 import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState;
@@ -50,40 +42,46 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.net.HostAndPort;
+
 
 /**
  * The local subcluster (where the writer is) should be tried first. The writer
  * is defined from the RPC query received in the RPC server.
  */
-public class LocalResolver implements OrderedResolver {
+public class LocalResolver extends RouterResolver<String, String> {
 
   private static final Logger LOG =
       LoggerFactory.getLogger(LocalResolver.class);
 
-  /** Configuration key to set the minimum time to update the local cache.*/
-  public static final String MIN_UPDATE_PERIOD_KEY =
-      RBFConfigKeys.FEDERATION_ROUTER_PREFIX + "local-resolver.update-period";
-  /** 10 seconds by default. */
-  private static final long MIN_UPDATE_PERIOD_DEFAULT =
-      TimeUnit.SECONDS.toMillis(10);
-
-
-  /** Router service. */
-  private final Router router;
-  /** Minimum update time. */
-  private final long minUpdateTime;
+  public LocalResolver(final Configuration conf, final Router routerService) {
+    super(conf, routerService);
+  }
 
-  /** Node IP -> Subcluster. */
-  private Map<String, String> nodeSubcluster = null;
-  /** Last time the subcluster map was updated. */
-  private long lastUpdated;
+  /**
+   * Get the mapping from nodes to subcluster. It gets this mapping from the
+   * subclusters through expensive calls (e.g., RPC) and uses caching to avoid
+   * too many calls. The cache might be updated asynchronously to reduce
+   * latency.
+   *
+   * @return Node IP -> Subcluster.
+   */
+  @Override
+  protected Map<String, String> getSubclusterInfo(
+      MembershipStore membershipStore) {
+    Map<String, String> mapping = new HashMap<>();
 
+    Map<String, String> dnSubcluster = getDatanodesSubcluster();
+    if (dnSubcluster != null) {
+      mapping.putAll(dnSubcluster);
+    }
 
-  public LocalResolver(final Configuration conf, final Router routerService) {
-    this.minUpdateTime = conf.getTimeDuration(
-        MIN_UPDATE_PERIOD_KEY, MIN_UPDATE_PERIOD_DEFAULT,
-        TimeUnit.MILLISECONDS);
-    this.router = routerService;
+    Map<String, String> nnSubcluster = getNamenodesSubcluster(membershipStore);
+    if (nnSubcluster != null) {
+      mapping.putAll(nnSubcluster);
+    }
+    return mapping;
   }
 
   /**
@@ -98,12 +96,12 @@ public class LocalResolver implements OrderedResolver {
    * @return Local name space. Null if we don't know about this machine.
    */
   @Override
-  public String getFirstNamespace(final String path, final PathLocation loc) {
+  protected String chooseFirstNamespace(String path, PathLocation loc) {
     String localSubcluster = null;
     String clientAddr = getClientAddr();
-    Map<String, String> nodeToSubcluster = getSubclusterMappings();
-    if (nodeToSubcluster != null) {
-      localSubcluster = nodeToSubcluster.get(clientAddr);
+    Map<String, String> subclusterInfo = getSubclusterMapping();
+    if (subclusterInfo != null) {
+      localSubcluster = subclusterInfo.get(clientAddr);
       if (localSubcluster != null) {
         LOG.debug("Local namespace for {} is {}", clientAddr, localSubcluster);
       } else {
@@ -121,52 +119,6 @@ public class LocalResolver implements OrderedResolver {
     return Server.getRemoteAddress();
   }
 
-  /**
-   * Get the mapping from nodes to subcluster. It gets this mapping from the
-   * subclusters through expensive calls (e.g., RPC) and uses caching to avoid
-   * too many calls. The cache might be updated asynchronously to reduce
-   * latency.
-   *
-   * @return Node IP -> Subcluster.
-   */
-  @VisibleForTesting
-  synchronized Map<String, String> getSubclusterMappings() {
-    if (nodeSubcluster == null ||
-        (monotonicNow() - lastUpdated) > minUpdateTime) {
-      // Fetch the mapping asynchronously
-      Thread updater = new Thread(new Runnable() {
-        @Override
-        public void run() {
-          Map<String, String> mapping = new HashMap<>();
-
-          Map<String, String> dnSubcluster = getDatanodesSubcluster();
-          if (dnSubcluster != null) {
-            mapping.putAll(dnSubcluster);
-          }
-
-          Map<String, String> nnSubcluster = getNamenodesSubcluster();
-          if (nnSubcluster != null) {
-            mapping.putAll(nnSubcluster);
-          }
-          nodeSubcluster = mapping;
-          lastUpdated = monotonicNow();
-        }
-      });
-      updater.start();
-
-      // Wait until initialized
-      if (nodeSubcluster == null) {
-        try {
-          LOG.debug("Wait to get the mapping for the first time");
-          updater.join();
-        } catch (InterruptedException e) {
-          LOG.error("Cannot wait for the updater to finish");
-        }
-      }
-    }
-    return nodeSubcluster;
-  }
-
   /**
    * Get the Datanode mapping from the subclusters from the Namenodes. This
    * needs to be done as a privileged action to use the user for the Router and
@@ -221,14 +173,8 @@ public class LocalResolver implements OrderedResolver {
    *
    * @return NN IP -> Subcluster.
    */
-  private Map<String, String> getNamenodesSubcluster() {
-
-    final MembershipStore membershipStore = getMembershipStore();
-    if (membershipStore == null) {
-      LOG.error("Cannot access the Membership store");
-      return null;
-    }
-
+  private Map<String, String> getNamenodesSubcluster(
+      MembershipStore membershipStore) {
     // Manage requests from this hostname (127.0.0.1)
     String localIp = "127.0.0.1";
     String localHostname = localIp;
@@ -269,29 +215,4 @@ public class LocalResolver implements OrderedResolver {
     }
     return ret;
   }
-
-  /**
-   * Get the Router RPC server.
-   *
-   * @return Router RPC server. Null if not possible.
-   */
-  private RouterRpcServer getRpcServer() {
-    if (this.router == null) {
-      return null;
-    }
-    return router.getRpcServer();
-  }
-
-  /**
-   * Get the Membership store.
-   *
-   * @return Membership store.
-   */
-  private MembershipStore getMembershipStore() {
-    StateStoreService stateStore = router.getStateStore();
-    if (stateStore == null) {
-      return null;
-    }
-    return stateStore.getRegisteredRecordStore(MembershipStore.class);
-  }
 }

+ 160 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/order/RouterResolver.java

@@ -0,0 +1,160 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.federation.resolver.order;
+
+import static org.apache.hadoop.util.Time.monotonicNow;
+
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.federation.resolver.PathLocation;
+import org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.router.Router;
+import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer;
+import org.apache.hadoop.hdfs.server.federation.store.MembershipStore;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The order resolver that depends upon the Router service.
+ *
+ * @param <K> The key type of subcluster mapping info queried from Router.
+ * @param <V> The value type of subcluster mapping info queried from Router.
+ */
+public abstract class RouterResolver<K, V> implements OrderedResolver {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RouterResolver.class);
+
+  /** Configuration key to set the minimum time to update subcluster info. */
+  public static final String MIN_UPDATE_PERIOD_KEY =
+      RBFConfigKeys.FEDERATION_ROUTER_PREFIX + "router-resolver.update-period";
+  /** 10 seconds by default. */
+  private static final long MIN_UPDATE_PERIOD_DEFAULT = TimeUnit.SECONDS
+      .toMillis(10);
+
+  /** Router service. */
+  private final Router router;
+  /** Minimum update time. */
+  private final long minUpdateTime;
+
+  /** K -> T template mapping. */
+  private Map<K, V> subclusterMapping = null;
+  /** Last time the subcluster mapping was updated. */
+  private long lastUpdated;
+
+  public RouterResolver(final Configuration conf, final Router routerService) {
+    this.minUpdateTime = conf.getTimeDuration(MIN_UPDATE_PERIOD_KEY,
+        MIN_UPDATE_PERIOD_DEFAULT, TimeUnit.MILLISECONDS);
+    this.router = routerService;
+  }
+
+  @Override
+  public String getFirstNamespace(String path, PathLocation loc) {
+    updateSubclusterMapping();
+    return chooseFirstNamespace(path, loc);
+  }
+
+  /**
+   * The implementation for getting desired subcluster mapping info.
+   *
+   * @param membershipStore Membership store the resolver queried from.
+   * @return The map of desired type info.
+   */
+  protected abstract Map<K, V> getSubclusterInfo(
+      MembershipStore membershipStore);
+
+  /**
+   * Choose the first namespace from queried subcluster mapping info.
+   *
+   * @param path Path to check.
+   * @param loc Federated location with multiple destinations.
+   * @return First namespace out of the locations.
+   */
+  protected abstract String chooseFirstNamespace(String path, PathLocation loc);
+
+  /**
+   * Update <NamespaceId, Subcluster Info> mapping info periodically.
+   */
+  private synchronized void updateSubclusterMapping() {
+    if (subclusterMapping == null
+        || (monotonicNow() - lastUpdated) > minUpdateTime) {
+      // Fetch the mapping asynchronously
+      Thread updater = new Thread(new Runnable() {
+        @Override
+        public void run() {
+          final MembershipStore membershipStore = getMembershipStore();
+          if (membershipStore == null) {
+            LOG.error("Cannot access the Membership store.");
+            return;
+          }
+
+          subclusterMapping = getSubclusterInfo(membershipStore);
+          lastUpdated = monotonicNow();
+        }
+      });
+      updater.start();
+
+      // Wait until initialized
+      if (subclusterMapping == null) {
+        try {
+          LOG.debug("Wait to get the mapping for the first time");
+          updater.join();
+        } catch (InterruptedException e) {
+          LOG.error("Cannot wait for the updater to finish");
+        }
+      }
+    }
+  }
+
+  /**
+   * Get the Router RPC server.
+   *
+   * @return Router RPC server. Null if not possible.
+   */
+  protected RouterRpcServer getRpcServer() {
+    if (this.router == null) {
+      return null;
+    }
+    return router.getRpcServer();
+  }
+
+  /**
+   * Get the Membership store.
+   *
+   * @return Membership store.
+   */
+  protected MembershipStore getMembershipStore() {
+    StateStoreService stateStore = router.getStateStore();
+    if (stateStore == null) {
+      return null;
+    }
+    return stateStore.getRegisteredRecordStore(MembershipStore.class);
+  }
+
+  /**
+   * Get subcluster mapping info.
+   *
+   * @return The map of subcluster info.
+   */
+  protected Map<K, V> getSubclusterMapping() {
+    return this.subclusterMapping;
+  }
+}

+ 13 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java

@@ -34,6 +34,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.server.federation.metrics.FederationMetrics;
+import org.apache.hadoop.hdfs.server.federation.metrics.NamenodeBeanMetrics;
 import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
 import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
 import org.apache.hadoop.hdfs.server.federation.store.RouterStore;
@@ -552,6 +553,18 @@ public class Router extends CompositeService {
     return null;
   }
 
+  /**
+   * Get the Namenode metrics.
+   *
+   * @return Namenode metrics.
+   */
+  public NamenodeBeanMetrics getNamenodeMetrics() {
+    if (this.metrics != null) {
+      return this.metrics.getNamenodeMetrics();
+    }
+    return null;
+  }
+
   /**
    * Get the subcluster resolver for files.
    *

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterHttpServer.java

@@ -23,6 +23,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeHttpServer;
 import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.service.AbstractService;
 
@@ -86,6 +87,9 @@ public class RouterHttpServer extends AbstractService {
 
     this.httpServer = builder.build();
 
+    NameNodeHttpServer.initWebHdfs(conf, httpAddress.getHostName(), httpServer,
+        RouterWebHdfsMethods.class.getPackage().getName());
+
     this.httpServer.setAttribute(NAMENODE_ATTRIBUTE_KEY, this.router);
     this.httpServer.setAttribute(JspHelper.CURRENT_CONF, this.conf);
     setupServlets(this.httpServer, this.conf);

+ 9 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterMetricsService.java

@@ -94,6 +94,15 @@ public class RouterMetricsService extends AbstractService {
     return this.federationMetrics;
   }
 
+  /**
+   * Get the Namenode metrics.
+   *
+   * @return Namenode metrics.
+   */
+  public NamenodeBeanMetrics getNamenodeMetrics() {
+    return this.nnMetrics;
+  }
+
   /**
    * Get the JVM metrics for the Router.
    *

+ 15 - 26
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java

@@ -534,31 +534,20 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
   }
 
   /**
-   * Get the permissions for the parent of a child with given permissions. If
-   * the child has r--, we will set it to r-x.
+   * Get the permissions for the parent of a child with given permissions.
+   * Add implicit u+wx permission for parent. This is based on
+   * @{FSDirMkdirOp#addImplicitUwx}.
    * @param mask The permission mask of the child.
    * @return The permission mask of the parent.
    */
   private static FsPermission getParentPermission(final FsPermission mask) {
     FsPermission ret = new FsPermission(
-        applyExecute(mask.getUserAction()),
-        applyExecute(mask.getGroupAction()),
-        applyExecute(mask.getOtherAction()));
+        mask.getUserAction().or(FsAction.WRITE_EXECUTE),
+        mask.getGroupAction(),
+        mask.getOtherAction());
     return ret;
   }
 
-  /**
-   * Apply the execute permissions if it can be read.
-   * @param action Input permission.
-   * @return Output permission.
-   */
-  private static FsAction applyExecute(final FsAction action) {
-    if (action.and(FsAction.READ) == FsAction.READ) {
-      return action.or(FsAction.EXECUTE);
-    }
-    return action;
-  }
-
   /**
    * Get the location to create a file. It checks if the file already existed
    * in one of the locations.
@@ -567,7 +556,7 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
    * @return The remote location for this file.
    * @throws IOException If the file has no creation location.
    */
-  private RemoteLocation getCreateLocation(final String src)
+  protected RemoteLocation getCreateLocation(final String src)
       throws IOException {
 
     final List<RemoteLocation> locations = getLocationsForPath(src, true);
@@ -1248,18 +1237,20 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
   public DatanodeInfo[] getDatanodeReport(DatanodeReportType type)
       throws IOException {
     checkOperation(OperationCategory.UNCHECKED);
-    return getDatanodeReport(type, 0);
+    return getDatanodeReport(type, true, 0);
   }
 
   /**
    * Get the datanode report with a timeout.
    * @param type Type of the datanode.
+   * @param requireResponse If we require all the namespaces to report.
    * @param timeOutMs Time out for the reply in milliseconds.
    * @return List of datanodes.
    * @throws IOException If it cannot get the report.
    */
   public DatanodeInfo[] getDatanodeReport(
-      DatanodeReportType type, long timeOutMs) throws IOException {
+      DatanodeReportType type, boolean requireResponse, long timeOutMs)
+          throws IOException {
     checkOperation(OperationCategory.UNCHECKED);
 
     Map<String, DatanodeInfo> datanodesMap = new LinkedHashMap<>();
@@ -1268,8 +1259,8 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
 
     Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
     Map<FederationNamespaceInfo, DatanodeInfo[]> results =
-        rpcClient.invokeConcurrent(
-            nss, method, true, false, timeOutMs, DatanodeInfo[].class);
+        rpcClient.invokeConcurrent(nss, method, requireResponse, false,
+            timeOutMs, DatanodeInfo[].class);
     for (Entry<FederationNamespaceInfo, DatanodeInfo[]> entry :
         results.entrySet()) {
       FederationNamespaceInfo ns = entry.getKey();
@@ -1289,9 +1280,7 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     }
     // Map -> Array
     Collection<DatanodeInfo> datanodes = datanodesMap.values();
-    DatanodeInfo[] combinedData = new DatanodeInfo[datanodes.size()];
-    combinedData = datanodes.toArray(combinedData);
-    return combinedData;
+    return toArray(datanodes, DatanodeInfo.class);
   }
 
   @Override // ClientProtocol
@@ -2306,7 +2295,7 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
    * @param clazz Class of the values.
    * @return Array with the values in set.
    */
-  private static <T> T[] toArray(Set<T> set, Class<T> clazz) {
+  private static <T> T[] toArray(Collection<T> set, Class<T> clazz) {
     @SuppressWarnings("unchecked")
     T[] combinedData = (T[]) Array.newInstance(clazz, set.size());
     combinedData = set.toArray(combinedData);

+ 655 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterWebHdfsMethods.java

@@ -0,0 +1,655 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.federation.router;
+
+import static org.apache.hadoop.util.StringUtils.getTrimmedStringCollection;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.server.common.JspHelper;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeContext;
+import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
+import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.Path;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+
+import com.sun.jersey.spi.container.ResourceFilters;
+import org.apache.hadoop.hdfs.web.JsonUtil;
+import org.apache.hadoop.hdfs.web.ParamFilter;
+import org.apache.hadoop.hdfs.web.URLConnectionFactory;
+import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
+import org.apache.hadoop.hdfs.web.resources.AccessTimeParam;
+import org.apache.hadoop.hdfs.web.resources.AclPermissionParam;
+import org.apache.hadoop.hdfs.web.resources.BlockSizeParam;
+import org.apache.hadoop.hdfs.web.resources.BufferSizeParam;
+import org.apache.hadoop.hdfs.web.resources.ConcatSourcesParam;
+import org.apache.hadoop.hdfs.web.resources.CreateFlagParam;
+import org.apache.hadoop.hdfs.web.resources.CreateParentParam;
+import org.apache.hadoop.hdfs.web.resources.DelegationParam;
+import org.apache.hadoop.hdfs.web.resources.DestinationParam;
+import org.apache.hadoop.hdfs.web.resources.DoAsParam;
+import org.apache.hadoop.hdfs.web.resources.ExcludeDatanodesParam;
+import org.apache.hadoop.hdfs.web.resources.FsActionParam;
+import org.apache.hadoop.hdfs.web.resources.GetOpParam;
+import org.apache.hadoop.hdfs.web.resources.GroupParam;
+import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
+import org.apache.hadoop.hdfs.web.resources.LengthParam;
+import org.apache.hadoop.hdfs.web.resources.ModificationTimeParam;
+import org.apache.hadoop.hdfs.web.resources.NewLengthParam;
+import org.apache.hadoop.hdfs.web.resources.NoRedirectParam;
+import org.apache.hadoop.hdfs.web.resources.OffsetParam;
+import org.apache.hadoop.hdfs.web.resources.OldSnapshotNameParam;
+import org.apache.hadoop.hdfs.web.resources.OverwriteParam;
+import org.apache.hadoop.hdfs.web.resources.OwnerParam;
+import org.apache.hadoop.hdfs.web.resources.Param;
+import org.apache.hadoop.hdfs.web.resources.PermissionParam;
+import org.apache.hadoop.hdfs.web.resources.PostOpParam;
+import org.apache.hadoop.hdfs.web.resources.PutOpParam;
+import org.apache.hadoop.hdfs.web.resources.RenameOptionSetParam;
+import org.apache.hadoop.hdfs.web.resources.RenewerParam;
+import org.apache.hadoop.hdfs.web.resources.ReplicationParam;
+import org.apache.hadoop.hdfs.web.resources.SnapshotNameParam;
+import org.apache.hadoop.hdfs.web.resources.StartAfterParam;
+import org.apache.hadoop.hdfs.web.resources.StoragePolicyParam;
+import org.apache.hadoop.hdfs.web.resources.TokenArgumentParam;
+import org.apache.hadoop.hdfs.web.resources.TokenKindParam;
+import org.apache.hadoop.hdfs.web.resources.TokenServiceParam;
+import org.apache.hadoop.hdfs.web.resources.UnmaskedPermissionParam;
+import org.apache.hadoop.hdfs.web.resources.UriFsPathParam;
+import org.apache.hadoop.hdfs.web.resources.UserParam;
+import org.apache.hadoop.hdfs.web.resources.XAttrEncodingParam;
+import org.apache.hadoop.hdfs.web.resources.XAttrNameParam;
+import org.apache.hadoop.hdfs.web.resources.XAttrSetFlagParam;
+import org.apache.hadoop.hdfs.web.resources.XAttrValueParam;
+import org.apache.hadoop.ipc.ExternalCall;
+import org.apache.hadoop.ipc.RetriableException;
+import org.apache.hadoop.net.Node;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.net.URLDecoder;
+import java.security.PrivilegedAction;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Random;
+
+/**
+ * WebHDFS Router implementation. This is an extension of
+ * {@link NamenodeWebHdfsMethods}, and tries to reuse as much as possible.
+ */
+@Path("")
+@ResourceFilters(ParamFilter.class)
+public class RouterWebHdfsMethods extends NamenodeWebHdfsMethods {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RouterWebHdfsMethods.class);
+
+  private static final ThreadLocal<String> REMOTE_ADDRESS =
+      new ThreadLocal<String>();
+
+  private @Context HttpServletRequest request;
+  private String method;
+  private String query;
+  private String reqPath;
+
+  public RouterWebHdfsMethods(@Context HttpServletRequest request) {
+    super(request);
+    this.method = request.getMethod();
+    this.query = request.getQueryString();
+    this.reqPath = request.getServletPath();
+    REMOTE_ADDRESS.set(JspHelper.getRemoteAddr(request));
+  }
+
+  @Override
+  protected void init(final UserGroupInformation ugi,
+                      final DelegationParam delegation,
+                      final UserParam username, final DoAsParam doAsUser,
+                      final UriFsPathParam path, final HttpOpParam<?> op,
+                      final Param<?, ?>... parameters) {
+    super.init(ugi, delegation, username, doAsUser, path, op, parameters);
+
+    REMOTE_ADDRESS.set(JspHelper.getRemoteAddr(request));
+  }
+
+  @Override
+  protected ClientProtocol getRpcClientProtocol() throws IOException {
+    final Router router = getRouter();
+    final RouterRpcServer routerRpcServer = router.getRpcServer();
+    if (routerRpcServer == null) {
+      throw new RetriableException("Router is in startup mode");
+    }
+    return routerRpcServer;
+  }
+
+  private void reset() {
+    REMOTE_ADDRESS.set(null);
+  }
+
+  @Override
+  protected String getRemoteAddr() {
+    return REMOTE_ADDRESS.get();
+  }
+
+  @Override
+  protected void queueExternalCall(ExternalCall call)
+      throws IOException, InterruptedException {
+    getRouter().getRpcServer().getServer().queueCall(call);
+  }
+
+  private Router getRouter() {
+    return (Router)getContext().getAttribute("name.node");
+  }
+
+  private static RouterRpcServer getRPCServer(final Router router)
+      throws IOException {
+    final RouterRpcServer routerRpcServer = router.getRpcServer();
+    if (routerRpcServer == null) {
+      throw new RetriableException("Router is in startup mode");
+    }
+    return routerRpcServer;
+  }
+
+  @Override
+  protected Response put(
+      final UserGroupInformation ugi,
+      final DelegationParam delegation,
+      final UserParam username,
+      final DoAsParam doAsUser,
+      final String fullpath,
+      final PutOpParam op,
+      final DestinationParam destination,
+      final OwnerParam owner,
+      final GroupParam group,
+      final PermissionParam permission,
+      final UnmaskedPermissionParam unmaskedPermission,
+      final OverwriteParam overwrite,
+      final BufferSizeParam bufferSize,
+      final ReplicationParam replication,
+      final BlockSizeParam blockSize,
+      final ModificationTimeParam modificationTime,
+      final AccessTimeParam accessTime,
+      final RenameOptionSetParam renameOptions,
+      final CreateParentParam createParent,
+      final TokenArgumentParam delegationTokenArgument,
+      final AclPermissionParam aclPermission,
+      final XAttrNameParam xattrName,
+      final XAttrValueParam xattrValue,
+      final XAttrSetFlagParam xattrSetFlag,
+      final SnapshotNameParam snapshotName,
+      final OldSnapshotNameParam oldSnapshotName,
+      final ExcludeDatanodesParam exclDatanodes,
+      final CreateFlagParam createFlagParam,
+      final NoRedirectParam noredirectParam,
+      final StoragePolicyParam policyName
+  ) throws IOException, URISyntaxException {
+
+    switch(op.getValue()) {
+    case CREATE:
+    {
+      final Router router = getRouter();
+      final URI uri = redirectURI(router, fullpath);
+      if (!noredirectParam.getValue()) {
+        return Response.temporaryRedirect(uri)
+            .type(MediaType.APPLICATION_OCTET_STREAM).build();
+      } else {
+        final String js = JsonUtil.toJsonString("Location", uri);
+        return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+      }
+    }
+    case MKDIRS:
+    case CREATESYMLINK:
+    case RENAME:
+    case SETREPLICATION:
+    case SETOWNER:
+    case SETPERMISSION:
+    case SETTIMES:
+    case RENEWDELEGATIONTOKEN:
+    case CANCELDELEGATIONTOKEN:
+    case MODIFYACLENTRIES:
+    case REMOVEACLENTRIES:
+    case REMOVEDEFAULTACL:
+    case REMOVEACL:
+    case SETACL:
+    case SETXATTR:
+    case REMOVEXATTR:
+    case ALLOWSNAPSHOT:
+    case CREATESNAPSHOT:
+    case RENAMESNAPSHOT:
+    case DISALLOWSNAPSHOT:
+    case SETSTORAGEPOLICY:
+    {
+      // Whitelist operations that can handled by NamenodeWebHdfsMethods
+      return super.put(ugi, delegation, username, doAsUser, fullpath, op,
+          destination, owner, group, permission, unmaskedPermission,
+          overwrite, bufferSize, replication, blockSize, modificationTime,
+          accessTime, renameOptions, createParent, delegationTokenArgument,
+          aclPermission, xattrName, xattrValue, xattrSetFlag, snapshotName,
+          oldSnapshotName, exclDatanodes, createFlagParam, noredirectParam,
+          policyName);
+    }
+    default:
+      throw new UnsupportedOperationException(op + " is not supported");
+    }
+  }
+
+  @Override
+  protected Response post(
+      final UserGroupInformation ugi,
+      final DelegationParam delegation,
+      final UserParam username,
+      final DoAsParam doAsUser,
+      final String fullpath,
+      final PostOpParam op,
+      final ConcatSourcesParam concatSrcs,
+      final BufferSizeParam bufferSize,
+      final ExcludeDatanodesParam excludeDatanodes,
+      final NewLengthParam newLength,
+      final NoRedirectParam noRedirectParam
+  ) throws IOException, URISyntaxException {
+    switch(op.getValue()) {
+    case APPEND:
+    {
+      final Router router = getRouter();
+      final URI uri = redirectURI(router, ugi, delegation, username,
+          doAsUser, fullpath, op.getValue(), -1L,
+          excludeDatanodes.getValue(), bufferSize);
+      if (!noRedirectParam.getValue()) {
+        return Response.temporaryRedirect(uri)
+            .type(MediaType.APPLICATION_OCTET_STREAM).build();
+      } else {
+        final String js = JsonUtil.toJsonString("Location", uri);
+        return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+      }
+    }
+    case CONCAT:
+    case TRUNCATE:
+    case UNSETSTORAGEPOLICY:
+    {
+      return super.post(ugi, delegation, username, doAsUser, fullpath, op,
+          concatSrcs, bufferSize, excludeDatanodes, newLength,
+          noRedirectParam);
+    }
+    default:
+      throw new UnsupportedOperationException(op + " is not supported");
+    }
+  }
+
+  @Override
+  protected Response get(
+      final UserGroupInformation ugi,
+      final DelegationParam delegation,
+      final UserParam username,
+      final DoAsParam doAsUser,
+      final String fullpath,
+      final GetOpParam op,
+      final OffsetParam offset,
+      final LengthParam length,
+      final RenewerParam renewer,
+      final BufferSizeParam bufferSize,
+      final List<XAttrNameParam> xattrNames,
+      final XAttrEncodingParam xattrEncoding,
+      final ExcludeDatanodesParam excludeDatanodes,
+      final FsActionParam fsAction,
+      final SnapshotNameParam snapshotName,
+      final OldSnapshotNameParam oldSnapshotName,
+      final TokenKindParam tokenKind,
+      final TokenServiceParam tokenService,
+      final NoRedirectParam noredirectParam,
+      final StartAfterParam startAfter
+  ) throws IOException, URISyntaxException {
+    try {
+      final Router router = getRouter();
+
+      switch (op.getValue()) {
+      case OPEN:
+      {
+        final URI uri = redirectURI(router, ugi, delegation, username,
+            doAsUser, fullpath, op.getValue(), offset.getValue(),
+            excludeDatanodes.getValue(), offset, length, bufferSize);
+        if (!noredirectParam.getValue()) {
+          return Response.temporaryRedirect(uri)
+              .type(MediaType.APPLICATION_OCTET_STREAM).build();
+        } else {
+          final String js = JsonUtil.toJsonString("Location", uri);
+          return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+        }
+      }
+      case GETFILECHECKSUM:
+      {
+        final URI uri = redirectURI(router, ugi, delegation, username,
+            doAsUser, fullpath, op.getValue(), -1L, null);
+        if (!noredirectParam.getValue()) {
+          return Response.temporaryRedirect(uri)
+              .type(MediaType.APPLICATION_OCTET_STREAM).build();
+        } else {
+          final String js = JsonUtil.toJsonString("Location", uri);
+          return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+        }
+      }
+      case GET_BLOCK_LOCATIONS:
+      case GETFILESTATUS:
+      case LISTSTATUS:
+      case GETCONTENTSUMMARY:
+      case GETHOMEDIRECTORY:
+      case GETACLSTATUS:
+      case GETXATTRS:
+      case LISTXATTRS:
+      case CHECKACCESS:
+      {
+        return super.get(ugi, delegation, username, doAsUser, fullpath, op,
+            offset, length, renewer, bufferSize, xattrNames, xattrEncoding,
+            excludeDatanodes, fsAction, snapshotName, oldSnapshotName,
+            tokenKind, tokenService, noredirectParam, startAfter);
+      }
+      default:
+        throw new UnsupportedOperationException(op + " is not supported");
+      }
+    } finally {
+      reset();
+    }
+  }
+
+  /**
+   * Get the redirect URI from the Namenode responsible for a path.
+   * @param router Router to check.
+   * @param path Path to get location for.
+   * @return URI returned by the Namenode.
+   * @throws IOException If it cannot get the redirect URI.
+   */
+  private URI redirectURI(final Router router, final String path)
+      throws IOException {
+    // Forward the request to the proper Namenode
+    final HttpURLConnection conn = forwardRequest(router, path);
+    try {
+      conn.setInstanceFollowRedirects(false);
+      conn.setDoOutput(true);
+      conn.connect();
+
+      // Read the reply from the Namenode
+      int responseCode = conn.getResponseCode();
+      if (responseCode != HttpServletResponse.SC_TEMPORARY_REDIRECT) {
+        LOG.info("We expected a redirection from the Namenode, not {}",
+            responseCode);
+        return null;
+      }
+
+      // Extract the redirect location and return it
+      String redirectLocation = conn.getHeaderField("Location");
+      try {
+        // We modify the namenode location and the path
+        redirectLocation = redirectLocation
+            .replaceAll("(?<=[?&;])namenoderpcaddress=.*?(?=[&;])",
+                "namenoderpcaddress=" + router.getRouterId())
+            .replaceAll("(?<=[/])webhdfs/v1/.*?(?=[?])",
+                "webhdfs/v1" + path);
+        return new URI(redirectLocation);
+      } catch (URISyntaxException e) {
+        LOG.error("Cannot parse redirect location {}", redirectLocation);
+      }
+    } finally {
+      if (conn != null) {
+        conn.disconnect();
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Forwards a request to a subcluster.
+   * @param router Router to check.
+   * @param path Path in HDFS.
+   * @return Reply from the subcluster.
+   * @throws IOException
+   */
+  private HttpURLConnection forwardRequest(
+      final Router router, final String path) throws IOException {
+    final Configuration conf =
+        (Configuration)getContext().getAttribute(JspHelper.CURRENT_CONF);
+    URLConnectionFactory connectionFactory =
+        URLConnectionFactory.newDefaultURLConnectionFactory(conf);
+
+    // Find the namespace responsible for a path
+    final RouterRpcServer rpcServer = getRPCServer(router);
+    RemoteLocation createLoc = rpcServer.getCreateLocation(path);
+    String nsId = createLoc.getNameserviceId();
+    String dest = createLoc.getDest();
+    ActiveNamenodeResolver nnResolver = router.getNamenodeResolver();
+    List<? extends FederationNamenodeContext> namenodes =
+        nnResolver.getNamenodesForNameserviceId(nsId);
+
+    // Go over the namenodes responsible for that namespace
+    for (FederationNamenodeContext namenode : namenodes) {
+      try {
+        // Generate the request for the namenode
+        String nnWebAddress = namenode.getWebAddress();
+        String[] nnWebAddressSplit = nnWebAddress.split(":");
+        String host = nnWebAddressSplit[0];
+        int port = Integer.parseInt(nnWebAddressSplit[1]);
+
+        // Avoid double-encoding here
+        query = URLDecoder.decode(query, "UTF-8");
+        URI uri = new URI(getScheme(), null, host, port,
+            reqPath + dest, query, null);
+        URL url = uri.toURL();
+
+        // Send a request to the proper Namenode
+        final HttpURLConnection conn =
+            (HttpURLConnection)connectionFactory.openConnection(url);
+        conn.setRequestMethod(method);
+
+        return conn;
+      } catch (Exception e) {
+        LOG.error("Cannot redirect request to {}", namenode, e);
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Get a URI to redirect an operation to.
+   * @param router Router to check.
+   * @param ugi User group information.
+   * @param delegation Delegation token.
+   * @param username User name.
+   * @param doAsUser Do as user.
+   * @param path Path to check.
+   * @param op Operation to perform.
+   * @param openOffset Offset for opening a file.
+   * @param excludeDatanodes Blocks to excluded.
+   * @param parameters Other parameters.
+   * @return Redirection URI.
+   * @throws URISyntaxException If it cannot parse the URI.
+   * @throws IOException If it cannot create the URI.
+   */
+  private URI redirectURI(final Router router, final UserGroupInformation ugi,
+      final DelegationParam delegation, final UserParam username,
+      final DoAsParam doAsUser, final String path, final HttpOpParam.Op op,
+      final long openOffset, final String excludeDatanodes,
+      final Param<?, ?>... parameters) throws URISyntaxException, IOException {
+    final DatanodeInfo dn =
+        chooseDatanode(router, path, op, openOffset, excludeDatanodes);
+
+    if (dn == null) {
+      throw new IOException("Failed to find datanode, suggest to check cluster"
+          + " health. excludeDatanodes=" + excludeDatanodes);
+    }
+
+    final String delegationQuery;
+    if (!UserGroupInformation.isSecurityEnabled()) {
+      // security disabled
+      delegationQuery = Param.toSortedString("&", doAsUser, username);
+    } else if (delegation.getValue() != null) {
+      // client has provided a token
+      delegationQuery = "&" + delegation;
+    } else {
+      // generate a token
+      final Token<? extends TokenIdentifier> t = generateDelegationToken(
+          router, ugi, request.getUserPrincipal().getName());
+      delegationQuery = "&delegation=" + t.encodeToUrlString();
+    }
+
+    final String redirectQuery = op.toQueryString() + delegationQuery
+        + "&namenoderpcaddress=" + router.getRouterId()
+        + Param.toSortedString("&", parameters);
+    final String uripath = WebHdfsFileSystem.PATH_PREFIX + path;
+
+    int port = "http".equals(getScheme()) ? dn.getInfoPort() :
+        dn.getInfoSecurePort();
+    final URI uri = new URI(getScheme(), null, dn.getHostName(), port, uripath,
+        redirectQuery, null);
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("redirectURI={}", uri);
+    }
+    return uri;
+  }
+
+  private DatanodeInfo chooseDatanode(final Router router,
+      final String path, final HttpOpParam.Op op, final long openOffset,
+      final String excludeDatanodes) throws IOException {
+    // We need to get the DNs as a privileged user
+    final RouterRpcServer rpcServer = getRPCServer(router);
+    UserGroupInformation loginUser = UserGroupInformation.getLoginUser();
+
+    DatanodeInfo[] dns = loginUser.doAs(
+        new PrivilegedAction<DatanodeInfo[]>() {
+          @Override
+          public DatanodeInfo[] run() {
+            try {
+              return rpcServer.getDatanodeReport(DatanodeReportType.LIVE);
+            } catch (IOException e) {
+              LOG.error("Cannot get the datanodes from the RPC server", e);
+              return null;
+            }
+          }
+        });
+
+    HashSet<Node> excludes = new HashSet<Node>();
+    if (excludeDatanodes != null) {
+      Collection<String> collection =
+          getTrimmedStringCollection(excludeDatanodes);
+      for (DatanodeInfo dn : dns) {
+        if (collection.contains(dn.getName())) {
+          excludes.add(dn);
+        }
+      }
+    }
+
+    if (op == GetOpParam.Op.OPEN ||
+        op == PostOpParam.Op.APPEND ||
+        op == GetOpParam.Op.GETFILECHECKSUM) {
+      // Choose a datanode containing a replica
+      final ClientProtocol cp = getRpcClientProtocol();
+      final HdfsFileStatus status = cp.getFileInfo(path);
+      if (status == null) {
+        throw new FileNotFoundException("File " + path + " not found.");
+      }
+      final long len = status.getLen();
+      if (op == GetOpParam.Op.OPEN) {
+        if (openOffset < 0L || (openOffset >= len && len > 0)) {
+          throw new IOException("Offset=" + openOffset
+              + " out of the range [0, " + len + "); " + op + ", path=" + path);
+        }
+      }
+
+      if (len > 0) {
+        final long offset = op == GetOpParam.Op.OPEN ? openOffset : len - 1;
+        final LocatedBlocks locations = cp.getBlockLocations(path, offset, 1);
+        final int count = locations.locatedBlockCount();
+        if (count > 0) {
+          LocatedBlock location0 = locations.get(0);
+          return bestNode(location0.getLocations(), excludes);
+        }
+      }
+    }
+
+    return getRandomDatanode(dns, excludes);
+  }
+
+  /**
+   * Get a random Datanode from a subcluster.
+   * @param dns Nodes to be chosen from.
+   * @param excludes Nodes to be excluded from.
+   * @return Random datanode from a particular subluster.
+   */
+  private static DatanodeInfo getRandomDatanode(
+      final DatanodeInfo[] dns, final HashSet<Node> excludes) {
+    DatanodeInfo dn = null;
+
+    if (dns == null) {
+      return dn;
+    }
+
+    int numDNs = dns.length;
+    int availableNodes = 0;
+    if (excludes.isEmpty()) {
+      availableNodes = numDNs;
+    } else {
+      for (DatanodeInfo di : dns) {
+        if (!excludes.contains(di)) {
+          availableNodes++;
+        }
+      }
+    }
+
+    // Return a random one from the list
+    if (availableNodes > 0) {
+      while (dn == null || excludes.contains(dn)) {
+        Random rnd = new Random();
+        int idx = rnd.nextInt(numDNs);
+        dn = dns[idx];
+      }
+    }
+    return dn;
+  }
+
+  /**
+   * Generate the delegation tokens for this request.
+   * @param router Router.
+   * @param ugi User group information.
+   * @param renewer Who is asking for the renewal.
+   * @return The delegation tokens.
+   * @throws IOException If it cannot create the tokens.
+   */
+  private Token<? extends TokenIdentifier> generateDelegationToken(
+      final Router router, final UserGroupInformation ugi,
+      final String renewer) throws IOException {
+    throw new UnsupportedOperationException("TODO Generate token for ugi=" +
+        ugi + " request=" + request);
+  }
+}

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MountTable.java

@@ -421,7 +421,8 @@ public abstract class MountTable extends BaseRecord {
   public boolean isAll() {
     DestinationOrder order = getDestOrder();
     return order == DestinationOrder.HASH_ALL ||
-        order == DestinationOrder.RANDOM;
+        order == DestinationOrder.RANDOM ||
+        order == DestinationOrder.SPACE;
   }
 
   /**

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MountTablePBImpl.java

@@ -297,6 +297,8 @@ public class MountTablePBImpl extends MountTable implements PBRecord {
       return DestinationOrder.RANDOM;
     case HASH_ALL:
       return DestinationOrder.HASH_ALL;
+    case SPACE:
+      return DestinationOrder.SPACE;
     default:
       return DestinationOrder.HASH;
     }
@@ -310,6 +312,8 @@ public class MountTablePBImpl extends MountTable implements PBRecord {
       return DestOrder.RANDOM;
     case HASH_ALL:
       return DestOrder.HASH_ALL;
+    case SPACE:
+      return DestOrder.SPACE;
     default:
       return DestOrder.HASH;
     }

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/proto/FederationProtocol.proto

@@ -130,6 +130,7 @@ message MountTableRecordProto {
     LOCAL = 1;
     RANDOM = 2;
     HASH_ALL = 3;
+    SPACE = 4;
   }
   optional DestOrder destOrder = 6 [default = HASH];
 

+ 21 - 21
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/webapps/router/federationhealth.html

@@ -21,7 +21,7 @@
 <meta http-equiv="X-UA-Compatible" content="IE=9" />
 <link rel="stylesheet" type="text/css" href="/static/bootstrap-3.0.2/css/bootstrap.min.css" />
 <link rel="stylesheet" type="text/css" href="/static/dataTables.bootstrap.css" />
-<link rel="stylesheet" type="text/css" href="/static/hadoop.css" />
+<link rel="stylesheet" type="text/css" href="/static/rbf.css" />
 <title>Router Information</title>
 </head>
 <body>
@@ -115,11 +115,11 @@
 <script type="text/x-dust-template" id="tmpl-namenode">
 <div class="page-header"><h1>Nameservice Information</h1></div>
 <div>
-  <ul class="dfshealth-node-legend">
-    <li class="dfshealth-node-icon dfshealth-node-alive">Active</li>
-    <li class="dfshealth-node-icon dfshealth-node-down-decommissioned">Standby</li>
-    <li class="dfshealth-node-icon dfshealth-node-down-maintenance">Safe mode</li>
-    <li class="dfshealth-node-icon dfshealth-node-down">Unavailable</li>
+  <ul class="federationhealth-namenode-legend">
+    <li class="federationhealth-namenode-icon federationhealth-namenode-active">Active</li>
+    <li class="federationhealth-namenode-icon federationhealth-namenode-standby">Standby</li>
+    <li class="federationhealth-namenode-icon federationhealth-namenode-safemode">Safe mode</li>
+    <li class="federationhealth-namenode-icon federationhealth-namenode-unavailable">Unavailable</li>
   </ul>
 </div>
 <small>
@@ -151,14 +151,14 @@
   <tbody>
     {#Nameservices}
     <tr>
-      <td class="dfshealth-node-icon dfshealth-node-{iconState}" title="{title}"></td>
+      <td class="federationhealth-namenode-icon federationhealth-namenode-{iconState}" title="{title}"></td>
       <td><a href="http://{webAddress}">{nameserviceId}</a></td>
       <td><a href="http://{webAddress}">{namenodeId}</a></td>
       <td>{lastHeartbeat}</td>
       <td ng-value="{usedPercentage}" style="width:210px">
         <div>
           <div style="display:inline-block; float: left; padding-right: 10px; width:60px;">{totalSpace|fmt_bytes}</div>
-          <div class="clearfix progress dfshealth-node-capacity-bar" title="Used: {used|fmt_bytes}">
+          <div class="clearfix progress federationhealth-namenode-capacity-bar" title="Used: {used|fmt_bytes}">
             <div class="progress-bar {#helper_usage_bar value="{usedPercentage}"/}" style="width: {usedPercentage}%">
             </div>
           </div>
@@ -181,11 +181,11 @@
 
 <div class="page-header"><h1>Namenode Information</h1></div>
 <div>
-  <ul class="dfshealth-node-legend">
-    <li class="dfshealth-node-icon dfshealth-node-alive">Active</li>
-    <li class="dfshealth-node-icon dfshealth-node-down-decommissioned">Standby</li>
-    <li class="dfshealth-node-icon dfshealth-node-down-maintenance">Safe mode</li>
-    <li class="dfshealth-node-icon dfshealth-node-down">Unavailable</li>
+  <ul class="federationhealth-namenode-legend">
+    <li class="federationhealth-namenode-icon federationhealth-namenode-active">Active</li>
+    <li class="federationhealth-namenode-icon federationhealth-namenode-standby">Standby</li>
+    <li class="federationhealth-namenode-icon federationhealth-namenode-safemode">Safe mode</li>
+    <li class="federationhealth-namenode-icon federationhealth-namenode-unavailable">Unavailable</li>
   </ul>
 </div>
 <small>
@@ -217,7 +217,7 @@
   <tbody>
     {#Namenodes}
     <tr>
-      <td class="dfshealth-node-icon dfshealth-node-{iconState}" title="{title}"></td>
+      <td class="federationhealth-namenode-icon federationhealth-namenode-{iconState}" title="{title}"></td>
       <td>{nameserviceId}</td>
       <td>{namenodeId}</td>
       <td><a href="http://{webAddress}">{webAddress}</a></td>
@@ -225,7 +225,7 @@
       <td ng-value="{usedPercentage}" style="width:210px">
         <div>
           <div style="display:inline-block; float: left; padding-right: 10px; width:60px;">{totalSpace|fmt_bytes}</div>
-          <div class="clearfix progress dfshealth-node-capacity-bar" title="Used: {used|fmt_bytes}">
+          <div class="clearfix progress federationhealth-namenode-capacity-bar" title="Used: {used|fmt_bytes}">
             <div class="progress-bar {#helper_usage_bar value="{usedPercentage}"/}" style="width: {usedPercentage}%">
             </div>
           </div>
@@ -251,10 +251,10 @@
 <script type="text/x-dust-template" id="tmpl-router">
 <div class="page-header"><h1>Routers Information</h1></div>
 <div>
-  <ul class="dfshealth-node-legend">
-    <li class="dfshealth-node-icon dfshealth-node-alive">Active</li>
-    <li class="dfshealth-node-icon dfshealth-node-decommissioned">Safe mode</li>
-    <li class="dfshealth-node-icon dfshealth-node-down">Unavailable</li>
+  <ul class="federationhealth-router-legend">
+    <li class="federationhealth-router-icon federationhealth-router-active">Active</li>
+    <li class="federationhealth-router-icon federationhealth-router-safemode">Safe mode</li>
+    <li class="federationhealth-router-icon federationhealth-router-unavailable">Unavailable</li>
   </ul>
 </div>
 <table class="table">
@@ -277,7 +277,7 @@
   <tbody>
     {#Routers}
     <tr>
-      <td class="dfshealth-node-icon dfshealth-node-{iconState}" title="{title}"></td>
+      <td class="federationhealth-router-icon federationhealth-router-{iconState}" title="{title}"></td>
       <td>{address}</td>
       <td>{status}</td>
       <td>{lastHeartbeat} sec ago</td>
@@ -395,7 +395,7 @@
       <td>{nameserviceId}</td>
       <td>{path}</td>
       <td>{order}</td>
-      <td class="dfshealth-node-icon dfshealth-mount-read-only-{readonly}"/>
+      <td class="mount-table-icon mount-table-read-only-{readonly}"/>
       <td>{ownerName}</td>
       <td>{groupName}</td>
       <td>{mode}</td>

+ 23 - 12
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/webapps/router/federationhealth.js

@@ -114,19 +114,19 @@
           var n = nodes[i];
           n.usedPercentage = Math.round(n.used * 1.0 / n.totalSpace * 100);
           n.title = "Unavailable";
-          n.iconState = "down";
+          n.iconState = "unavailable";
           if (n.isSafeMode === true) {
             n.title = capitalise(n.state) + " (safe mode)"
-            n.iconState = "decommissioned";
+            n.iconState = "safemode";
           } else if (n.state === "ACTIVE") {
             n.title = capitalise(n.state);
-            n.iconState = "alive";
+            n.iconState = "active";
           } else if (nodes[i].state === "STANDBY") {
             n.title = capitalise(n.state);
-            n.iconState = "down-decommissioned";
+            n.iconState = "standby";
           } else if (nodes[i].state === "UNAVAILABLE") {
             n.title = capitalise(n.state);
-            n.iconState = "down";
+            n.iconState = "unavailable";
           }
           if (n.namenodeId === "null") {
             n.namenodeId = "";
@@ -180,22 +180,22 @@
         for (var i = 0, e = nodes.length; i < e; ++i) {
           var n = nodes[i];
           n.title = "Unavailable"
-          n.iconState = "down";
+          n.iconState = "unavailable";
           if (n.status === "INITIALIZING") {
             n.title = capitalise(n.status);
-            n.iconState = "alive";
+            n.iconState = "active";
           } else if (n.status === "RUNNING") {
             n.title = capitalise(n.status);
-            n.iconState = "alive";
+            n.iconState = "active";
           } else if (n.status === "SAFEMODE") {
             n.title = capitalise(n.status);
-            n.iconState = "down-decommissioned";
+            n.iconState = "safemode";
           } else if (n.status === "STOPPING") {
             n.title = capitalise(n.status);
-            n.iconState = "decommissioned";
+            n.iconState = "unavailable";
           } else if (n.status === "SHUTDOWN") {
             n.title = capitalise(n.status);
-            n.iconState = "down";
+            n.iconState = "unavailable";
           }
         }
       }
@@ -307,7 +307,18 @@
     var HELPERS = {}
 
     function workaround(resource) {
+      function augment_read_only(mountTable) {
+        for (var i = 0, e = mountTable.length; i < e; ++i) {
+          if (mountTable[i].readonly == true) {
+            mountTable[i].readonly = "true"
+          } else {
+            mountTable[i].readonly = "false"
+          }
+        }
+      }
+
       resource.MountTable = JSON.parse(resource.MountTable)
+      augment_read_only(resource.MountTable)
       return resource;
     }
 
@@ -378,4 +389,4 @@
   $(window).bind('hashchange', function () {
     load_page();
   });
-})();
+})();

+ 138 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/webapps/static/rbf.css

@@ -0,0 +1,138 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+@import url("hadoop.css");
+
+.federationhealth-namenode-capacity-bar {
+    margin-bottom:0;
+    width: 60%;
+}
+
+.federationhealth-namenode-icon:before {
+    font-size: 10pt;
+    padding-right: 1pt;
+    font-family: 'Glyphicons Halflings';
+    font-style: normal;
+    font-weight: normal;
+    line-height: 1;
+    -webkit-font-smoothing: antialiased;
+    -moz-osx-font-smoothing: grayscale;
+}
+
+.federationhealth-namenode-active:before {
+    color: #5fa341;
+    content: "\e013";
+}
+
+.federationhealth-namenode-unavailable:before {
+    color: #c7254e;
+    content: "\e101";
+}
+
+.federationhealth-namenode-standby:before {
+    color: #2e6da6;
+    content: "\e017";
+}
+
+.federationhealth-namenode-safemode:before {
+    color: #bc5f04;
+    content: "\e090";
+}
+
+.federationhealth-namenode-legend {
+    list-style-type: none;
+    text-align: right;
+}
+
+.federationhealth-namenode-legend li {
+    display: inline;
+    padding: 10pt;
+    padding-left: 10pt;
+}
+
+.federationhealth-namenode-legend li:before {
+    padding-right: 5pt;
+}
+
+
+.federationhealth-router-capacity-bar {
+    margin-bottom:0;
+    width: 60%;
+}
+
+.federationhealth-router-icon:before {
+    font-size: 10pt;
+    padding-right: 1pt;
+    font-family: 'Glyphicons Halflings';
+    font-style: normal;
+    font-weight: normal;
+    line-height: 1;
+    -webkit-font-smoothing: antialiased;
+    -moz-osx-font-smoothing: grayscale;
+}
+
+.federationhealth-router-active:before {
+    color: #5fa341;
+    content: "\e013";
+}
+
+.federationhealth-router-unavailable:before {
+    color: #c7254e;
+    content: "\e101";
+}
+
+.federationhealth-router-safemode:before {
+    color: #bc5f04;
+    content: "\e090";
+}
+
+.federationhealth-router-legend {
+    list-style-type: none;
+    text-align: right;
+}
+
+.federationhealth-router-legend li {
+    display: inline;
+    padding: 10pt;
+    padding-left: 10pt;
+}
+
+.federationhealth-router-legend li:before {
+    padding-right: 5pt;
+}
+
+.mount-table-icon:before {
+    font-size: 10pt;
+    padding-right: 1pt;
+    font-family: 'Glyphicons Halflings';
+    font-style: normal;
+    font-weight: normal;
+    line-height: 1;
+    -webkit-font-smoothing: antialiased;
+    -moz-osx-font-smoothing: grayscale;
+}
+
+.mount-table-read-only-true:before {
+    color: #c7254e;
+    content: "\e033";
+}
+
+.mount-table-read-only-false:before {
+    color: #5fa341;
+    content: "\e013";
+}
+

+ 37 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/xsl/configuration.xsl

@@ -0,0 +1,37 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+<xsl:stylesheet xmlns:xsl="http://www.w3.org/1999/XSL/Transform" version="1.0">
+<xsl:output method="html"/>
+<xsl:template match="configuration">
+<html>
+<body>
+<table border="1">
+<tr>
+ <td>name</td>
+ <td>value</td>
+ <td>description</td>
+</tr>
+<xsl:for-each select="property">
+<tr>
+  <td><a name="{name}"><xsl:value-of select="name"/></a></td>
+  <td><xsl:value-of select="value"/></td>
+  <td><xsl:value-of select="description"/></td>
+</tr>
+</xsl:for-each>
+</table>
+</body>
+</html>
+</xsl:template>
+</xsl:stylesheet>

+ 129 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/web/RouterWebHDFSContract.java

@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.fs.contract.router.web;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.contract.hdfs.HDFSContract;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster;
+import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster.RouterContext;
+import org.apache.hadoop.hdfs.web.WebHdfsConstants;
+import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
+import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.NAMENODES;
+
+/**
+ * The contract of Router-based Federated HDFS
+ * This changes its feature set from platform for platform -the default
+ * set is updated during initialization.
+ */
+public class RouterWebHDFSContract extends HDFSContract {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(WebHdfsFileSystem.class);
+
+  public static final String CONTRACT_WEBHDFS_XML = "contract/webhdfs.xml";
+  private static MiniRouterDFSCluster cluster;
+
+  public RouterWebHDFSContract(Configuration conf) {
+    super(conf);
+    addConfResource(CONTRACT_WEBHDFS_XML);
+  }
+
+  public static void createCluster() throws IOException {
+    try {
+      HdfsConfiguration conf = new HdfsConfiguration();
+      conf.addResource(CONTRACT_HDFS_XML);
+      conf.addResource(CONTRACT_WEBHDFS_XML);
+
+      cluster = new MiniRouterDFSCluster(true, 2);
+
+      // Start NNs and DNs and wait until ready
+      cluster.startCluster();
+
+      // Start routers with only an RPC service
+      cluster.startRouters();
+
+      // Register and verify all NNs with all routers
+      cluster.registerNamenodes();
+      cluster.waitNamenodeRegistration();
+
+      // Setup the mount table
+      cluster.installMockLocations();
+
+      // Making one Namenodes active per nameservice
+      if (cluster.isHighAvailability()) {
+        for (String ns : cluster.getNameservices()) {
+          cluster.switchToActive(ns, NAMENODES[0]);
+          cluster.switchToStandby(ns, NAMENODES[1]);
+        }
+      }
+    } catch (Exception e) {
+      cluster = null;
+      throw new IOException("Cannot start federated cluster", e);
+    }
+  }
+
+  public static void destroyCluster() {
+    if (cluster != null) {
+      cluster.shutdown();
+      cluster = null;
+    }
+  }
+
+  public static MiniDFSCluster getCluster() {
+    return cluster.getCluster();
+  }
+
+  @Override
+  public FileSystem getTestFileSystem() throws IOException {
+    return getFileSystem();
+  }
+
+  public static FileSystem getFileSystem() throws IOException {
+    //assumes cluster is not null
+    Assert.assertNotNull("cluster not created", cluster);
+
+    // Create a connection to WebHDFS
+    try {
+      RouterContext router = cluster.getRandomRouter();
+      String uriStr =
+          WebHdfsConstants.WEBHDFS_SCHEME + "://" + router.getHttpAddress();
+      URI uri = new URI(uriStr);
+      Configuration conf = new HdfsConfiguration();
+      return FileSystem.get(uri, conf);
+    } catch (URISyntaxException e) {
+      LOG.error("Cannot create URI for the WebHDFS filesystem", e);
+    }
+    return null;
+  }
+
+  @Override
+  public String getScheme() {
+    return WebHdfsConstants.WEBHDFS_SCHEME;
+  }
+}

+ 45 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/web/TestRouterWebHDFSContractAppend.java

@@ -0,0 +1,45 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License. See accompanying LICENSE file.
+ */
+
+package org.apache.hadoop.fs.contract.router.web;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractAppendTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+/**
+ * Test append operations on a Router WebHDFS FS.
+ */
+public class TestRouterWebHDFSContractAppend
+    extends AbstractContractAppendTest {
+
+  @BeforeClass
+  public static void createCluster() throws IOException {
+    RouterWebHDFSContract.createCluster();
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    RouterWebHDFSContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RouterWebHDFSContract(conf);
+  }
+}

+ 52 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/web/TestRouterWebHDFSContractConcat.java

@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.contract.router.web;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.AbstractContractConcatTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+/**
+ * Test concat operations on a Router WebHDFS FS.
+ */
+public class TestRouterWebHDFSContractConcat
+    extends AbstractContractConcatTest {
+
+  @BeforeClass
+  public static void createCluster() throws IOException {
+    RouterWebHDFSContract.createCluster();
+    // perform a simple operation on the cluster to verify it is up
+    RouterWebHDFSContract.getFileSystem().getDefaultBlockSize(new Path("/"));
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    RouterWebHDFSContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RouterWebHDFSContract(conf);
+  }
+}

+ 49 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/web/TestRouterWebHDFSContractCreate.java

@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.contract.router.web;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+/**
+ * Test create operations on a Router WebHDFS FS.
+ */
+public class TestRouterWebHDFSContractCreate
+    extends AbstractContractCreateTest {
+
+  @BeforeClass
+  public static void createCluster() throws IOException {
+    RouterWebHDFSContract.createCluster();
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    RouterWebHDFSContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RouterWebHDFSContract(conf);
+  }
+}

+ 49 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/web/TestRouterWebHDFSContractDelete.java

@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.contract.router.web;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+/**
+ * Test delete operations on a Router WebHDFS FS.
+ */
+public class TestRouterWebHDFSContractDelete
+    extends AbstractContractDeleteTest {
+
+  @BeforeClass
+  public static void createCluster() throws IOException {
+    RouterWebHDFSContract.createCluster();
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    RouterWebHDFSContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RouterWebHDFSContract(conf);
+  }
+}

+ 48 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/web/TestRouterWebHDFSContractMkdir.java

@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.contract.router.web;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+/**
+ * Test dir operations on a Router WebHDFS FS.
+ */
+public class TestRouterWebHDFSContractMkdir extends AbstractContractMkdirTest {
+
+  @BeforeClass
+  public static void createCluster() throws IOException {
+    RouterWebHDFSContract.createCluster();
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    RouterWebHDFSContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RouterWebHDFSContract(conf);
+  }
+}

+ 63 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/web/TestRouterWebHDFSContractOpen.java

@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.contract.router.web;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+
+/**
+ * Test open operations on a Router WebHDFS FS.
+ */
+public class TestRouterWebHDFSContractOpen extends AbstractContractOpenTest {
+
+  @BeforeClass
+  public static void createCluster() throws IOException {
+    RouterWebHDFSContract.createCluster();
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    RouterWebHDFSContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RouterWebHDFSContract(conf);
+  }
+
+  @Override
+  @Test
+  public void testOpenReadDir() throws Throwable {
+    // WebHDFS itself allows open read on directory, we may need to
+    // fix this first before make this test work
+  }
+
+  @Override
+  @Test
+  public void testOpenReadDirWithChild() throws Throwable {
+    // WebHDFS itself allows open read on directory, we may need to
+    // fix this first before make this test work
+  }
+}

+ 49 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/web/TestRouterWebHDFSContractRename.java

@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.contract.router.web;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+/**
+ * Test rename operations on a Router WebHDFS FS.
+ */
+public class TestRouterWebHDFSContractRename
+    extends AbstractContractRenameTest {
+
+  @BeforeClass
+  public static void createCluster() throws IOException {
+    RouterWebHDFSContract.createCluster();
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    RouterWebHDFSContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RouterWebHDFSContract(conf);
+  }
+}

+ 64 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/web/TestRouterWebHDFSContractRootDirectory.java

@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.contract.router.web;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+/**
+ * Test dir operations on a Router WebHDFS FS.
+ */
+public class TestRouterWebHDFSContractRootDirectory extends
+    AbstractContractRootDirectoryTest {
+
+  @BeforeClass
+  public static void createCluster() throws IOException {
+    RouterWebHDFSContract.createCluster();
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    RouterWebHDFSContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RouterWebHDFSContract(conf);
+  }
+
+  @Override
+  public void testListEmptyRootDirectory() throws IOException {
+    // It doesn't apply because we still have the mount points here
+  }
+
+  @Override
+  public void testRmEmptyRootDirNonRecursive() throws IOException {
+    // It doesn't apply because we still have the mount points here
+  }
+
+  @Override
+  public void testRecursiveRootListing() throws IOException {
+    // It doesn't apply because we still have the mount points here
+  }
+}

+ 63 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/web/TestRouterWebHDFSContractSeek.java

@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.contract.router.web;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+/**
+ * Test seek operations on a Router WebHDFS FS.
+ */
+public class TestRouterWebHDFSContractSeek extends AbstractContractSeekTest {
+
+  @BeforeClass
+  public static void createCluster() throws IOException {
+    RouterWebHDFSContract.createCluster();
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    RouterWebHDFSContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RouterWebHDFSContract(conf);
+  }
+
+  @Override
+  public void testNegativeSeek() throws Throwable {
+    System.out.println("Not supported");
+  }
+
+  @Override
+  public void testSeekReadClosedFile() throws Throwable {
+    System.out.println("Not supported");
+  }
+
+  @Override
+  public void testSeekPastEndOfFileThenReseekAndRead() throws Throwable {
+    System.out.println("Not supported");
+  }
+}

+ 22 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/fs/contract/router/web/package-info.java

@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+/**
+ * Test the WebHDFS contract.
+ */
+package org.apache.hadoop.fs.contract.router.web;

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MiniRouterDFSCluster.java

@@ -181,6 +181,11 @@ public class MiniRouterDFSCluster {
       return this.fileContext;
     }
 
+    public String getHttpAddress() {
+      InetSocketAddress httpAddress = router.getHttpServerAddress();
+      return NetUtils.getHostPortString(httpAddress);
+    }
+
     public void initRouter() throws URISyntaxException {
       // Store the bound points for the router interfaces
       InetSocketAddress rpcAddress = router.getRpcServerAddress();

+ 232 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/order/TestAvailableSpaceResolver.java

@@ -0,0 +1,232 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.federation.resolver.order;
+
+import static org.apache.hadoop.hdfs.server.federation.resolver.order.AvailableSpaceResolver.BALANCER_PREFERENCE_DEFAULT;
+import static org.apache.hadoop.hdfs.server.federation.resolver.order.AvailableSpaceResolver.BALANCER_PREFERENCE_KEY;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.federation.resolver.MultipleDestinationMountTableResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.PathLocation;
+import org.apache.hadoop.hdfs.server.federation.resolver.order.AvailableSpaceResolver.SubclusterAvailableSpace;
+import org.apache.hadoop.hdfs.server.federation.resolver.order.AvailableSpaceResolver.SubclusterSpaceComparator;
+import org.apache.hadoop.hdfs.server.federation.router.Router;
+import org.apache.hadoop.hdfs.server.federation.store.MembershipStore;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamenodeRegistrationsRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetNamenodeRegistrationsResponse;
+import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState;
+import org.apache.hadoop.hdfs.server.federation.store.records.MembershipStats;
+import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.MembershipStatsPBImpl;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Test;
+
+/**
+ * Test the {@link AvailableSpaceResolver}.
+ */
+public class TestAvailableSpaceResolver {
+
+  private static final int SUBCLUSTER_NUM = 10;
+
+  @Test
+  public void testResolverWithNoPreference() throws IOException {
+    MultipleDestinationMountTableResolver mountTableResolver =
+        mockAvailableSpaceResolver(1.0f);
+    // Since we don't have any preference, it will
+    // always chose the maximum-available-space subcluster.
+    PathLocation loc = mountTableResolver.getDestinationForPath("/space");
+    assertEquals("subcluster9",
+        loc.getDestinations().get(0).getNameserviceId());
+
+    loc = mountTableResolver.getDestinationForPath("/space/subdir");
+    assertEquals("subcluster9",
+        loc.getDestinations().get(0).getNameserviceId());
+  }
+
+  @Test
+  public void testResolverWithDefaultPreference() throws IOException {
+    MultipleDestinationMountTableResolver mountTableResolver =
+        mockAvailableSpaceResolver(BALANCER_PREFERENCE_DEFAULT);
+
+    int retries = 10;
+    int retryTimes = 0;
+    // There is chance we won't always chose the
+    // maximum-available-space subcluster.
+    for (retryTimes = 0; retryTimes < retries; retryTimes++) {
+      PathLocation loc = mountTableResolver.getDestinationForPath("/space");
+      if (!"subcluster9"
+          .equals(loc.getDestinations().get(0).getNameserviceId())) {
+        break;
+      }
+    }
+    assertNotEquals(retries, retryTimes);
+  }
+
+  /**
+   * Mock the available space based resolver.
+   *
+   * @param balancerPreference The balancer preference for the resolver.
+   * @throws IOException
+   * @return MultipleDestinationMountTableResolver instance.
+   */
+  @SuppressWarnings("unchecked")
+  private MultipleDestinationMountTableResolver mockAvailableSpaceResolver(
+      float balancerPreference) throws IOException {
+    Configuration conf = new Configuration();
+    conf.setFloat(BALANCER_PREFERENCE_KEY, balancerPreference);
+    Router router = mock(Router.class);
+    StateStoreService stateStore = mock(StateStoreService.class);
+    MembershipStore membership = mock(MembershipStore.class);
+    when(router.getStateStore()).thenReturn(stateStore);
+    when(stateStore.getRegisteredRecordStore(any(Class.class)))
+        .thenReturn(membership);
+    GetNamenodeRegistrationsResponse response =
+        GetNamenodeRegistrationsResponse.newInstance();
+    // Set the mapping for each client
+    List<MembershipState> records = new LinkedList<>();
+    for (int i = 0; i < SUBCLUSTER_NUM; i++) {
+      records.add(newMembershipState("subcluster" + i, i));
+    }
+    response.setNamenodeMemberships(records);
+
+    when(membership
+        .getNamenodeRegistrations(any(GetNamenodeRegistrationsRequest.class)))
+            .thenReturn(response);
+
+    // construct available space resolver
+    AvailableSpaceResolver resolver = new AvailableSpaceResolver(conf, router);
+    MultipleDestinationMountTableResolver mountTableResolver =
+        new MultipleDestinationMountTableResolver(conf, router);
+    mountTableResolver.addResolver(DestinationOrder.SPACE, resolver);
+
+    // We point /space to subclusters [0,..9] with the SPACE order
+    Map<String, String> destinations = new HashMap<>();
+    for (int i = 0; i < SUBCLUSTER_NUM; i++) {
+      destinations.put("subcluster" + i, "/space");
+    }
+    MountTable spaceEntry = MountTable.newInstance("/space", destinations);
+    spaceEntry.setDestOrder(DestinationOrder.SPACE);
+    mountTableResolver.addEntry(spaceEntry);
+
+    return mountTableResolver;
+  }
+
+  public static MembershipState newMembershipState(String nameservice,
+      long availableSpace) {
+    MembershipState record = MembershipState.newInstance();
+    record.setNameserviceId(nameservice);
+
+    MembershipStats stats = new MembershipStatsPBImpl();
+    stats.setAvailableSpace(availableSpace);
+    record.setStats(stats);
+    return record;
+  }
+
+  @Test
+  public void testSubclusterSpaceComparator() {
+    verifyRank(0.0f, true, false);
+    verifyRank(1.0f, true, true);
+    verifyRank(0.5f, false, false);
+    verifyRank(BALANCER_PREFERENCE_DEFAULT, false, false);
+
+    // test for illegal cases
+    try {
+      verifyRank(2.0f, false, false);
+      fail("Subcluster comparison should be failed.");
+    } catch (IllegalArgumentException e) {
+      GenericTestUtils.assertExceptionContains(
+          "The balancer preference value should be in the range 0.0 - 1.0", e);
+    }
+
+    try {
+      verifyRank(-1.0f, false, false);
+      fail("Subcluster comparison should be failed.");
+    } catch (IllegalArgumentException e) {
+      GenericTestUtils.assertExceptionContains(
+          "The balancer preference value should be in the range 0.0 - 1.0", e);
+    }
+  }
+
+  /**
+   * Verify result rank with {@link SubclusterSpaceComparator}.
+   * @param balancerPreference The balancer preference used
+   *        in {@link SubclusterSpaceComparator}.
+   * @param shouldOrdered The result rank should be ordered.
+   * @param isDesc If the rank result is in a descending order.
+   */
+  private void verifyRank(float balancerPreference, boolean shouldOrdered,
+      boolean isDesc) {
+    List<SubclusterAvailableSpace> subclusters = new LinkedList<>();
+    for (int i = 0; i < SUBCLUSTER_NUM; i++) {
+      subclusters.add(new SubclusterAvailableSpace("subcluster" + i, i));
+    }
+
+    // shuffle the cluster list if we expect rank to be ordered
+    if (shouldOrdered) {
+      Collections.shuffle(subclusters);
+    }
+
+    SubclusterSpaceComparator comparator = new SubclusterSpaceComparator(
+        balancerPreference);
+    Collections.sort(subclusters, comparator);
+
+    int i = SUBCLUSTER_NUM - 1;
+    for (; i >= 0; i--) {
+      SubclusterAvailableSpace cluster = subclusters
+          .get(SUBCLUSTER_NUM - 1 - i);
+
+      if (shouldOrdered) {
+        if (isDesc) {
+          assertEquals("subcluster" + i, cluster.getNameserviceId());
+          assertEquals(i, cluster.getAvailableSpace());
+        } else {
+          assertEquals("subcluster" + (SUBCLUSTER_NUM - 1 - i),
+              cluster.getNameserviceId());
+          assertEquals(SUBCLUSTER_NUM - 1 - i, cluster.getAvailableSpace());
+        }
+      } else {
+        // If catch one cluster is not in ordered, that's expected behavior.
+        if (!cluster.getNameserviceId().equals("subcluster" + i)
+            && cluster.getAvailableSpace() != i) {
+          break;
+        }
+      }
+    }
+
+    // The var i won't reach to 0 since cluster list won't be completely
+    // ordered.
+    if (!shouldOrdered) {
+      assertNotEquals(0, i);
+    }
+    subclusters.clear();
+  }
+}

+ 17 - 5
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestConnectionManager.java

@@ -29,6 +29,7 @@ import java.util.Map;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
  * Test functionalities of {@link ConnectionManager}, which manages a pool
@@ -94,14 +95,20 @@ public class TestConnectionManager {
     // Make sure the number of connections doesn't go below minSize
     ConnectionPool pool3 = new ConnectionPool(
         conf, TEST_NN_ADDRESS, TEST_USER3, 2, 10);
-    addConnectionsToPool(pool3, 10, 0);
-    poolMap.put(new ConnectionPoolId(TEST_USER2, TEST_NN_ADDRESS), pool3);
-    connManager.cleanup(pool3);
+    addConnectionsToPool(pool3, 8, 0);
+    poolMap.put(new ConnectionPoolId(TEST_USER3, TEST_NN_ADDRESS), pool3);
+    checkPoolConnections(TEST_USER3, 10, 0);
+    for (int i = 0; i < 10; i++) {
+      connManager.cleanup(pool3);
+    }
     checkPoolConnections(TEST_USER3, 2, 0);
     // With active connections added to pool, make sure it honors the
     // MIN_ACTIVE_RATIO again
-    addConnectionsToPool(pool3, 10, 2);
-    connManager.cleanup(pool3);
+    addConnectionsToPool(pool3, 8, 2);
+    checkPoolConnections(TEST_USER3, 10, 2);
+    for (int i = 0; i < 10; i++) {
+      connManager.cleanup(pool3);
+    }
     checkPoolConnections(TEST_USER3, 4, 2);
   }
 
@@ -145,13 +152,18 @@ public class TestConnectionManager {
 
   private void checkPoolConnections(UserGroupInformation ugi,
       int numOfConns, int numOfActiveConns) {
+    boolean connPoolFoundForUser = false;
     for (Map.Entry<ConnectionPoolId, ConnectionPool> e :
         connManager.getPools().entrySet()) {
       if (e.getKey().getUgi() == ugi) {
         assertEquals(numOfConns, e.getValue().getNumConnections());
         assertEquals(numOfActiveConns, e.getValue().getNumActiveConnections());
+        connPoolFoundForUser = true;
       }
     }
+    if (!connPoolFoundForUser) {
+      fail("Connection pool not found for user " + ugi.getUserName());
+    }
   }
 
 }

+ 9 - 1
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAllResolver.java

@@ -60,8 +60,10 @@ public class TestRouterAllResolver {
 
   /** Directory that will be in a HASH_ALL mount point. */
   private static final String TEST_DIR_HASH_ALL = "/hashall";
-  /** Directory that will be in a HASH_ALL mount point. */
+  /** Directory that will be in a RANDOM mount point. */
   private static final String TEST_DIR_RANDOM = "/random";
+  /** Directory that will be in a SPACE mount point. */
+  private static final String TEST_DIR_SPACE = "/space";
 
   /** Number of namespaces. */
   private static final int NUM_NAMESPACES = 2;
@@ -103,6 +105,7 @@ public class TestRouterAllResolver {
     // Setup the test mount point
     createMountTableEntry(TEST_DIR_HASH_ALL, DestinationOrder.HASH_ALL);
     createMountTableEntry(TEST_DIR_RANDOM, DestinationOrder.RANDOM);
+    createMountTableEntry(TEST_DIR_SPACE, DestinationOrder.SPACE);
 
     // Get filesystems for federated and each namespace
     routerFs = routerContext.getFileSystem();
@@ -135,6 +138,11 @@ public class TestRouterAllResolver {
     testAll(TEST_DIR_RANDOM);
   }
 
+  @Test
+  public void testSpaceAll() throws Exception {
+    testAll(TEST_DIR_SPACE);
+  }
+
   /**
    * Tests that the resolver spreads files across subclusters in the whole
    * tree.

+ 39 - 2
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java

@@ -27,6 +27,7 @@ import static org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster.TEST
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
@@ -45,6 +46,7 @@ import java.util.Map.Entry;
 import java.util.Random;
 import java.util.Set;
 import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.CryptoProtocolVersion;
@@ -71,10 +73,11 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder;
 import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster;
 import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster.NamenodeContext;
 import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster.RouterContext;
+import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder;
+import org.apache.hadoop.hdfs.server.federation.metrics.NamenodeBeanMetrics;
 import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.io.EnumSetWritable;
@@ -83,6 +86,7 @@ import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.Service.STATE;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.codehaus.jettison.json.JSONObject;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -150,7 +154,14 @@ public class TestRouterRpc {
     cluster.startCluster();
 
     // Start routers with only an RPC service
-    cluster.addRouterOverrides((new RouterConfigBuilder()).rpc().build());
+    Configuration routerConf = new RouterConfigBuilder()
+        .metrics()
+        .rpc()
+        .build();
+    // We decrease the DN cache times to make the test faster
+    routerConf.setTimeDuration(
+        NamenodeBeanMetrics.DN_REPORT_CACHE_EXPIRE, 1, TimeUnit.SECONDS);
+    cluster.addRouterOverrides(routerConf);
     cluster.startRouters();
 
     // Register and verify all NNs with all routers
@@ -1032,6 +1043,32 @@ public class TestRouterRpc {
     assertEquals(statsNamenode.toString(), statsRouter.toString());
   }
 
+  @Test
+  public void testNamenodeMetrics() throws Exception {
+    final NamenodeBeanMetrics metrics =
+        router.getRouter().getNamenodeMetrics();
+    final String jsonString0 = metrics.getLiveNodes();
+
+    // We should have 12 nodes in total
+    JSONObject jsonObject = new JSONObject(jsonString0);
+    assertEquals(12, jsonObject.names().length());
+
+    // We should be caching this information
+    String jsonString1 = metrics.getLiveNodes();
+    assertEquals(jsonString0, jsonString1);
+
+    // We wait until the cached value is updated
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        return !jsonString0.equals(metrics.getLiveNodes());
+      }
+    }, 500, 5 * 1000);
+
+    // The cache should be updated now
+    assertNotEquals(jsonString0, metrics.getLiveNodes());
+  }
+
   /**
    * Check the erasure coding policies in the Router and the Namenode.
    * @return The erasure coding policies.

+ 26 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/resources/contract/webhdfs.xml

@@ -0,0 +1,26 @@
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~  or more contributor license agreements.  See the NOTICE file
+  ~  distributed with this work for additional information
+  ~  regarding copyright ownership.  The ASF licenses this file
+  ~  to you under the Apache License, Version 2.0 (the
+  ~  "License"); you may not use this file except in compliance
+  ~  with the License.  You may obtain a copy of the License at
+  ~
+  ~       http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~  Unless required by applicable law or agreed to in writing, software
+  ~  distributed under the License is distributed on an "AS IS" BASIS,
+  ~  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~  See the License for the specific language governing permissions and
+  ~  limitations under the License.
+  -->
+
+<configuration>
+
+  <property>
+    <name>fs.contract.supports-strict-exceptions</name>
+    <value>false</value>
+  </property>
+
+</configuration>

Filskillnaden har hållts tillbaka eftersom den är för stor
+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_3.0.1.xml


+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclStorage.java

@@ -281,7 +281,8 @@ public final class AclStorage {
       // Only directories may have a default ACL.
       if (!defaultEntries.isEmpty() && !inode.isDirectory()) {
         throw new AclException(
-          "Invalid ACL: only directories may have a default ACL.");
+          "Invalid ACL: only directories may have a default ACL. "
+            + "Path: " + inode.getFullPathName());
       }
 
       // Attach entries to the feature.

+ 66 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java

@@ -33,6 +33,9 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.CipherSuite;
 import org.apache.hadoop.crypto.CryptoProtocolVersion;
@@ -50,6 +53,7 @@ import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.security.AccessControlException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -107,6 +111,34 @@ public class EncryptionZoneManager {
     String getKeyName() {
       return keyName;
     }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (!(o instanceof EncryptionZoneInt)) {
+        return false;
+      }
+
+      EncryptionZoneInt b = (EncryptionZoneInt)o;
+      return new EqualsBuilder()
+          .append(inodeId, b.getINodeId())
+          .append(suite, b.getSuite())
+          .append(version, b.getVersion())
+          .append(keyName, b.getKeyName())
+          .isEquals();
+    }
+
+    @Override
+    public int hashCode() {
+      return new HashCodeBuilder().
+          append(inodeId).
+          append(suite).
+          append(version).
+          append(keyName).
+          toHashCode();
+    }
   }
 
   private TreeMap<Long, EncryptionZoneInt> encryptionZones = null;
@@ -315,8 +347,8 @@ public class EncryptionZoneManager {
    * <p/>
    * Called while holding the FSDirectory lock.
    */
-  boolean isInAnEZ(INodesInPath iip)
-      throws UnresolvedLinkException, SnapshotAccessControlException {
+  boolean isInAnEZ(INodesInPath iip) throws UnresolvedLinkException,
+      SnapshotAccessControlException, IOException {
     assert dir.hasReadLock();
     return (getEncryptionZoneForPath(iip) != null);
   }
@@ -341,7 +373,7 @@ public class EncryptionZoneManager {
    * <p/>
    * Called while holding the FSDirectory lock.
    */
-  String getKeyName(final INodesInPath iip) {
+  String getKeyName(final INodesInPath iip) throws IOException {
     assert dir.hasReadLock();
     EncryptionZoneInt ezi = getEncryptionZoneForPath(iip);
     if (ezi == null) {
@@ -356,19 +388,43 @@ public class EncryptionZoneManager {
    * <p/>
    * Called while holding the FSDirectory lock.
    */
-  private EncryptionZoneInt getEncryptionZoneForPath(INodesInPath iip) {
+  private EncryptionZoneInt getEncryptionZoneForPath(INodesInPath iip)
+      throws  IOException{
     assert dir.hasReadLock();
     Preconditions.checkNotNull(iip);
     if (!hasCreatedEncryptionZone()) {
       return null;
     }
+
+    int snapshotID = iip.getPathSnapshotId();
     for (int i = iip.length() - 1; i >= 0; i--) {
       final INode inode = iip.getINode(i);
-      if (inode != null) {
+      if (inode == null || !inode.isDirectory()) {
+        //not found or not a directory, encryption zone is supported on
+        //directory only.
+        continue;
+      }
+      if (snapshotID == Snapshot.CURRENT_STATE_ID) {
         final EncryptionZoneInt ezi = encryptionZones.get(inode.getId());
         if (ezi != null) {
           return ezi;
         }
+      } else {
+        XAttr xAttr = FSDirXAttrOp.unprotectedGetXAttrByPrefixedName(
+            inode, snapshotID, CRYPTO_XATTR_ENCRYPTION_ZONE);
+        if (xAttr != null) {
+          try {
+            final HdfsProtos.ZoneEncryptionInfoProto ezProto =
+                HdfsProtos.ZoneEncryptionInfoProto.parseFrom(xAttr.getValue());
+            return new EncryptionZoneInt(
+                inode.getId(), PBHelperClient.convert(ezProto.getSuite()),
+                PBHelperClient.convert(ezProto.getCryptoProtocolVersion()),
+                ezProto.getKeyName());
+          } catch (InvalidProtocolBufferException e) {
+            throw new IOException("Could not parse encryption zone for inode "
+                + iip.getPath(), e);
+          }
+        }
       }
     }
     return null;
@@ -381,7 +437,8 @@ public class EncryptionZoneManager {
    * <p/>
    * Called while holding the FSDirectory lock.
    */
-  private EncryptionZoneInt getParentEncryptionZoneForPath(INodesInPath iip) {
+  private EncryptionZoneInt getParentEncryptionZoneForPath(INodesInPath iip)
+      throws  IOException {
     assert dir.hasReadLock();
     Preconditions.checkNotNull(iip);
     INodesInPath parentIIP = iip.getParentINodesInPath();
@@ -395,7 +452,8 @@ public class EncryptionZoneManager {
    * @param iip The INodesInPath of the path to check
    * @return the EncryptionZone representing the ez for the path.
    */
-  EncryptionZone getEZINodeForPath(INodesInPath iip) {
+  EncryptionZone getEZINodeForPath(INodesInPath iip)
+      throws IOException {
     final EncryptionZoneInt ezi = getEncryptionZoneForPath(iip);
     if (ezi == null) {
       return null;
@@ -437,7 +495,7 @@ public class EncryptionZoneManager {
     }
 
     if (srcInEZ) {
-      if (srcParentEZI != dstParentEZI) {
+      if (!srcParentEZI.equals(dstParentEZI)) {
         final String srcEZPath = getFullPathName(srcParentEZI.getINodeId());
         final String dstEZPath = getFullPathName(dstParentEZI.getINodeId());
         final StringBuilder sb = new StringBuilder(srcIIP.getPath());

+ 9 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java

@@ -205,7 +205,7 @@ final class FSDirEncryptionZoneOp {
   }
 
   static EncryptionZone getEZForPath(final FSDirectory fsd,
-      final INodesInPath iip) {
+      final INodesInPath iip) throws IOException {
     fsd.readLock();
     try {
       return fsd.ezManager.getEZINodeForPath(iip);
@@ -360,8 +360,9 @@ final class FSDirEncryptionZoneOp {
 
   private static ZoneEncryptionInfoProto getZoneEncryptionInfoProto(
       final INodesInPath iip) throws IOException {
-    final XAttr fileXAttr = FSDirXAttrOp
-        .unprotectedGetXAttrByPrefixedName(iip, CRYPTO_XATTR_ENCRYPTION_ZONE);
+    final XAttr fileXAttr = FSDirXAttrOp.unprotectedGetXAttrByPrefixedName(
+        iip.getLastINode(), iip.getPathSnapshotId(),
+        CRYPTO_XATTR_ENCRYPTION_ZONE);
     if (fileXAttr == null) {
       throw new IOException(
           "Could not find reencryption XAttr for file " + iip.getPath());
@@ -457,7 +458,8 @@ final class FSDirEncryptionZoneOp {
       }
 
       XAttr fileXAttr = FSDirXAttrOp.unprotectedGetXAttrByPrefixedName(
-          iip, CRYPTO_XATTR_FILE_ENCRYPTION_INFO);
+          iip.getLastINode(), iip.getPathSnapshotId(),
+          CRYPTO_XATTR_FILE_ENCRYPTION_INFO);
       if (fileXAttr == null) {
         NameNode.LOG.warn("Could not find encryption XAttr for file " +
             iip.getPath() + " in encryption zone " + encryptionZone.getPath());
@@ -494,7 +496,7 @@ final class FSDirEncryptionZoneOp {
    */
   static FileEncryptionInfo getFileEncryptionInfo(FSDirectory dir,
       INodesInPath iip, EncryptionKeyInfo ezInfo)
-          throws RetryStartFileException {
+          throws RetryStartFileException, IOException {
     FileEncryptionInfo feInfo = null;
     final EncryptionZone zone = getEZForPath(dir, iip);
     if (zone != null) {
@@ -517,7 +519,8 @@ final class FSDirEncryptionZoneOp {
   }
 
   static boolean isInAnEZ(final FSDirectory fsd, final INodesInPath iip)
-      throws UnresolvedLinkException, SnapshotAccessControlException {
+      throws UnresolvedLinkException, SnapshotAccessControlException,
+      IOException {
     if (!fsd.ezManager.hasCreatedEncryptionZone()) {
       return false;
     }

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java

@@ -378,16 +378,18 @@ class FSDirXAttrOp {
       String prefixedName) throws IOException {
     fsd.readLock();
     try {
-      return XAttrStorage.readINodeXAttrByPrefixedName(iip, prefixedName);
+      return XAttrStorage.readINodeXAttrByPrefixedName(iip.getLastINode(),
+          iip.getPathSnapshotId(), prefixedName);
     } finally {
       fsd.readUnlock();
     }
   }
 
   static XAttr unprotectedGetXAttrByPrefixedName(
-      INodesInPath iip, String prefixedName)
+      INode inode, int snapshotId, String prefixedName)
       throws IOException {
-    return XAttrStorage.readINodeXAttrByPrefixedName(iip, prefixedName);
+    return XAttrStorage.readINodeXAttrByPrefixedName(
+        inode, snapshotId, prefixedName);
   }
 
   private static void checkXAttrChangeAccess(

+ 25 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java

@@ -35,6 +35,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -68,6 +69,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.util.Canceler;
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
 import org.apache.hadoop.io.MD5Hash;
+import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.Time;
 
 import com.google.common.base.Joiner;
@@ -86,6 +88,10 @@ public class FSImage implements Closeable {
   protected FSEditLog editLog = null;
   private boolean isUpgradeFinalized = false;
 
+  // If true, then image corruption was detected. The NameNode process will
+  // exit immediately after saving the image.
+  private AtomicBoolean exitAfterSave = new AtomicBoolean(false);
+
   protected NNStorage storage;
   
   /**
@@ -954,8 +960,14 @@ public class FSImage implements Closeable {
     
     FSImageFormatProtobuf.Saver saver = new FSImageFormatProtobuf.Saver(context);
     FSImageCompression compression = FSImageCompression.createCompression(conf);
-    saver.save(newFile, compression);
-    
+    long numErrors = saver.save(newFile, compression);
+    if (numErrors > 0) {
+      // The image is likely corrupted.
+      LOG.error("Detected " + numErrors + " errors while saving FsImage " +
+          dstFile);
+      exitAfterSave.set(true);
+    }
+
     MD5FileUtils.saveMD5File(dstFile, saver.getSavedDigest());
     storage.setMostRecentCheckpointInfo(txid, Time.now());
   }
@@ -1117,6 +1129,12 @@ public class FSImage implements Closeable {
     }
     //Update NameDirSize Metric
     getStorage().updateNameDirSize();
+
+    if (exitAfterSave.get()) {
+      LOG.fatal("NameNode process will exit now... The saved FsImage " +
+          nnf + " is potentially corrupted.");
+      ExitUtil.terminate(-1);
+    }
   }
 
   /**
@@ -1184,8 +1202,11 @@ public class FSImage implements Closeable {
   
       // Since we now have a new checkpoint, we can clean up some
       // old edit logs and checkpoints.
-      purgeOldStorage(nnf);
-      archivalManager.purgeCheckpoints(NameNodeFile.IMAGE_NEW);
+      // Do not purge anything if we just wrote a corrupted FsImage.
+      if (!exitAfterSave.get()) {
+        purgeOldStorage(nnf);
+        archivalManager.purgeCheckpoints(NameNodeFile.IMAGE_NEW);
+      }
     } finally {
       // Notify any threads waiting on the checkpoint to be canceled
       // that it is complete.

+ 24 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java

@@ -444,15 +444,22 @@ public final class FSImageFormatProtobuf {
       sectionOutputStream.flush();
     }
 
-    void save(File file, FSImageCompression compression) throws IOException {
+    /**
+     * @return number of non-fatal errors detected while writing the image.
+     * @throws IOException on fatal error.
+     */
+    long save(File file, FSImageCompression compression) throws IOException {
       FileOutputStream fout = new FileOutputStream(file);
       fileChannel = fout.getChannel();
       try {
         LOG.info("Saving image file {} using {}", file, compression);
         long startTime = monotonicNow();
-        saveInternal(fout, compression, file.getAbsolutePath());
-        LOG.info("Image file {} of size {} bytes saved in {} seconds.", file,
-            file.length(), (monotonicNow() - startTime) / 1000);
+        long numErrors = saveInternal(
+            fout, compression, file.getAbsolutePath());
+        LOG.info("Image file {} of size {} bytes saved in {} seconds {}.", file,
+            file.length(), (monotonicNow() - startTime) / 1000,
+            (numErrors > 0 ? (" with" + numErrors + " errors") : ""));
+        return numErrors;
       } finally {
         fout.close();
       }
@@ -476,7 +483,11 @@ public final class FSImageFormatProtobuf {
       saver.serializeFilesUCSection(sectionOutputStream);
     }
 
-    private void saveSnapshots(FileSummary.Builder summary) throws IOException {
+    /**
+     * @return number of non-fatal errors detected while saving the image.
+     * @throws IOException on fatal error.
+     */
+    private long saveSnapshots(FileSummary.Builder summary) throws IOException {
       FSImageFormatPBSnapshot.Saver snapshotSaver = new FSImageFormatPBSnapshot.Saver(
           this, summary, context, context.getSourceNamesystem());
 
@@ -487,9 +498,14 @@ public final class FSImageFormatProtobuf {
         snapshotSaver.serializeSnapshotDiffSection(sectionOutputStream);
       }
       snapshotSaver.serializeINodeReferenceSection(sectionOutputStream);
+      return snapshotSaver.getNumImageErrors();
     }
 
-    private void saveInternal(FileOutputStream fout,
+    /**
+     * @return number of non-fatal errors detected while writing the FsImage.
+     * @throws IOException on fatal error.
+     */
+    private long saveInternal(FileOutputStream fout,
         FSImageCompression compression, String filePath) throws IOException {
       StartupProgress prog = NameNode.getStartupProgress();
       MessageDigest digester = MD5Hash.getDigester();
@@ -528,7 +544,7 @@ public final class FSImageFormatProtobuf {
       step = new Step(StepType.INODES, filePath);
       prog.beginStep(Phase.SAVING_CHECKPOINT, step);
       saveInodes(b);
-      saveSnapshots(b);
+      long numErrors = saveSnapshots(b);
       prog.endStep(Phase.SAVING_CHECKPOINT, step);
 
       step = new Step(StepType.DELEGATION_TOKENS, filePath);
@@ -551,6 +567,7 @@ public final class FSImageFormatProtobuf {
       saveFileSummary(underlyingOutputStream, summary);
       underlyingOutputStream.close();
       savedDigest = new MD5Hash(digester.digest());
+      return numErrors;
     }
 
     private void saveSecretManagerSection(FileSummary.Builder summary)

+ 15 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java

@@ -76,7 +76,9 @@ public class NameNodeHttpServer {
     this.bindAddress = bindAddress;
   }
 
-  private void initWebHdfs(Configuration conf) throws IOException {
+  public static void initWebHdfs(Configuration conf, String hostname,
+      HttpServer2 httpServer2, String jerseyResourcePackage)
+      throws IOException {
     // set user pattern based on configuration file
     UserParam.setUserPattern(conf.get(
         HdfsClientConfigKeys.DFS_WEBHDFS_USER_PATTERN_KEY,
@@ -92,8 +94,8 @@ public class NameNodeHttpServer {
     final String name = className;
 
     final String pathSpec = WebHdfsFileSystem.PATH_PREFIX + "/*";
-    Map<String, String> params = getAuthFilterParams(conf);
-    HttpServer2.defineFilter(httpServer.getWebAppContext(), name, className,
+    Map<String, String> params = getAuthFilterParams(conf, hostname);
+    HttpServer2.defineFilter(httpServer2.getWebAppContext(), name, className,
         params, new String[] { pathSpec });
     HttpServer2.LOG.info("Added filter '" + name + "' (class=" + className
         + ")");
@@ -104,13 +106,14 @@ public class NameNodeHttpServer {
       Map<String, String> restCsrfParams = RestCsrfPreventionFilter
           .getFilterParams(conf, "dfs.webhdfs.rest-csrf.");
       String restCsrfClassName = RestCsrfPreventionFilter.class.getName();
-      HttpServer2.defineFilter(httpServer.getWebAppContext(), restCsrfClassName,
-          restCsrfClassName, restCsrfParams, new String[] {pathSpec});
+      HttpServer2.defineFilter(httpServer2.getWebAppContext(),
+          restCsrfClassName, restCsrfClassName, restCsrfParams,
+          new String[] {pathSpec});
     }
 
     // add webhdfs packages
-    httpServer.addJerseyResourcePackage(NamenodeWebHdfsMethods.class
-        .getPackage().getName() + ";" + Param.class.getPackage().getName(),
+    httpServer2.addJerseyResourcePackage(
+        jerseyResourcePackage + ";" + Param.class.getPackage().getName(),
         pathSpec);
   }
 
@@ -165,7 +168,8 @@ public class NameNodeHttpServer {
           datanodeSslPort.getPort());
     }
 
-    initWebHdfs(conf);
+    initWebHdfs(conf, bindAddress.getHostName(), httpServer,
+        NamenodeWebHdfsMethods.class.getPackage().getName());
 
     httpServer.setAttribute(NAMENODE_ATTRIBUTE_KEY, nn);
     httpServer.setAttribute(JspHelper.CURRENT_CONF, conf);
@@ -186,8 +190,8 @@ public class NameNodeHttpServer {
     }
   }
   
-  private Map<String, String> getAuthFilterParams(Configuration conf)
-      throws IOException {
+  private static Map<String, String> getAuthFilterParams(Configuration conf,
+      String hostname) throws IOException {
     Map<String, String> params = new HashMap<String, String>();
     // Select configs beginning with 'dfs.web.authentication.'
     Iterator<Map.Entry<String, String>> iterator = conf.iterator();
@@ -203,8 +207,7 @@ public class NameNodeHttpServer {
       params
           .put(
               DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY,
-              SecurityUtil.getServerPrincipal(principalInConf,
-                                              bindAddress.getHostName()));
+              SecurityUtil.getServerPrincipal(principalInConf, hostname));
     } else if (UserGroupInformation.isSecurityEnabled()) {
       HttpServer2.LOG.error(
           "WebHDFS and security are enabled, but configuration property '" +

+ 4 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/XAttrStorage.java

@@ -47,14 +47,13 @@ public class XAttrStorage {
    * <p/>
    *
    * @param inode INode to read
-   * @param snapshotId
+   * @param snapshotId the snapshotId of the requested path
    * @param prefixedName xAttr name with prefix
    * @return the xAttr
    */
-  public static XAttr readINodeXAttrByPrefixedName(INodesInPath iip,
-      String prefixedName) {
-    XAttrFeature f =
-        iip.getLastINode().getXAttrFeature(iip.getPathSnapshotId());
+  public static XAttr readINodeXAttrByPrefixedName(INode inode, int snapshotId,
+                                                   String prefixedName) {
+    XAttrFeature f = inode.getXAttrFeature(snapshotId);
     return f == null ? null : f.getXAttr(prefixedName);
   }
 

+ 52 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java

@@ -1,4 +1,4 @@
-/**
+ /**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -27,6 +27,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
@@ -49,6 +50,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.AclEntryStatusFormat;
 import org.apache.hadoop.hdfs.server.namenode.AclFeature;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FSImage;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext;
@@ -415,6 +417,7 @@ public class FSImageFormatPBSnapshot {
     private final FileSummary.Builder headers;
     private final FSImageFormatProtobuf.Saver parent;
     private final SaveNamespaceContext context;
+    private long numImageErrors;
 
     public Saver(FSImageFormatProtobuf.Saver parent,
         FileSummary.Builder headers, SaveNamespaceContext context,
@@ -423,6 +426,7 @@ public class FSImageFormatPBSnapshot {
       this.headers = headers;
       this.context = context;
       this.fsn = fsn;
+      this.numImageErrors = 0;
     }
 
     /**
@@ -471,15 +475,17 @@ public class FSImageFormatPBSnapshot {
         throws IOException {
       final List<INodeReference> refList = parent.getSaverContext()
           .getRefList();
+      long i = 0;
       for (INodeReference ref : refList) {
-        INodeReferenceSection.INodeReference.Builder rb = buildINodeReference(ref);
+        INodeReferenceSection.INodeReference.Builder rb =
+            buildINodeReference(ref, i++);
         rb.build().writeDelimitedTo(out);
       }
       parent.commitSection(headers, SectionName.INODE_REFERENCE);
     }
 
     private INodeReferenceSection.INodeReference.Builder buildINodeReference(
-        INodeReference ref) throws IOException {
+        final INodeReference ref, final long refIndex) throws IOException {
       INodeReferenceSection.INodeReference.Builder rb =
           INodeReferenceSection.INodeReference.newBuilder().
             setReferredId(ref.getId());
@@ -489,6 +495,16 @@ public class FSImageFormatPBSnapshot {
       } else if (ref instanceof DstReference) {
         rb.setDstSnapshotId(ref.getDstSnapshotId());
       }
+
+      if (fsn.getFSDirectory().getInode(ref.getId()) == null) {
+        FSImage.LOG.error(
+            "FSImageFormatPBSnapshot: Missing referred INodeId " +
+            ref.getId() + " for INodeReference index " + refIndex +
+            "; path=" + ref.getFullPathName() +
+            "; parent=" + (ref.getParent() == null ? "null" :
+                ref.getParent().getFullPathName()));
+        ++numImageErrors;
+      }
       return rb;
     }
 
@@ -583,7 +599,23 @@ public class FSImageFormatPBSnapshot {
           List<INode> created = diff.getChildrenDiff().getCreatedUnmodifiable();
           db.setCreatedListSize(created.size());
           List<INode> deleted = diff.getChildrenDiff().getDeletedUnmodifiable();
+          INode previousNode = null;
+          boolean misordered = false;
           for (INode d : deleted) {
+            // getBytes() may return null below, and that is okay.
+            final int result = previousNode == null ? -1 :
+                previousNode.compareTo(d.getLocalNameBytes());
+            if (result == 0) {
+              FSImage.LOG.error(
+                  "Name '" + d.getLocalName() + "' is repeated in the " +
+                      "'deleted' difflist of directory " +
+                      dir.getFullPathName() + ", INodeId=" + dir.getId());
+              ++numImageErrors;
+            } else if (result > 0 && !misordered) {
+              misordered = true;
+              ++numImageErrors;
+            }
+            previousNode = d;
             if (d.isReference()) {
               refList.add(d.asReference());
               db.addDeletedINodeRef(refList.size() - 1);
@@ -591,11 +623,28 @@ public class FSImageFormatPBSnapshot {
               db.addDeletedINode(d.getId());
             }
           }
+          if (misordered) {
+            FSImage.LOG.error(
+                "Misordered entries in the 'deleted' difflist of directory " +
+                    dir.getFullPathName() + ", INodeId=" + dir.getId() +
+                    ". The full list is " +
+                    Arrays.toString(deleted.toArray()));
+          }
           db.build().writeDelimitedTo(out);
           saveCreatedList(created, out);
         }
       }
     }
+
+
+    /**
+     * Number of non-fatal errors detected while writing the
+     * SnapshotDiff and INodeReference sections.
+     * @return the number of non-fatal errors detected.
+     */
+    public long getNumImageErrors() {
+      return numImageErrors;
+    }
   }
 
   private FSImageFormatPBSnapshot(){}

+ 29 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java

@@ -143,7 +143,7 @@ public class NamenodeWebHdfsMethods {
         Boolean.valueOf(request.getHeader(WebHdfsFileSystem.EZ_HEADER));
   }
 
-  private void init(final UserGroupInformation ugi,
+  protected void init(final UserGroupInformation ugi,
       final DelegationParam delegation,
       final UserParam username, final DoAsParam doAsUser,
       final UriFsPathParam path, final HttpOpParam<?> op,
@@ -184,6 +184,14 @@ public class NamenodeWebHdfsMethods {
     return cp;
   }
 
+  protected String getScheme() {
+    return scheme;
+  }
+
+  protected ServletContext getContext() {
+    return context;
+  }
+
   private <T> T doAs(final UserGroupInformation ugi,
       final PrivilegedExceptionAction<T> action)
           throws IOException, InterruptedException {
@@ -206,7 +214,7 @@ public class NamenodeWebHdfsMethods {
       }
       @Override
       public String getHostAddress() {
-        return remoteAddr;
+        return getRemoteAddr();
       }
       @Override
       public InetAddress getHostInetAddress() {
@@ -217,8 +225,8 @@ public class NamenodeWebHdfsMethods {
         }
       }
     };
-    final NameNode namenode = (NameNode)context.getAttribute("name.node");
-    namenode.queueExternalCall(call);
+
+    queueExternalCall(call);
     T result = null;
     try {
       result = call.get();
@@ -235,6 +243,16 @@ public class NamenodeWebHdfsMethods {
     return result;
   }
 
+  protected String getRemoteAddr() {
+    return remoteAddr;
+  }
+
+  protected void queueExternalCall(ExternalCall call)
+      throws IOException, InterruptedException {
+    final NameNode namenode = (NameNode)context.getAttribute("name.node");
+    namenode.queueExternalCall(call);
+  }
+
   @VisibleForTesting
   static DatanodeInfo chooseDatanode(final NameNode namenode,
       final String path, final HttpOpParam.Op op, final long openOffset,
@@ -306,7 +324,7 @@ public class NamenodeWebHdfsMethods {
    * sorted based on availability and network distances, thus it is sufficient
    * to return the first element of the node here.
    */
-  private static DatanodeInfo bestNode(DatanodeInfo[] nodes,
+  protected static DatanodeInfo bestNode(DatanodeInfo[] nodes,
       HashSet<Node> excludes) throws IOException {
     for (DatanodeInfo dn: nodes) {
       if (false == dn.isDecommissioned() && false == excludes.contains(dn)) {
@@ -470,7 +488,7 @@ public class NamenodeWebHdfsMethods {
 
   /** Validate all required params. */
   @SuppressWarnings("rawtypes")
-  private void validateOpParams(HttpOpParam<?> op, Param... params) {
+  protected void validateOpParams(HttpOpParam<?> op, Param... params) {
     for (Param param : params) {
       if (param.getValue() == null || param.getValueString() == null || param
           .getValueString().isEmpty()) {
@@ -570,7 +588,7 @@ public class NamenodeWebHdfsMethods {
     });
   }
 
-  private Response put(
+  protected Response put(
       final UserGroupInformation ugi,
       final DelegationParam delegation,
       final UserParam username,
@@ -602,14 +620,13 @@ public class NamenodeWebHdfsMethods {
       final NoRedirectParam noredirectParam,
       final StoragePolicyParam policyName
       ) throws IOException, URISyntaxException {
-
     final Configuration conf = (Configuration)context.getAttribute(JspHelper.CURRENT_CONF);
-    final NameNode namenode = (NameNode)context.getAttribute("name.node");
     final ClientProtocol cp = getRpcClientProtocol();
 
     switch(op.getValue()) {
     case CREATE:
     {
+      final NameNode namenode = (NameNode)context.getAttribute("name.node");
       final URI uri = redirectURI(null, namenode, ugi, delegation, username,
           doAsUser, fullpath, op.getValue(), -1L, blockSize.getValue(conf),
           exclDatanodes.getValue(), permission, unmaskedPermission,
@@ -840,7 +857,7 @@ public class NamenodeWebHdfsMethods {
     });
   }
 
-  private Response post(
+  protected Response post(
       final UserGroupInformation ugi,
       final DelegationParam delegation,
       final UserParam username,
@@ -1014,7 +1031,7 @@ public class NamenodeWebHdfsMethods {
     return encodedValue;
   }
 
-  private Response get(
+  protected Response get(
       final UserGroupInformation ugi,
       final DelegationParam delegation,
       final UserParam username,
@@ -1344,7 +1361,7 @@ public class NamenodeWebHdfsMethods {
     });
   }
 
-  private Response delete(
+  protected Response delete(
       final UserGroupInformation ugi,
       final DelegationParam delegation,
       final UserParam username,

+ 15 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/TestRefreshCallQueue.java

@@ -30,8 +30,10 @@ import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
@@ -141,8 +143,16 @@ public class TestRefreshCallQueue {
 
     // throw an error when we double-initialize JvmMetrics
     DefaultMetricsSystem.setMiniClusterMode(false);
-
+    int serviceHandlerCount = config.getInt(
+        DFSConfigKeys.DFS_NAMENODE_SERVICE_HANDLER_COUNT_KEY,
+        DFSConfigKeys.DFS_NAMENODE_SERVICE_HANDLER_COUNT_DEFAULT);
     NameNodeRpcServer rpcServer = (NameNodeRpcServer) cluster.getNameNodeRpc();
+    // check callqueue size
+    assertEquals(CommonConfigurationKeys.IPC_SERVER_HANDLER_QUEUE_SIZE_DEFAULT
+        * serviceHandlerCount, rpcServer.getClientRpcServer().getMaxQueueSize());
+    // Replace queue and update queue size
+    config.setInt(CommonConfigurationKeys.IPC_SERVER_HANDLER_QUEUE_SIZE_KEY,
+        150);
     try {
       rpcServer.getClientRpcServer().refreshCallQueue(config);
     } catch (Exception e) {
@@ -158,6 +168,9 @@ public class TestRefreshCallQueue {
     } finally {
       DefaultMetricsSystem.setMiniClusterMode(oldValue);
     }
-  }
+    // check callQueueSize has changed
+    assertEquals(150 * serviceHandlerCount, rpcServer.getClientRpcServer()
+        .getMaxQueueSize());
+ }
 
 }

+ 13 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java

@@ -1413,11 +1413,20 @@ public class TestEncryptionZones {
     fsWrapper.mkdir(zone, FsPermission.getDirDefault(), true);
     final Path snap2 = fs.createSnapshot(zoneParent, "snap2");
     final Path snap2Zone = new Path(snap2, zone.getName());
+    assertEquals("Got unexpected ez path", zone.toString(),
+        dfsAdmin.getEncryptionZoneForPath(snap1Zone).getPath().toString());
     assertNull("Expected null ez path",
         dfsAdmin.getEncryptionZoneForPath(snap2Zone));
 
-    // Create the encryption zone again
+    // Create the encryption zone again, and that shouldn't affect old snapshot
     dfsAdmin.createEncryptionZone(zone, TEST_KEY2, NO_TRASH);
+    EncryptionZone ezSnap1 = dfsAdmin.getEncryptionZoneForPath(snap1Zone);
+    assertEquals("Got unexpected ez path", zone.toString(),
+        ezSnap1.getPath().toString());
+    assertEquals("Unexpected ez key", TEST_KEY, ezSnap1.getKeyName());
+    assertNull("Expected null ez path",
+        dfsAdmin.getEncryptionZoneForPath(snap2Zone));
+
     final Path snap3 = fs.createSnapshot(zoneParent, "snap3");
     final Path snap3Zone = new Path(snap3, zone.getName());
     // Check that snap3's EZ has the correct settings
@@ -1426,10 +1435,12 @@ public class TestEncryptionZones {
         ezSnap3.getPath().toString());
     assertEquals("Unexpected ez key", TEST_KEY2, ezSnap3.getKeyName());
     // Check that older snapshots still have the old EZ settings
-    EncryptionZone ezSnap1 = dfsAdmin.getEncryptionZoneForPath(snap1Zone);
+    ezSnap1 = dfsAdmin.getEncryptionZoneForPath(snap1Zone);
     assertEquals("Got unexpected ez path", zone.toString(),
         ezSnap1.getPath().toString());
     assertEquals("Unexpected ez key", TEST_KEY, ezSnap1.getKeyName());
+    assertNull("Expected null ez path",
+        dfsAdmin.getEncryptionZoneForPath(snap2Zone));
 
     // Check that listEZs only shows the current filesystem state
     ArrayList<EncryptionZone> listZones = Lists.newArrayList();

+ 6 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalDistributedCacheManager.java

@@ -37,7 +37,7 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.atomic.AtomicLong;
+import java.util.UUID;
 
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileUtil;
@@ -82,7 +82,7 @@ class LocalDistributedCacheManager {
    * @param conf
    * @throws IOException
    */
-  public void setup(JobConf conf) throws IOException {
+  public void setup(JobConf conf, JobID jobId) throws IOException {
     File workDir = new File(System.getProperty("user.dir"));
     
     // Generate YARN local resources objects corresponding to the distributed
@@ -91,9 +91,7 @@ class LocalDistributedCacheManager {
       new LinkedHashMap<String, LocalResource>();
     MRApps.setupDistributedCache(conf, localResources);
     // Generating unique numbers for FSDownload.
-    AtomicLong uniqueNumberGenerator =
-        new AtomicLong(System.currentTimeMillis());
-    
+
     // Find which resources are to be put on the local classpath
     Map<String, Path> classpaths = new HashMap<String, Path>();
     Path[] archiveClassPaths = DistributedCache.getArchiveClassPaths(conf);
@@ -124,9 +122,10 @@ class LocalDistributedCacheManager {
       Path destPath = localDirAllocator.getLocalPathForWrite(".", conf);
       Map<LocalResource, Future<Path>> resourcesToPaths = Maps.newHashMap();
       for (LocalResource resource : localResources.values()) {
+        Path destPathForDownload = new Path(destPath,
+            jobId.toString() + "_" + UUID.randomUUID().toString());
         Callable<Path> download =
-            new FSDownload(localFSFileContext, ugi, conf, new Path(destPath,
-                Long.toString(uniqueNumberGenerator.incrementAndGet())),
+            new FSDownload(localFSFileContext, ugi, conf, destPathForDownload,
                 resource);
         Future<Path> future = exec.submit(download);
         resourcesToPaths.put(resource, future);

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalJobRunner.java

@@ -169,7 +169,7 @@ public class LocalJobRunner implements ClientProtocol {
       // Manage the distributed cache.  If there are files to be copied,
       // this will trigger localFile to be re-written again.
       localDistributedCacheManager = new LocalDistributedCacheManager();
-      localDistributedCacheManager.setup(conf);
+      localDistributedCacheManager.setup(conf, jobid);
       
       // Write out configuration file.  Instead of copying it from
       // systemJobFile, we re-write it, since setup(), above, may have

+ 71 - 24
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapred/TestLocalDistributedCacheManager.java

@@ -32,6 +32,13 @@ import java.io.IOException;
 import java.net.URI;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.ArrayList;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
 
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -54,37 +61,37 @@ import org.mockito.stubbing.Answer;
 @SuppressWarnings("deprecation")
 public class TestLocalDistributedCacheManager {
 
-  private static FileSystem mockfs; 
-  
+  private static FileSystem mockfs;
+
   public static class MockFileSystem extends FilterFileSystem {
     public MockFileSystem() {
       super(mockfs);
     }
   }
-  
+
   private File localDir;
-  
+
   private static void delete(File file) throws IOException {
-    if (file.getAbsolutePath().length() < 5) { 
+    if (file.getAbsolutePath().length() < 5) {
       throw new IllegalArgumentException(
           "Path [" + file + "] is too short, not deleting");
     }
-    if (file.exists()) {  
+    if (file.exists()) {
       if (file.isDirectory()) {
         File[] children = file.listFiles();
         if (children != null) {
           for (File child : children) {
             delete(child);
-          } 
-        } 
-      } 
+          }
+        }
+      }
       if (!file.delete()) {
         throw new RuntimeException(
           "Could not delete path [" + file + "]");
       }
     }
   }
-  
+
   @Before
   public void setup() throws Exception {
     mockfs = mock(FileSystem.class);
@@ -93,7 +100,7 @@ public class TestLocalDistributedCacheManager {
     delete(localDir);
     localDir.mkdirs();
   }
-  
+
   @After
   public void cleanup() throws Exception {
     delete(localDir);
@@ -120,9 +127,10 @@ public class TestLocalDistributedCacheManager {
 
   @Test
   public void testDownload() throws Exception {
+    JobID jobId = new JobID();
     JobConf conf = new JobConf();
     conf.setClass("fs.mock.impl", MockFileSystem.class, FileSystem.class);
-    
+
     URI mockBase = new URI("mock://test-nn1/");
     when(mockfs.getUri()).thenReturn(mockBase);
     Path working = new Path("mock://test-nn1/user/me/");
@@ -137,14 +145,14 @@ public class TestLocalDistributedCacheManager {
     final URI file = new URI("mock://test-nn1/user/me/file.txt#link");
     final Path filePath = new Path(file);
     File link = new File("link");
-    
+
     when(mockfs.getFileStatus(any(Path.class))).thenAnswer(new Answer<FileStatus>() {
       @Override
       public FileStatus answer(InvocationOnMock args) throws Throwable {
         Path p = (Path)args.getArguments()[0];
         if("file.txt".equals(p.getName())) {
-         return new FileStatus(201, false, 1, 500, 101, 101, 
-             FsPermission.getDefault(), "me", "me", filePath);
+          return new FileStatus(201, false, 1, 500, 101, 101,
+              FsPermission.getDefault(), "me", "me", filePath);
         }  else {
           throw new FileNotFoundException(p+" not supported by mocking");
         }
@@ -176,7 +184,7 @@ public class TestLocalDistributedCacheManager {
     conf.set(MRConfig.LOCAL_DIR, localDir.getAbsolutePath());
     LocalDistributedCacheManager manager = new LocalDistributedCacheManager();
     try {
-      manager.setup(conf);
+      manager.setup(conf, jobId);
       assertTrue(link.exists());
     } finally {
       manager.close();
@@ -186,9 +194,10 @@ public class TestLocalDistributedCacheManager {
 
   @Test
   public void testEmptyDownload() throws Exception {
+    JobID jobId = new JobID();
     JobConf conf = new JobConf();
     conf.setClass("fs.mock.impl", MockFileSystem.class, FileSystem.class);
-    
+
     URI mockBase = new URI("mock://test-nn1/");
     when(mockfs.getUri()).thenReturn(mockBase);
     Path working = new Path("mock://test-nn1/user/me/");
@@ -199,7 +208,7 @@ public class TestLocalDistributedCacheManager {
         return (Path) args.getArguments()[0];
       }
     });
-    
+
     when(mockfs.getFileStatus(any(Path.class))).thenAnswer(new Answer<FileStatus>() {
       @Override
       public FileStatus answer(InvocationOnMock args) throws Throwable {
@@ -221,7 +230,7 @@ public class TestLocalDistributedCacheManager {
     conf.set(MRConfig.LOCAL_DIR, localDir.getAbsolutePath());
     LocalDistributedCacheManager manager = new LocalDistributedCacheManager();
     try {
-      manager.setup(conf);
+      manager.setup(conf, jobId);
     } finally {
       manager.close();
     }
@@ -230,9 +239,10 @@ public class TestLocalDistributedCacheManager {
 
   @Test
   public void testDuplicateDownload() throws Exception {
+    JobID jobId = new JobID();
     JobConf conf = new JobConf();
     conf.setClass("fs.mock.impl", MockFileSystem.class, FileSystem.class);
-    
+
     URI mockBase = new URI("mock://test-nn1/");
     when(mockfs.getUri()).thenReturn(mockBase);
     Path working = new Path("mock://test-nn1/user/me/");
@@ -247,14 +257,14 @@ public class TestLocalDistributedCacheManager {
     final URI file = new URI("mock://test-nn1/user/me/file.txt#link");
     final Path filePath = new Path(file);
     File link = new File("link");
-    
+
     when(mockfs.getFileStatus(any(Path.class))).thenAnswer(new Answer<FileStatus>() {
       @Override
       public FileStatus answer(InvocationOnMock args) throws Throwable {
         Path p = (Path)args.getArguments()[0];
         if("file.txt".equals(p.getName())) {
-         return new FileStatus(201, false, 1, 500, 101, 101, 
-             FsPermission.getDefault(), "me", "me", filePath);
+          return new FileStatus(201, false, 1, 500, 101, 101,
+              FsPermission.getDefault(), "me", "me", filePath);
         }  else {
           throw new FileNotFoundException(p+" not supported by mocking");
         }
@@ -287,11 +297,48 @@ public class TestLocalDistributedCacheManager {
     conf.set(MRConfig.LOCAL_DIR, localDir.getAbsolutePath());
     LocalDistributedCacheManager manager = new LocalDistributedCacheManager();
     try {
-      manager.setup(conf);
+      manager.setup(conf, jobId);
       assertTrue(link.exists());
     } finally {
       manager.close();
     }
     assertFalse(link.exists());
   }
+
+  /**
+   * This test tries to replicate the issue with the previous version of
+   * {@ref LocalDistributedCacheManager} when the resulting timestamp is
+   * identical as that in another process.  Unfortunately, it is difficult
+   * to mimic such behavior in a single process unit test.  And mocking
+   * the unique id (timestamp previously, UUID otherwise) won't prove the
+   * validity of one approach over the other.
+   */
+  @Test
+  public void testMultipleCacheSetup() throws Exception {
+    JobID jobId = new JobID();
+    JobConf conf = new JobConf();
+    LocalDistributedCacheManager manager = new LocalDistributedCacheManager();
+
+    final int threadCount = 10;
+    final CyclicBarrier barrier = new CyclicBarrier(threadCount);
+
+    ArrayList<Callable<Void>> setupCallable = new ArrayList<>();
+    for (int i = 0; i < threadCount; ++i) {
+      setupCallable.add(() -> {
+        barrier.await();
+        manager.setup(conf, jobId);
+        return null;
+      });
+    }
+
+    ExecutorService ePool = Executors.newFixedThreadPool(threadCount);
+    try {
+      for (Future<Void> future : ePool.invokeAll(setupCallable)) {
+        future.get();
+      }
+    } finally {
+      ePool.shutdown();
+      manager.close();
+    }
+  }
 }

+ 13 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestTaskID.java

@@ -17,8 +17,11 @@
  */
 package org.apache.hadoop.mapreduce;
 
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.nio.ByteBuffer;
+
 import org.apache.hadoop.io.DataInputByteBuffer;
-import org.apache.hadoop.io.DataOutputByteBuffer;
 import org.apache.hadoop.io.WritableUtils;
 import org.junit.Test;
 import static org.junit.Assert.*;
@@ -73,7 +76,7 @@ public class TestTaskID {
    * Test of getTaskType method, of class TaskID.
    */
   @Test
-  public void testGetTaskType_0args() {
+  public void testGetTaskType0args() {
     JobID jobId = new JobID("1234", 0);
 
     for (TaskType type : TaskType.values()) {
@@ -253,17 +256,18 @@ public class TestTaskID {
    */
   @Test
   public void testReadFields() throws Exception {
-    DataOutputByteBuffer out = new DataOutputByteBuffer();
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    DataOutputStream out = new DataOutputStream(baos);
 
     out.writeInt(0);
     out.writeInt(1);
     WritableUtils.writeVInt(out, 4);
-    out.write(new byte[] { 0x31, 0x32, 0x33, 0x34});
+    out.write(new byte[] {0x31, 0x32, 0x33, 0x34});
     WritableUtils.writeEnum(out, TaskType.REDUCE);
 
     DataInputByteBuffer in = new DataInputByteBuffer();
 
-    in.reset(out.getData());
+    in.reset(ByteBuffer.wrap(baos.toByteArray()));
 
     TaskID instance = new TaskID();
 
@@ -280,14 +284,15 @@ public class TestTaskID {
   public void testWrite() throws Exception {
     JobID jobId = new JobID("1234", 1);
     TaskID taskId = new TaskID(jobId, TaskType.JOB_SETUP, 0);
-    DataOutputByteBuffer out = new DataOutputByteBuffer();
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    DataOutputStream out = new DataOutputStream(baos);
 
     taskId.write(out);
 
     DataInputByteBuffer in = new DataInputByteBuffer();
     byte[] buffer = new byte[4];
 
-    in.reset(out.getData());
+    in.reset(ByteBuffer.wrap(baos.toByteArray()));
 
     assertEquals("The write() method did not write the expected task ID",
         0, in.readInt());
@@ -430,7 +435,7 @@ public class TestTaskID {
    * Test of getTaskType method, of class TaskID.
    */
   @Test
-  public void testGetTaskType_char() {
+  public void testGetTaskTypeChar() {
     assertEquals("The getTaskType() method did not return the expected type",
         TaskType.MAP,
         TaskID.getTaskType('m'));

+ 1 - 1
hadoop-project-dist/pom.xml

@@ -145,7 +145,7 @@
         <activeByDefault>false</activeByDefault>
       </activation>
       <properties>
-        <jdiff.stable.api>3.0.0</jdiff.stable.api>
+        <jdiff.stable.api>3.0.1</jdiff.stable.api>
         <jdiff.stability>-unstable</jdiff.stability>
         <!-- Commented out for HADOOP-11776 -->
         <!-- Uncomment param name="${jdiff.compatibility}" in javadoc doclet if compatibility is not empty -->

+ 1 - 1
hadoop-project/pom.xml

@@ -69,7 +69,7 @@
 
     <!-- jackson versions -->
     <jackson.version>1.9.13</jackson.version>
-    <jackson2.version>2.7.8</jackson2.version>
+    <jackson2.version>2.9.4</jackson2.version>
 
     <!-- SLF4J version -->
     <slf4j.version>1.7.25</slf4j.version>

+ 1 - 2
hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunCredentialsProvider.java

@@ -35,8 +35,7 @@ import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
 public class AliyunCredentialsProvider implements CredentialsProvider {
   private Credentials credentials = null;
 
-  public AliyunCredentialsProvider(Configuration conf)
-      throws IOException {
+  public AliyunCredentialsProvider(Configuration conf) throws IOException {
     String accessKeyId;
     String accessKeySecret;
     String securityToken;

+ 206 - 0
hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSBlockOutputStream.java

@@ -0,0 +1,206 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.aliyun.oss;
+
+import com.aliyun.oss.model.PartETag;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import org.apache.hadoop.conf.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+
+/**
+ * Asynchronous multi-part based uploading mechanism to support huge file
+ * which is larger than 5GB. Data will be buffered on local disk, then uploaded
+ * to OSS in {@link #close()} method.
+ */
+public class AliyunOSSBlockOutputStream extends OutputStream {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AliyunOSSBlockOutputStream.class);
+  private AliyunOSSFileSystemStore store;
+  private Configuration conf;
+  private boolean closed;
+  private String key;
+  private File blockFile;
+  private List<File> blockFiles = new ArrayList<>();
+  private long blockSize;
+  private int blockId = 0;
+  private long blockWritten = 0L;
+  private String uploadId = null;
+  private final List<ListenableFuture<PartETag>> partETagsFutures;
+  private final ListeningExecutorService executorService;
+  private OutputStream blockStream;
+  private final byte[] singleByte = new byte[1];
+
+  public AliyunOSSBlockOutputStream(Configuration conf,
+      AliyunOSSFileSystemStore store,
+      String key,
+      Long blockSize,
+      ExecutorService executorService) throws IOException {
+    this.store = store;
+    this.conf = conf;
+    this.key = key;
+    this.blockSize = blockSize;
+    this.blockFile = newBlockFile();
+    this.blockStream =
+        new BufferedOutputStream(new FileOutputStream(blockFile));
+    this.partETagsFutures = new ArrayList<>(2);
+    this.executorService = MoreExecutors.listeningDecorator(executorService);
+  }
+
+  private File newBlockFile() throws IOException {
+    return AliyunOSSUtils.createTmpFileForWrite(
+        String.format("oss-block-%04d-", blockId), blockSize, conf);
+  }
+
+  @Override
+  public synchronized void flush() throws IOException {
+    blockStream.flush();
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    if (closed) {
+      return;
+    }
+
+    blockStream.flush();
+    blockStream.close();
+    if (!blockFiles.contains(blockFile)) {
+      blockFiles.add(blockFile);
+    }
+
+    try {
+      if (blockFiles.size() == 1) {
+        // just upload it directly
+        store.uploadObject(key, blockFile);
+      } else {
+        if (blockWritten > 0) {
+          ListenableFuture<PartETag> partETagFuture =
+              executorService.submit(() -> {
+                PartETag partETag = store.uploadPart(blockFile, key, uploadId,
+                    blockId + 1);
+                return partETag;
+              });
+          partETagsFutures.add(partETagFuture);
+        }
+        // wait for the partial uploads to finish
+        final List<PartETag> partETags = waitForAllPartUploads();
+        if (null == partETags) {
+          throw new IOException("Failed to multipart upload to oss, abort it.");
+        }
+        store.completeMultipartUpload(key, uploadId, partETags);
+      }
+    } finally {
+      for (File tFile: blockFiles) {
+        if (tFile.exists() && !tFile.delete()) {
+          LOG.warn("Failed to delete temporary file {}", tFile);
+        }
+      }
+      closed = true;
+    }
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    singleByte[0] = (byte)b;
+    write(singleByte, 0, 1);
+  }
+
+  @Override
+  public synchronized void write(byte[] b, int off, int len)
+      throws IOException {
+    if (closed) {
+      throw new IOException("Stream closed.");
+    }
+    try {
+      blockStream.write(b, off, len);
+      blockWritten += len;
+      if (blockWritten >= blockSize) {
+        uploadCurrentPart();
+        blockWritten = 0L;
+      }
+    } finally {
+      for (File tFile: blockFiles) {
+        if (tFile.exists() && !tFile.delete()) {
+          LOG.warn("Failed to delete temporary file {}", tFile);
+        }
+      }
+    }
+  }
+
+  private void uploadCurrentPart() throws IOException {
+    blockFiles.add(blockFile);
+    blockStream.flush();
+    blockStream.close();
+    if (blockId == 0) {
+      uploadId = store.getUploadId(key);
+    }
+    ListenableFuture<PartETag> partETagFuture =
+        executorService.submit(() -> {
+          PartETag partETag = store.uploadPart(blockFile, key, uploadId,
+              blockId + 1);
+          return partETag;
+        });
+    partETagsFutures.add(partETagFuture);
+    blockFile = newBlockFile();
+    blockId++;
+    blockStream = new BufferedOutputStream(new FileOutputStream(blockFile));
+  }
+
+  /**
+   * Block awaiting all outstanding uploads to complete.
+   * @return list of results
+   * @throws IOException IO Problems
+   */
+  private List<PartETag> waitForAllPartUploads() throws IOException {
+    LOG.debug("Waiting for {} uploads to complete", partETagsFutures.size());
+    try {
+      return Futures.allAsList(partETagsFutures).get();
+    } catch (InterruptedException ie) {
+      LOG.warn("Interrupted partUpload", ie);
+      Thread.currentThread().interrupt();
+      return null;
+    } catch (ExecutionException ee) {
+      //there is no way of recovering so abort
+      //cancel all partUploads
+      LOG.debug("While waiting for upload completion", ee);
+      LOG.debug("Cancelling futures");
+      for (ListenableFuture<PartETag> future : partETagsFutures) {
+        future.cancel(true);
+      }
+      //abort multipartupload
+      store.abortMultipartUpload(key, uploadId);
+      throw new IOException("Multi-part upload with id '" + uploadId
+        + "' to " + key, ee);
+    }
+  }
+}

+ 24 - 10
hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java

@@ -56,6 +56,8 @@ import org.apache.hadoop.util.SemaphoredDelegatingExecutor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.hadoop.fs.aliyun.oss.AliyunOSSUtils.intOption;
+import static org.apache.hadoop.fs.aliyun.oss.AliyunOSSUtils.longOption;
 import static org.apache.hadoop.fs.aliyun.oss.AliyunOSSUtils.objectRepresentsDirectory;
 import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
 
@@ -69,6 +71,7 @@ public class AliyunOSSFileSystem extends FileSystem {
   private URI uri;
   private String bucket;
   private Path workingDir;
+  private int blockOutputActiveBlocks;
   private AliyunOSSFileSystemStore store;
   private int maxKeys;
   private int maxReadAheadPartNumber;
@@ -125,8 +128,15 @@ public class AliyunOSSFileSystem extends FileSystem {
       // this means the file is not found
     }
 
-    return new FSDataOutputStream(new AliyunOSSOutputStream(getConf(),
-        store, key, progress, statistics), (Statistics)(null));
+    long uploadPartSize = AliyunOSSUtils.getMultipartSizeProperty(getConf(),
+        MULTIPART_UPLOAD_PART_SIZE_KEY, MULTIPART_UPLOAD_PART_SIZE_DEFAULT);
+    return new FSDataOutputStream(
+        new AliyunOSSBlockOutputStream(getConf(),
+            store,
+            key,
+            uploadPartSize,
+            new SemaphoredDelegatingExecutor(boundedThreadPool,
+                blockOutputActiveBlocks, true)), (Statistics)(null));
   }
 
   /**
@@ -149,9 +159,8 @@ public class AliyunOSSFileSystem extends FileSystem {
         throw new FileAlreadyExistsException("Not a directory: " + parent);
       }
     }
-    return create(path, permission,
-      flags.contains(CreateFlag.OVERWRITE), bufferSize,
-      replication, blockSize, progress);
+    return create(path, permission, flags.contains(CreateFlag.OVERWRITE),
+        bufferSize, replication, blockSize, progress);
   }
 
   @Override
@@ -270,7 +279,7 @@ public class AliyunOSSFileSystem extends FileSystem {
       }
     } else if (objectRepresentsDirectory(key, meta.getContentLength())) {
       return new FileStatus(0, true, 1, 0, meta.getLastModified().getTime(),
-           qualifiedPath);
+          qualifiedPath);
     } else {
       return new FileStatus(meta.getContentLength(), false, 1,
           getDefaultBlockSize(path), meta.getLastModified().getTime(),
@@ -318,6 +327,10 @@ public class AliyunOSSFileSystem extends FileSystem {
     uri = java.net.URI.create(name.getScheme() + "://" + name.getAuthority());
     workingDir = new Path("/user",
         System.getProperty("user.name")).makeQualified(uri, null);
+    long keepAliveTime = longOption(conf,
+        KEEPALIVE_TIME_KEY, KEEPALIVE_TIME_DEFAULT, 0);
+    blockOutputActiveBlocks = intOption(conf,
+        UPLOAD_ACTIVE_BLOCKS_KEY, UPLOAD_ACTIVE_BLOCKS_DEFAULT, 1);
 
     store = new AliyunOSSFileSystemStore();
     store.initialize(name, conf, statistics);
@@ -335,7 +348,8 @@ public class AliyunOSSFileSystem extends FileSystem {
         Constants.MULTIPART_DOWNLOAD_AHEAD_PART_MAX_NUM_DEFAULT);
 
     this.boundedThreadPool = BlockingThreadPoolExecutorService.newInstance(
-        threadNum, totalTasks, 60L, TimeUnit.SECONDS, "oss-read-shared");
+        threadNum, totalTasks, keepAliveTime, TimeUnit.SECONDS,
+        "oss-transfer-shared");
 
     maxConcurrentCopyTasksPerDir = AliyunOSSUtils.intPositiveOption(conf,
         Constants.MAX_CONCURRENT_COPY_TASKS_PER_DIR_KEY,
@@ -490,12 +504,12 @@ public class AliyunOSSFileSystem extends FileSystem {
     if (status.isFile()) {
       LOG.debug("{} is a File", qualifiedPath);
       final BlockLocation[] locations = getFileBlockLocations(status,
-        0, status.getLen());
+          0, status.getLen());
       return store.singleStatusRemoteIterator(filter.accept(f) ? status : null,
-        locations);
+          locations);
     } else {
       return store.createLocatedFileStatusIterator(key, maxKeys, this, filter,
-        acceptor, recursive ? null : "/");
+          acceptor, recursive ? null : "/");
     }
   }
 

+ 90 - 83
hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java

@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.hadoop.fs.aliyun.oss;
 
 import com.aliyun.oss.ClientConfiguration;
@@ -62,8 +63,11 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.Serializable;
 import java.net.URI;
 import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
 import java.util.List;
 import java.util.ListIterator;
 import java.util.NoSuchElementException;
@@ -83,7 +87,6 @@ public class AliyunOSSFileSystemStore {
   private String bucketName;
   private long uploadPartSize;
   private long multipartThreshold;
-  private long partSize;
   private int maxKeys;
   private String serverSideEncryptionAlgorithm;
 
@@ -143,28 +146,18 @@ public class AliyunOSSFileSystemStore {
     String endPoint = conf.getTrimmed(ENDPOINT_KEY, "");
     if (StringUtils.isEmpty(endPoint)) {
       throw new IllegalArgumentException("Aliyun OSS endpoint should not be " +
-        "null or empty. Please set proper endpoint with 'fs.oss.endpoint'.");
+          "null or empty. Please set proper endpoint with 'fs.oss.endpoint'.");
     }
     CredentialsProvider provider =
         AliyunOSSUtils.getCredentialsProvider(conf);
     ossClient = new OSSClient(endPoint, provider, clientConf);
-    uploadPartSize = conf.getLong(MULTIPART_UPLOAD_SIZE_KEY,
-        MULTIPART_UPLOAD_SIZE_DEFAULT);
+    uploadPartSize = AliyunOSSUtils.getMultipartSizeProperty(conf,
+        MULTIPART_UPLOAD_PART_SIZE_KEY, MULTIPART_UPLOAD_PART_SIZE_DEFAULT);
     multipartThreshold = conf.getLong(MIN_MULTIPART_UPLOAD_THRESHOLD_KEY,
         MIN_MULTIPART_UPLOAD_THRESHOLD_DEFAULT);
-    partSize = conf.getLong(MULTIPART_UPLOAD_SIZE_KEY,
-        MULTIPART_UPLOAD_SIZE_DEFAULT);
-    if (partSize < MIN_MULTIPART_UPLOAD_PART_SIZE) {
-      partSize = MIN_MULTIPART_UPLOAD_PART_SIZE;
-    }
     serverSideEncryptionAlgorithm =
         conf.get(SERVER_SIDE_ENCRYPTION_ALGORITHM_KEY, "");
 
-    if (uploadPartSize < 5 * 1024 * 1024) {
-      LOG.warn(MULTIPART_UPLOAD_SIZE_KEY + " must be at least 5 MB");
-      uploadPartSize = 5 * 1024 * 1024;
-    }
-
     if (multipartThreshold < 5 * 1024 * 1024) {
       LOG.warn(MIN_MULTIPART_UPLOAD_THRESHOLD_KEY + " must be at least 5 MB");
       multipartThreshold = 5 * 1024 * 1024;
@@ -419,71 +412,6 @@ public class AliyunOSSFileSystemStore {
     }
   }
 
-  /**
-   * Upload a file as an OSS object, using multipart upload.
-   *
-   * @param key object key.
-   * @param file local file to upload.
-   * @throws IOException if failed to upload object.
-   */
-  public void multipartUploadObject(String key, File file) throws IOException {
-    File object = file.getAbsoluteFile();
-    long dataLen = object.length();
-    long realPartSize = AliyunOSSUtils.calculatePartSize(dataLen, partSize);
-    int partNum = (int) (dataLen / realPartSize);
-    if (dataLen % realPartSize != 0) {
-      partNum += 1;
-    }
-
-    InitiateMultipartUploadRequest initiateMultipartUploadRequest =
-        new InitiateMultipartUploadRequest(bucketName, key);
-    ObjectMetadata meta = new ObjectMetadata();
-    if (StringUtils.isNotEmpty(serverSideEncryptionAlgorithm)) {
-      meta.setServerSideEncryption(serverSideEncryptionAlgorithm);
-    }
-    initiateMultipartUploadRequest.setObjectMetadata(meta);
-    InitiateMultipartUploadResult initiateMultipartUploadResult =
-        ossClient.initiateMultipartUpload(initiateMultipartUploadRequest);
-    List<PartETag> partETags = new ArrayList<PartETag>();
-    String uploadId = initiateMultipartUploadResult.getUploadId();
-
-    try {
-      for (int i = 0; i < partNum; i++) {
-        // TODO: Optimize this, avoid opening the object multiple times
-        FileInputStream fis = new FileInputStream(object);
-        try {
-          long skipBytes = realPartSize * i;
-          AliyunOSSUtils.skipFully(fis, skipBytes);
-          long size = (realPartSize < dataLen - skipBytes) ?
-              realPartSize : dataLen - skipBytes;
-          UploadPartRequest uploadPartRequest = new UploadPartRequest();
-          uploadPartRequest.setBucketName(bucketName);
-          uploadPartRequest.setKey(key);
-          uploadPartRequest.setUploadId(uploadId);
-          uploadPartRequest.setInputStream(fis);
-          uploadPartRequest.setPartSize(size);
-          uploadPartRequest.setPartNumber(i + 1);
-          UploadPartResult uploadPartResult =
-              ossClient.uploadPart(uploadPartRequest);
-          statistics.incrementWriteOps(1);
-          partETags.add(uploadPartResult.getPartETag());
-        } finally {
-          fis.close();
-        }
-      }
-      CompleteMultipartUploadRequest completeMultipartUploadRequest =
-          new CompleteMultipartUploadRequest(bucketName, key,
-              uploadId, partETags);
-      CompleteMultipartUploadResult completeMultipartUploadResult =
-          ossClient.completeMultipartUpload(completeMultipartUploadRequest);
-      LOG.debug(completeMultipartUploadResult.getETag());
-    } catch (OSSException | ClientException e) {
-      AbortMultipartUploadRequest abortMultipartUploadRequest =
-          new AbortMultipartUploadRequest(bucketName, key, uploadId);
-      ossClient.abortMultipartUpload(abortMultipartUploadRequest);
-    }
-  }
-
   /**
    * list objects.
    *
@@ -494,7 +422,7 @@ public class AliyunOSSFileSystemStore {
    * @return a list of matches.
    */
   public ObjectListing listObjects(String prefix, int maxListingLength,
-                                   String marker, boolean recursive) {
+      String marker, boolean recursive) {
     String delimiter = recursive ? null : "/";
     prefix = AliyunOSSUtils.maybeAddTrailingSlash(prefix);
     ListObjectsRequest listRequest = new ListObjectsRequest(bucketName);
@@ -605,7 +533,7 @@ public class AliyunOSSFileSystemStore {
         if (hasNext()) {
           FileStatus status = batchIterator.next();
           BlockLocation[] locations = fs.getFileBlockLocations(status,
-            0, status.getLen());
+              0, status.getLen());
           return new LocatedFileStatus(
               status, status.isFile() ? locations : null);
         } else {
@@ -626,7 +554,7 @@ public class AliyunOSSFileSystemStore {
         List<FileStatus> stats = new ArrayList<>(
             listing.getObjectSummaries().size() +
             listing.getCommonPrefixes().size());
-        for(OSSObjectSummary summary: listing.getObjectSummaries()) {
+        for (OSSObjectSummary summary : listing.getObjectSummaries()) {
           String key = summary.getKey();
           Path path = fs.makeQualified(new Path("/" + key));
           if (filter.accept(path) && acceptor.accept(path, summary)) {
@@ -637,7 +565,7 @@ public class AliyunOSSFileSystemStore {
           }
         }
 
-        for(String commonPrefix: listing.getCommonPrefixes()) {
+        for (String commonPrefix : listing.getCommonPrefixes()) {
           Path path = fs.makeQualified(new Path("/" + commonPrefix));
           if (filter.accept(path) && acceptor.accept(path, commonPrefix)) {
             FileStatus status = new FileStatus(0, true, 1, 0, 0, path);
@@ -656,4 +584,83 @@ public class AliyunOSSFileSystemStore {
       }
     };
   }
+
+  public PartETag uploadPart(File file, String key, String uploadId, int idx)
+      throws IOException {
+    InputStream instream = null;
+    Exception caught = null;
+    int tries = 3;
+    while (tries > 0) {
+      try {
+        instream = new FileInputStream(file);
+        UploadPartRequest uploadRequest = new UploadPartRequest();
+        uploadRequest.setBucketName(bucketName);
+        uploadRequest.setKey(key);
+        uploadRequest.setUploadId(uploadId);
+        uploadRequest.setInputStream(instream);
+        uploadRequest.setPartSize(file.length());
+        uploadRequest.setPartNumber(idx);
+        UploadPartResult uploadResult = ossClient.uploadPart(uploadRequest);
+        return uploadResult.getPartETag();
+      } catch (Exception e) {
+        LOG.debug("Failed to upload "+ file.getPath() +", " +
+            "try again.", e);
+        caught = e;
+      } finally {
+        if (instream != null) {
+          instream.close();
+          instream = null;
+        }
+      }
+      tries--;
+    }
+
+    assert (caught != null);
+    throw new IOException("Failed to upload " + file.getPath() +
+        " for 3 times.", caught);
+  }
+
+  /**
+   * Initiate multipart upload.
+   */
+  public String getUploadId(String key) {
+    InitiateMultipartUploadRequest initiateMultipartUploadRequest =
+        new InitiateMultipartUploadRequest(bucketName, key);
+    InitiateMultipartUploadResult initiateMultipartUploadResult =
+        ossClient.initiateMultipartUpload(initiateMultipartUploadRequest);
+    return initiateMultipartUploadResult.getUploadId();
+  }
+
+  /**
+   * Complete the specific multipart upload.
+   */
+  public CompleteMultipartUploadResult completeMultipartUpload(String key,
+      String uploadId, List<PartETag> partETags) {
+    Collections.sort(partETags, new PartNumberAscendComparator());
+    CompleteMultipartUploadRequest completeMultipartUploadRequest =
+        new CompleteMultipartUploadRequest(bucketName, key, uploadId,
+            partETags);
+    return ossClient.completeMultipartUpload(completeMultipartUploadRequest);
+  }
+
+  /**
+   * Abort the specific multipart upload.
+   */
+  public void abortMultipartUpload(String key, String uploadId) {
+    AbortMultipartUploadRequest request = new AbortMultipartUploadRequest(
+        bucketName, key, uploadId);
+    ossClient.abortMultipartUpload(request);
+  }
+
+  private static class PartNumberAscendComparator
+      implements Comparator<PartETag>, Serializable {
+    @Override
+    public int compare(PartETag o1, PartETag o2) {
+      if (o1.getPartNumber() > o2.getPartNumber()) {
+        return 1;
+      } else {
+        return -1;
+      }
+    }
+  }
 }

+ 0 - 111
hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSOutputStream.java

@@ -1,111 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.aliyun.oss;
-
-import java.io.BufferedOutputStream;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem.Statistics;
-import org.apache.hadoop.fs.LocalDirAllocator;
-import org.apache.hadoop.util.Progressable;
-
-import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
-
-/**
- * The output stream for OSS blob system.
- * Data will be buffered on local disk, then uploaded to OSS in
- * {@link #close()} method.
- */
-public class AliyunOSSOutputStream extends OutputStream {
-  public static final Log LOG = LogFactory.getLog(AliyunOSSOutputStream.class);
-  private AliyunOSSFileSystemStore store;
-  private final String key;
-  private Statistics statistics;
-  private Progressable progress;
-  private long partSizeThreshold;
-  private LocalDirAllocator dirAlloc;
-  private boolean closed;
-  private File tmpFile;
-  private BufferedOutputStream backupStream;
-
-  public AliyunOSSOutputStream(Configuration conf,
-      AliyunOSSFileSystemStore store, String key, Progressable progress,
-      Statistics statistics) throws IOException {
-    this.store = store;
-    this.key = key;
-    // The caller cann't get any progress information
-    this.progress = progress;
-    this.statistics = statistics;
-    partSizeThreshold = conf.getLong(MIN_MULTIPART_UPLOAD_THRESHOLD_KEY,
-        MIN_MULTIPART_UPLOAD_THRESHOLD_DEFAULT);
-
-    if (conf.get(BUFFER_DIR_KEY) == null) {
-      conf.set(BUFFER_DIR_KEY, conf.get("hadoop.tmp.dir") + "/oss");
-    }
-    dirAlloc = new LocalDirAllocator(BUFFER_DIR_KEY);
-
-    tmpFile = dirAlloc.createTmpFileForWrite("output-",
-        LocalDirAllocator.SIZE_UNKNOWN, conf);
-    backupStream = new BufferedOutputStream(new FileOutputStream(tmpFile));
-    closed = false;
-  }
-
-  @Override
-  public synchronized void close() throws IOException {
-    if (closed) {
-      return;
-    }
-    closed = true;
-    if (backupStream != null) {
-      backupStream.close();
-    }
-    long dataLen = tmpFile.length();
-    try {
-      if (dataLen <= partSizeThreshold) {
-        store.uploadObject(key, tmpFile);
-      } else {
-        store.multipartUploadObject(key, tmpFile);
-      }
-    } finally {
-      if (!tmpFile.delete()) {
-        LOG.warn("Can not delete file: " + tmpFile);
-      }
-    }
-  }
-
-
-
-  @Override
-  public synchronized void flush() throws IOException {
-    backupStream.flush();
-  }
-
-  @Override
-  public synchronized void write(int b) throws IOException {
-    backupStream.write(b);
-    statistics.incrementBytesWritten(1);
-  }
-
-}

+ 87 - 28
hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java

@@ -18,12 +18,14 @@
 
 package org.apache.hadoop.fs.aliyun.oss;
 
+import java.io.File;
 import java.io.IOException;
-import java.io.InputStream;
 
 import com.aliyun.oss.common.auth.CredentialsProvider;
+import com.google.common.base.Preconditions;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.security.ProviderUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -36,6 +38,7 @@ import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
 final public class AliyunOSSUtils {
   private static final Logger LOG =
       LoggerFactory.getLogger(AliyunOSSUtils.class);
+  private static LocalDirAllocator directoryAllocator;
 
   private AliyunOSSUtils() {
   }
@@ -74,31 +77,6 @@ final public class AliyunOSSUtils {
     }
   }
 
-  /**
-   * Skip the requested number of bytes or fail if there are no enough bytes
-   * left. This allows for the possibility that {@link InputStream#skip(long)}
-   * may not skip as many bytes as requested (most likely because of reaching
-   * EOF).
-   *
-   * @param is the input stream to skip.
-   * @param n the number of bytes to skip.
-   * @throws IOException thrown when skipped less number of bytes.
-   */
-  public static void skipFully(InputStream is, long n) throws IOException {
-    long total = 0;
-    long cur = 0;
-
-    do {
-      cur = is.skip(n - total);
-      total += cur;
-    } while((total < n) && (cur > 0));
-
-    if (total < n) {
-      throw new IOException("Failed to skip " + n + " bytes, possibly due " +
-              "to EOF.");
-    }
-  }
-
   /**
    * Calculate a proper size of multipart piece. If <code>minPartSize</code>
    * is too small, the number of multipart pieces may exceed the limit of
@@ -126,7 +104,7 @@ final public class AliyunOSSUtils {
       throws IOException {
     CredentialsProvider credentials;
 
-    String className = conf.getTrimmed(ALIYUN_OSS_CREDENTIALS_PROVIDER_KEY);
+    String className = conf.getTrimmed(CREDENTIALS_PROVIDER_KEY);
     if (StringUtils.isEmpty(className)) {
       Configuration newConf =
           ProviderUtils.excludeIncompatibleCredentialProviders(conf,
@@ -151,7 +129,7 @@ final public class AliyunOSSUtils {
         throw new IOException(String.format("%s constructor exception.  A " +
             "class specified in %s must provide an accessible constructor " +
             "accepting URI and Configuration, or an accessible default " +
-            "constructor.", className, ALIYUN_OSS_CREDENTIALS_PROVIDER_KEY),
+            "constructor.", className, CREDENTIALS_PROVIDER_KEY),
             e);
       } catch (ReflectiveOperationException | IllegalArgumentException e) {
         throw new IOException(className + " instantiation exception.", e);
@@ -188,4 +166,85 @@ final public class AliyunOSSUtils {
       final long size) {
     return StringUtils.isNotEmpty(name) && name.endsWith("/") && size == 0L;
   }
+
+  /**
+   * Demand create the directory allocator, then create a temporary file.
+   *  @param path prefix for the temporary file
+   *  @param size the size of the file that is going to be written
+   *  @param conf the Configuration object
+   *  @return a unique temporary file
+   *  @throws IOException IO problems
+   */
+  public static File createTmpFileForWrite(String path, long size,
+      Configuration conf) throws IOException {
+    if (conf.get(BUFFER_DIR_KEY) == null) {
+      conf.set(BUFFER_DIR_KEY, conf.get("hadoop.tmp.dir") + "/oss");
+    }
+    if (directoryAllocator == null) {
+      directoryAllocator = new LocalDirAllocator(BUFFER_DIR_KEY);
+    }
+    return directoryAllocator.createTmpFileForWrite(path, size, conf);
+  }
+
+  /**
+   * Get a integer option >= the minimum allowed value.
+   * @param conf configuration
+   * @param key key to look up
+   * @param defVal default value
+   * @param min minimum value
+   * @return the value
+   * @throws IllegalArgumentException if the value is below the minimum
+   */
+  static int intOption(Configuration conf, String key, int defVal, int min) {
+    int v = conf.getInt(key, defVal);
+    Preconditions.checkArgument(v >= min,
+        String.format("Value of %s: %d is below the minimum value %d",
+            key, v, min));
+    LOG.debug("Value of {} is {}", key, v);
+    return v;
+  }
+
+  /**
+   * Get a long option >= the minimum allowed value.
+   * @param conf configuration
+   * @param key key to look up
+   * @param defVal default value
+   * @param min minimum value
+   * @return the value
+   * @throws IllegalArgumentException if the value is below the minimum
+   */
+  static long longOption(Configuration conf, String key, long defVal,
+      long min) {
+    long v = conf.getLong(key, defVal);
+    Preconditions.checkArgument(v >= min,
+        String.format("Value of %s: %d is below the minimum value %d",
+            key, v, min));
+    LOG.debug("Value of {} is {}", key, v);
+    return v;
+  }
+
+  /**
+   * Get a size property from the configuration: this property must
+   * be at least equal to {@link Constants#MULTIPART_MIN_SIZE}.
+   * If it is too small, it is rounded up to that minimum, and a warning
+   * printed.
+   * @param conf configuration
+   * @param property property name
+   * @param defVal default value
+   * @return the value, guaranteed to be above the minimum size
+   */
+  public static long getMultipartSizeProperty(Configuration conf,
+      String property, long defVal) {
+    long partSize = conf.getLong(property, defVal);
+    if (partSize < MULTIPART_MIN_SIZE) {
+      LOG.warn("{} must be at least 100 KB; configured value is {}",
+          property, partSize);
+      partSize = MULTIPART_MIN_SIZE;
+    } else if (partSize > Integer.MAX_VALUE) {
+      LOG.warn("oss: {} capped to ~2.14GB(maximum allowed size with " +
+          "current output mechanism)", MULTIPART_UPLOAD_PART_SIZE_KEY);
+      partSize = Integer.MAX_VALUE;
+    }
+    return partSize;
+  }
 }

+ 15 - 7
hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java

@@ -31,10 +31,10 @@ public final class Constants {
   // User agent
   public static final String USER_AGENT_PREFIX = "fs.oss.user.agent.prefix";
   public static final String USER_AGENT_PREFIX_DEFAULT =
-          VersionInfoUtils.getDefaultUserAgent();
+      VersionInfoUtils.getDefaultUserAgent();
 
   // Class of credential provider
-  public static final String ALIYUN_OSS_CREDENTIALS_PROVIDER_KEY =
+  public static final String CREDENTIALS_PROVIDER_KEY =
       "fs.oss.credentials.provider";
 
   // OSS access verification
@@ -82,10 +82,14 @@ public final class Constants {
   public static final int MAX_PAGING_KEYS_DEFAULT = 1000;
 
   // Size of each of or multipart pieces in bytes
-  public static final String MULTIPART_UPLOAD_SIZE_KEY =
+  public static final String MULTIPART_UPLOAD_PART_SIZE_KEY =
       "fs.oss.multipart.upload.size";
+  public static final long MULTIPART_UPLOAD_PART_SIZE_DEFAULT =
+      104857600; // 100 MB
+
+  /** The minimum multipart size which Aliyun OSS supports. */
+  public static final int MULTIPART_MIN_SIZE = 100 * 1024;
 
-  public static final long MULTIPART_UPLOAD_SIZE_DEFAULT = 10 * 1024 * 1024;
   public static final int MULTIPART_UPLOAD_PART_NUM_LIMIT = 10000;
 
   // Minimum size in bytes before we start a multipart uploads or copy
@@ -96,7 +100,6 @@ public final class Constants {
 
   public static final String MULTIPART_DOWNLOAD_SIZE_KEY =
       "fs.oss.multipart.download.size";
-
   public static final long MULTIPART_DOWNLOAD_SIZE_DEFAULT = 512 * 1024;
 
   public static final String MULTIPART_DOWNLOAD_THREAD_NUMBER_KEY =
@@ -139,9 +142,14 @@ public final class Constants {
 
   public static final String FS_OSS_BLOCK_SIZE_KEY = "fs.oss.block.size";
   public static final int FS_OSS_BLOCK_SIZE_DEFAULT = 64 * 1024 * 1024;
+
   public static final String FS_OSS = "oss";
 
-  public static final long MIN_MULTIPART_UPLOAD_PART_SIZE = 100 * 1024L;
-  public static final int MAX_RETRIES = 10;
+  public static final String KEEPALIVE_TIME_KEY =
+      "fs.oss.threads.keepalivetime";
+  public static final int KEEPALIVE_TIME_DEFAULT = 60;
 
+  public static final String UPLOAD_ACTIVE_BLOCKS_KEY =
+      "fs.oss.upload.active.blocks";
+  public static final int UPLOAD_ACTIVE_BLOCKS_DEFAULT = 4;
 }

+ 28 - 4
hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSOutputStream.java → hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSBlockOutputStream.java

@@ -30,10 +30,13 @@ import org.junit.rules.Timeout;
 
 import java.io.IOException;
 
+import static org.apache.hadoop.fs.aliyun.oss.Constants.MULTIPART_UPLOAD_PART_SIZE_DEFAULT;
+
 /**
- * Tests regular and multi-part upload functionality for AliyunOSSOutputStream.
+ * Tests regular and multi-part upload functionality for
+ * AliyunOSSBlockOutputStream.
  */
-public class TestAliyunOSSOutputStream {
+public class TestAliyunOSSBlockOutputStream {
   private FileSystem fs;
   private static String testRootPath =
       AliyunOSSTestUtils.generateUniqueTestPath();
@@ -45,7 +48,7 @@ public class TestAliyunOSSOutputStream {
   public void setUp() throws Exception {
     Configuration conf = new Configuration();
     conf.setLong(Constants.MIN_MULTIPART_UPLOAD_THRESHOLD_KEY, 5 * 1024 * 1024);
-    conf.setInt(Constants.MULTIPART_UPLOAD_SIZE_KEY, 5 * 1024 * 1024);
+    conf.setInt(Constants.MULTIPART_UPLOAD_PART_SIZE_KEY, 5 * 1024 * 1024);
     fs = AliyunOSSTestUtils.createTestFileSystem(conf);
   }
 
@@ -56,18 +59,39 @@ public class TestAliyunOSSOutputStream {
     }
   }
 
-  protected Path getTestPath() {
+  private Path getTestPath() {
     return new Path(testRootPath + "/test-aliyun-oss");
   }
 
+  @Test
+  public void testZeroByteUpload() throws IOException {
+    ContractTestUtils.createAndVerifyFile(fs, getTestPath(), 0);
+  }
+
   @Test
   public void testRegularUpload() throws IOException {
+    ContractTestUtils.createAndVerifyFile(fs, getTestPath(), 1024 * 1024 - 1);
     ContractTestUtils.createAndVerifyFile(fs, getTestPath(), 1024 * 1024);
+    ContractTestUtils.createAndVerifyFile(fs, getTestPath(), 1024 * 1024 + 1);
   }
 
   @Test
   public void testMultiPartUpload() throws IOException {
+    ContractTestUtils.createAndVerifyFile(fs, getTestPath(),
+        6 * 1024 * 1024 - 1);
     ContractTestUtils.createAndVerifyFile(fs, getTestPath(), 6 * 1024 * 1024);
+    ContractTestUtils.createAndVerifyFile(fs, getTestPath(),
+        6 * 1024 * 1024 + 1);
+  }
+
+  @Test
+  public void testHugeUpload() throws IOException {
+    ContractTestUtils.createAndVerifyFile(fs, getTestPath(),
+        MULTIPART_UPLOAD_PART_SIZE_DEFAULT - 1);
+    ContractTestUtils.createAndVerifyFile(fs, getTestPath(),
+        MULTIPART_UPLOAD_PART_SIZE_DEFAULT);
+    ContractTestUtils.createAndVerifyFile(fs, getTestPath(),
+        MULTIPART_UPLOAD_PART_SIZE_DEFAULT + 1);
   }
 
   @Test

+ 5 - 5
hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSInputStream.java

@@ -123,15 +123,15 @@ public class TestAliyunOSSInputStream {
         + fsDataInputStream.getPos(), fsDataInputStream.getPos() == 0);
 
     assertTrue("expected position at:"
-            + Constants.MULTIPART_DOWNLOAD_SIZE_DEFAULT + ", but got:"
-            + in.getExpectNextPos(),
+        + Constants.MULTIPART_DOWNLOAD_SIZE_DEFAULT + ", but got:"
+        + in.getExpectNextPos(),
         in.getExpectNextPos() == Constants.MULTIPART_DOWNLOAD_SIZE_DEFAULT);
     fsDataInputStream.seek(4 * 1024 * 1024);
     assertTrue("expected position at:" + 4 * 1024 * 1024
-            + Constants.MULTIPART_DOWNLOAD_SIZE_DEFAULT + ", but got:"
-            + in.getExpectNextPos(),
+        + Constants.MULTIPART_DOWNLOAD_SIZE_DEFAULT + ", but got:"
+        + in.getExpectNextPos(),
         in.getExpectNextPos() == 4 * 1024 * 1024
-            + Constants.MULTIPART_DOWNLOAD_SIZE_DEFAULT);
+        + Constants.MULTIPART_DOWNLOAD_SIZE_DEFAULT);
     IOUtils.closeStream(fsDataInputStream);
   }
 

+ 1 - 1
hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractDistCp.java

@@ -33,7 +33,7 @@ public class TestAliyunOSSContractDistCp extends AbstractContractDistCpTest {
   protected Configuration createConfiguration() {
     Configuration newConf = super.createConfiguration();
     newConf.setLong(MIN_MULTIPART_UPLOAD_THRESHOLD_KEY, MULTIPART_SETTING);
-    newConf.setLong(MULTIPART_UPLOAD_SIZE_KEY, MULTIPART_SETTING);
+    newConf.setLong(MULTIPART_UPLOAD_PART_SIZE_KEY, MULTIPART_SETTING);
     return newConf;
   }
 

+ 1 - 1
hadoop-tools/hadoop-azure-datalake/pom.xml

@@ -33,7 +33,7 @@
     <minimalJsonVersion>0.9.1</minimalJsonVersion>
     <file.encoding>UTF-8</file.encoding>
     <downloadSources>true</downloadSources>
-    <azure.data.lake.store.sdk.version>2.2.5</azure.data.lake.store.sdk.version>
+    <azure.data.lake.store.sdk.version>2.2.7</azure.data.lake.store.sdk.version>
   </properties>
   <build>
     <plugins>

+ 0 - 40
hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java

@@ -46,7 +46,6 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary.Builder;
 import org.apache.hadoop.fs.CreateFlag;
@@ -910,45 +909,6 @@ public class AdlFileSystem extends FileSystem {
     return ADL_BLOCK_SIZE;
   }
 
-  @Override
-  public BlockLocation[] getFileBlockLocations(final FileStatus status,
-      final long offset, final long length) throws IOException {
-    if (status == null) {
-      return null;
-    }
-
-    if ((offset < 0) || (length < 0)) {
-      throw new IllegalArgumentException("Invalid start or len parameter");
-    }
-
-    if (status.getLen() < offset) {
-      return new BlockLocation[0];
-    }
-
-    final String[] name = {"localhost"};
-    final String[] host = {"localhost"};
-    long blockSize = ADL_BLOCK_SIZE;
-    int numberOfLocations =
-        (int) (length / blockSize) + ((length % blockSize == 0) ? 0 : 1);
-    BlockLocation[] locations = new BlockLocation[numberOfLocations];
-    for (int i = 0; i < locations.length; i++) {
-      long currentOffset = offset + (i * blockSize);
-      long currentLength = Math.min(blockSize, offset + length - currentOffset);
-      locations[i] = new BlockLocation(name, host, currentOffset,
-          currentLength);
-    }
-
-    return locations;
-  }
-
-  @Override
-  public BlockLocation[] getFileBlockLocations(final Path p, final long offset,
-      final long length) throws IOException {
-    // read ops incremented in getFileStatus
-    FileStatus fileStatus = getFileStatus(p);
-    return getFileBlockLocations(fileStatus, offset, length);
-  }
-
   /**
    * Get replication.
    *

+ 0 - 46
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java

@@ -52,7 +52,6 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BufferedFSInputStream;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -726,10 +725,6 @@ public class NativeAzureFileSystem extends FileSystem {
 
   static final String AZURE_CHMOD_USERLIST_PROPERTY_DEFAULT_VALUE = "*";
 
-  static final String AZURE_BLOCK_LOCATION_HOST_PROPERTY_NAME =
-      "fs.azure.block.location.impersonatedhost";
-  private static final String AZURE_BLOCK_LOCATION_HOST_DEFAULT =
-      "localhost";
   static final String AZURE_RINGBUFFER_CAPACITY_PROPERTY_NAME =
       "fs.azure.ring.buffer.capacity";
   static final String AZURE_OUTPUT_STREAM_BUFFER_SIZE_PROPERTY_NAME =
@@ -3469,47 +3464,6 @@ public class NativeAzureFileSystem extends FileSystem {
     }
   }
 
-  /**
-   * Return an array containing hostnames, offset and size of
-   * portions of the given file. For WASB we'll just lie and give
-   * fake hosts to make sure we get many splits in MR jobs.
-   */
-  @Override
-  public BlockLocation[] getFileBlockLocations(FileStatus file,
-      long start, long len) throws IOException {
-    if (file == null) {
-      return null;
-    }
-
-    if ((start < 0) || (len < 0)) {
-      throw new IllegalArgumentException("Invalid start or len parameter");
-    }
-
-    if (file.getLen() < start) {
-      return new BlockLocation[0];
-    }
-    final String blobLocationHost = getConf().get(
-        AZURE_BLOCK_LOCATION_HOST_PROPERTY_NAME,
-        AZURE_BLOCK_LOCATION_HOST_DEFAULT);
-    final String[] name = { blobLocationHost };
-    final String[] host = { blobLocationHost };
-    long blockSize = file.getBlockSize();
-    if (blockSize <= 0) {
-      throw new IllegalArgumentException(
-          "The block size for the given file is not a positive number: "
-              + blockSize);
-    }
-    int numberOfLocations = (int) (len / blockSize)
-        + ((len % blockSize == 0) ? 0 : 1);
-    BlockLocation[] locations = new BlockLocation[numberOfLocations];
-    for (int i = 0; i < locations.length; i++) {
-      long currentOffset = start + (i * blockSize);
-      long currentLength = Math.min(blockSize, start + len - currentOffset);
-      locations[i] = new BlockLocation(name, host, currentOffset, currentLength);
-    }
-    return locations;
-  }
-
   /**
    * Set the working directory to the given directory.
    */

+ 0 - 141
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemBlockLocations.java

@@ -1,141 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.azure;
-
-import java.io.OutputStream;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.BlockLocation;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.junit.Test;
-
-/**
- * Test block location logic.
- */
-public class TestNativeAzureFileSystemBlockLocations
-    extends AbstractWasbTestWithTimeout {
-  @Test
-  public void testNumberOfBlocks() throws Exception {
-    Configuration conf = new Configuration();
-    conf.set(NativeAzureFileSystem.AZURE_BLOCK_SIZE_PROPERTY_NAME, "500");
-    AzureBlobStorageTestAccount testAccount = AzureBlobStorageTestAccount
-        .createMock(conf);
-    FileSystem fs = testAccount.getFileSystem();
-    Path testFile = createTestFile(fs, 1200);
-    FileStatus stat = fs.getFileStatus(testFile);
-    assertEquals(500, stat.getBlockSize());
-    testAccount.cleanup();
-  }
-
-  @Test
-  public void testBlockLocationsTypical() throws Exception {
-    BlockLocation[] locations = getBlockLocationsOutput(210, 50, 0, 210);
-    assertEquals(5, locations.length);
-    assertEquals("localhost", locations[0].getHosts()[0]);
-    assertEquals(50, locations[0].getLength());
-    assertEquals(10, locations[4].getLength());
-    assertEquals(100, locations[2].getOffset());
-  }
-
-  @Test
-  public void testBlockLocationsEmptyFile() throws Exception {
-    BlockLocation[] locations = getBlockLocationsOutput(0, 50, 0, 0);
-    assertEquals(0, locations.length);
-  }
-
-  @Test
-  public void testBlockLocationsSmallFile() throws Exception {
-    BlockLocation[] locations = getBlockLocationsOutput(1, 50, 0, 1);
-    assertEquals(1, locations.length);
-    assertEquals(1, locations[0].getLength());
-  }
-
-  @Test
-  public void testBlockLocationsExactBlockSizeMultiple() throws Exception {
-    BlockLocation[] locations = getBlockLocationsOutput(200, 50, 0, 200);
-    assertEquals(4, locations.length);
-    assertEquals(150, locations[3].getOffset());
-    assertEquals(50, locations[3].getLength());
-  }
-
-  @Test
-  public void testBlockLocationsSubsetOfFile() throws Exception {
-    BlockLocation[] locations = getBlockLocationsOutput(205, 10, 15, 35);
-    assertEquals(4, locations.length);
-    assertEquals(10, locations[0].getLength());
-    assertEquals(15, locations[0].getOffset());
-    assertEquals(5, locations[3].getLength());
-    assertEquals(45, locations[3].getOffset());
-  }
-
-  @Test
-  public void testBlockLocationsOutOfRangeSubsetOfFile() throws Exception {
-    BlockLocation[] locations = getBlockLocationsOutput(205, 10, 300, 10);
-    assertEquals(0, locations.length);
-  }
-
-  @Test
-  public void testBlockLocationsEmptySubsetOfFile() throws Exception {
-    BlockLocation[] locations = getBlockLocationsOutput(205, 10, 0, 0);
-    assertEquals(0, locations.length);
-  }
-
-  @Test
-  public void testBlockLocationsDifferentLocationHost() throws Exception {
-    BlockLocation[] locations = getBlockLocationsOutput(100, 10, 0, 100,
-        "myblobhost");
-    assertEquals(10, locations.length);
-    assertEquals("myblobhost", locations[0].getHosts()[0]);
-  }
-
-  private static BlockLocation[] getBlockLocationsOutput(int fileSize,
-      int blockSize, long start, long len) throws Exception {
-    return getBlockLocationsOutput(fileSize, blockSize, start, len, null);
-  }
-
-  private static BlockLocation[] getBlockLocationsOutput(int fileSize,
-      int blockSize, long start, long len, String blockLocationHost)
-      throws Exception {
-    Configuration conf = new Configuration();
-    conf.set(NativeAzureFileSystem.AZURE_BLOCK_SIZE_PROPERTY_NAME, ""
-        + blockSize);
-    if (blockLocationHost != null) {
-      conf.set(NativeAzureFileSystem.AZURE_BLOCK_LOCATION_HOST_PROPERTY_NAME,
-          blockLocationHost);
-    }
-    AzureBlobStorageTestAccount testAccount = AzureBlobStorageTestAccount
-        .createMock(conf);
-    FileSystem fs = testAccount.getFileSystem();
-    Path testFile = createTestFile(fs, fileSize);
-    FileStatus stat = fs.getFileStatus(testFile);
-    BlockLocation[] locations = fs.getFileBlockLocations(stat, start, len);
-    testAccount.cleanup();
-    return locations;
-  }
-
-  private static Path createTestFile(FileSystem fs, int size) throws Exception {
-    Path testFile = new Path("/testFile");
-    OutputStream outputStream = fs.create(testFile);
-    outputStream.write(new byte[size]);
-    outputStream.close();
-    return testFile;
-  }
-}

+ 4 - 4
hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/state/StatePool.java

@@ -18,11 +18,11 @@
 package org.apache.hadoop.tools.rumen.state;
 
 import java.io.DataInput;
-import java.io.DataInputStream;
 import java.io.DataOutput;
-import java.io.DataOutputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
 import java.util.Calendar;
@@ -216,7 +216,7 @@ public class StatePool {
     // register the module with the object-mapper
     mapper.registerModule(module);
 
-    JsonParser parser = mapper.getFactory().createParser((DataInputStream)in);
+    JsonParser parser = mapper.getFactory().createParser((InputStream)in);
     StatePool statePool = mapper.readValue(parser, StatePool.class);
     this.setStates(statePool.getStates());
     parser.close();
@@ -285,7 +285,7 @@ public class StatePool {
 
     JsonFactory outFactory = outMapper.getFactory();
     JsonGenerator jGen =
-        outFactory.createGenerator((DataOutputStream)out, JsonEncoding.UTF8);
+        outFactory.createGenerator((OutputStream)out, JsonEncoding.UTF8);
     jGen.useDefaultPrettyPrinter();
 
     jGen.writeObject(this);

+ 2 - 1
hadoop-tools/hadoop-rumen/src/test/java/org/apache/hadoop/tools/rumen/TestHistograms.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.tools.rumen;
 import java.io.IOException;
 
+import java.io.OutputStream;
 import java.util.List;
 
 import com.fasterxml.jackson.core.JsonEncoding;
@@ -142,7 +143,7 @@ public class TestHistograms {
         ObjectMapper mapper = new ObjectMapper();
         JsonFactory factory = mapper.getFactory();
         FSDataOutputStream ostream = lfs.create(goldFilePath, true);
-        JsonGenerator gen = factory.createGenerator(ostream,
+        JsonGenerator gen = factory.createGenerator((OutputStream)ostream,
             JsonEncoding.UTF8);
         gen.useDefaultPrettyPrinter();
         

+ 2 - 0
hadoop-tools/hadoop-sls/src/main/bin/slsrun.sh

@@ -77,6 +77,8 @@ function parse_args()
 function calculate_classpath
 {
   hadoop_add_to_classpath_tools hadoop-sls
+
+  hadoop_add_classpath "${HADOOP_YARN_HOME}/${YARN_DIR}/timelineservice"'/*'
 }
 
 function run_simulation() {

+ 5 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java

@@ -130,6 +130,11 @@ public class PlacementConstraint {
     public PlacementConstraint build() {
       return new PlacementConstraint(this);
     }
+
+    @Override
+    public String toString() {
+      return super.toString();
+    }
   }
 
   static final String NODE_SCOPE = "node";

+ 23 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -268,6 +268,8 @@ public class YarnConfiguration extends Configuration {
   /** UserGroupMappingPlacementRule configuration string. */
   public static final String USER_GROUP_PLACEMENT_RULE = "user-group";
 
+  public static final String APP_NAME_PLACEMENT_RULE = "app-name";
+
   /** Enable Resource Manager webapp ui actions */
   public static final String RM_WEBAPP_UI_ACTIONS_ENABLED =
     RM_PREFIX + "webapp.ui-actions.enabled";
@@ -774,12 +776,15 @@ public class YarnConfiguration extends Configuration {
   public static final String MEMORY_CONFIGURATION_STORE = "memory";
   @Private
   @Unstable
-  public static final String LEVELDB_CONFIGURATION_STORE = "leveldb";
+  public static final String FS_CONFIGURATION_STORE = "fs";
   @Private
   @Unstable
   public static final String ZK_CONFIGURATION_STORE = "zk";
   @Private
   @Unstable
+  public static final String LEVELDB_CONFIGURATION_STORE = "leveldb";
+  @Private
+  @Unstable
   public static final String DEFAULT_CONFIGURATION_STORE =
       FILE_CONFIGURATION_STORE;
   @Private
@@ -807,6 +812,17 @@ public class YarnConfiguration extends Configuration {
   @Private
   @Unstable
   public static final long DEFAULT_RM_SCHEDCONF_ZK_MAX_LOGS = 1000;
+  @Private
+  @Unstable
+  public static final String SCHEDULER_CONFIGURATION_FS_PATH =
+      YARN_PREFIX + "scheduler.configuration.fs.path";
+  @Private
+  @Unstable
+  public static final String SCHEDULER_CONFIGURATION_FS_MAX_VERSION =
+      YARN_PREFIX + "scheduler.configuration.max.version";
+  @Private
+  @Unstable
+  public static final int DEFAULT_SCHEDULER_CONFIGURATION_FS_MAX_VERSION = 100;
 
   /** Parent znode path under which ZKConfigurationStore will create znodes. */
   @Private
@@ -3087,15 +3103,18 @@ public class YarnConfiguration extends Configuration {
 
   public static final String FEDERATION_CACHE_TIME_TO_LIVE_SECS =
       FEDERATION_PREFIX + "cache-ttl.secs";
+  // 5 minutes
+  public static final int DEFAULT_FEDERATION_CACHE_TIME_TO_LIVE_SECS = 5 * 60;
+
+  public static final String FEDERATION_FLUSH_CACHE_FOR_RM_ADDR =
+      FEDERATION_PREFIX + "flush-cache-for-rm-addr";
+  public static final boolean DEFAULT_FEDERATION_FLUSH_CACHE_FOR_RM_ADDR = true;
 
   public static final String FEDERATION_REGISTRY_BASE_KEY =
       FEDERATION_PREFIX + "registry.base-dir";
   public static final String DEFAULT_FEDERATION_REGISTRY_BASE_KEY =
       "yarnfederation/";
 
-  // 5 minutes
-  public static final int DEFAULT_FEDERATION_CACHE_TIME_TO_LIVE_SECS = 5 * 60;
-
   public static final String FEDERATION_STATESTORE_HEARTBEAT_INTERVAL_SECS =
       FEDERATION_PREFIX + "state-store.heartbeat-interval-secs";
 

+ 2 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java

@@ -79,6 +79,8 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
         .add(YarnConfiguration.FEDERATION_FAILOVER_ENABLED);
     configurationPropsToSkipCompare
         .add(YarnConfiguration.FEDERATION_STATESTORE_HEARTBEAT_INTERVAL_SECS);
+    configurationPrefixToSkipCompare
+        .add(YarnConfiguration.FEDERATION_FLUSH_CACHE_FOR_RM_ADDR);
     configurationPropsToSkipCompare
         .add(YarnConfiguration.RM_EPOCH);
     configurationPropsToSkipCompare

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

@@ -1147,7 +1147,6 @@ public class ApplicationMaster {
     public void onError(Throwable e) {
       LOG.error("Error in RMCallbackHandler: ", e);
       done = true;
-      amRMClient.stop();
     }
   }
 

+ 15 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java

@@ -230,6 +230,9 @@ public class Client {
   // Docker client configuration
   private String dockerClientConfig = null;
 
+  // Application tags
+  private Set<String> applicationTags = new HashSet<>();
+
   // Command line options
   private Options opts;
 
@@ -384,6 +387,7 @@ public class Client {
         "Placement specification. Please note, if this option is specified,"
             + " The \"num_containers\" option will be ignored. All requested"
             + " containers will be of type GUARANTEED" );
+    opts.addOption("application_tags", true, "Application tags.");
   }
 
   /**
@@ -604,6 +608,14 @@ public class Client {
     if (cliParser.hasOption("docker_client_config")) {
       dockerClientConfig = cliParser.getOptionValue("docker_client_config");
     }
+
+    if (cliParser.hasOption("application_tags")) {
+      String applicationTagsStr = cliParser.getOptionValue("application_tags");
+      String[] appTags = applicationTagsStr.split(",");
+      for (String appTag : appTags) {
+        this.applicationTags.add(appTag.trim());
+      }
+    }
     return true;
   }
 
@@ -729,6 +741,9 @@ public class Client {
     }
 
     Set<String> tags = new HashSet<String>();
+    if (applicationTags != null) {
+      tags.addAll(applicationTags);
+    }
     if (flowName != null) {
       tags.add(TimelineUtils.generateFlowNameTag(flowName));
     }

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/DistributedShellTimelinePlugin.java

@@ -52,7 +52,7 @@ public class DistributedShellTimelinePlugin extends TimelineEntityGroupPlugin {
   @Override
   public Set<TimelineEntityGroupId> getTimelineEntityGroupId(String entityId,
       String entityType) {
-    if (ApplicationMaster.DSEntity.DS_CONTAINER.toString().equals(entityId)) {
+    if (ApplicationMaster.DSEntity.DS_CONTAINER.toString().equals(entityType)) {
       ContainerId containerId = ContainerId.fromString(entityId);
       ApplicationId appId = containerId.getApplicationAttemptId()
           .getApplicationId();

+ 9 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java

@@ -494,6 +494,15 @@ public class TestDistributedShell {
     Assert.assertEquals(2, entities.getEntities().size());
     Assert.assertEquals(entities.getEntities().get(0).getEntityType()
         .toString(), ApplicationMaster.DSEntity.DS_CONTAINER.toString());
+
+    String entityId = entities.getEntities().get(0).getEntityId();
+    org.apache.hadoop.yarn.api.records.timeline.TimelineEntity entity =
+        yarnCluster.getApplicationHistoryServer().getTimelineStore()
+            .getEntity(entityId,
+                ApplicationMaster.DSEntity.DS_CONTAINER.toString(), null);
+    Assert.assertNotNull(entity);
+    Assert.assertEquals(entityId, entity.getEntityId());
+
     if (haveDomain) {
       Assert.assertEquals(domain.getId(),
           entities.getEntities().get(0).getDomainId());

+ 18 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/ApiServiceClient.java

@@ -469,4 +469,22 @@ public class ApiServiceClient extends AppAdminClient {
     return output;
   }
 
+  @Override
+  public int actionUpgrade(String appName,
+      String fileName) throws IOException, YarnException {
+    int result;
+    try {
+      Service service =
+          loadAppJsonFromLocalFS(fileName, appName, null, null);
+      service.setState(ServiceState.UPGRADING);
+      String buffer = jsonSerDeser.toJson(service);
+      ClientResponse response = getApiClient()
+          .post(ClientResponse.class, buffer);
+      result = processResponse(response);
+    } catch (Exception e) {
+      LOG.error("Failed to upgrade application: ", e);
+      result = EXIT_EXCEPTION_THROWN;
+    }
+    return result;
+  }
 }

+ 24 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java

@@ -375,6 +375,12 @@ public class ApiServer {
           && updateServiceData.getLifetime() > 0) {
         return updateLifetime(appName, updateServiceData, ugi);
       }
+
+      // If an UPGRADE is requested
+      if (updateServiceData.getState() != null &&
+          updateServiceData.getState() == ServiceState.UPGRADING) {
+        return upgradeService(updateServiceData, ugi);
+      }
     } catch (UndeclaredThrowableException e) {
       return formatResponse(Status.BAD_REQUEST,
           e.getCause().getMessage());
@@ -475,6 +481,24 @@ public class ApiServer {
     return formatResponse(Status.OK, status);
   }
 
+  private Response upgradeService(Service service,
+      final UserGroupInformation ugi) throws IOException, InterruptedException {
+    ServiceStatus status = new ServiceStatus();
+    ugi.doAs((PrivilegedExceptionAction<Void>) () -> {
+      ServiceClient sc = getServiceClient();
+      sc.init(YARN_CONFIG);
+      sc.start();
+      sc.actionUpgrade(service);
+      sc.close();
+      return null;
+    });
+    LOG.info("Service {} version {} upgrade initialized");
+    status.setDiagnostics("Service " + service.getName() +
+        " version " + service.getVersion() + " saved.");
+    status.setState(ServiceState.ACCEPTED);
+    return formatResponse(Status.ACCEPTED, status);
+  }
+
   /**
    * Used by negative test case.
    *

+ 79 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Simplified-V1-API-Layer-For-Services.yaml

@@ -229,9 +229,6 @@ definitions:
         type: integer
         format: int64
         description: Life time (in seconds) of the service from the time it reaches the STARTED state (after which it is automatically destroyed by YARN). For unlimited lifetime do not set a lifetime value.
-      placement_policy:
-        description: (TBD) Advanced scheduling and placement policies. If not specified, it defaults to the default placement policy of the service owner. The design of placement policies are in the works. It is not very clear at this point, how policies in conjunction with labels be exposed to service owners. This is a placeholder for now. The advanced structure of this attribute will be determined by YARN-4902.
-        $ref: '#/definitions/PlacementPolicy'
       components:
         description: Components of a service.
         type: array
@@ -256,7 +253,7 @@ definitions:
         $ref: '#/definitions/KerberosPrincipal'
   ResourceInformation:
     description:
-      ResourceInformation determines unit/value of resource types in addition to memory and vcores. It will be part of Resource object
+      ResourceInformation determines unit/value of resource types in addition to memory and vcores. It will be part of Resource object.
     properties:
       value:
         type: integer
@@ -264,8 +261,7 @@ definitions:
         description: Integer value of the resource.
       unit:
         type: string
-        description:
-          Unit of the resource, acceptable values are: p/n/u/m/k/M/G/T/P/Ki/Mi/Gi/Ti/Pi. By default it is empty means no unit
+        description: Unit of the resource, acceptable values are - p/n/u/m/k/M/G/T/P/Ki/Mi/Gi/Ti/Pi. By default it is empty means no unit.
   Resource:
     description:
       Resource determines the amount of resources (vcores, memory, network, etc.) usable by a container. This field determines the resource to be applied for all the containers of a component or service. The resource specified at the service (or global) level can be overriden at the component level. Only one of profile OR cpu & memory are expected. It raises a validation exception otherwise.
@@ -286,11 +282,75 @@ definitions:
           $ref: '#/definitions/ResourceInformation'
         description: Map of resource name to ResourceInformation
   PlacementPolicy:
-    description: Placement policy of an instance of a service. This feature is in the works in YARN-6592.
+    description: Advanced placement policy of the components of a service.
+    required:
+      - constraints
     properties:
-      label:
+      constraints:
+        description: Placement constraint details.
+        type: array
+        items:
+          $ref: '#/definitions/PlacementConstraint'
+  PlacementConstraint:
+    description: Placement constraint details.
+    required:
+      - type
+      - scope
+    properties:
+      name:
+        description: An optional name associated to this constraint.
         type: string
-        description: Assigns a service to a named partition of the cluster where the service desires to run (optional). If not specified all services are submitted to a default label of the service owner. One or more labels can be setup for each service owner account with required constraints like no-preemption, sla-99999, preemption-ok, etc.
+        example: C1
+      type:
+        description: The type of placement.
+        $ref: '#/definitions/PlacementType'
+      scope:
+        description: The scope of placement.
+        $ref: '#/definitions/PlacementScope'
+      target_tags:
+        description: The name of the components that this component's placement policy is depending upon are added as target tags. So for affinity say, this component's containers are requesting to be placed on hosts where containers of the target tag component(s) are running on. Target tags can also contain the name of this component, in which case it implies that for anti-affinity say, no more than one container of this component can be placed on a host. Similarly, for cardinality, it would mean that containers of this component is requesting to be placed on hosts where at least minCardinality but no more than maxCardinality containers of the target tag component(s) are running.
+        type: array
+        items:
+          type: string
+      node_attributes:
+        description: Node attributes are a set of key:value(s) pairs associated with nodes.
+        type: object
+        additionalProperties:
+          type: array
+          items:
+            type: string
+      node_partitions:
+        description: Node partitions where the containers of this component can run.
+        type: array
+        items:
+          type: string
+      min_cardinality:
+        type: integer
+        format: int64
+        description: When placement type is cardinality, the minimum number of containers of the depending component that a host should have, where containers of this component can be allocated on.
+        example: 2
+      max_cardinality:
+        type: integer
+        format: int64
+        description: When placement type is cardinality, the maximum number of containers of the depending component that a host should have, where containers of this component can be allocated on.
+        example: 3
+  PlacementType:
+    description: The type of placement - affinity/anti-affinity/affinity-with-cardinality with containers of another component or containers of the same component (self).
+    properties:
+      type:
+        type: string
+        enum:
+          - AFFINITY
+          - ANTI_AFFINITY
+          - AFFINITY_WITH_CARDINALITY
+  PlacementScope:
+    description: The scope of placement for the containers of a component.
+    properties:
+      type:
+        type: string
+        enum:
+          - NODE
+          - RACK
   Artifact:
     description: Artifact of a service component. If not specified, component will just run the bare launch command and no artifact will be localized.
     required:
@@ -342,11 +402,16 @@ definitions:
         type: integer
         format: int64
         description: Number of containers for this component (optional). If not specified, the service level global number_of_containers takes effect.
+      containers:
+        type: array
+        description: Containers of a started component. Specifying a value for this attribute for the POST payload raises a validation error. This blob is available only in the GET response of a started service.
+        items:
+          $ref: '#/definitions/Container'
       run_privileged_container:
         type: boolean
         description: Run all containers of this component in privileged mode (YARN-4262).
       placement_policy:
-        description: Advanced scheduling and placement policies for all containers of this component (optional). If not specified, the service level placement_policy takes effect. Refer to the description at the global level for more details.
+        description: Advanced scheduling and placement policies for all containers of this component.
         $ref: '#/definitions/PlacementPolicy'
       configuration:
         description: Config properties for this component.
@@ -380,7 +445,7 @@ definitions:
     properties:
       properties:
         type: object
-        description: A blob of key-value pairs for configuring the YARN service AM
+        description: A blob of key-value pairs for configuring the YARN service AM.
         additionalProperties:
           type: string
       env:
@@ -405,7 +470,6 @@ definitions:
           - JSON
           - YAML
           - TEMPLATE
-          - ENV
           - HADOOP_XML
       dest_file:
         type: string
@@ -416,6 +480,8 @@ definitions:
       properties:
         type: object
         description: A blob of key value pairs that will be dumped in the dest_file in the format as specified in type. If src_file is specified, src_file content are dumped in the dest_file and these properties will overwrite, if any, existing properties in src_file or be added as new properties in src_file.
+        additionalProperties:
+          type: string
   Container:
     description: An instance of a running service container.
     properties:
@@ -464,6 +530,7 @@ definitions:
           - STABLE
           - STOPPED
           - FAILED
+          - FLEX
   ContainerState:
     description: The current state of the container of a service.
     properties:

+ 12 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ClientAMProtocol.java

@@ -23,8 +23,14 @@ import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsRequestProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsResponseProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusResponseProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusRequestProto;
+import org.apache.hadoop.yarn.proto.ClientAMProtocol.RestartServiceRequestProto;
+import org.apache.hadoop.yarn.proto.ClientAMProtocol.RestartServiceResponseProto;
+
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.StopResponseProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.StopRequestProto;
+import org.apache.hadoop.yarn.proto.ClientAMProtocol.UpgradeServiceRequestProto;
+import org.apache.hadoop.yarn.proto.ClientAMProtocol.UpgradeServiceResponseProto;
+
 
 import java.io.IOException;
 
@@ -37,4 +43,10 @@ public interface ClientAMProtocol {
 
   StopResponseProto stop(StopRequestProto requestProto)
       throws IOException, YarnException;
+
+  UpgradeServiceResponseProto upgrade(UpgradeServiceRequestProto request)
+      throws IOException, YarnException;
+
+  RestartServiceResponseProto restart(RestartServiceRequestProto request)
+      throws IOException, YarnException;
 }

Vissa filer visades inte eftersom för många filer har ändrats