فهرست منبع

Merge branch 'trunk' into HDFS-7240

Anu Engineer 7 سال پیش
والد
کامیت
c04492ac8b
99فایلهای تغییر یافته به همراه2687 افزوده شده و 709 حذف شده
  1. 11 0
      hadoop-common-project/hadoop-common/dev-support/jdiff/Apache_Hadoop_Common_2.8.3.xml
  2. 10 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java
  3. 90 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/EtagChecksum.java
  4. 28 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/package-info.java
  5. 4 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
  6. 2 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/CryptoStreamsTestBase.java
  7. 45 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreams.java
  8. 0 258
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java
  9. 246 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractPathHandleTest.java
  10. 85 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/store/TestEtagChecksum.java
  11. 77 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestFixKerberosTicketOrder.java
  12. 3 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsPathHandle.java
  13. 11 0
      hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_2.8.3.xml
  14. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferServer.java
  15. 63 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
  16. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
  17. 4 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
  18. 46 25
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  19. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java
  20. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsImageViewer.md
  21. 55 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/contract/hdfs/TestHDFSContractPathHandle.java
  22. 118 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java
  23. 2 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java
  24. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
  25. 11 0
      hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_Common_2.8.3.xml
  26. 11 0
      hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_Core_2.8.3.xml
  27. 11 0
      hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_JobClient_2.8.3.xml
  28. 2 0
      hadoop-project/pom.xml
  29. 80 13
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
  30. 24 4
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
  31. 13 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
  32. 3 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
  33. 12 1
      hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
  34. 92 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClosedFS.java
  35. 116 17
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java
  36. 5 1
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
  37. 12 4
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
  38. 18 7
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
  39. 18 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemHelper.java
  40. 4 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java
  41. 6 2
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SecureStorageInterfaceImpl.java
  42. 1 1
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java
  43. 6 2
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java
  44. 72 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemLive.java
  45. 8 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java
  46. 10 0
      hadoop-tools/hadoop-fs2img/pom.xml
  47. 0 1
      hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
  48. 11 0
      hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Client_2.8.3.xml
  49. 11 0
      hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Common_2.8.3.xml
  50. 11 0
      hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Server_Common_2.8.3.xml
  51. 44 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  52. 17 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Simplified-V1-API-Layer-For-Services.yaml
  53. 8 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/dev-support/findbugs-exclude.xml
  54. 37 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceMaster.java
  55. 11 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
  56. 39 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Resource.java
  57. 119 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ResourceInformation.java
  58. 16 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java
  59. 80 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java
  60. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java
  61. 10 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
  62. 4 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceUtils.java
  63. 25 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/MockServiceAM.java
  64. 57 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceAM.java
  65. 29 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestYarnNativeServices.java
  66. 17 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/conf/TestAppJsonResolve.java
  67. 10 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/resources/org/apache/hadoop/yarn/service/conf/examples/app.json
  68. 32 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
  69. 24 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
  70. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/RecoveredContainerLaunch.java
  71. 13 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
  72. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java
  73. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
  74. 73 58
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
  75. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/security/TestTimelineAuthFilterForV2.java
  76. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/AbstractTimelineReaderHBaseTestBase.java
  77. 3 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java
  78. 17 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/NodeTimelineCollectorManager.java
  79. 18 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
  80. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderServer.java
  81. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java
  82. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
  83. 12 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/YarnServiceAPI.md
  84. 23 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/queue-navigator.js
  85. 187 114
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js
  86. 48 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/yarn-queue-partition-capacity-labels.js
  87. 3 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/constants.js
  88. 28 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-queues.js
  89. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/initializers/loader.js
  90. 33 23
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/capacity-queue.js
  91. 6 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/capacity-queue.js
  92. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.scss
  93. 29 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/yarn-queues.scss
  94. 18 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/queue-navigator.hbs
  95. 54 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue-partition-capacity-labels.hbs
  96. 3 31
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue.hbs
  97. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queues.hbs
  98. 43 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/integration/components/yarn-queue-partition-capacity-labels-test.js
  99. 1 1
      pom.xml

تفاوت فایلی نمایش داده نمی شود زیرا این فایل بسیار بزرگ است
+ 11 - 0
hadoop-common-project/hadoop-common/dev-support/jdiff/Apache_Hadoop_Common_2.8.3.xml


+ 10 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java

@@ -26,6 +26,7 @@ import java.security.GeneralSecurityException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.CanSetDropBehind;
+import org.apache.hadoop.fs.StreamCapabilities;
 import org.apache.hadoop.fs.Syncable;
 
 import com.google.common.base.Preconditions;
@@ -47,7 +48,7 @@ import com.google.common.base.Preconditions;
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class CryptoOutputStream extends FilterOutputStream implements 
-    Syncable, CanSetDropBehind {
+    Syncable, CanSetDropBehind, StreamCapabilities {
   private final byte[] oneByteBuf = new byte[1];
   private final CryptoCodec codec;
   private final Encryptor encryptor;
@@ -304,4 +305,12 @@ public class CryptoOutputStream extends FilterOutputStream implements
     CryptoStreamUtils.freeDB(inBuffer);
     CryptoStreamUtils.freeDB(outBuffer);
   }
+
+  @Override
+  public boolean hasCapability(String capability) {
+    if (out instanceof StreamCapabilities) {
+      return ((StreamCapabilities) out).hasCapability(capability);
+    }
+    return false;
+  }
 }

+ 90 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/EtagChecksum.java

@@ -0,0 +1,90 @@
+/*
+ * 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.store;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+
+import org.apache.hadoop.fs.FileChecksum;
+
+/**
+ * An etag as a checksum.
+ * Consider these suitable for checking if an object has changed, but
+ * not suitable for comparing two different objects for equivalence,
+ * especially between object stores.
+ */
+public class EtagChecksum extends FileChecksum {
+
+  /** The algorithm name: {@value}. */
+  private static final String ETAG = "etag";
+
+  /**
+   * Etag string.
+   */
+  private String eTag = "";
+
+  /**
+   * Create with an empty etag.
+   */
+  public EtagChecksum() {
+  }
+
+  /**
+   * Create with a string etag.
+   * @param eTag etag
+   */
+  public EtagChecksum(String eTag) {
+    this.eTag = eTag;
+  }
+
+  @Override
+  public String getAlgorithmName() {
+    return ETAG;
+  }
+
+  @Override
+  public int getLength() {
+    return eTag.getBytes(StandardCharsets.UTF_8).length;
+  }
+
+  @Override
+  public byte[] getBytes() {
+    return eTag != null
+        ? eTag.getBytes(StandardCharsets.UTF_8)
+        : new byte[0];
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeUTF(eTag != null ? eTag : "");
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    eTag = in.readUTF();
+  }
+
+  @Override
+  public String toString() {
+    return "etag: \"" + eTag  + '"';
+  }
+
+}

+ 28 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/package-info.java

@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+/**
+ * This package is for classes to be shared across object stores; for internal
+ * use within the hadoop-* modules only. No stability guarantees.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.fs.store;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

+ 4 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java

@@ -1253,7 +1253,10 @@ public class UserGroupInformation {
         Object cred = iter.next();
         if (cred instanceof KerberosTicket) {
           KerberosTicket ticket = (KerberosTicket) cred;
-          if (!ticket.getServer().getName().startsWith("krbtgt")) {
+          if (ticket.isDestroyed() || ticket.getServer() == null) {
+            LOG.warn("Ticket is already destroyed, remove it.");
+            iter.remove();
+          } else if (!ticket.getServer().getName().startsWith("krbtgt")) {
             LOG.warn(
                 "The first kerberos ticket is not TGT"
                     + "(the server principal is {}), remove and destroy it.",

+ 2 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/CryptoStreamsTestBase.java

@@ -50,9 +50,9 @@ public abstract class CryptoStreamsTestBase {
       CryptoStreamsTestBase.class);
 
   protected static CryptoCodec codec;
-  private static final byte[] key = {0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 
+  protected static final byte[] key = {0x01, 0x02, 0x03, 0x04, 0x05, 0x06,
     0x07, 0x08, 0x09, 0x10, 0x11, 0x12, 0x13, 0x14, 0x15, 0x16};
-  private static final byte[] iv = {0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 
+  protected static final byte[] iv = {0x01, 0x02, 0x03, 0x04, 0x05, 0x06,
     0x07, 0x08, 0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08};
   
   protected static final int count = 10000;

+ 45 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreams.java

@@ -42,6 +42,10 @@ import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 public class TestCryptoStreams extends CryptoStreamsTestBase {
   /**
@@ -91,7 +95,7 @@ public class TestCryptoStreams extends CryptoStreamsTestBase {
   }
   
   private class FakeOutputStream extends OutputStream 
-      implements Syncable, CanSetDropBehind{
+      implements Syncable, CanSetDropBehind, StreamCapabilities{
     private final byte[] oneByteBuf = new byte[1];
     private final DataOutputBuffer out;
     private boolean closed;
@@ -153,7 +157,19 @@ public class TestCryptoStreams extends CryptoStreamsTestBase {
       checkStream();
       flush();
     }
-    
+
+    @Override
+    public boolean hasCapability(String capability) {
+      switch (capability.toLowerCase()) {
+      case StreamCapabilities.HFLUSH:
+      case StreamCapabilities.HSYNC:
+      case StreamCapabilities.DROPBEHIND:
+        return true;
+      default:
+        return false;
+      }
+    }
+
     private void checkStream() throws IOException {
       if (closed) {
         throw new IOException("Stream is closed!");
@@ -393,4 +409,31 @@ public class TestCryptoStreams extends CryptoStreamsTestBase {
       return ( ret <= 0 ) ? -1 : (oneByteBuf[0] & 0xff);
     }
   }
+
+  /**
+   * This tests {@link StreamCapabilities#hasCapability(String)} for the
+   * the underlying streams.
+   */
+  @Test(timeout = 120000)
+  public void testHasCapability() throws Exception {
+    // verify hasCapability returns what FakeOutputStream is set up for
+    CryptoOutputStream cos =
+        (CryptoOutputStream) getOutputStream(defaultBufferSize, key, iv);
+    assertTrue(cos instanceof StreamCapabilities);
+    assertTrue(cos.hasCapability(StreamCapabilities.HFLUSH));
+    assertTrue(cos.hasCapability(StreamCapabilities.HSYNC));
+    assertTrue(cos.hasCapability(StreamCapabilities.DROPBEHIND));
+    assertFalse(cos.hasCapability(StreamCapabilities.READAHEAD));
+    assertFalse(cos.hasCapability(StreamCapabilities.UNBUFFER));
+
+    // verify hasCapability for input stream
+    CryptoInputStream cis =
+        (CryptoInputStream) getInputStream(defaultBufferSize, key, iv);
+    assertTrue(cis instanceof StreamCapabilities);
+    assertTrue(cis.hasCapability(StreamCapabilities.DROPBEHIND));
+    assertTrue(cis.hasCapability(StreamCapabilities.READAHEAD));
+    assertTrue(cis.hasCapability(StreamCapabilities.UNBUFFER));
+    assertFalse(cis.hasCapability(StreamCapabilities.HFLUSH));
+    assertFalse(cis.hasCapability(StreamCapabilities.HSYNC));
+  }
 }

+ 0 - 258
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java

@@ -19,27 +19,17 @@ package org.apache.hadoop.fs.contract;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.InvalidPathHandleException;
-import org.apache.hadoop.fs.Options.HandleOpt;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathHandle;
-import org.apache.hadoop.fs.RawPathHandle;
 import org.apache.hadoop.io.IOUtils;
 
-import static org.apache.hadoop.fs.contract.ContractTestUtils.appendFile;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
-import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
-import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyRead;
-import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyFileContents;
 
 import org.junit.Test;
 
@@ -173,252 +163,4 @@ public abstract class AbstractContractOpenTest
     instream.close();
   }
 
-  /**
-   * Skip a test case if the FS doesn't support file references.
-   * The feature is assumed to be unsupported unless stated otherwise.
-   */
-  protected void assumeSupportsFileReference() throws IOException {
-    if (getContract().isSupported(SUPPORTS_FILE_REFERENCE, false)) {
-      return;
-    }
-    skip("Skipping as unsupported feature: " + SUPPORTS_FILE_REFERENCE);
-  }
-
-  /**
-   * Skip a test case if the FS doesn't support content validation.
-   * The feature is assumed to be unsupported unless stated otherwise.
-   */
-  protected void assumeSupportsContentCheck() throws IOException {
-    if (getContract().isSupported(SUPPORTS_CONTENT_CHECK, false)) {
-      return;
-    }
-    skip("Skipping as unsupported feature: " + SUPPORTS_CONTENT_CHECK);
-  }
-
-  private PathHandle getHandleOrSkip(FileStatus stat, HandleOpt... opts) {
-    try {
-      return getFileSystem().getPathHandle(stat, opts);
-    } catch (UnsupportedOperationException e) {
-      skip("FileSystem does not support " + Arrays.toString(opts));
-    }
-    // unreachable
-    return null;
-  }
-
-  /**
-   * Verify {@link HandleOpt#exact()} handle semantics.
-   * @throws Throwable on error
-   */
-  @Test
-  public void testOpenFileByExact() throws Throwable {
-    describe("verify open(getPathHandle(FileStatus, exact())) operations" +
-        "detect changes");
-    assumeSupportsContentCheck();
-    assumeSupportsFileReference();
-    Path path1 = path("testopenfilebyexact1");
-    Path path2 = path("testopenfilebyexact2");
-    byte[] file1 = dataset(TEST_FILE_LEN, 43, 255);
-    createFile(getFileSystem(), path1, false, file1);
-    FileStatus stat1 = getFileSystem().getFileStatus(path1);
-    assertNotNull(stat1);
-    assertEquals(path1, stat1.getPath());
-    ContractTestUtils.rename(getFileSystem(), path1, path2);
-    FileStatus stat2 = getFileSystem().getFileStatus(path2);
-    assertNotNull(stat2);
-    assertEquals(path2, stat2.getPath());
-    // create identical file at same location, orig still exists at path2
-    createFile(getFileSystem(), path1, false, file1);
-
-    PathHandle fd1 = getHandleOrSkip(stat1, HandleOpt.exact());
-    PathHandle fd2 = getHandleOrSkip(stat2, HandleOpt.exact());
-
-    // verify path1, path2 contents identical
-    verifyFileContents(getFileSystem(), path1, file1);
-    verifyFileContents(getFileSystem(), path2, file1);
-    try {
-      // the PathHandle will not resolve, even though
-      // the original entity exists, it has not been modified, and an
-      // identical file exists at the old path. The handle would also
-      // fail to resolve if path1 had been modified
-      instream = getFileSystem().open(fd1);
-      fail("Expected an exception");
-    } catch (InvalidPathHandleException e) {
-      // expected
-    }
-
-    // verify unchanged resolves
-    instream = getFileSystem().open(fd2);
-    verifyRead(instream, file1, 0, TEST_FILE_LEN);
-  }
-
-  /**
-   * Verify {@link HandleOpt#content()} handle semantics.
-   * @throws Throwable on error
-   */
-  @Test
-  public void testOpenFileByContent() throws Throwable {
-    describe("verify open(getPathHandle(FileStatus, content())) operations" +
-        "follow relocation");
-    assumeSupportsContentCheck();
-    assumeSupportsFileReference();
-    Path path1 = path("testopenfilebycontent1");
-    Path path2 = path("testopenfilebycontent2");
-    byte[] file1 = dataset(TEST_FILE_LEN, 43, 255);
-    createFile(getFileSystem(), path1, false, file1);
-    FileStatus stat = getFileSystem().getFileStatus(path1);
-    assertNotNull(stat);
-    assertEquals(path1, stat.getPath());
-    // rename the file after obtaining FileStatus
-    ContractTestUtils.rename(getFileSystem(), path1, path2);
-
-    // obtain handle to entity from #getFileStatus call
-    PathHandle fd = getHandleOrSkip(stat, HandleOpt.content());
-
-    try (FSDataInputStream in = getFileSystem().open(fd)) {
-      // verify read of consistent content at new location
-      verifyRead(in, file1, 0, TEST_FILE_LEN);
-    }
-
-    // modify the file at its new location by appending data
-    byte[] file1a = dataset(TEST_FILE_LEN, 44, 255);
-    appendFile(getFileSystem(), path2, file1a);
-    byte[] file1x = Arrays.copyOf(file1, file1.length + file1a.length);
-    System.arraycopy(file1a, 0, file1x, file1.length, file1a.length);
-    // verify fd entity contains contents of file1 + appended bytes
-    verifyFileContents(getFileSystem(), path2, file1x);
-
-    try {
-      // handle should not resolve when content changed
-      instream = getFileSystem().open(fd);
-      fail("Failed to detect change to content");
-    } catch (InvalidPathHandleException e) {
-      // expected
-    }
-  }
-
-
-  /**
-   * Verify {@link HandleOpt#path()} handle semantics.
-   * @throws Throwable on error
-   */
-  @Test
-  public void testOpenFileByPath() throws Throwable {
-    describe("verify open(getPathHandle(FileStatus, path())) operations" +
-        "detect changes");
-    assumeSupportsContentCheck();
-    Path path1 = path("testopenfilebypath1");
-    Path path2 = path("testopenfilebypath2");
-
-    byte[] file1 = dataset(TEST_FILE_LEN, 43, 255);
-    createFile(getFileSystem(), path1, false, file1);
-    FileStatus stat1 = getFileSystem().getFileStatus(path1);
-    assertNotNull(stat1);
-    assertEquals(path1, stat1.getPath());
-    ContractTestUtils.rename(getFileSystem(), path1, path2);
-    FileStatus stat2 = getFileSystem().getFileStatus(path2);
-    assertNotNull(stat2);
-    assertEquals(path2, stat2.getPath());
-    // create identical file at same location, orig still exists at path2
-    createFile(getFileSystem(), path1, false, file1);
-
-    PathHandle fd1 = getHandleOrSkip(stat1, HandleOpt.path());
-    PathHandle fd2 = getHandleOrSkip(stat2, HandleOpt.path());
-
-    // verify path1, path2 contents identical
-    verifyFileContents(getFileSystem(), path1, file1);
-    verifyFileContents(getFileSystem(), path2, file1);
-    try {
-      // verify attempt to resolve the handle fails
-      instream = getFileSystem().open(fd1);
-      fail("Expected an exception");
-    } catch (InvalidPathHandleException e) {
-      // expected
-    }
-
-    // verify content change OK
-    byte[] file2a = dataset(TEST_FILE_LEN, 44, 255);
-    ContractTestUtils.appendFile(getFileSystem(), path2, file2a);
-    byte[] file2x = Arrays.copyOf(file1, file1.length + file2a.length);
-    System.arraycopy(file2a, 0, file2x, file1.length, file2a.length);
-    // verify path2 contains contents of orig + appended bytes
-    verifyFileContents(getFileSystem(), path2, file2x);
-    // verify open by fd succeeds
-    instream = getFileSystem().open(fd2);
-    verifyRead(instream, file2x, 0, 2 * TEST_FILE_LEN);
-  }
-
-  /**
-   * Verify {@link HandleOpt#reference()} handle semantics.
-   * @throws Throwable on error
-   */
-  @Test
-  public void testOpenFileByReference() throws Throwable {
-    describe("verify open(getPathHandle(FileStatus, reference())) operations" +
-        " are independent of rename");
-    assumeSupportsFileReference();
-    Path path1 = path("testopenfilebyref1");
-    Path path2 = path("testopenfilebyref2");
-
-    byte[] file1 = dataset(TEST_FILE_LEN, 43, 255);
-    createFile(getFileSystem(), path1, false, file1);
-    FileStatus stat = getFileSystem().getFileStatus(path1);
-    assertNotNull(stat);
-    assertEquals(path1, stat.getPath());
-    ContractTestUtils.rename(getFileSystem(), path1, path2);
-
-    byte[] file2 = dataset(TEST_FILE_LEN, 44, 255);
-    createFile(getFileSystem(), path1, false, file2);
-    byte[] file1a = dataset(TEST_FILE_LEN, 42, 255);
-    appendFile(getFileSystem(), path2, file1a);
-    byte[] file1x = Arrays.copyOf(file1, file1.length + file1a.length);
-    System.arraycopy(file1a, 0, file1x, file1.length, file1a.length);
-
-    PathHandle fd = getHandleOrSkip(stat, HandleOpt.reference());
-
-    // verify path2 contains contents of file1 + appended bytes
-    verifyFileContents(getFileSystem(), path2, file1x);
-    // verify path1 contents contents of file2
-    verifyFileContents(getFileSystem(), path1, file2);
-
-    // verify fd contains contents of file1 + appended bytes
-    instream = getFileSystem().open(fd);
-    verifyRead(instream, file1x, 0, 2 * TEST_FILE_LEN);
-  }
-
-  /**
-   * Verify {@link PathHandle} may be serialized and restored.
-   * @throws Throwable on error
-   */
-  @Test
-  public void testOpenFileBySerializedReference() throws Throwable {
-    describe("verify PathHandle supports generic serialization");
-    assumeSupportsFileReference();
-    Path path1 = path("testopenfilebyref1");
-    Path path2 = path("testopenfilebyref2");
-
-    byte[] file1 = dataset(TEST_FILE_LEN, 43, 255);
-    createFile(getFileSystem(), path1, false, file1);
-    FileStatus stat = getFileSystem().getFileStatus(path1);
-    assertNotNull(stat);
-    assertEquals(path1, stat.getPath());
-    ContractTestUtils.rename(getFileSystem(), path1, path2);
-
-    byte[] file2 = dataset(TEST_FILE_LEN, 44, 255);
-    createFile(getFileSystem(), path1, false, file2);
-
-    PathHandle fd = getHandleOrSkip(stat, HandleOpt.reference());
-
-    // serialize PathHandle
-    ByteBuffer sb = fd.bytes();
-    PathHandle fdb = new RawPathHandle(sb);
-
-    instream = getFileSystem().open(fdb);
-    // verify stat contains contents of file1
-    verifyRead(instream, file1, 0, TEST_FILE_LEN);
-    // verify path2 contains contents of file1
-    verifyFileContents(getFileSystem(), path2, file1);
-    // verify path1 contents contents of file2
-    verifyFileContents(getFileSystem(), path1, file2);
-  }
-
 }

+ 246 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractPathHandleTest.java

@@ -0,0 +1,246 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.stream.Collectors;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.InvalidPathHandleException;
+import org.apache.hadoop.fs.Options.HandleOpt;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathHandle;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.appendFile;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyRead;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyFileContents;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
+
+import org.apache.hadoop.fs.RawPathHandle;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+/**
+ * Test {@link PathHandle} operations and semantics.
+ * @see ContractOptions#SUPPORTS_FILE_REFERENCE
+ * @see ContractOptions#SUPPORTS_CONTENT_CHECK
+ * @see org.apache.hadoop.fs.FileSystem#getPathHandle(FileStatus, HandleOpt...)
+ * @see org.apache.hadoop.fs.FileSystem#open(PathHandle)
+ * @see org.apache.hadoop.fs.FileSystem#open(PathHandle, int)
+ */
+@RunWith(Parameterized.class)
+public abstract class AbstractContractPathHandleTest
+    extends AbstractFSContractTestBase {
+
+  private final HandleOpt[] opts;
+  private final boolean serialized;
+
+  private static final byte[] B1 = dataset(TEST_FILE_LEN, 43, 255);
+  private static final byte[] B2 = dataset(TEST_FILE_LEN, 44, 255);
+
+  /**
+   * Create an instance of the test from {@link #params()}.
+   * @param testname Name of the set of options under test
+   * @param opts Set of {@link HandleOpt} params under test.
+   * @param serialized Serialize the handle before using it.
+   */
+  public AbstractContractPathHandleTest(String testname, HandleOpt[] opts,
+      boolean serialized) {
+    this.opts = opts;
+    this.serialized = serialized;
+  }
+
+  /**
+   * Run test against all combinations of default options. Also run each
+   * after converting the PathHandle to bytes and back.
+   * @return
+   */
+  @Parameterized.Parameters(name="Test{0}")
+  public static Collection<Object[]> params() {
+    return Arrays.asList(
+        Arrays.asList("Exact", HandleOpt.exact()),
+        Arrays.asList("Content", HandleOpt.content()),
+        Arrays.asList("Path", HandleOpt.path()),
+        Arrays.asList("Reference", HandleOpt.reference())
+    ).stream()
+    .flatMap((x) -> Arrays.asList(true, false).stream()
+        .map((b) -> {
+          ArrayList<Object> y = new ArrayList<>(x);
+          y.add(b);
+          return y;
+        }))
+    .map(ArrayList::toArray)
+    .collect(Collectors.toList());
+  }
+
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    conf.setInt(IO_FILE_BUFFER_SIZE_KEY, 4096);
+    return conf;
+  }
+
+  @Test
+  public void testIdent() throws IOException {
+    describe("verify simple open, no changes");
+    FileStatus stat = testFile(B1);
+    PathHandle fd = getHandleOrSkip(stat);
+    verifyFileContents(getFileSystem(), stat.getPath(), B1);
+
+    try (FSDataInputStream in = getFileSystem().open(fd)) {
+      verifyRead(in, B1, 0, TEST_FILE_LEN);
+    }
+  }
+
+  @Test
+  public void testChanged() throws IOException {
+    describe("verify open(PathHandle, changed(*))");
+    assumeSupportsContentCheck();
+    HandleOpt.Data data = HandleOpt.getOpt(HandleOpt.Data.class, opts)
+        .orElseThrow(IllegalArgumentException::new);
+    FileStatus stat = testFile(B1);
+    // modify the file by appending data
+    appendFile(getFileSystem(), stat.getPath(), B2);
+    byte[] b12 = Arrays.copyOf(B1, B1.length + B2.length);
+    System.arraycopy(B2, 0, b12, B1.length, B2.length);
+    // verify fd entity contains contents of file1 + appended bytes
+    verifyFileContents(getFileSystem(), stat.getPath(), b12);
+    // get the handle *after* the file has been modified
+    PathHandle fd = getHandleOrSkip(stat);
+
+    try (FSDataInputStream in = getFileSystem().open(fd)) {
+      assertTrue("Failed to detect content change", data.allowChange());
+      verifyRead(in, b12, 0, b12.length);
+    } catch (InvalidPathHandleException e) {
+      assertFalse("Failed to allow content change", data.allowChange());
+    }
+  }
+
+  @Test
+  public void testMoved() throws IOException {
+    describe("verify open(PathHandle, moved(*))");
+    assumeSupportsFileReference();
+    HandleOpt.Location loc = HandleOpt.getOpt(HandleOpt.Location.class, opts)
+        .orElseThrow(IllegalArgumentException::new);
+    FileStatus stat = testFile(B1);
+    // rename the file after obtaining FileStatus
+    ContractTestUtils.rename(getFileSystem(), stat.getPath(),
+        path(stat.getPath() + "2"));
+    // obtain handle to entity from #getFileStatus call
+    PathHandle fd = getHandleOrSkip(stat);
+
+    try (FSDataInputStream in = getFileSystem().open(fd)) {
+      assertTrue("Failed to detect location change", loc.allowChange());
+      verifyRead(in, B1, 0, B1.length);
+    } catch (InvalidPathHandleException e) {
+      assertFalse("Failed to allow location change", loc.allowChange());
+    }
+  }
+
+  @Test
+  public void testChangedAndMoved() throws IOException {
+    describe("verify open(PathHandle, changed(*), moved(*))");
+    assumeSupportsFileReference();
+    assumeSupportsContentCheck();
+    HandleOpt.Data data = HandleOpt.getOpt(HandleOpt.Data.class, opts)
+        .orElseThrow(IllegalArgumentException::new);
+    HandleOpt.Location loc = HandleOpt.getOpt(HandleOpt.Location.class, opts)
+        .orElseThrow(IllegalArgumentException::new);
+    FileStatus stat = testFile(B1);
+    Path dst = path(stat.getPath() + "2");
+    ContractTestUtils.rename(getFileSystem(), stat.getPath(), dst);
+    appendFile(getFileSystem(), dst, B2);
+    PathHandle fd = getHandleOrSkip(stat);
+
+    byte[] b12 = Arrays.copyOf(B1, B1.length + B2.length);
+    System.arraycopy(B2, 0, b12, B1.length, B2.length);
+    try (FSDataInputStream in = getFileSystem().open(fd)) {
+      assertTrue("Failed to detect location change", loc.allowChange());
+      assertTrue("Failed to detect content change", data.allowChange());
+      verifyRead(in, b12, 0, b12.length);
+    } catch (InvalidPathHandleException e) {
+      if (data.allowChange()) {
+        assertFalse("Failed to allow location change", loc.allowChange());
+      }
+      if (loc.allowChange()) {
+        assertFalse("Failed to allow content change", data.allowChange());
+      }
+    }
+  }
+
+  private FileStatus testFile(byte[] content) throws IOException {
+    Path path = path(methodName.getMethodName());
+    createFile(getFileSystem(), path, false, content);
+    FileStatus stat = getFileSystem().getFileStatus(path);
+    assertNotNull(stat);
+    assertEquals(path, stat.getPath());
+    return stat;
+  }
+
+  /**
+   * Skip a test case if the FS doesn't support file references.
+   * The feature is assumed to be unsupported unless stated otherwise.
+   */
+  protected void assumeSupportsFileReference() throws IOException {
+    if (getContract().isSupported(SUPPORTS_FILE_REFERENCE, false)) {
+      return;
+    }
+    skip("Skipping as unsupported feature: " + SUPPORTS_FILE_REFERENCE);
+  }
+
+  /**
+   * Skip a test case if the FS doesn't support content validation.
+   * The feature is assumed to be unsupported unless stated otherwise.
+   */
+  protected void assumeSupportsContentCheck() throws IOException {
+    if (getContract().isSupported(SUPPORTS_CONTENT_CHECK, false)) {
+      return;
+    }
+    skip("Skipping as unsupported feature: " + SUPPORTS_CONTENT_CHECK);
+  }
+
+  /**
+   * Utility method to obtain a handle or skip the test if the set of opts
+   * are not supported.
+   * @param stat Target file status
+   * @return Handle to the indicated entity or skip the test
+   */
+  protected PathHandle getHandleOrSkip(FileStatus stat) {
+    try {
+      PathHandle fd = getFileSystem().getPathHandle(stat, opts);
+      if (serialized) {
+        ByteBuffer sb = fd.bytes();
+        return new RawPathHandle(sb);
+      }
+      return fd;
+    } catch (UnsupportedOperationException e) {
+      skip("FileSystem does not support " + Arrays.toString(opts));
+    }
+    // unreachable
+    return null;
+  }
+}

+ 85 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/store/TestEtagChecksum.java

@@ -0,0 +1,85 @@
+/*
+ * 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.store;
+
+import java.io.IOException;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+
+/**
+ * Unit test of etag operations.
+ */
+public class TestEtagChecksum extends Assert {
+
+  private final EtagChecksum empty1 = tag("");
+  private final EtagChecksum empty2 = tag("");
+  private final EtagChecksum valid1 = tag("valid");
+  private final EtagChecksum valid2 = tag("valid");
+
+  @Test
+  public void testEmptyTagsEqual() {
+    assertEquals(empty1, empty2);
+  }
+
+  @Test
+  public void testEmptyTagRoundTrip() throws Throwable {
+    assertEquals(empty1, roundTrip(empty1));
+  }
+
+  @Test
+  public void testValidTagsEqual() {
+    assertEquals(valid1, valid2);
+  }
+
+  @Test
+  public void testValidTagRoundTrip() throws Throwable {
+    assertEquals(valid1, roundTrip(valid1));
+  }
+
+  @Test
+  public void testValidAndEmptyTagsDontMatch() {
+    assertNotEquals(valid1, empty1);
+    assertNotEquals(valid1, tag("other valid one"));
+  }
+
+  @Test
+  public void testDifferentTagsDontMatch() {
+    assertNotEquals(valid1, tag("other valid one"));
+  }
+
+  private EtagChecksum tag(String t) {
+    return new EtagChecksum(t);
+  }
+
+  private EtagChecksum roundTrip(EtagChecksum tag) throws IOException {
+    try (DataOutputBuffer dob = new DataOutputBuffer();
+         DataInputBuffer dib = new DataInputBuffer()) {
+      tag.write(dob);
+      dib.reset(dob.getData(), dob.getLength());
+      EtagChecksum t2 = new EtagChecksum();
+      t2.readFields(dib);
+      return t2;
+    }
+  }
+
+}

+ 77 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestFixKerberosTicketOrder.java

@@ -155,4 +155,81 @@ public class TestFixKerberosTicketOrder extends KerberosSecurityTestcase {
             .filter(t -> t.getServer().getName().startsWith(server2Protocol))
             .findAny().isPresent());
   }
+
+  @Test
+  public void testWithDestroyedTGT() throws Exception {
+    UserGroupInformation ugi =
+        UserGroupInformation.loginUserFromKeytabAndReturnUGI(clientPrincipal,
+            keytabFile.getCanonicalPath());
+    ugi.doAs(new PrivilegedExceptionAction<Void>() {
+
+      @Override
+      public Void run() throws Exception {
+        SaslClient client = Sasl.createSaslClient(
+            new String[] {AuthMethod.KERBEROS.getMechanismName()},
+            clientPrincipal, server1Protocol, host, props, null);
+        client.evaluateChallenge(new byte[0]);
+        client.dispose();
+        return null;
+      }
+    });
+
+    Subject subject = ugi.getSubject();
+
+    // mark the ticket as destroyed
+    for (KerberosTicket ticket : subject
+        .getPrivateCredentials(KerberosTicket.class)) {
+      if (ticket.getServer().getName().startsWith("krbtgt")) {
+        ticket.destroy();
+        break;
+      }
+    }
+
+    ugi.fixKerberosTicketOrder();
+
+    // verify that after fixing, the tgt ticket should be removed
+    assertFalse("The first ticket is not tgt",
+        subject.getPrivateCredentials().stream()
+            .filter(c -> c instanceof KerberosTicket)
+            .map(c -> ((KerberosTicket) c).getServer().getName()).findFirst()
+            .isPresent());
+
+
+    // should fail as we send a service ticket instead of tgt to KDC.
+    intercept(SaslException.class,
+        () -> ugi.doAs(new PrivilegedExceptionAction<Void>() {
+
+          @Override
+          public Void run() throws Exception {
+            SaslClient client = Sasl.createSaslClient(
+                new String[] {AuthMethod.KERBEROS.getMechanismName()},
+                clientPrincipal, server2Protocol, host, props, null);
+            client.evaluateChallenge(new byte[0]);
+            client.dispose();
+            return null;
+          }
+        }));
+
+    // relogin to get a new ticket
+    ugi.reloginFromKeytab();
+
+    // make sure we can get new service ticket after the relogin.
+    ugi.doAs(new PrivilegedExceptionAction<Void>() {
+
+      @Override
+      public Void run() throws Exception {
+        SaslClient client = Sasl.createSaslClient(
+            new String[] {AuthMethod.KERBEROS.getMechanismName()},
+            clientPrincipal, server2Protocol, host, props, null);
+        client.evaluateChallenge(new byte[0]);
+        client.dispose();
+        return null;
+      }
+    });
+
+    assertTrue("No service ticket for " + server2Protocol + " found",
+        subject.getPrivateCredentials(KerberosTicket.class).stream()
+            .filter(t -> t.getServer().getName().startsWith(server2Protocol))
+            .findAny().isPresent());
+  }
 }

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsPathHandle.java

@@ -66,6 +66,9 @@ public final class HdfsPathHandle implements PathHandle {
 
   public void verify(HdfsLocatedFileStatus stat)
       throws InvalidPathHandleException {
+    if (null == stat) {
+      throw new InvalidPathHandleException("Could not resolve handle");
+    }
     if (mtime != null && mtime != stat.getModificationTime()) {
       throw new InvalidPathHandleException("Content changed");
     }

تفاوت فایلی نمایش داده نمی شود زیرا این فایل بسیار بزرگ است
+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_2.8.3.xml


+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferServer.java

@@ -97,7 +97,7 @@ public class SaslDataTransferServer {
    * @param peer connection peer
    * @param underlyingOut connection output stream
    * @param underlyingIn connection input stream
-   * @param int xferPort data transfer port of DataNode accepting connection
+   * @param xferPort data transfer port of DataNode accepting connection
    * @param datanodeId ID of DataNode accepting connection
    * @return new pair of streams, wrapped after SASL negotiation
    * @throws IOException for any error

+ 63 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java

@@ -62,6 +62,28 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
         }
       };
 
+  private static final ThreadLocal<HashMap<NodeNotChosenReason, Integer>>
+      CHOOSE_RANDOM_REASONS = ThreadLocal
+      .withInitial(() -> new HashMap<NodeNotChosenReason, Integer>());
+
+  private enum NodeNotChosenReason {
+    NOT_IN_SERVICE("the node isn't in service"),
+    NODE_STALE("the node is stale"),
+    NODE_TOO_BUSY("the node is too busy"),
+    TOO_MANY_NODES_ON_RACK("the rack has too many chosen nodes"),
+    NOT_ENOUGH_STORAGE_SPACE("no enough storage space to place the block");
+
+    private final String text;
+
+    NodeNotChosenReason(final String logText) {
+      text = logText;
+    }
+
+    private String getText() {
+      return text;
+    }
+  }
+
   protected boolean considerLoad; 
   protected double considerLoadFactor;
   private boolean preferLocalNode;
@@ -711,6 +733,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       builder.setLength(0);
       builder.append("[");
     }
+    CHOOSE_RANDOM_REASONS.get().clear();
     boolean badTarget = false;
     DatanodeStorageInfo firstChosen = null;
     while (numOfReplicas > 0) {
@@ -781,14 +804,24 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     }
     if (numOfReplicas>0) {
       String detail = enableDebugLogging;
-      if (LOG.isDebugEnabled()) {
-        if (badTarget && builder != null) {
-          detail = builder.toString();
+      if (LOG.isDebugEnabled() && builder != null) {
+        detail = builder.toString();
+        if (badTarget) {
           builder.setLength(0);
         } else {
+          if (detail.length() > 1) {
+            // only log if there's more than "[", which is always appended at
+            // the beginning of this method.
+            LOG.debug(detail);
+          }
           detail = "";
         }
       }
+      final HashMap<NodeNotChosenReason, Integer> reasonMap =
+          CHOOSE_RANDOM_REASONS.get();
+      if (!reasonMap.isEmpty()) {
+        LOG.info("Not enough replicas was chosen. Reason:{}", reasonMap);
+      }
       throw new NotEnoughReplicasException(detail);
     }
     
@@ -834,19 +867,38 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     if (storage != null) {
       results.add(storage);
     } else {
-      logNodeIsNotChosen(dnd, "no good storage to place the block ");
+      logNodeIsNotChosen(dnd, NodeNotChosenReason.NOT_ENOUGH_STORAGE_SPACE,
+          " for storage type " + storageType);
     }
     return storage;
   }
 
   private static void logNodeIsNotChosen(DatanodeDescriptor node,
-      String reason) {
+      NodeNotChosenReason reason) {
+    logNodeIsNotChosen(node, reason, null);
+  }
+
+  private static void logNodeIsNotChosen(DatanodeDescriptor node,
+      NodeNotChosenReason reason, String reasonDetails) {
+    assert reason != null;
     if (LOG.isDebugEnabled()) {
       // build the error message for later use.
       debugLoggingBuilder.get()
           .append("\n  Datanode ").append(node)
-          .append(" is not chosen since ").append(reason).append(".");
+          .append(" is not chosen since ").append(reason.getText());
+      if (reasonDetails != null) {
+        debugLoggingBuilder.get().append(" ").append(reasonDetails);
+      }
+      debugLoggingBuilder.get().append(".");
+    }
+    // always populate reason map to log high level reasons.
+    final HashMap<NodeNotChosenReason, Integer> reasonMap =
+        CHOOSE_RANDOM_REASONS.get();
+    Integer base = reasonMap.get(reason);
+    if (base == null) {
+      base = 0;
     }
+    reasonMap.put(reason, base + 1);
   }
 
   /**
@@ -868,13 +920,13 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
                          boolean avoidStaleNodes) {
     // check if the node is (being) decommissioned
     if (!node.isInService()) {
-      logNodeIsNotChosen(node, "the node isn't in service.");
+      logNodeIsNotChosen(node, NodeNotChosenReason.NOT_IN_SERVICE);
       return false;
     }
 
     if (avoidStaleNodes) {
       if (node.isStale(this.staleInterval)) {
-        logNodeIsNotChosen(node, "the node is stale ");
+        logNodeIsNotChosen(node, NodeNotChosenReason.NODE_STALE);
         return false;
       }
     }
@@ -885,8 +937,8 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
           stats.getInServiceXceiverAverage();
       final int nodeLoad = node.getXceiverCount();
       if (nodeLoad > maxLoad) {
-        logNodeIsNotChosen(node, "the node is too busy (load: " + nodeLoad
-            + " > " + maxLoad + ") ");
+        logNodeIsNotChosen(node, NodeNotChosenReason.NODE_TOO_BUSY,
+            "(load: " + nodeLoad + " > " + maxLoad + ")");
         return false;
       }
     }
@@ -901,7 +953,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       }
     }
     if (counter > maxTargetPerRack) {
-      logNodeIsNotChosen(node, "the rack has too many chosen nodes ");
+      logNodeIsNotChosen(node, NodeNotChosenReason.TOO_MANY_NODES_ON_RACK);
       return false;
     }
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java

@@ -764,7 +764,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
       }
     }
     if (requiredSize > remaining - scheduledSize) {
-      LOG.debug(
+      BlockPlacementPolicy.LOG.debug(
           "The node {} does not have enough {} space (required={},"
           + " scheduled={}, remaining={}).",
           this, t, requiredSize, scheduledSize, remaining);

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java

@@ -414,9 +414,12 @@ public class FSDirAttrOp {
       if (oldBR > targetReplication) {
         FSDirectory.LOG.info("Decreasing replication from {} to {} for {}",
                              oldBR, targetReplication, iip.getPath());
-      } else {
+      } else if (oldBR < targetReplication) {
         FSDirectory.LOG.info("Increasing replication from {} to {} for {}",
                              oldBR, targetReplication, iip.getPath());
+      } else {
+        FSDirectory.LOG.info("Replication remains unchanged at {} for {}",
+                             oldBR, iip.getPath());
       }
     }
     return file.getBlocks();

+ 46 - 25
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -33,10 +33,11 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
@@ -45,7 +46,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
-import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
@@ -53,11 +53,11 @@ import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshotFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.util.LongBitFormat;
+import org.apache.hadoop.util.StringUtils;
+import static org.apache.hadoop.io.erasurecode.ErasureCodeConstants.REPLICATION_POLICY_ID;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
-import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
-import org.apache.hadoop.util.StringUtils;
 
 /** I-node for closed file. */
 @InterfaceAudience.Private
@@ -186,28 +186,49 @@ public class INodeFile extends INodeWithAdditionalFields
      * Construct block layout redundancy based on the given BlockType,
      * replication factor and EC PolicyID.
      */
-    static long getBlockLayoutRedundancy(final BlockType blockType,
-        final Short replication, final Byte erasureCodingPolicyID) {
-      long layoutRedundancy = 0;
-      if (blockType == STRIPED) {
-        Preconditions.checkArgument(replication == null &&
-            erasureCodingPolicyID != null);
-        Preconditions.checkArgument(ErasureCodingPolicyManager.getInstance()
-                .getByID(erasureCodingPolicyID) != null,
-            "Could not find EC policy with ID 0x" + StringUtils
-                .byteToHexString(erasureCodingPolicyID));
+    static long getBlockLayoutRedundancy(BlockType blockType,
+        Short replication, Byte erasureCodingPolicyID) {
+      if (null == erasureCodingPolicyID) {
+        erasureCodingPolicyID = REPLICATION_POLICY_ID;
+      }
+      long layoutRedundancy = 0xFF & erasureCodingPolicyID;
+      switch (blockType) {
+      case STRIPED:
+        if (replication != null) {
+          throw new IllegalArgumentException(
+              "Illegal replication for STRIPED block type");
+        }
+        if (erasureCodingPolicyID == REPLICATION_POLICY_ID) {
+          throw new IllegalArgumentException(
+              "Illegal REPLICATION policy for STRIPED block type");
+        }
+        if (null == ErasureCodingPolicyManager.getInstance()
+            .getByID(erasureCodingPolicyID)) {
+          throw new IllegalArgumentException(String.format(
+                "Could not find EC policy with ID 0x%02x",
+                erasureCodingPolicyID));
+        }
+
+        // valid parameters for STRIPED
         layoutRedundancy |= BLOCK_TYPE_MASK_STRIPED;
-        // Following bitwise OR with signed byte erasureCodingPolicyID is safe
-        // as the PolicyID can never be in negative.
-        layoutRedundancy |= erasureCodingPolicyID;
-      } else {
-        Preconditions.checkArgument(erasureCodingPolicyID == null ||
-                erasureCodingPolicyID ==
-                    ErasureCodeConstants.REPLICATION_POLICY_ID);
-        Preconditions.checkArgument(replication != null && replication >= 0 &&
-            replication <= MAX_REDUNDANCY,
-            "Invalid replication value " + replication);
+        break;
+      case CONTIGUOUS:
+        if (erasureCodingPolicyID != REPLICATION_POLICY_ID) {
+          throw new IllegalArgumentException(String.format(
+              "Illegal EC policy 0x%02x for CONTIGUOUS block type",
+              erasureCodingPolicyID));
+        }
+        if (null == replication ||
+            replication < 0 || replication > MAX_REDUNDANCY) {
+          throw new IllegalArgumentException("Invalid replication value "
+              + replication);
+        }
+
+        // valid parameters for CONTIGUOUS
         layoutRedundancy |= replication;
+        break;
+      default:
+        throw new IllegalArgumentException("Unknown blockType: " + blockType);
       }
       return layoutRedundancy;
     }
@@ -599,7 +620,7 @@ public class INodeFile extends INodeWithAdditionalFields
     if (isStriped()) {
       return HeaderFormat.getECPolicyID(header);
     }
-    return ErasureCodeConstants.REPLICATION_POLICY_ID;
+    return REPLICATION_POLICY_ID;
   }
 
   /**

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java

@@ -132,11 +132,11 @@ public class RouterAdmin extends Configured implements Tool {
     try {
       if ("-add".equals(cmd)) {
         if (addMount(argv, i)) {
-          System.err.println("Successfuly added mount point " + argv[i]);
+          System.out.println("Successfuly added mount point " + argv[i]);
         }
       } else if ("-rm".equals(cmd)) {
         if (removeMount(argv[i])) {
-          System.err.println("Successfully removed mount point " + argv[i]);
+          System.out.println("Successfully removed mount point " + argv[i]);
         }
       } else if ("-ls".equals(cmd)) {
         if (argv.length > 1) {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsImageViewer.md

@@ -99,7 +99,7 @@ The Web processor now supports the following operations:
 * [GETACLSTATUS](./WebHDFS.html#Get_ACL_Status)
 * [GETXATTRS](./WebHDFS.html#Get_an_XAttr)
 * [LISTXATTRS](./WebHDFS.html#List_all_XAttrs)
-* [CONTENTSUMMARY] (./WebHDFS.html#Get_Content_Summary_of_a_Directory)
+* [CONTENTSUMMARY](./WebHDFS.html#Get_Content_Summary_of_a_Directory)
 
 ### XML Processor
 

+ 55 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/contract/hdfs/TestHDFSContractPathHandle.java

@@ -0,0 +1,55 @@
+/**
+ * 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.hdfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.contract.AbstractContractPathHandleTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+/**
+ * Verify HDFS compliance with {@link org.apache.hadoop.fs.PathHandle}
+ * semantics.
+ */
+public class TestHDFSContractPathHandle
+    extends AbstractContractPathHandleTest {
+
+  public TestHDFSContractPathHandle(String testname, Options.HandleOpt[] opts,
+      boolean serialized) {
+    super(testname, opts, serialized);
+  }
+
+  @BeforeClass
+  public static void createCluster() throws IOException {
+    HDFSContract.createCluster();
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    HDFSContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new HDFSContract(conf);
+  }
+}

+ 118 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java

@@ -18,16 +18,20 @@
 package org.apache.hadoop.hdfs.server.federation.router;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 import java.io.ByteArrayOutputStream;
 import java.io.PrintStream;
 import java.net.InetSocketAddress;
+import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder;
 import org.apache.hadoop.hdfs.server.federation.RouterDFSCluster.RouterContext;
 import org.apache.hadoop.hdfs.server.federation.StateStoreDFSCluster;
+import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
 import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
 import org.apache.hadoop.hdfs.server.federation.store.impl.MountTableStoreImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesRequest;
@@ -36,6 +40,7 @@ import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
 import org.apache.hadoop.hdfs.tools.federation.RouterAdmin;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ToolRunner;
+import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -83,12 +88,123 @@ public class TestRouterAdminCLI {
   }
 
   @AfterClass
-  public static void tearDown() {
+  public static void tearDownCluster() {
     cluster.stopRouter(routerContext);
     cluster.shutdown();
     cluster = null;
   }
 
+  @After
+  public void tearDown() {
+    // set back system out
+    System.setOut(OLD_OUT);
+  }
+
+  @Test
+  public void testAddMountTable() throws Exception {
+    String nsId = "ns0";
+    String src = "/test-addmounttable";
+    String dest = "/addmounttable";
+    String[] argv = new String[] {"-add", src, nsId, dest};
+    Assert.assertEquals(0, ToolRunner.run(admin, argv));
+
+    stateStore.loadCache(MountTableStoreImpl.class, true);
+    GetMountTableEntriesRequest getRequest = GetMountTableEntriesRequest
+        .newInstance(src);
+    GetMountTableEntriesResponse getResponse = client.getMountTableManager()
+        .getMountTableEntries(getRequest);
+    MountTable mountTable = getResponse.getEntries().get(0);
+
+    List<RemoteLocation> destinations = mountTable.getDestinations();
+    assertEquals(1, destinations.size());
+
+    assertEquals(src, mountTable.getSourcePath());
+    assertEquals(nsId, destinations.get(0).getNameserviceId());
+    assertEquals(dest, destinations.get(0).getDest());
+    assertFalse(mountTable.isReadOnly());
+
+    // test mount table update behavior
+    dest = dest + "-new";
+    argv = new String[] {"-add", src, nsId, dest, "-readonly"};
+    Assert.assertEquals(0, ToolRunner.run(admin, argv));
+    stateStore.loadCache(MountTableStoreImpl.class, true);
+
+    getResponse = client.getMountTableManager()
+        .getMountTableEntries(getRequest);
+    mountTable = getResponse.getEntries().get(0);
+    assertEquals(2, mountTable.getDestinations().size());
+    assertEquals(nsId, mountTable.getDestinations().get(1).getNameserviceId());
+    assertEquals(dest, mountTable.getDestinations().get(1).getDest());
+    assertTrue(mountTable.isReadOnly());
+  }
+
+  @Test
+  public void testListMountTable() throws Exception {
+    String nsId = "ns0";
+    String src = "/test-lsmounttable";
+    String dest = "/lsmounttable";
+    String[] argv = new String[] {"-add", src, nsId, dest};
+    assertEquals(0, ToolRunner.run(admin, argv));
+
+    // re-set system out for testing
+    System.setOut(new PrintStream(out));
+    stateStore.loadCache(MountTableStoreImpl.class, true);
+    argv = new String[] {"-ls", src};
+    assertEquals(0, ToolRunner.run(admin, argv));
+    assertTrue(out.toString().contains(src));
+
+    out.reset();
+    GetMountTableEntriesRequest getRequest = GetMountTableEntriesRequest
+        .newInstance("/");
+    GetMountTableEntriesResponse getResponse = client.getMountTableManager()
+        .getMountTableEntries(getRequest);
+
+    // Test ls command without input path, it will list
+    // mount table under root path.
+    argv = new String[] {"-ls"};
+    assertEquals(0, ToolRunner.run(admin, argv));
+    assertTrue(out.toString().contains(src));
+    String outStr = out.toString();
+    // verify if all the mount table are listed
+    for(MountTable entry: getResponse.getEntries()) {
+      assertTrue(outStr.contains(entry.getSourcePath()));
+    }
+  }
+
+  @Test
+  public void testRemoveMountTable() throws Exception {
+    String nsId = "ns0";
+    String src = "/test-rmmounttable";
+    String dest = "/rmmounttable";
+    String[] argv = new String[] {"-add", src, nsId, dest};
+    assertEquals(0, ToolRunner.run(admin, argv));
+
+    stateStore.loadCache(MountTableStoreImpl.class, true);
+    GetMountTableEntriesRequest getRequest = GetMountTableEntriesRequest
+        .newInstance(src);
+    GetMountTableEntriesResponse getResponse = client.getMountTableManager()
+        .getMountTableEntries(getRequest);
+    // ensure mount table added successfully
+    MountTable mountTable = getResponse.getEntries().get(0);
+    assertEquals(src, mountTable.getSourcePath());
+
+    argv = new String[] {"-rm", src};
+    assertEquals(0, ToolRunner.run(admin, argv));
+
+    stateStore.loadCache(MountTableStoreImpl.class, true);
+    getResponse = client.getMountTableManager()
+        .getMountTableEntries(getRequest);
+    assertEquals(0, getResponse.getEntries().size());
+
+    // remove an invalid mount table
+    String invalidPath = "/invalid";
+    System.setOut(new PrintStream(out));
+    argv = new String[] {"-rm", invalidPath};
+    assertEquals(0, ToolRunner.run(admin, argv));
+    assertTrue(out.toString().contains(
+        "Cannot remove mount point " + invalidPath));
+  }
+
   @Test
   public void testMountTableDefaultACL() throws Exception {
     String[] argv = new String[] {"-add", "/testpath0", "ns0", "/testdir0"};
@@ -140,8 +256,7 @@ public class TestRouterAdminCLI {
     assertEquals(0, ToolRunner.run(admin, argv));
     verifyExecutionResult("/testpath2-3", true, 0, 0);
 
-    // set back system out and login user
-    System.setOut(OLD_OUT);
+    // set back login user
     remoteUser = UserGroupInformation.createRemoteUser(superUser);
     UserGroupInformation.setLoginUser(remoteUser);
   }

+ 2 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java

@@ -66,7 +66,8 @@ public class TestOpenFilesWithSnapshot {
   public void setup() throws IOException {
     conf.setBoolean(
         DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_CAPTURE_OPENFILES, true);
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
+    cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION).build();
     conf.set("dfs.blocksize", "1048576");
     fs = cluster.getFileSystem();
   }
@@ -252,8 +253,6 @@ public class TestOpenFilesWithSnapshot {
    */
   @Test (timeout = 120000)
   public void testPointInTimeSnapshotCopiesForOpenFiles() throws Exception {
-    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_CAPTURE_OPENFILES,
-        true);
     // Construct the directory tree
     final Path level0A = new Path("/level_0_A");
     final Path level0B = new Path("/level_0_B");
@@ -738,8 +737,6 @@ public class TestOpenFilesWithSnapshot {
    */
   @Test (timeout = 120000)
   public void testOpenFilesSnapChecksumWithTrunkAndAppend() throws Exception {
-    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_CAPTURE_OPENFILES,
-        true);
     // Construct the directory tree
     final Path dir = new Path("/A/B/C");
     fs.mkdirs(dir);

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml

@@ -311,7 +311,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>Warning: setting erasure coding policy on an non-empty directory will not automatically convert existing data to RS-6-3-1024</expected-output>
+          <expected-output>Warning: setting erasure coding policy on a non-empty directory will not automatically convert existing files to RS-6-3-1024</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -353,7 +353,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>Warning: unsetting erasure coding policy on an non-empty directory will not automatically convert existing data to replicated data</expected-output>
+          <expected-output>Warning: unsetting erasure coding policy on a non-empty directory will not automatically convert existing files to replicated data</expected-output>
         </comparator>
       </comparators>
     </test>

تفاوت فایلی نمایش داده نمی شود زیرا این فایل بسیار بزرگ است
+ 11 - 0
hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_Common_2.8.3.xml


تفاوت فایلی نمایش داده نمی شود زیرا این فایل بسیار بزرگ است
+ 11 - 0
hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_Core_2.8.3.xml


تفاوت فایلی نمایش داده نمی شود زیرا این فایل بسیار بزرگ است
+ 11 - 0
hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_JobClient_2.8.3.xml


+ 2 - 0
hadoop-project/pom.xml

@@ -1614,6 +1614,8 @@
             <ignore>sun.misc.*</ignore>
             <ignore>sun.net.*</ignore>
             <ignore>sun.nio.ch.*</ignore>
+            <ignore>com.sun.javadoc.*</ignore>
+            <ignore>com.sun.tools.*</ignore>
           </ignores>
         </configuration>
       </plugin>

+ 80 - 13
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java

@@ -112,6 +112,7 @@ import org.apache.hadoop.fs.s3a.s3guard.PathMetadata;
 import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
 import org.apache.hadoop.fs.s3native.S3xLoginHelper;
 import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.fs.store.EtagChecksum;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.BlockingThreadPoolExecutorService;
 import org.apache.hadoop.util.Progressable;
@@ -187,6 +188,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   private long readAhead;
   private S3AInputPolicy inputPolicy;
   private final AtomicBoolean closed = new AtomicBoolean(false);
+  private volatile boolean isClosed = false;
   private MetadataStore metadataStore;
   private boolean allowAuthoritative;
 
@@ -537,6 +539,14 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
     return inputPolicy;
   }
 
+  /**
+   * Get the encryption algorithm of this endpoint.
+   * @return the encryption algorithm.
+   */
+  public S3AEncryptionMethods getServerSideEncryptionAlgorithm() {
+    return serverSideEncryptionAlgorithm;
+  }
+
   /**
    * Demand create the directory allocator, then create a temporary file.
    * {@link LocalDirAllocator#createTmpFileForWrite(String, long, Configuration)}.
@@ -678,7 +688,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
    */
   public FSDataInputStream open(Path f, int bufferSize)
       throws IOException {
-
+    checkNotClosed();
     LOG.debug("Opening '{}' for reading; input policy = {}", f, inputPolicy);
     final FileStatus fileStatus = getFileStatus(f);
     if (fileStatus.isDirectory()) {
@@ -722,6 +732,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   public FSDataOutputStream create(Path f, FsPermission permission,
       boolean overwrite, int bufferSize, short replication, long blockSize,
       Progressable progress) throws IOException {
+    checkNotClosed();
     final Path path = qualify(f);
     String key = pathToKey(path);
     FileStatus status = null;
@@ -871,7 +882,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
     Path dst = qualify(dest);
 
     LOG.debug("Rename path {} to {}", src, dst);
-    incrementStatistic(INVOCATION_RENAME);
+    entryPoint(INVOCATION_RENAME);
 
     String srcKey = pathToKey(src);
     String dstKey = pathToKey(dst);
@@ -1067,6 +1078,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
    * @throws IOException IO and object access problems.
    */
   @VisibleForTesting
+  @Retries.RetryRaw
   public ObjectMetadata getObjectMetadata(Path path) throws IOException {
     return getObjectMetadata(pathToKey(path));
   }
@@ -1097,6 +1109,17 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
     metadataStore = ms;
   }
 
+  /**
+   * Entry point to an operation.
+   * Increments the statistic; verifies the FS is active.
+   * @param operation The operation to increment
+   * @throws IOException if the
+   */
+  protected void entryPoint(Statistic operation) throws IOException {
+    checkNotClosed();
+    incrementStatistic(operation);
+  }
+
   /**
    * Increment a statistic by 1.
    * @param statistic The operation to increment
@@ -1660,6 +1683,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   @Retries.RetryTranslated
   public boolean delete(Path f, boolean recursive) throws IOException {
     try {
+      checkNotClosed();
       return innerDelete(innerGetFileStatus(f, true), recursive);
     } catch (FileNotFoundException e) {
       LOG.debug("Couldn't delete {} - does not exist", f);
@@ -1838,7 +1862,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
     Path path = qualify(f);
     String key = pathToKey(path);
     LOG.debug("List status for path: {}", path);
-    incrementStatistic(INVOCATION_LIST_STATUS);
+    entryPoint(INVOCATION_LIST_STATUS);
 
     List<FileStatus> result;
     final FileStatus fileStatus =  getFileStatus(path);
@@ -1981,7 +2005,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
       throws IOException, FileAlreadyExistsException, AmazonClientException {
     Path f = qualify(p);
     LOG.debug("Making directory: {}", f);
-    incrementStatistic(INVOCATION_MKDIRS);
+    entryPoint(INVOCATION_MKDIRS);
     FileStatus fileStatus;
     List<Path> metadataStoreDirs = null;
     if (hasMetadataStore()) {
@@ -2058,7 +2082,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   @Retries.RetryTranslated
   S3AFileStatus innerGetFileStatus(final Path f,
       boolean needEmptyDirectoryFlag) throws IOException {
-    incrementStatistic(INVOCATION_GET_FILE_STATUS);
+    entryPoint(INVOCATION_GET_FILE_STATUS);
     final Path path = qualify(f);
     String key = pathToKey(path);
     LOG.debug("Getting path status for {}  ({})", path, key);
@@ -2319,7 +2343,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   private void innerCopyFromLocalFile(boolean delSrc, boolean overwrite,
       Path src, Path dst)
       throws IOException, FileAlreadyExistsException, AmazonClientException {
-    incrementStatistic(INVOCATION_COPY_FROM_LOCAL_FILE);
+    entryPoint(INVOCATION_COPY_FROM_LOCAL_FILE);
     LOG.debug("Copying local file from {} to {}", src, dst);
 
     // Since we have a local file, we don't need to stream into a temporary file
@@ -2418,6 +2442,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
       // already closed
       return;
     }
+    isClosed = true;
+    LOG.debug("Filesystem {} is closed", uri);
     try {
       super.close();
     } finally {
@@ -2434,6 +2460,17 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
     }
   }
 
+  /**
+   * Verify that the input stream is open. Non blocking; this gives
+   * the last state of the volatile {@link #closed} field.
+   * @throws IOException if the connection is closed.
+   */
+  private void checkNotClosed() throws IOException {
+    if (isClosed) {
+      throw new IOException(uri + ": " + E_FS_CLOSED);
+    }
+  }
+
   /**
    * Override getCanonicalServiceName because we don't support token in S3A.
    */
@@ -2860,7 +2897,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
    */
   @Override
   public FileStatus[] globStatus(Path pathPattern) throws IOException {
-    incrementStatistic(INVOCATION_GLOB_STATUS);
+    entryPoint(INVOCATION_GLOB_STATUS);
     return super.globStatus(pathPattern);
   }
 
@@ -2871,7 +2908,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   @Override
   public FileStatus[] globStatus(Path pathPattern, PathFilter filter)
       throws IOException {
-    incrementStatistic(INVOCATION_GLOB_STATUS);
+    entryPoint(INVOCATION_GLOB_STATUS);
     return super.globStatus(pathPattern, filter);
   }
 
@@ -2881,7 +2918,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
    */
   @Override
   public boolean exists(Path f) throws IOException {
-    incrementStatistic(INVOCATION_EXISTS);
+    entryPoint(INVOCATION_EXISTS);
     return super.exists(f);
   }
 
@@ -2892,7 +2929,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   @Override
   @SuppressWarnings("deprecation")
   public boolean isDirectory(Path f) throws IOException {
-    incrementStatistic(INVOCATION_IS_DIRECTORY);
+    entryPoint(INVOCATION_IS_DIRECTORY);
     return super.isDirectory(f);
   }
 
@@ -2903,10 +2940,40 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   @Override
   @SuppressWarnings("deprecation")
   public boolean isFile(Path f) throws IOException {
-    incrementStatistic(INVOCATION_IS_FILE);
+    entryPoint(INVOCATION_IS_FILE);
     return super.isFile(f);
   }
 
+  /**
+   * Get the etag of a object at the path via HEAD request and return it
+   * as a checksum object. This has the whatever guarantees about equivalence
+   * the S3 implementation offers.
+   * <ol>
+   *   <li>If a tag has not changed, consider the object unchanged.</li>
+   *   <li>Two tags being different does not imply the data is different.</li>
+   * </ol>
+   * Different S3 implementations may offer different guarantees.
+   * @param f The file path
+   * @param length The length of the file range for checksum calculation
+   * @return The EtagChecksum or null if checksums are not supported.
+   * @throws IOException IO failure
+   * @see <a href="http://docs.aws.amazon.com/AmazonS3/latest/API/RESTCommonResponseHeaders.html">Common Response Headers</a>
+   */
+
+  public EtagChecksum getFileChecksum(Path f, final long length)
+      throws IOException {
+    Preconditions.checkArgument(length >= 0);
+    Path path = qualify(f);
+    LOG.debug("getFileChecksum({})", path);
+    return once("getFileChecksum", path.toString(),
+        () -> {
+          // this always does a full HEAD to the object
+          ObjectMetadata headers = getObjectMetadata(path);
+          String eTag = headers.getETag();
+          return eTag != null ? new EtagChecksum(eTag) : null;
+        });
+  }
+
   /**
    * {@inheritDoc}.
    *
@@ -2948,7 +3015,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
 
   private RemoteIterator<LocatedFileStatus> innerListFiles(Path f, boolean
       recursive, Listing.FileStatusAcceptor acceptor) throws IOException {
-    incrementStatistic(INVOCATION_LIST_FILES);
+    entryPoint(INVOCATION_LIST_FILES);
     Path path = qualify(f);
     LOG.debug("listFiles({}, {})", path, recursive);
     try {
@@ -3033,7 +3100,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   public RemoteIterator<LocatedFileStatus> listLocatedStatus(final Path f,
       final PathFilter filter)
       throws FileNotFoundException, IOException {
-    incrementStatistic(INVOCATION_LIST_LOCATED_STATUS);
+    entryPoint(INVOCATION_LIST_LOCATED_STATUS);
     Path path = qualify(f);
     LOG.debug("listLocatedStatus({}, {}", path, filter);
     try {

+ 24 - 4
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java

@@ -83,7 +83,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
   private final S3AInstrumentation.InputStreamStatistics streamStatistics;
   private S3AEncryptionMethods serverSideEncryptionAlgorithm;
   private String serverSideEncryptionKey;
-  private final S3AInputPolicy inputPolicy;
+  private S3AInputPolicy inputPolicy;
   private long readahead = Constants.DEFAULT_READAHEAD_RANGE;
   private final Invoker invoker;
 
@@ -139,11 +139,21 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
     this.serverSideEncryptionAlgorithm =
         s3Attributes.getServerSideEncryptionAlgorithm();
     this.serverSideEncryptionKey = s3Attributes.getServerSideEncryptionKey();
-    this.inputPolicy = inputPolicy;
+    setInputPolicy(inputPolicy);
     setReadahead(readahead);
     this.invoker = invoker;
   }
 
+  /**
+   * Set/update the input policy of the stream.
+   * This updates the stream statistics.
+   * @param inputPolicy new input policy.
+   */
+  private void setInputPolicy(S3AInputPolicy inputPolicy) {
+    this.inputPolicy = inputPolicy;
+    streamStatistics.inputPolicySet(inputPolicy.ordinal());
+  }
+
   /**
    * Opens up the stream at specified target position and for given length.
    *
@@ -162,8 +172,9 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
     contentRangeFinish = calculateRequestLimit(inputPolicy, targetPos,
         length, contentLength, readahead);
     LOG.debug("reopen({}) for {} range[{}-{}], length={}," +
-        " streamPosition={}, nextReadPosition={}",
-        uri, reason, targetPos, contentRangeFinish, length,  pos, nextReadPos);
+        " streamPosition={}, nextReadPosition={}, policy={}",
+        uri, reason, targetPos, contentRangeFinish, length,  pos, nextReadPos,
+        inputPolicy);
 
     long opencount = streamStatistics.streamOpened();
     GetObjectRequest request = new GetObjectRequest(bucket, key)
@@ -274,6 +285,12 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
     } else if (diff < 0) {
       // backwards seek
       streamStatistics.seekBackwards(diff);
+      // if the stream is in "Normal" mode, switch to random IO at this
+      // point, as it is indicative of columnar format IO
+      if (inputPolicy.equals(S3AInputPolicy.Normal)) {
+        LOG.info("Switching to Random IO seek policy");
+        setInputPolicy(S3AInputPolicy.Random);
+      }
     } else {
       // targetPos == pos
       if (remainingInCurrentRequest() > 0) {
@@ -443,6 +460,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
       try {
         // close or abort the stream
         closeStream("close() operation", this.contentRangeFinish, false);
+        LOG.debug("Statistics of stream {}\n{}", key, streamStatistics);
         // this is actually a no-op
         super.close();
       } finally {
@@ -713,6 +731,8 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
       break;
 
     case Normal:
+      // normal is considered sequential until a backwards seek switches
+      // it to 'Random'
     default:
       rangeLimit = contentLength;
 

+ 13 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java

@@ -667,6 +667,8 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
     public long readsIncomplete;
     public long bytesReadInClose;
     public long bytesDiscardedInAbort;
+    public long policySetCount;
+    public long inputPolicy;
 
     private InputStreamStatistics() {
     }
@@ -782,6 +784,15 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
       mergeInputStreamStatistics(this);
     }
 
+    /**
+     * The input policy has been switched.
+     * @param updatedPolicy enum value of new policy.
+     */
+    public void inputPolicySet(int updatedPolicy) {
+      policySetCount++;
+      inputPolicy = updatedPolicy;
+    }
+
     /**
      * String operator describes all the current statistics.
      * <b>Important: there are no guarantees as to the stability
@@ -813,6 +824,8 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
       sb.append(", ReadsIncomplete=").append(readsIncomplete);
       sb.append(", BytesReadInClose=").append(bytesReadInClose);
       sb.append(", BytesDiscardedInAbort=").append(bytesDiscardedInAbort);
+      sb.append(", InputPolicy=").append(inputPolicy);
+      sb.append(", InputPolicySetCount=").append(policySetCount);
       sb.append('}');
       return sb.toString();
     }

+ 3 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java

@@ -89,6 +89,9 @@ public final class S3AUtils {
       "is abstract and therefore cannot be created";
   static final String ENDPOINT_KEY = "Endpoint";
 
+  /** Filesystem is closed; kept here to keep the errors close. */
+  static final String E_FS_CLOSED = "FileSystem is closed!";
+
   /**
    * Core property for provider path. Duplicated here for consistent
    * code across Hadoop version: {@value}.

+ 12 - 1
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md

@@ -1553,7 +1553,18 @@ backward seeks.
 
 *"normal" (default)*
 
-This is currently the same as "sequential", though it may evolve in future.
+The "Normal" policy starts off reading a file  in "sequential" mode,
+but if the caller seeks backwards in the stream, it switches from
+sequential to "random".
+
+This policy effectively recognizes the initial read pattern of columnar
+storage formats (e.g. Apache ORC and Apache Parquet), which seek to the end
+of a file, read in index data and then seek backwards to selectively read
+columns. The first seeks may be be expensive compared to the random policy,
+however the overall process is much less expensive than either sequentially
+reading through a file with the "random" policy, or reading columnar data
+with the "sequential" policy. When the exact format/recommended
+seek policy of data are known in advance, this policy
 
 *"random"*
 

+ 92 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClosedFS.java

@@ -0,0 +1,92 @@
+/*
+ * 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.s3a;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.Path;
+
+import static org.apache.hadoop.test.LambdaTestUtils.*;
+import static org.apache.hadoop.fs.s3a.S3AUtils.E_FS_CLOSED;
+
+/**
+ * Tests of the S3A FileSystem which is closed; just make sure
+ * that that basic file Ops fail meaningfully.
+ */
+public class ITestS3AClosedFS extends AbstractS3ATestBase {
+
+  private Path root = new Path("/");
+
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    root = getFileSystem().makeQualified(new Path("/"));
+    getFileSystem().close();
+  }
+
+  @Override
+  public void teardown()  {
+    // no op, as the FS is closed
+  }
+
+  @Test
+  public void testClosedGetFileStatus() throws Exception {
+    intercept(IOException.class, E_FS_CLOSED,
+        () -> getFileSystem().getFileStatus(root));
+  }
+
+  @Test
+  public void testClosedListStatus() throws Exception {
+    intercept(IOException.class, E_FS_CLOSED,
+        () -> getFileSystem().listStatus(root));
+  }
+
+  @Test
+  public void testClosedListFile() throws Exception {
+    intercept(IOException.class, E_FS_CLOSED,
+        () -> getFileSystem().listFiles(root, false));
+  }
+
+  @Test
+  public void testClosedListLocatedStatus() throws Exception {
+    intercept(IOException.class, E_FS_CLOSED,
+        () -> getFileSystem().listLocatedStatus(root));
+  }
+
+  @Test
+  public void testClosedCreate() throws Exception {
+    intercept(IOException.class, E_FS_CLOSED,
+        () -> getFileSystem().create(path("to-create")).close());
+  }
+
+  @Test
+  public void testClosedDelete() throws Exception {
+    intercept(IOException.class, E_FS_CLOSED,
+        () ->  getFileSystem().delete(path("to-delete"), false));
+  }
+
+  @Test
+  public void testClosedOpen() throws Exception {
+    intercept(IOException.class, E_FS_CLOSED,
+        () ->  getFileSystem().open(path("to-open")));
+  }
+
+}

+ 116 - 17
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java

@@ -18,21 +18,24 @@
 
 package org.apache.hadoop.fs.s3a;
 
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.apache.hadoop.test.LambdaTestUtils;
+import java.io.ByteArrayInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 
 import com.amazonaws.services.s3.model.ObjectMetadata;
 import com.amazonaws.services.s3.model.PutObjectRequest;
-import com.amazonaws.services.s3.model.PutObjectResult;
+import org.junit.Assume;
 import org.junit.Test;
 
-import java.io.ByteArrayInputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.concurrent.Callable;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.store.EtagChecksum;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
 
 /**
  * Tests of the S3A FileSystem which don't have a specific home and can share
@@ -40,6 +43,8 @@ import java.util.concurrent.Callable;
  */
 public class ITestS3AMiscOperations extends AbstractS3ATestBase {
 
+  private static final byte[] HELLO = "hello".getBytes(StandardCharsets.UTF_8);
+
   @Test
   public void testCreateNonRecursiveSuccess() throws IOException {
     Path shouldWork = path("nonrecursivenode");
@@ -58,7 +63,7 @@ public class ITestS3AMiscOperations extends AbstractS3ATestBase {
   @Test(expected = FileAlreadyExistsException.class)
   public void testCreateNonRecursiveParentIsFile() throws IOException {
     Path parent = path("/file.txt");
-    ContractTestUtils.touch(getFileSystem(), parent);
+    touch(getFileSystem(), parent);
     createNonRecursive(new Path(parent, "fail"));
   }
 
@@ -73,12 +78,7 @@ public class ITestS3AMiscOperations extends AbstractS3ATestBase {
         new ByteArrayInputStream("PUT".getBytes()),
         metadata);
     LambdaTestUtils.intercept(IllegalStateException.class,
-        new Callable<PutObjectResult>() {
-          @Override
-          public PutObjectResult call() throws Exception {
-            return fs.putObjectDirect(put);
-          }
-        });
+        () -> fs.putObjectDirect(put));
     assertPathDoesNotExist("put object was created", path);
   }
 
@@ -87,4 +87,103 @@ public class ITestS3AMiscOperations extends AbstractS3ATestBase {
         (short) 3, (short) 4096,
         null);
   }
+
+  /**
+   * Touch a path, return the full path.
+   * @param name relative name
+   * @return the path
+   * @throws IOException IO failure
+   */
+  Path touchFile(String name) throws IOException {
+    Path path = path(name);
+    touch(getFileSystem(), path);
+    return path;
+  }
+
+  /**
+   * Create a file with the data, return the path.
+   * @param name relative name
+   * @param data data to write
+   * @return the path
+   * @throws IOException IO failure
+   */
+  Path mkFile(String name, byte[] data) throws IOException {
+    final Path f = path(name);
+    createFile(getFileSystem(), f, true, data);
+    return f;
+  }
+
+  /**
+   * The assumption here is that 0-byte files uploaded in a single PUT
+   * always have the same checksum, including stores with encryption.
+   * @throws Throwable on a failure
+   */
+  @Test
+  public void testEmptyFileChecksums() throws Throwable {
+    final S3AFileSystem fs = getFileSystem();
+    Path file1 = touchFile("file1");
+    EtagChecksum checksum1 = fs.getFileChecksum(file1, 0);
+    LOG.info("Checksum for {}: {}", file1, checksum1);
+    assertNotNull("file 1 checksum", checksum1);
+    assertNotEquals("file 1 checksum", 0, checksum1.getLength());
+    assertEquals("checksums", checksum1,
+        fs.getFileChecksum(touchFile("file2"), 0));
+  }
+
+  /**
+   * Verify that different file contents have different
+   * checksums, and that that they aren't the same as the empty file.
+   * @throws Throwable failure
+   */
+  @Test
+  public void testNonEmptyFileChecksums() throws Throwable {
+    final S3AFileSystem fs = getFileSystem();
+    final Path file3 = mkFile("file3", HELLO);
+    final EtagChecksum checksum1 = fs.getFileChecksum(file3, 0);
+    assertNotNull("file 3 checksum", checksum1);
+    final Path file4 = touchFile("file4");
+    final EtagChecksum checksum2 = fs.getFileChecksum(file4, 0);
+    assertNotEquals("checksums", checksum1, checksum2);
+    // overwrite
+    createFile(fs, file4, true,
+        "hello, world".getBytes(StandardCharsets.UTF_8));
+    assertNotEquals(checksum2, fs.getFileChecksum(file4, 0));
+  }
+
+  /**
+   * Verify that on an unencrypted store, the checksum of two non-empty
+   * (single PUT) files is the same if the data is the same.
+   * This will fail if the bucket has S3 default encryption enabled.
+   * @throws Throwable failure
+   */
+  @Test
+  public void testNonEmptyFileChecksumsUnencrypted() throws Throwable {
+    Assume.assumeTrue(encryptionAlgorithm().equals(S3AEncryptionMethods.NONE));
+    final S3AFileSystem fs = getFileSystem();
+    final EtagChecksum checksum1 =
+        fs.getFileChecksum(mkFile("file5", HELLO), 0);
+    assertNotNull("file 3 checksum", checksum1);
+    assertEquals("checksums", checksum1,
+        fs.getFileChecksum(mkFile("file6", HELLO), 0));
+  }
+
+  private S3AEncryptionMethods encryptionAlgorithm() {
+    return getFileSystem().getServerSideEncryptionAlgorithm();
+  }
+
+  @Test
+  public void testNegativeLength() throws Throwable {
+    LambdaTestUtils.intercept(IllegalArgumentException.class,
+        () -> getFileSystem().getFileChecksum(mkFile("negative", HELLO), -1));
+  }
+
+  @Test
+  public void testLengthPastEOF() throws Throwable {
+    final S3AFileSystem fs = getFileSystem();
+    Path f = mkFile("file5", HELLO);
+    assertEquals(
+        fs.getFileChecksum(f, HELLO.length),
+        fs.getFileChecksum(f, HELLO.length * 2));
+  }
+
 }

+ 5 - 1
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java

@@ -427,7 +427,11 @@ public class ITestS3AInputStreamPerformance extends S3AScaleTestBase {
     long expectedOpenCount = RANDOM_IO_SEQUENCE.length;
     executeRandomIO(S3AInputPolicy.Normal, expectedOpenCount);
     assertEquals("streams aborted in " + streamStatistics,
-        4, streamStatistics.aborted);
+        1, streamStatistics.aborted);
+    assertEquals("policy changes in " + streamStatistics,
+        2, streamStatistics.policySetCount);
+    assertEquals("input policy in " + streamStatistics,
+        S3AInputPolicy.Random.ordinal(), streamStatistics.inputPolicy);
   }
 
   /**

+ 12 - 4
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java

@@ -2605,12 +2605,18 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
 
   @Override
   public void rename(String srcKey, String dstKey) throws IOException {
-    rename(srcKey, dstKey, false, null);
+    rename(srcKey, dstKey, false, null, true);
   }
 
   @Override
   public void rename(String srcKey, String dstKey, boolean acquireLease,
-      SelfRenewingLease existingLease) throws IOException {
+                     SelfRenewingLease existingLease) throws IOException {
+    rename(srcKey, dstKey, acquireLease, existingLease, true);
+  }
+
+    @Override
+  public void rename(String srcKey, String dstKey, boolean acquireLease,
+      SelfRenewingLease existingLease, boolean overwriteDestination) throws IOException {
 
     LOG.debug("Moving {} to {}", srcKey, dstKey);
 
@@ -2672,7 +2678,8 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
       // a more intensive exponential retry policy when the cluster is getting
       // throttled.
       try {
-        dstBlob.startCopyFromBlob(srcBlob, null, getInstrumentedContext());
+        dstBlob.startCopyFromBlob(srcBlob, null,
+            getInstrumentedContext(), overwriteDestination);
       } catch (StorageException se) {
         if (se.getHttpStatusCode() == HttpURLConnection.HTTP_UNAVAILABLE) {
           int copyBlobMinBackoff = sessionConfiguration.getInt(
@@ -2695,7 +2702,8 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
           options.setRetryPolicyFactory(new RetryExponentialRetry(
             copyBlobMinBackoff, copyBlobDeltaBackoff, copyBlobMaxBackoff,
             copyBlobMaxRetries));
-          dstBlob.startCopyFromBlob(srcBlob, options, getInstrumentedContext());
+          dstBlob.startCopyFromBlob(srcBlob, options,
+              getInstrumentedContext(), overwriteDestination);
         } else {
           throw se;
         }

+ 18 - 7
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java

@@ -3269,16 +3269,27 @@ public class NativeAzureFileSystem extends FileSystem {
     } else if (!srcMetadata.isDir()) {
       LOG.debug("Source {} found as a file, renaming.", src);
       try {
-        store.rename(srcKey, dstKey);
+        // HADOOP-15086 - file rename must ensure that the destination does
+        // not exist.  The fix is targeted to this call only to avoid
+        // regressions.  Other call sites are attempting to rename temporary
+        // files, redo a failed rename operation, or rename a directory
+        // recursively; for these cases the destination may exist.
+        store.rename(srcKey, dstKey, false, null,
+            false);
       } catch(IOException ex) {
-
         Throwable innerException = NativeAzureFileSystemHelper.checkForAzureStorageException(ex);
 
-        if (innerException instanceof StorageException
-            && NativeAzureFileSystemHelper.isFileNotFoundException((StorageException) innerException)) {
-
-          LOG.debug("BlobNotFoundException encountered. Failing rename", src);
-          return false;
+        if (innerException instanceof StorageException) {
+          if (NativeAzureFileSystemHelper.isFileNotFoundException(
+              (StorageException) innerException)) {
+            LOG.debug("BlobNotFoundException encountered. Failing rename", src);
+            return false;
+          }
+          if (NativeAzureFileSystemHelper.isBlobAlreadyExistsConflict(
+              (StorageException) innerException)) {
+            LOG.debug("Destination BlobAlreadyExists. Failing rename", src);
+            return false;
+          }
         }
 
         throw ex;

+ 18 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemHelper.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.fs.azure;
 
 import java.io.EOFException;
 import java.io.IOException;
+import java.net.HttpURLConnection;
 import java.util.Map;
 
 import com.google.common.base.Preconditions;
@@ -95,6 +96,23 @@ final class NativeAzureFileSystemHelper {
     return false;
   }
 
+  /*
+   * Determines if a conditional request failed because the blob already
+   * exists.
+   *
+   * @param e - the storage exception thrown by the failed operation.
+   *
+   * @return true if a conditional request failed because the blob already
+   * exists; otherwise, returns false.
+   */
+  static boolean isBlobAlreadyExistsConflict(StorageException e) {
+    if (e.getHttpStatusCode() == HttpURLConnection.HTTP_CONFLICT
+        && StorageErrorCodeStrings.BLOB_ALREADY_EXISTS.equals(e.getErrorCode())) {
+      return true;
+    }
+    return false;
+  }
+
   /*
    * Helper method that logs stack traces from all live threads.
    */

+ 4 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java

@@ -91,6 +91,10 @@ interface NativeFileSystemStore {
   void rename(String srcKey, String dstKey, boolean acquireLease, SelfRenewingLease existingLease)
       throws IOException;
 
+  void rename(String srcKey, String dstKey, boolean acquireLease,
+              SelfRenewingLease existingLease, boolean overwriteDestination)
+      throws IOException;
+
   /**
    * Delete all keys with the given prefix. Used for testing.
    *

+ 6 - 2
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SecureStorageInterfaceImpl.java

@@ -503,10 +503,14 @@ public class SecureStorageInterfaceImpl extends StorageInterface {
 
     @Override
     public void startCopyFromBlob(CloudBlobWrapper sourceBlob, BlobRequestOptions options,
-        OperationContext opContext)
+        OperationContext opContext, boolean overwriteDestination)
             throws StorageException, URISyntaxException {
+      AccessCondition dstAccessCondition =
+          overwriteDestination
+              ? null
+              : AccessCondition.generateIfNotExistsCondition();
       getBlob().startCopy(sourceBlob.getBlob().getQualifiedUri(),
-          null, null, options, opContext);
+          null, dstAccessCondition, options, opContext);
     }
 
     @Override

+ 1 - 1
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java

@@ -406,7 +406,7 @@ abstract class StorageInterface {
      *
      */
     public abstract void startCopyFromBlob(CloudBlobWrapper sourceBlob,
-        BlobRequestOptions options, OperationContext opContext)
+        BlobRequestOptions options, OperationContext opContext, boolean overwriteDestination)
         throws StorageException, URISyntaxException;
     
     /**

+ 6 - 2
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java

@@ -425,10 +425,14 @@ class StorageInterfaceImpl extends StorageInterface {
 
     @Override
     public void startCopyFromBlob(CloudBlobWrapper sourceBlob, BlobRequestOptions options,
-        OperationContext opContext)
+        OperationContext opContext, boolean overwriteDestination)
             throws StorageException, URISyntaxException {
+      AccessCondition dstAccessCondition =
+          overwriteDestination
+              ? null
+              : AccessCondition.generateIfNotExistsCondition();
       getBlob().startCopy(sourceBlob.getBlob().getQualifiedUri(),
-          null, null, options, opContext);
+          null, dstAccessCondition, options, opContext);
     }
 
     @Override

+ 72 - 0
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemLive.java

@@ -18,8 +18,16 @@
 
 package org.apache.hadoop.fs.azure;
 
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
 
+import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
@@ -40,6 +48,70 @@ public class ITestNativeAzureFileSystemLive extends
     return AzureBlobStorageTestAccount.create();
   }
 
+  /**
+   * Tests the rename file operation to ensure that when there are multiple
+   * attempts to rename a file to the same destination, only one rename
+   * operation is successful (HADOOP-15086).
+   */
+  @Test
+  public void testMultipleRenameFileOperationsToSameDestination()
+      throws IOException, InterruptedException {
+    final CountDownLatch latch = new CountDownLatch(1);
+    final AtomicInteger successfulRenameCount = new AtomicInteger(0);
+    final AtomicReference<IOException> unexpectedError = new AtomicReference<IOException>();
+    final Path dest = path("dest");
+
+    // Run 10 threads to rename multiple files to the same target path
+    List<Thread> threads = new ArrayList<>();
+
+    for (int i = 0; i < 10; i++) {
+      final int threadNumber = i;
+      Path src = path("test" + threadNumber);
+      threads.add(new Thread(() -> {
+        try {
+          latch.await(Long.MAX_VALUE, TimeUnit.SECONDS);
+        } catch (InterruptedException e) {
+        }
+        try {
+          try (OutputStream output = fs.create(src)) {
+            output.write(("Source file number " + threadNumber).getBytes());
+          }
+
+          if (fs.rename(src, dest)) {
+            LOG.info("rename succeeded for thread " + threadNumber);
+            successfulRenameCount.incrementAndGet();
+          }
+        } catch (IOException e) {
+          unexpectedError.compareAndSet(null, e);
+          ContractTestUtils.fail("Exception unexpected", e);
+        }
+      }));
+    }
+
+    // Start each thread
+    threads.forEach(t -> t.start());
+
+    // Wait for threads to start and wait on latch
+    Thread.sleep(2000);
+
+    // Now start to rename
+    latch.countDown();
+
+    // Wait for all threads to complete
+    threads.forEach(t -> {
+      try {
+        t.join();
+      } catch (InterruptedException e) {
+      }
+    });
+
+    if (unexpectedError.get() != null) {
+      throw unexpectedError.get();
+    }
+    assertEquals(1, successfulRenameCount.get());
+    LOG.info("Success, only one rename operation succeeded!");
+  }
+
   @Test
   public void testLazyRenamePendingCanOverwriteExistingFile()
     throws Exception {

+ 8 - 1
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java

@@ -425,7 +425,14 @@ public class MockStorageInterface extends StorageInterface {
 
     @Override
     public void startCopyFromBlob(CloudBlobWrapper sourceBlob, BlobRequestOptions options,
-        OperationContext opContext) throws StorageException, URISyntaxException {
+        OperationContext opContext, boolean overwriteDestination) throws StorageException, URISyntaxException {
+      if (!overwriteDestination && backingStore.exists(convertUriToDecodedString(uri))) {
+        throw new StorageException("BlobAlreadyExists",
+            "The blob already exists.",
+            HttpURLConnection.HTTP_CONFLICT,
+            null,
+            null);
+      }
       backingStore.copy(convertUriToDecodedString(sourceBlob.getUri()), convertUriToDecodedString(uri));
       //TODO: set the backingStore.properties.CopyState and
       //      update azureNativeFileSystemStore.waitForCopyToComplete

+ 10 - 0
hadoop-tools/hadoop-fs2img/pom.xml

@@ -87,6 +87,16 @@
          </archive>
         </configuration>
        </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+        <configuration>
+          <findbugsXmlOutput>true</findbugsXmlOutput>
+          <xmlOutput>true</xmlOutput>
+          <excludeFilterFile>${basedir}/dev-support/findbugs-exclude.xml</excludeFilterFile>
+          <effort>Max</effort>
+        </configuration>
+      </plugin>
     </plugins>
   </build>
 

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

@@ -650,5 +650,4 @@
     <Method name="equals" />
     <Bug pattern="EQ_OVERRIDING_EQUALS_NOT_SYMMETRIC" />
   </Match>
-
 </FindBugsFilter>

تفاوت فایلی نمایش داده نمی شود زیرا این فایل بسیار بزرگ است
+ 11 - 0
hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Client_2.8.3.xml


تفاوت فایلی نمایش داده نمی شود زیرا این فایل بسیار بزرگ است
+ 11 - 0
hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Common_2.8.3.xml


تفاوت فایلی نمایش داده نمی شود زیرا این فایل بسیار بزرگ است
+ 11 - 0
hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Server_Common_2.8.3.xml


+ 44 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -2389,6 +2389,9 @@ public class YarnConfiguration extends Configuration {
   /**
    * Settings for timeline service v2.0.
    */
+  public static final String TIMELINE_SERVICE_READER_PREFIX =
+      TIMELINE_SERVICE_PREFIX + "reader.";
+
   public static final String TIMELINE_SERVICE_WRITER_CLASS =
       TIMELINE_SERVICE_PREFIX + "writer.class";
 
@@ -2397,7 +2400,7 @@ public class YarnConfiguration extends Configuration {
           + ".storage.HBaseTimelineWriterImpl";
 
   public static final String TIMELINE_SERVICE_READER_CLASS =
-      TIMELINE_SERVICE_PREFIX + "reader.class";
+      TIMELINE_SERVICE_READER_PREFIX + "class";
 
   public static final String DEFAULT_TIMELINE_SERVICE_READER_CLASS =
       "org.apache.hadoop.yarn.server.timelineservice.storage" +
@@ -3422,6 +3425,46 @@ public class YarnConfiguration extends Configuration {
   public static final String TIMELINE_XFS_OPTIONS =
       TIMELINE_XFS_PREFIX + "xframe-options";
 
+  /**
+   * Settings for timeline reader.
+   */
+  public static final String TIMELINE_SERVICE_READER_BIND_HOST =
+      TIMELINE_SERVICE_READER_PREFIX + "bind-host";
+
+  public static final String TIMELINE_SERVICE_READER_WEBAPP_ADDRESS =
+      TIMELINE_SERVICE_READER_PREFIX + "webapp.address";
+  public static final String DEFAULT_TIMELINE_SERVICE_READER_WEBAPP_ADDRESS =
+      DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS;
+
+  public static final String TIMELINE_SERVICE_READER_WEBAPP_HTTPS_ADDRESS =
+      TIMELINE_SERVICE_READER_PREFIX + "webapp.https.address";
+  public static final String
+      DEFAULT_TIMELINE_SERVICE_READER_WEBAPP_HTTPS_ADDRESS =
+      DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS;
+
+  /**
+   * Marked collector properties as Private since it run as auxillary service.
+   */
+  public static final String TIMELINE_SERVICE_COLLECTOR_PREFIX =
+      TIMELINE_SERVICE_PREFIX + "collector.";
+
+  @Private
+  public static final String TIMELINE_SERVICE_COLLECTOR_BIND_HOST =
+      TIMELINE_SERVICE_COLLECTOR_PREFIX + "bind-host";
+
+  @Private
+  public static final String TIMELINE_SERVICE_COLLECTOR_WEBAPP_ADDRESS =
+      TIMELINE_SERVICE_COLLECTOR_PREFIX + "webapp.address";
+  public static final String DEFAULT_TIMELINE_SERVICE_COLLECTOR_WEBAPP_ADDRESS =
+      DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS;
+
+  @Private
+  public static final String TIMELINE_SERVICE_COLLECTOR_WEBAPP_HTTPS_ADDRESS =
+      TIMELINE_SERVICE_COLLECTOR_PREFIX + "webapp.https.address";
+  public static final String
+      DEFAULT_TIMELINE_SERVICE_COLLECTOR_WEBAPP_HTTPS_ADDRESS =
+      DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS;
+
   public YarnConfiguration() {
     super();
   }

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

@@ -247,7 +247,18 @@ definitions:
       kerberos_principal:
         description: The Kerberos Principal of the service
         $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
+    properties:
+      value:
+        type: integer
+        format: int64
+        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
   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.
@@ -262,6 +273,11 @@ definitions:
       memory:
         type: string
         description: Amount of memory allocated to each container (optional but overrides memory in profile if specified). Currently accepts only an integer value and default unit is in MB.
+      additional:
+        type: object
+        additionalProperties:
+          $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.
     properties:

+ 8 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/dev-support/findbugs-exclude.xml

@@ -48,4 +48,12 @@
         <Class name="org.apache.hadoop.yarn.service.ClientAMPolicyProvider"/>
         <Bug pattern="EI_EXPOSE_REP"/>
     </Match>
+    <!-- SE_BAD_FIELD -->
+    <Match>
+      <Class name="org.apache.hadoop.yarn.service.api.records.Resource" />
+      <Or>
+        <Field name="additional"/>
+      </Or>
+      <Bug pattern="SE_BAD_FIELD" />
+  </Match>
 </FindBugsFilter>

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

@@ -44,6 +44,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager;
+import org.apache.hadoop.yarn.service.api.records.ServiceState;
 import org.apache.hadoop.yarn.service.exceptions.BadClusterStateException;
 import org.apache.hadoop.yarn.service.monitor.ServiceMonitor;
 import org.apache.hadoop.yarn.service.utils.ServiceApiUtil;
@@ -237,6 +238,7 @@ public class ServiceMaster extends CompositeService {
       SliderFileSystem fs) throws IOException {
     context.service = ServiceApiUtil
         .loadServiceFrom(fs, new Path(serviceDefPath));
+    context.service.setState(ServiceState.ACCEPTED);
     LOG.info(context.service.toString());
   }
 
@@ -257,6 +259,41 @@ public class ServiceMaster extends CompositeService {
     super.serviceStop();
   }
 
+  // This method should be called whenever there is an increment or decrement
+  // of a READY state component of a service
+  public static synchronized void checkAndUpdateServiceState(
+      ServiceScheduler scheduler, boolean isIncrement) {
+    ServiceState curState = scheduler.getApp().getState();
+    if (!isIncrement) {
+      // set it to STARTED every time a component moves out of STABLE state
+      scheduler.getApp().setState(ServiceState.STARTED);
+    } else {
+      // otherwise check the state of all components
+      boolean isStable = true;
+      for (org.apache.hadoop.yarn.service.api.records.Component comp : scheduler
+          .getApp().getComponents()) {
+        if (comp.getState() !=
+            org.apache.hadoop.yarn.service.api.records.ComponentState.STABLE) {
+          isStable = false;
+          break;
+        }
+      }
+      if (isStable) {
+        scheduler.getApp().setState(ServiceState.STABLE);
+      } else {
+        // mark new state as started only if current state is stable, otherwise
+        // leave it as is
+        if (curState == ServiceState.STABLE) {
+          scheduler.getApp().setState(ServiceState.STARTED);
+        }
+      }
+    }
+    if (curState != scheduler.getApp().getState()) {
+      LOG.info("Service state changed from {} -> {}", curState,
+          scheduler.getApp().getState());
+    }
+  }
+
   private void printSystemEnv() {
     for (Map.Entry<String, String> envs : System.getenv().entrySet()) {
       LOG.info("{} = {}", envs.getKey(), envs.getValue());

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

@@ -58,6 +58,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.service.api.ServiceApiConstants;
 import org.apache.hadoop.yarn.service.api.records.Service;
+import org.apache.hadoop.yarn.service.api.records.ServiceState;
 import org.apache.hadoop.yarn.service.api.records.ConfigFile;
 import org.apache.hadoop.yarn.service.component.instance.ComponentInstance;
 import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEvent;
@@ -75,6 +76,7 @@ import org.apache.hadoop.yarn.service.timelineservice.ServiceTimelinePublisher;
 import org.apache.hadoop.yarn.service.utils.ServiceApiUtil;
 import org.apache.hadoop.yarn.service.utils.ServiceRegistryUtils;
 import org.apache.hadoop.yarn.util.BoundedAppender;
+import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -277,6 +279,12 @@ public class ServiceScheduler extends CompositeService {
     RegisterApplicationMasterResponse response = amRMClient
         .registerApplicationMaster(bindAddress.getHostName(),
             bindAddress.getPort(), "N/A");
+
+    // Update internal resource types according to response.
+    if (response.getResourceTypes() != null) {
+      ResourceUtils.reinitializeResources(response.getResourceTypes());
+    }
+
     if (response.getClientToAMTokenMasterKey() != null
         && response.getClientToAMTokenMasterKey().remaining() != 0) {
       context.secretManager
@@ -284,6 +292,9 @@ public class ServiceScheduler extends CompositeService {
     }
     registerServiceInstance(context.attemptId, app);
 
+    // Since AM has been started and registered, the service is in STARTED state
+    app.setState(ServiceState.STARTED);
+
     // recover components based on containers sent from RM
     recoverComponents(response);
 

+ 39 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Resource.java

@@ -17,16 +17,17 @@
 
 package org.apache.hadoop.yarn.service.api.records;
 
-import io.swagger.annotations.ApiModel;
-import io.swagger.annotations.ApiModelProperty;
-
-import java.util.Objects;
-
 import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import com.fasterxml.jackson.annotation.JsonProperty;
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
+import javax.xml.bind.annotation.XmlElement;
+import java.util.Map;
+import java.util.Objects;
+
 /**
  * Resource determines the amount of resources (vcores, memory, network, etc.)
  * usable by a container. This field determines the resource to be applied for
@@ -46,6 +47,10 @@ public class Resource extends BaseResource implements Cloneable {
   private Integer cpus = 1;
   private String memory = null;
 
+  @JsonProperty("additional")
+  @XmlElement(name = "additional")
+  private Map<String, ResourceInformation> additional = null;
+
   /**
    * Each resource profile has a unique id which is associated with a
    * cluster-level predefined memory, cpus, etc.
@@ -112,6 +117,28 @@ public class Resource extends BaseResource implements Cloneable {
     return Long.parseLong(memory);
   }
 
+  public Resource setResourceInformations(
+      Map<String, ResourceInformation> resourceInformations) {
+    this.additional = resourceInformations;
+    return this;
+  }
+
+  public Resource resourceInformations(
+      Map<String, ResourceInformation> resourceInformations) {
+    this.additional = resourceInformations;
+    return this;
+  }
+
+  /**
+   * Map of resource name to ResourceInformation
+   * @return additional
+   **/
+  @ApiModelProperty(value = "Map of resource name to ResourceInformation")
+  @JsonProperty("additional")
+  public Map<String, ResourceInformation> getAdditional() {
+    return additional;
+  }
+
   @Override
   public boolean equals(java.lang.Object o) {
     if (this == o) {
@@ -121,14 +148,15 @@ public class Resource extends BaseResource implements Cloneable {
       return false;
     }
     Resource resource = (Resource) o;
-    return Objects.equals(this.profile, resource.profile)
-        && Objects.equals(this.cpus, resource.cpus)
-        && Objects.equals(this.memory, resource.memory);
+    return Objects.equals(this.profile, resource.profile) && Objects.equals(
+        this.cpus, resource.cpus) && Objects.equals(this.memory,
+        resource.memory) && Objects.equals(this.additional,
+        resource.additional);
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(profile, cpus, memory);
+    return Objects.hash(profile, cpus, memory, additional);
   }
 
   @Override
@@ -139,6 +167,8 @@ public class Resource extends BaseResource implements Cloneable {
     sb.append("    profile: ").append(toIndentedString(profile)).append("\n");
     sb.append("    cpus: ").append(toIndentedString(cpus)).append("\n");
     sb.append("    memory: ").append(toIndentedString(memory)).append("\n");
+    sb.append("    additional: ").append(
+        toIndentedString(additional)).append("\n");
     sb.append("}");
     return sb.toString();
   }

+ 119 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ResourceInformation.java

@@ -0,0 +1,119 @@
+/*
+ * 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.yarn.service.api.records;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.gson.annotations.SerializedName;
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import java.util.Objects;
+
+/**
+ * ResourceInformation determines unit/name/value of resource types in addition to memory and vcores. It will be part of Resource object
+ */
+@ApiModel(description = "ResourceInformation determines unit/value of resource types in addition to memory and vcores. It will be part of Resource object")
+@javax.annotation.Generated(value = "io.swagger.codegen.languages.JavaClientCodegen",
+                            date = "2017-11-22T15:15:49.495-08:00")
+public class ResourceInformation {
+  @SerializedName("value")
+  private Long value = null;
+
+  @SerializedName("unit")
+  private String unit = null;
+
+  public ResourceInformation value(Long value) {
+    this.value = value;
+    return this;
+  }
+
+  /**
+   * Integer value of the resource.
+   *
+   * @return value
+   **/
+  @ApiModelProperty(value = "Integer value of the resource.")
+  @JsonProperty("value")
+  public Long getValue() {
+    return value;
+  }
+
+  public void setValue(Long value) {
+    this.value = value;
+  }
+
+  public ResourceInformation unit(String unit) {
+    this.unit = unit;
+    return this;
+  }
+
+  /**
+   * @return unit
+   **/
+  @ApiModelProperty(value = "")
+  @JsonProperty("unit")
+  public String getUnit() {
+    return unit == null ? "" : unit;
+  }
+
+  public void setUnit(String unit) {
+    this.unit = unit;
+  }
+
+  @Override
+  public boolean equals(java.lang.Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    ResourceInformation resourceInformation = (ResourceInformation) o;
+    return Objects
+        .equals(this.value, resourceInformation.value) && Objects.equals(
+        this.unit, resourceInformation.unit);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(value, unit);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("class ResourceInformation {\n");
+    sb.append("    value: ").append(toIndentedString(value)).append("\n");
+    sb.append("    unit: ").append(toIndentedString(unit)).append("\n");
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * Convert the given object to string with each line indented by 4 spaces
+   * (except the first line).
+   */
+  private String toIndentedString(java.lang.Object o) {
+    if (o == null) {
+      return "null";
+    }
+    return o.toString().replace("\n", "\n    ");
+  }
+
+}
+

+ 16 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java

@@ -268,7 +268,8 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
       long ret = orig - Long.parseLong(newNumber.substring(1));
       if (ret < 0) {
         LOG.warn(MessageFormat.format(
-            "[COMPONENT {}]: component count goes to negative ({}{} = {}), reset it to 0.",
+            "[COMPONENT {0}]: component count goes to negative ({1}{2} = {3}),"
+                + " ignore and reset it to 0.",
             component.getName(), orig, newNumber, ret));
         ret = 0;
       }
@@ -878,18 +879,23 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
     return newTimeout;
   }
 
-  public ServiceState convertState(FinalApplicationStatus status) {
-    switch (status) {
-    case UNDEFINED:
+  public ServiceState convertState(YarnApplicationState state) {
+    switch (state) {
+    case NEW:
+    case NEW_SAVING:
+    case SUBMITTED:
+    case ACCEPTED:
       return ServiceState.ACCEPTED;
-    case FAILED:
+    case RUNNING:
+      return ServiceState.STARTED;
+    case FINISHED:
     case KILLED:
-      return ServiceState.FAILED;
-    case ENDED:
-    case SUCCEEDED:
       return ServiceState.STOPPED;
+    case FAILED:
+      return ServiceState.FAILED;
+    default:
+      return ServiceState.ACCEPTED;
     }
-    return ServiceState.ACCEPTED;
   }
 
   public String getStatusString(String appId)
@@ -917,7 +923,7 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
     ApplicationReport appReport = yarnClient.getApplicationReport(currentAppId);
     Service appSpec = new Service();
     appSpec.setName(serviceName);
-    appSpec.setState(convertState(appReport.getFinalApplicationStatus()));
+    appSpec.setState(convertState(appReport.getYarnApplicationState()));
     ApplicationTimeout lifetime =
         appReport.getApplicationTimeouts().get(ApplicationTimeoutType.LIFETIME);
     if (lifetime != null) {

+ 80 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java

@@ -26,12 +26,15 @@ import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
 import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.service.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.service.component.instance.ComponentInstance;
 import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceId;
 import org.apache.hadoop.yarn.service.ContainerFailureTracker;
 import org.apache.hadoop.yarn.service.ServiceContext;
 import org.apache.hadoop.yarn.service.ServiceScheduler;
+import org.apache.hadoop.yarn.service.api.records.ServiceState;
 import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEvent;
+import org.apache.hadoop.yarn.service.ServiceMaster;
 import org.apache.hadoop.yarn.service.ServiceMetrics;
 import org.apache.hadoop.yarn.service.provider.ProviderUtils;
 import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
@@ -209,6 +212,7 @@ public class Component implements EventHandler<ComponentEvent> {
         component.createNumCompInstances(delta);
         component.componentSpec.setState(
             org.apache.hadoop.yarn.service.api.records.ComponentState.FLEXING);
+        component.getScheduler().getApp().setState(ServiceState.STARTED);
         return FLEXING;
       } else if (delta < 0){
         delta = 0 - delta;
@@ -229,14 +233,11 @@ public class Component implements EventHandler<ComponentEvent> {
           component.instanceIdCounter.decrementAndGet();
           instance.destroy();
         }
-        component.componentSpec.setState(
-            org.apache.hadoop.yarn.service.api.records.ComponentState.STABLE);
+        checkAndUpdateComponentState(component, false);
         return STABLE;
       } else {
         LOG.info("[FLEX COMPONENT " + component.getName() + "]: already has " +
             event.getDesired() + " instances, ignoring");
-        component.componentSpec.setState(
-            org.apache.hadoop.yarn.service.api.records.ComponentState.STABLE);
         return STABLE;
       }
     }
@@ -289,7 +290,7 @@ public class Component implements EventHandler<ComponentEvent> {
 
   private static ComponentState checkIfStable(Component component) {
     // if desired == running
-    if (component.componentMetrics.containersRunning.value() == component
+    if (component.componentMetrics.containersReady.value() == component
         .getComponentSpec().getNumberOfContainers()) {
       component.componentSpec.setState(
           org.apache.hadoop.yarn.service.api.records.ComponentState.STABLE);
@@ -301,6 +302,46 @@ public class Component implements EventHandler<ComponentEvent> {
     }
   }
 
+  // This method should be called whenever there is an increment or decrement
+  // of a READY state container of a component
+  public static synchronized void checkAndUpdateComponentState(
+      Component component, boolean isIncrement) {
+    org.apache.hadoop.yarn.service.api.records.ComponentState curState =
+        component.componentSpec.getState();
+    if (isIncrement) {
+      // check if all containers are in READY state
+      if (component.componentMetrics.containersReady
+          .value() == component.componentMetrics.containersDesired.value()) {
+        component.componentSpec.setState(
+            org.apache.hadoop.yarn.service.api.records.ComponentState.STABLE);
+        if (curState != component.componentSpec.getState()) {
+          LOG.info("[COMPONENT {}] state changed from {} -> {}",
+              component.componentSpec.getName(), curState,
+              component.componentSpec.getState());
+        }
+        // component state change will trigger re-check of service state
+        ServiceMaster.checkAndUpdateServiceState(component.scheduler,
+            isIncrement);
+      }
+    } else {
+      // container moving out of READY state could be because of FLEX down so
+      // still need to verify the count before changing the component state
+      if (component.componentMetrics.containersReady
+          .value() < component.componentMetrics.containersDesired.value()) {
+        component.componentSpec.setState(
+            org.apache.hadoop.yarn.service.api.records.ComponentState.FLEXING);
+        if (curState != component.componentSpec.getState()) {
+          LOG.info("[COMPONENT {}] state changed from {} -> {}",
+              component.componentSpec.getName(), curState,
+              component.componentSpec.getState());
+        }
+        // component state change will trigger re-check of service state
+        ServiceMaster.checkAndUpdateServiceState(component.scheduler,
+            isIncrement);
+      }
+    }
+  }
+
   private static class ContainerCompletedTransition extends BaseTransition {
     @Override
     public void transition(Component component, ComponentEvent event) {
@@ -310,6 +351,7 @@ public class Component implements EventHandler<ComponentEvent> {
               STOP).setStatus(event.getStatus()));
       component.componentSpec.setState(
           org.apache.hadoop.yarn.service.api.records.ComponentState.FLEXING);
+      component.getScheduler().getApp().setState(ServiceState.STARTED);
     }
   }
 
@@ -351,9 +393,37 @@ public class Component implements EventHandler<ComponentEvent> {
 
   @SuppressWarnings({ "unchecked" })
   public void requestContainers(long count) {
-    Resource resource = Resource
-        .newInstance(componentSpec.getResource().calcMemoryMB(),
-            componentSpec.getResource().getCpus());
+    org.apache.hadoop.yarn.service.api.records.Resource componentResource =
+        componentSpec.getResource();
+
+    Resource resource = Resource.newInstance(componentResource.calcMemoryMB(),
+        componentResource.getCpus());
+
+    if (componentResource.getAdditional() != null) {
+      for (Map.Entry<String, ResourceInformation> entry : componentResource
+          .getAdditional().entrySet()) {
+
+        String resourceName = entry.getKey();
+
+        // Avoid setting memory/cpu under "additional"
+        if (resourceName.equals(
+            org.apache.hadoop.yarn.api.records.ResourceInformation.MEMORY_URI)
+            || resourceName.equals(
+            org.apache.hadoop.yarn.api.records.ResourceInformation.VCORES_URI)) {
+          LOG.warn("Please set memory/vcore in the main section of resource, "
+              + "ignoring this entry=" + resourceName);
+          continue;
+        }
+
+        ResourceInformation specInfo = entry.getValue();
+        org.apache.hadoop.yarn.api.records.ResourceInformation ri =
+            org.apache.hadoop.yarn.api.records.ResourceInformation.newInstance(
+                entry.getKey(),
+                specInfo.getUnit(),
+                specInfo.getValue());
+        resource.setResourceInformation(resourceName, ri);
+      }
+    }
 
     for (int i = 0; i < count; i++) {
       //TODO Once YARN-5468 is done, use that for anti-affinity
@@ -472,11 +542,13 @@ public class Component implements EventHandler<ComponentEvent> {
   public void incContainersReady() {
     componentMetrics.containersReady.incr();
     scheduler.getServiceMetrics().containersReady.incr();
+    checkAndUpdateComponentState(this, true);
   }
 
   public void decContainersReady() {
     componentMetrics.containersReady.decr();
     scheduler.getServiceMetrics().containersReady.decr();
+    checkAndUpdateComponentState(this, false);
   }
 
   public int getNumReadyInstances() {

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java

@@ -147,7 +147,6 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
               new ContainerStatusRetriever(compInstance.scheduler,
                   event.getContainerId(), compInstance), 0, 1,
               TimeUnit.SECONDS);
-      compInstance.component.incRunningContainers();
       long containerStartTime = System.currentTimeMillis();
       try {
         ContainerTokenIdentifier containerTokenIdentifier = BuilderUtils
@@ -171,6 +170,7 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
       compInstance.containerSpec = container;
       compInstance.getCompSpec().addContainer(container);
       compInstance.containerStartedTime = containerStartTime;
+      compInstance.component.incRunningContainers();
 
       if (compInstance.timelineServiceEnabled) {
         compInstance.serviceTimelinePublisher
@@ -183,8 +183,8 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
     @Override
     public void transition(ComponentInstance compInstance,
         ComponentInstanceEvent event) {
-      compInstance.component.incContainersReady();
       compInstance.containerSpec.setState(ContainerState.READY);
+      compInstance.component.incContainersReady();
       if (compInstance.timelineServiceEnabled) {
         compInstance.serviceTimelinePublisher
             .componentInstanceBecomeReady(compInstance.containerSpec);
@@ -196,8 +196,8 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
     @Override
     public void transition(ComponentInstance compInstance,
         ComponentInstanceEvent event) {
-      compInstance.component.decContainersReady();
       compInstance.containerSpec.setState(ContainerState.RUNNING_BUT_UNREADY);
+      compInstance.component.decContainersReady();
     }
   }
 

+ 10 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java

@@ -330,13 +330,19 @@ public class ServiceApiUtil {
       org.apache.hadoop.yarn.api.records.Resource maxResource,
       Service service) throws YarnException {
     for (Component component : service.getComponents()) {
-      // only handle mem now.
       long mem = Long.parseLong(component.getResource().getMemory());
       if (mem > maxResource.getMemorySize()) {
         throw new YarnException(
-            "Component " + component.getName() + " memory size (" + mem
-                + ") is larger than configured max container memory size ("
-                + maxResource.getMemorySize() + ")");
+            "Component " + component.getName() + ": specified memory size ("
+                + mem + ") is larger than configured max container memory " +
+                "size (" + maxResource.getMemorySize() + ")");
+      }
+      int cpu = component.getResource().getCpus();
+      if (cpu > maxResource.getVirtualCores()) {
+        throw new YarnException(
+            "Component " + component.getName() + ": specified number of " +
+                "virtual core (" + cpu + ") is larger than configured max " +
+                "virtual core size (" + maxResource.getVirtualCores() + ")");
       }
     }
   }

+ 4 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceUtils.java

@@ -411,6 +411,10 @@ public final class ServiceUtils {
         return;
       }
       for (File jarFile : listOfJars) {
+        if (!jarFile.exists()) {
+          log.debug("File does not exist, skipping: " + jarFile);
+          continue;
+        }
         LocalResource res = sliderFileSystem.submitFile(jarFile, tempPath, libDir, jarFile.getName());
         providerResources.put(libDir + "/" + jarFile.getName(), res);
       }

+ 25 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/MockServiceAM.java

@@ -29,8 +29,16 @@ import org.apache.hadoop.registry.client.types.yarn.YarnRegistryAttributes;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
-
-import org.apache.hadoop.yarn.api.records.*;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.client.api.AMRMClient;
 import org.apache.hadoop.yarn.client.api.NMClient;
 import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
@@ -47,11 +55,16 @@ import org.apache.hadoop.yarn.service.exceptions.BadClusterStateException;
 import org.apache.hadoop.yarn.service.registry.YarnRegistryViewForProviders;
 import org.apache.hadoop.yarn.service.utils.SliderFileSystem;
 import org.apache.hadoop.yarn.util.Records;
+import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
-import java.util.*;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeoutException;
 
@@ -88,7 +101,6 @@ public class MockServiceAM extends ServiceMaster {
     this.service = service;
   }
 
-
   @Override
   protected ContainerId getAMContainerId()
       throws BadClusterStateException {
@@ -185,7 +197,11 @@ public class MockServiceAM extends ServiceMaster {
           @Override
           public RegisterApplicationMasterResponse registerApplicationMaster(
               String appHostName, int appHostPort, String appTrackingUrl) {
-            return mock(RegisterApplicationMasterResponse.class);
+            RegisterApplicationMasterResponse response = mock(
+                RegisterApplicationMasterResponse.class);
+            when(response.getResourceTypes()).thenReturn(
+                ResourceUtils.getResourcesTypeInfo());
+            return response;
           }
 
           @Override public void unregisterApplicationMaster(
@@ -195,8 +211,11 @@ public class MockServiceAM extends ServiceMaster {
           }
         };
 
-        return AMRMClientAsync.createAMRMClientAsync(client1, 1000,
+        AMRMClientAsync<AMRMClient.ContainerRequest> amrmClientAsync =
+            AMRMClientAsync.createAMRMClientAsync(client1, 1000,
                 this.new AMRMClientCallback());
+
+        return amrmClientAsync;
       }
 
       @SuppressWarnings("SuspiciousMethodCalls")

+ 57 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceAM.java

@@ -18,17 +18,25 @@
 
 package org.apache.hadoop.yarn.service;
 
+import com.google.common.collect.ImmutableMap;
 import org.apache.commons.io.FileUtils;
 import org.apache.curator.test.TestingCluster;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ResourceTypeInfo;
+import org.apache.hadoop.yarn.client.api.AMRMClient;
+import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.service.api.records.Component;
+import org.apache.hadoop.yarn.service.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.service.api.records.Service;
 import org.apache.hadoop.yarn.service.component.ComponentState;
 import org.apache.hadoop.yarn.service.component.instance.ComponentInstance;
 import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceState;
 import org.apache.hadoop.yarn.service.conf.YarnServiceConf;
+import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -38,10 +46,12 @@ import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
 import java.util.concurrent.TimeoutException;
 
-import static org.apache.hadoop.registry.client.api.RegistryConstants
-    .KEY_REGISTRY_ZK_QUORUM;
+import static org.apache.hadoop.registry.client.api.RegistryConstants.KEY_REGISTRY_ZK_QUORUM;
 
 public class TestServiceAM extends ServiceTestUtils{
 
@@ -183,12 +193,12 @@ public class TestServiceAM extends ServiceTestUtils{
     am.init(conf);
     am.start();
     Thread.sleep(100);
-    GenericTestUtils.waitFor(() -> am.getComponent(comp1Name).getState().equals(
-        ComponentState.FLEXING), 100, 2000);
+    GenericTestUtils.waitFor(() -> am.getComponent(comp1Name).getState()
+        .equals(ComponentState.FLEXING), 100, 2000);
 
     // 1 pending instance
-    Assert.assertEquals(1,
-        am.getComponent(comp1Name).getPendingInstances().size());
+    Assert.assertEquals(1, am.getComponent(comp1Name).getPendingInstances()
+        .size());
 
     am.feedContainerToComp(exampleApp, 2, comp1Name);
 
@@ -198,6 +208,47 @@ public class TestServiceAM extends ServiceTestUtils{
         org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
         am.getCompInstance(comp1Name, comp1InstName).getContainerStatus()
             .getState());
+  }
+
+  @Test
+  public void testScheduleWithMultipleResourceTypes()
+      throws TimeoutException, InterruptedException, IOException {
+    ApplicationId applicationId = ApplicationId.newInstance(123456, 1);
+    Service exampleApp = new Service();
+    exampleApp.setId(applicationId.toString());
+    exampleApp.setName("testScheduleWithMultipleResourceTypes");
+
+    List<ResourceTypeInfo> resourceTypeInfos = new ArrayList<>(
+        ResourceUtils.getResourcesTypeInfo());
+    // Add 3rd resource type.
+    resourceTypeInfos.add(ResourceTypeInfo
+        .newInstance("resource-1", "", ResourceTypes.COUNTABLE));
+    // Reinitialize resource types
+    ResourceUtils.reinitializeResources(resourceTypeInfos);
+
+    Component serviceCompoent = createComponent("compa", 1, "pwd");
+    serviceCompoent.getResource().setResourceInformations(ImmutableMap
+        .of("resource-1", new ResourceInformation().value(3333L).unit("Gi")));
+    exampleApp.addComponent(serviceCompoent);
+
+    MockServiceAM am = new MockServiceAM(exampleApp);
+    am.init(conf);
+    am.start();
+
+    ServiceScheduler serviceScheduler = am.context.scheduler;
+    AMRMClientAsync<AMRMClient.ContainerRequest> amrmClientAsync =
+        serviceScheduler.getAmRMClient();
+
+    Collection<AMRMClient.ContainerRequest> rr =
+        amrmClientAsync.getMatchingRequests(0);
+    Assert.assertEquals(1, rr.size());
+
+    org.apache.hadoop.yarn.api.records.Resource capability =
+        rr.iterator().next().getCapability();
+    Assert.assertEquals(3333L, capability.getResourceValue("resource-1"));
+    Assert.assertEquals("Gi",
+        capability.getResourceInformation("resource-1").getUnits());
+
     am.stop();
   }
 }

+ 29 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestYarnNativeServices.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.api.records.*;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.service.api.records.Service;
+import org.apache.hadoop.yarn.service.api.records.ServiceState;
 import org.apache.hadoop.yarn.service.api.records.Component;
 import org.apache.hadoop.yarn.service.api.records.Container;
 import org.apache.hadoop.yarn.service.api.records.ContainerState;
@@ -90,25 +91,25 @@ public class TestYarnNativeServices extends ServiceTestUtils {
     // check app.json is persisted.
     Assert.assertTrue(
         getFS().exists(new Path(appDir, exampleApp.getName() + ".json")));
-    waitForAllCompToBeReady(client, exampleApp);
+    waitForServiceToBeStable(client, exampleApp);
 
     // Flex two components, each from 2 container to 3 containers.
     flexComponents(client, exampleApp, 3L);
     // wait for flex to be completed, increase from 2 to 3 containers.
-    waitForAllCompToBeReady(client, exampleApp);
+    waitForServiceToBeStable(client, exampleApp);
     // check all instances name for each component are in sequential order.
     checkCompInstancesInOrder(client, exampleApp);
 
     // flex down to 1
     flexComponents(client, exampleApp, 1L);
-    waitForAllCompToBeReady(client, exampleApp);
+    waitForServiceToBeStable(client, exampleApp);
     checkCompInstancesInOrder(client, exampleApp);
 
     // check component dir and registry are cleaned up.
 
     // flex up again to 2
     flexComponents(client, exampleApp, 2L);
-    waitForAllCompToBeReady(client, exampleApp);
+    waitForServiceToBeStable(client, exampleApp);
     checkCompInstancesInOrder(client, exampleApp);
 
     // stop the service
@@ -145,7 +146,7 @@ public class TestYarnNativeServices extends ServiceTestUtils {
     exampleApp.addComponent(compb);
 
     client.actionCreate(exampleApp);
-    waitForAllCompToBeReady(client, exampleApp);
+    waitForServiceToBeStable(client, exampleApp);
 
     // check that containers for compa are launched before containers for compb
     checkContainerLaunchDependencies(client, exampleApp, "compa", "compb");
@@ -372,6 +373,29 @@ public class TestYarnNativeServices extends ServiceTestUtils {
     return allContainers;
   }
 
+  /**
+   * Wait until service state becomes stable. A service is stable when all
+   * requested containers of all components are running and in ready state.
+   *
+   * @param client
+   * @param exampleApp
+   * @throws TimeoutException
+   * @throws InterruptedException
+   */
+  private void waitForServiceToBeStable(ServiceClient client,
+      Service exampleApp) throws TimeoutException, InterruptedException {
+    GenericTestUtils.waitFor(() -> {
+      try {
+        Service retrievedApp = client.getStatus(exampleApp.getName());
+        System.out.println(retrievedApp);
+        return retrievedApp.getState() == ServiceState.STABLE;
+      } catch (Exception e) {
+        e.printStackTrace();
+        return false;
+      }
+    }, 2000, 200000);
+  }
+
   private ServiceClient createClient() throws Exception {
     ServiceClient client = new ServiceClient() {
       @Override protected Path addJarResource(String appName,

+ 17 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/conf/TestAppJsonResolve.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.service.conf;
 
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.service.ServiceTestUtils;
+import org.apache.hadoop.yarn.service.api.records.Resource;
 import org.apache.hadoop.yarn.service.api.records.Service;
 import org.apache.hadoop.yarn.service.api.records.ConfigFile;
 import org.apache.hadoop.yarn.service.api.records.Configuration;
@@ -194,6 +195,22 @@ public class TestAppJsonResolve extends Assert {
     assertEquals("60",
         worker.getProperty("yarn.service.failure-count-reset.window"));
 
+    // Validate worker's resources
+    Resource workerResource = orig.getComponent("worker").getResource();
+    Assert.assertEquals(1, workerResource.getCpus().intValue());
+    Assert.assertEquals(1024, workerResource.calcMemoryMB());
+    Assert.assertNotNull(workerResource.getAdditional());
+    Assert.assertEquals(2, workerResource.getAdditional().size());
+    Assert.assertEquals(3333, workerResource.getAdditional().get(
+        "resource-1").getValue().longValue());
+    Assert.assertEquals("Gi", workerResource.getAdditional().get(
+        "resource-1").getUnit());
+
+    Assert.assertEquals(5, workerResource.getAdditional().get(
+        "yarn.io/gpu").getValue().longValue());
+    Assert.assertEquals("", workerResource.getAdditional().get(
+        "yarn.io/gpu").getUnit());
+
     other = orig.getComponent("other").getConfiguration();
     assertEquals(0, other.getProperties().size());
   }

+ 10 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/resources/org/apache/hadoop/yarn/service/conf/examples/app.json

@@ -37,7 +37,16 @@
       "launch_command": "sleep 3600",
       "resource": {
         "cpus": 1,
-        "memory": "1024"
+        "memory": "1024",
+        "additional": {
+          "resource-1": {
+            "value": 3333,
+            "unit": "Gi"
+          },
+          "yarn.io/gpu": {
+            "value": 5
+          }
+        }
       },
       "configuration": {
         "properties": {

+ 32 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java

@@ -314,16 +314,41 @@ public class WebAppUtils {
   }
 
   public static String getAHSWebAppURLWithoutScheme(Configuration conf) {
-    return getTimelineReaderWebAppURL(conf);
-  }
-
-  public static String getTimelineReaderWebAppURL(Configuration conf) {
     if (YarnConfiguration.useHttps(conf)) {
       return conf.get(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS,
-        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS);
+          YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS);
     } else {
       return conf.get(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
-        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS);
+          YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS);
+    }
+  }
+
+  public static String getTimelineReaderWebAppURLWithoutScheme(
+      Configuration conf) {
+    if (YarnConfiguration.useHttps(conf)) {
+      return conf
+          .get(YarnConfiguration.TIMELINE_SERVICE_READER_WEBAPP_HTTPS_ADDRESS,
+              YarnConfiguration.
+                  DEFAULT_TIMELINE_SERVICE_READER_WEBAPP_HTTPS_ADDRESS);
+    } else {
+      return conf.get(YarnConfiguration.TIMELINE_SERVICE_READER_WEBAPP_ADDRESS,
+          YarnConfiguration.
+              DEFAULT_TIMELINE_SERVICE_READER_WEBAPP_ADDRESS);
+    }
+  }
+
+  public static String getTimelineCollectorWebAppURLWithoutScheme(
+      Configuration conf) {
+    if (YarnConfiguration.useHttps(conf)) {
+      return conf.get(
+          YarnConfiguration.TIMELINE_SERVICE_COLLECTOR_WEBAPP_HTTPS_ADDRESS,
+          YarnConfiguration.
+              DEFAULT_TIMELINE_SERVICE_COLLECTOR_WEBAPP_HTTPS_ADDRESS);
+    } else {
+      return conf
+          .get(YarnConfiguration.TIMELINE_SERVICE_COLLECTOR_WEBAPP_ADDRESS,
+              YarnConfiguration.
+                  DEFAULT_TIMELINE_SERVICE_COLLECTOR_WEBAPP_ADDRESS);
     }
   }
 
@@ -342,7 +367,7 @@ public class WebAppUtils {
       return schemePrefix + url;
     }
   }
-  
+
   public static String getRunningLogURL(
       String nodeHttpAddress, String containerId, String user) {
     if (nodeHttpAddress == null || nodeHttpAddress.isEmpty() ||

+ 24 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml

@@ -3599,4 +3599,28 @@
     <value>0,1</value>
   </property>
 
+  <property>
+    <description>The http address of the timeline reader web application.</description>
+    <name>yarn.timeline-service.reader.webapp.address</name>
+    <value>${yarn.timeline-service.webapp.address}</value>
+  </property>
+
+  <property>
+    <description>The https address of the timeline reader web application.</description>
+    <name>yarn.timeline-service.reader.webapp.https.address</name>
+    <value>${yarn.timeline-service.webapp.https.address}</value>
+  </property>
+
+  <property>
+    <description>
+      The actual address timeline reader will bind to. If this optional address is
+      set, the reader server will bind to this address and the port specified in
+      yarn.timeline-service.reader.webapp.address.
+      This is most useful for making the service listen to all interfaces by setting to
+      0.0.0.0.
+    </description>
+    <name>yarn.timeline-service.reader.bind-host</name>
+    <value></value>
+  </property>
+
 </configuration>

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/RecoveredContainerLaunch.java

@@ -72,7 +72,7 @@ public class RecoveredContainerLaunch extends ContainerLaunch {
     String containerIdStr = containerId.toString();
 
     dispatcher.getEventHandler().handle(new ContainerEvent(containerId,
-        ContainerEventType.RECOVER_PAUSED_CONTAINER));
+        ContainerEventType.CONTAINER_LAUNCHED));
 
     boolean notInterrupted = true;
     try {

+ 13 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java

@@ -650,26 +650,34 @@ public class ContainersMonitorImpl extends AbstractService implements
       if (isVmemCheckEnabled()
               && isProcessTreeOverLimit(containerId.toString(),
               currentVmemUsage, curMemUsageOfAgedProcesses, vmemLimit)) {
+        // The current usage (age=0) is always higher than the aged usage. We
+        // do not show the aged size in the message, base the delta on the
+        // current usage
+        long delta = currentVmemUsage - vmemLimit;
         // Container (the root process) is still alive and overflowing
         // memory.
         // Dump the process-tree and then clean it up.
         msg = formatErrorMessage("virtual",
                 formatUsageString(currentVmemUsage, vmemLimit,
                   currentPmemUsage, pmemLimit),
-                pId, containerId, pTree);
+                pId, containerId, pTree, delta);
         isMemoryOverLimit = true;
         containerExitStatus = ContainerExitStatus.KILLED_EXCEEDED_VMEM;
       } else if (isPmemCheckEnabled()
               && isProcessTreeOverLimit(containerId.toString(),
               currentPmemUsage, curRssMemUsageOfAgedProcesses,
               pmemLimit)) {
+        // The current usage (age=0) is always higher than the aged usage. We
+        // do not show the aged size in the message, base the delta on the
+        // current usage
+        long delta = currentPmemUsage - pmemLimit;
         // Container (the root process) is still alive and overflowing
         // memory.
         // Dump the process-tree and then clean it up.
         msg = formatErrorMessage("physical",
                 formatUsageString(currentVmemUsage, vmemLimit,
                   currentPmemUsage, pmemLimit),
-                pId, containerId, pTree);
+                pId, containerId, pTree, delta);
         isMemoryOverLimit = true;
         containerExitStatus = ContainerExitStatus.KILLED_EXCEEDED_PMEM;
       }
@@ -726,11 +734,11 @@ public class ContainersMonitorImpl extends AbstractService implements
      */
     private String formatErrorMessage(String memTypeExceeded,
         String usageString, String pId, ContainerId containerId,
-        ResourceCalculatorProcessTree pTree) {
+        ResourceCalculatorProcessTree pTree, long delta) {
       return
         String.format("Container [pid=%s,containerID=%s] is " +
-            "running beyond %s memory limits. ",
-            pId, containerId, memTypeExceeded) +
+            "running %dB beyond the '%S' memory limit. ",
+            pId, containerId, delta, memTypeExceeded) +
         "Current usage: " + usageString +
         ". Killing container.\n" +
         "Dump of the process-tree for " + containerId + " :\n" +

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java

@@ -332,8 +332,8 @@ public class TestContainersMonitor extends BaseContainerManagerTest {
     Assert.assertEquals(ContainerExitStatus.KILLED_EXCEEDED_VMEM,
         containerStatus.getExitStatus());
     String expectedMsgPattern =
-        "Container \\[pid=" + pid + ",containerID=" + cId
-            + "\\] is running beyond virtual memory limits. Current usage: "
+        "Container \\[pid=" + pid + ",containerID=" + cId + "\\] is running "
+            + "[0-9]+B beyond the 'VIRTUAL' memory limit. Current usage: "
             + "[0-9.]+ ?[KMGTPE]?B of [0-9.]+ ?[KMGTPE]?B physical memory used; "
             + "[0-9.]+ ?[KMGTPE]?B of [0-9.]+ ?[KMGTPE]?B virtual memory used. "
             + "Killing container.\nDump of the process-tree for "

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java

@@ -177,7 +177,7 @@ public class MockAM {
       List<ContainerId> releases, String labelExpression) throws Exception {
     List<ResourceRequest> reqs =
         createReq(new String[] { host }, memory, priority, numContainers,
-            labelExpression, 0L);
+            labelExpression, -1);
     return allocate(reqs, releases);
   }
   

+ 73 - 58
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.applicationsmanager;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -45,6 +46,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockMemoryRMStateStore;
+import org.apache.hadoop.yarn.server.resourcemanager.ParameterizedSchedulerTestBase;
 import org.apache.hadoop.yarn.server.resourcemanager.TestRMRestart;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
@@ -63,14 +65,20 @@ import org.apache.hadoop.yarn.util.Records;
 import org.junit.Assert;
 import org.junit.Test;
 
-public class TestAMRestart {
+/**
+ * Test AM restart functions.
+ */
+public class TestAMRestart extends ParameterizedSchedulerTestBase {
+
+  public TestAMRestart(SchedulerType type) throws IOException {
+    super(type);
+  }
 
   @Test(timeout = 30000)
   public void testAMRestartWithExistingContainers() throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
+    getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
 
-    MockRM rm1 = new MockRM(conf);
+    MockRM rm1 = new MockRM(getConf());
     rm1.start();
     RMApp app1 =
         rm1.submitApp(200, "name", "user",
@@ -266,15 +274,14 @@ public class TestAMRestart {
 
   @Test(timeout = 30000)
   public void testNMTokensRebindOnAMRestart() throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 3);
+    getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 3);
     // To prevent test from blacklisting nm1 for AM, we sit threshold to half
     // of 2 nodes which is 1
-    conf.setFloat(
+    getConf().setFloat(
         YarnConfiguration.AM_SCHEDULING_NODE_BLACKLISTING_DISABLE_THRESHOLD,
         0.5f);
 
-    MockRM rm1 = new MockRM(conf);
+    MockRM rm1 = new MockRM(getConf());
     rm1.start();
     RMApp app1 =
         rm1.submitApp(200, "myname", "myuser",
@@ -378,11 +385,11 @@ public class TestAMRestart {
   // should not be counted towards AM max retry count.
   @Test(timeout = 100000)
   public void testShouldNotCountFailureToMaxAttemptRetry() throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
-    conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
-    conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
-    MockRM rm1 = new MockRM(conf);
+    getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
+    getConf().setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
+    getConf().set(
+        YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
+    MockRM rm1 = new MockRM(getConf());
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8000, rm1.getResourceTrackerService());
@@ -503,11 +510,11 @@ public class TestAMRestart {
 
   @Test(timeout = 100000)
   public void testMaxAttemptOneMeansOne() throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
-    conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
-    conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
-    MockRM rm1 = new MockRM(conf);
+    getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
+    getConf().setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
+    getConf().set(
+        YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
+    MockRM rm1 = new MockRM(getConf());
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8000, rm1.getResourceTrackerService());
@@ -537,14 +544,15 @@ public class TestAMRestart {
   // re-launch the AM.
   @Test(timeout = 60000)
   public void testPreemptedAMRestartOnRMRestart() throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
-    conf.setBoolean(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, false);
+    getConf().setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
+    getConf().setBoolean(
+        YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, false);
 
-    conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
-    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
+    getConf().set(
+        YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
+    getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
 
-    MockRM rm1 = new MockRM(conf);
+    MockRM rm1 = new MockRM(getConf());
     MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     rm1.start();
     MockNM nm1 =
@@ -584,12 +592,19 @@ public class TestAMRestart {
     ApplicationStateData appState =
         memStore.getState().getApplicationState().get(app1.getApplicationId());
     Assert.assertEquals(2, appState.getAttemptCount());
-    // attempt stored has the preempted container exit status.
-    Assert.assertEquals(ContainerExitStatus.PREEMPTED,
-        appState.getAttempt(am2.getApplicationAttemptId())
-            .getAMContainerExitStatus());
+    if (getSchedulerType().equals(SchedulerType.FAIR)) {
+      // attempt stored has the preempted container exit status.
+      Assert.assertEquals(ContainerExitStatus.KILLED_BY_RESOURCEMANAGER,
+          appState.getAttempt(am2.getApplicationAttemptId())
+              .getAMContainerExitStatus());
+    } else {
+      // attempt stored has the preempted container exit status.
+      Assert.assertEquals(ContainerExitStatus.PREEMPTED,
+          appState.getAttempt(am2.getApplicationAttemptId())
+              .getAMContainerExitStatus());
+    }
     // Restart rm.
-    MockRM rm2 = new MockRM(conf, memStore);
+    MockRM rm2 = new MockRM(getConf(), memStore);
     nm1.setResourceTrackerService(rm2.getResourceTrackerService());
     nm1.registerNode();
     rm2.start();
@@ -615,15 +630,16 @@ public class TestAMRestart {
   @Test(timeout = 50000)
   public void testRMRestartOrFailoverNotCountedForAMFailures()
       throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
-    conf.setBoolean(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, false);
+    getConf().setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
+    getConf().setBoolean(
+        YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, false);
 
-    conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
+    getConf().set(
+        YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
     // explicitly set max-am-retry count as 2.
-    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
+    getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
 
-    MockRM rm1 = new MockRM(conf);
+    MockRM rm1 = new MockRM(getConf());
     MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     rm1.start();
     AbstractYarnScheduler scheduler =
@@ -651,7 +667,7 @@ public class TestAMRestart {
     RMAppAttempt attempt2 = app1.getCurrentAppAttempt();
 
     // Restart rm.
-    MockRM rm2 = new MockRM(conf, memStore);
+    MockRM rm2 = new MockRM(getConf(), memStore);
     rm2.start();
     ApplicationStateData appState =
         memStore.getState().getApplicationState().get(app1.getApplicationId());
@@ -688,14 +704,15 @@ public class TestAMRestart {
 
   @Test (timeout = 120000)
   public void testRMAppAttemptFailuresValidityInterval() throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
-    conf.setBoolean(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, false);
+    getConf().setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
+    getConf().setBoolean(
+        YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, false);
 
-    conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
+    getConf().set(
+        YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
     // explicitly set max-am-retry count as 2.
-    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
-    MockRM rm1 = new MockRM(conf);
+    getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
+    MockRM rm1 = new MockRM(getConf());
     rm1.start();
 
     MockMemoryRMStateStore memStore =
@@ -765,7 +782,7 @@ public class TestAMRestart {
 
     // Restart rm.
     @SuppressWarnings("resource")
-    MockRM rm2 = new MockRM(conf, memStore);
+    MockRM rm2 = new MockRM(getConf(), memStore);
     rm2.start();
 
     MockMemoryRMStateStore memStore1 =
@@ -834,12 +851,11 @@ public class TestAMRestart {
     return false;
   }
 
-  @Test(timeout = 30000)
+  @Test(timeout = 40000)
   public void testAMRestartNotLostContainerCompleteMsg() throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
+    getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
 
-    MockRM rm1 = new MockRM(conf);
+    MockRM rm1 = new MockRM(getConf());
     rm1.start();
     RMApp app1 =
         rm1.submitApp(200, "name", "user",
@@ -934,11 +950,10 @@ public class TestAMRestart {
   @Test (timeout = 20000)
   public void testAMRestartNotLostContainerAfterAttemptFailuresValidityInterval()
       throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
     // explicitly set max-am-retry count as 2.
-    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
+    getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
 
-    MockRM rm1 = new MockRM(conf);
+    MockRM rm1 = new MockRM(getConf());
     rm1.start();
     MockNM nm1 =
             new MockNM("127.0.0.1:1234", 8000, rm1.getResourceTrackerService());
@@ -1019,16 +1034,16 @@ public class TestAMRestart {
   @Test(timeout = 200000)
   public void testContainersFromPreviousAttemptsWithRMRestart()
       throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
-    conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
-    conf.setBoolean(
+    getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
+    getConf().setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
+    getConf().setBoolean(
         YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, true);
-    conf.setLong(
+    getConf().setLong(
         YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_SCHEDULING_WAIT_MS, 0);
-    conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
+    getConf()
+        .set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
 
-    MockRM rm1 = new MockRM(conf);
+    MockRM rm1 = new MockRM(getConf());
     MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     rm1.start();
     YarnScheduler scheduler = rm1.getResourceScheduler();
@@ -1071,7 +1086,7 @@ public class TestAMRestart {
         (AbstractYarnScheduler)scheduler, am1.getApplicationAttemptId());
 
     // restart rm
-    MockRM rm2 = new MockRM(conf, memStore);
+    MockRM rm2 = new MockRM(getConf(), memStore);
     rm2.start();
     nm1.setResourceTrackerService(rm2.getResourceTrackerService());
     NMContainerStatus container2Status =

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/security/TestTimelineAuthFilterForV2.java

@@ -162,7 +162,8 @@ public class TestTimelineAuthFilterForV2 {
       conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
       conf.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS,
           FileSystemTimelineWriterImpl.class, TimelineWriter.class);
-      conf.set(YarnConfiguration.TIMELINE_SERVICE_BIND_HOST, "localhost");
+      conf.set(YarnConfiguration.TIMELINE_SERVICE_READER_BIND_HOST,
+          "localhost");
       conf.set(FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_DIR_ROOT,
           TEST_ROOT_DIR.getAbsolutePath());
       conf.set("hadoop.proxyuser.HTTP.hosts", "*");

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/AbstractTimelineReaderHBaseTestBase.java

@@ -79,7 +79,7 @@ public abstract class AbstractTimelineReaderHBaseTestBase {
       Configuration config = util.getConfiguration();
       config.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
       config.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
-      config.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
+      config.set(YarnConfiguration.TIMELINE_SERVICE_READER_WEBAPP_ADDRESS,
           "localhost:0");
       config.set(YarnConfiguration.RM_CLUSTER_ID, "cluster1");
       config.set(YarnConfiguration.TIMELINE_SERVICE_READER_CLASS,

+ 3 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java

@@ -98,7 +98,9 @@ public class FlowRunCoprocessor extends BaseRegionObserver {
     if ((attributes != null) && (attributes.size() > 0)) {
       for (Map.Entry<String, byte[]> attribute : attributes.entrySet()) {
         Tag t = HBaseTimelineStorageUtils.getTagFromAttribute(attribute);
-        tags.add(t);
+        if (t != null) {
+          tags.add(t);
+        }
       }
       byte[] tagByteArray = Tag.fromList(tags);
       NavigableMap<byte[], List<Cell>> newFamilyMap = new TreeMap<>(

+ 17 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/NodeTimelineCollectorManager.java

@@ -145,10 +145,9 @@ public class NodeTimelineCollectorManager extends TimelineCollectorManager {
 
   private void doSecureLogin() throws IOException {
     Configuration conf = getConfig();
-    InetSocketAddress addr = NetUtils.createSocketAddr(conf.getTrimmed(
-        YarnConfiguration.TIMELINE_SERVICE_BIND_HOST,
-            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_BIND_HOST), 0,
-                YarnConfiguration.TIMELINE_SERVICE_BIND_HOST);
+    String webAppURLWithoutScheme =
+        WebAppUtils.getTimelineCollectorWebAppURLWithoutScheme(conf);
+    InetSocketAddress addr = NetUtils.createSocketAddr(webAppURLWithoutScheme);
     SecurityUtil.login(conf, YarnConfiguration.TIMELINE_SERVICE_KEYTAB,
         YarnConfiguration.TIMELINE_SERVICE_PRINCIPAL, addr.getHostName());
   }
@@ -277,8 +276,20 @@ public class NodeTimelineCollectorManager extends TimelineCollectorManager {
         initializers, defaultInitializers, tokenMgrService);
     TimelineServerUtils.setTimelineFilters(
         conf, initializers, defaultInitializers);
-    String bindAddress = conf.get(YarnConfiguration.TIMELINE_SERVICE_BIND_HOST,
-        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_BIND_HOST) + ":0";
+
+    String bindAddress = null;
+    String host =
+        conf.getTrimmed(YarnConfiguration.TIMELINE_SERVICE_COLLECTOR_BIND_HOST);
+    if (host == null || host.isEmpty()) {
+      // if collector bind-host is not set, fall back to
+      // timeline-service.bind-host to maintain compatibility
+      bindAddress =
+          conf.get(YarnConfiguration.DEFAULT_TIMELINE_SERVICE_BIND_HOST,
+              YarnConfiguration.DEFAULT_TIMELINE_SERVICE_BIND_HOST) + ":0";
+    } else {
+      bindAddress = host + ":0";
+    }
+
     try {
       HttpServer2.Builder builder = new HttpServer2.Builder()
           .setName("timeline")

+ 18 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java

@@ -28,6 +28,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.http.HttpServer2;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.HttpCrossOriginFilterInitializer;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.service.CompositeService;
@@ -63,6 +64,8 @@ public class TimelineReaderServer extends CompositeService {
 
   private HttpServer2 readerWebServer;
   private TimelineReaderManager timelineReaderManager;
+  private String webAppURLWithoutScheme;
+
 
   public TimelineReaderServer() {
     super(TimelineReaderServer.class.getName());
@@ -73,10 +76,10 @@ public class TimelineReaderServer extends CompositeService {
     if (!YarnConfiguration.timelineServiceV2Enabled(conf)) {
       throw new YarnException("timeline service v.2 is not enabled");
     }
-    InetSocketAddress bindAddr = conf.getSocketAddr(
-        YarnConfiguration.TIMELINE_SERVICE_ADDRESS,
-            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ADDRESS,
-                YarnConfiguration.DEFAULT_TIMELINE_SERVICE_PORT);
+    webAppURLWithoutScheme =
+        WebAppUtils.getTimelineReaderWebAppURLWithoutScheme(conf);
+    InetSocketAddress bindAddr =
+        NetUtils.createSocketAddr(webAppURLWithoutScheme);
     // Login from keytab if security is enabled.
     try {
       SecurityUtil.login(conf, YarnConfiguration.TIMELINE_SERVICE_KEYTAB,
@@ -170,9 +173,17 @@ public class TimelineReaderServer extends CompositeService {
   private void startTimelineReaderWebApp() {
     Configuration conf = getConfig();
     addFilters(conf);
-    String bindAddress = WebAppUtils.getWebAppBindURL(conf,
-        YarnConfiguration.TIMELINE_SERVICE_BIND_HOST,
-        WebAppUtils.getTimelineReaderWebAppURL(conf));
+
+    String hostProperty = YarnConfiguration.TIMELINE_SERVICE_READER_BIND_HOST;
+    String host = conf.getTrimmed(hostProperty);
+    if (host == null || host.isEmpty()) {
+      // if reader bind-host is not set, fall back to timeline-service.bind-host
+      // to maintain compatibility
+      hostProperty = YarnConfiguration.TIMELINE_SERVICE_BIND_HOST;
+    }
+    String bindAddress = WebAppUtils
+        .getWebAppBindURL(conf, hostProperty, webAppURLWithoutScheme);
+
     LOG.info("Instantiating TimelineReaderWebApp at " + bindAddress);
     try {
       HttpServer2.Builder builder = new HttpServer2.Builder()

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderServer.java

@@ -37,7 +37,7 @@ public class TestTimelineReaderServer {
     Configuration config = new YarnConfiguration();
     config.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
     config.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
-    config.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
+    config.set(YarnConfiguration.TIMELINE_SERVICE_READER_WEBAPP_ADDRESS,
         "localhost:0");
     config.setClass(YarnConfiguration.TIMELINE_SERVICE_READER_CLASS,
         FileSystemTimelineReaderImpl.class, TimelineReader.class);
@@ -61,7 +61,7 @@ public class TestTimelineReaderServer {
     Configuration conf = new YarnConfiguration();
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
     conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
-    conf.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
+    conf.set(YarnConfiguration.TIMELINE_SERVICE_READER_WEBAPP_ADDRESS,
         "localhost:0");
     conf.set(YarnConfiguration.TIMELINE_SERVICE_READER_CLASS,
         Object.class.getName());
@@ -75,7 +75,7 @@ public class TestTimelineReaderServer {
     Configuration conf = new YarnConfiguration();
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
     conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
-    conf.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
+    conf.set(YarnConfiguration.TIMELINE_SERVICE_READER_WEBAPP_ADDRESS,
         "localhost:0");
     conf.set(YarnConfiguration.TIMELINE_SERVICE_READER_CLASS,
         nonexistentTimelineReaderClass);

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java

@@ -84,7 +84,7 @@ public class TestTimelineReaderWebServices {
       Configuration config = new YarnConfiguration();
       config.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
       config.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
-      config.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
+      config.set(YarnConfiguration.TIMELINE_SERVICE_READER_WEBAPP_ADDRESS,
           "localhost:0");
       config.set(YarnConfiguration.RM_CLUSTER_ID, "cluster1");
       config.setClass(YarnConfiguration.TIMELINE_SERVICE_READER_CLASS,

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md

@@ -133,9 +133,9 @@ New configuration parameters that are introduced with v.2 are marked bold.
 | Configuration Property | Description |
 |:---- |:---- |
 | `yarn.timeline-service.hostname` | The hostname of the Timeline service web application. Defaults to `0.0.0.0` |
-| `yarn.timeline-service.address` | Address for the Timeline server to start the RPC server. Defaults to `${yarn.timeline-service.hostname}:10200`. |
-| `yarn.timeline-service.webapp.address` | The http address of the Timeline service web application. Defaults to `${yarn.timeline-service.hostname}:8188`. |
-| `yarn.timeline-service.webapp.https.address` | The https address of the Timeline service web application. Defaults to `${yarn.timeline-service.hostname}:8190`. |
+| `yarn.timeline-service.reader.webapp.address` | The http address of the Timeline Reader web application. Defaults to `${yarn.timeline-service.hostname}:8188`. |
+| `yarn.timeline-service.reader.webapp.https.address` | The https address of the Timeline Reader web application. Defaults to `${yarn.timeline-service.hostname}:8190`. |
+| `yarn.timeline-service.reader.bind-host` | The actual address the timeline reader will bind to. If this optional address is set, reader server will bind to this address and the port specified in yarn.timeline-service.reader.webapp.address. This is most useful for making the service listen on all interfaces by setting to 0.0.0.0. |
 | **`yarn.timeline-service.hbase.configuration.file`** | Optional URL to an hbase-site.xml configuration file to be used to connect to the timeline-service hbase cluster. If empty or not specified, then the HBase configuration will be loaded from the classpath. When specified the values in the specified configuration file will override those from the ones that are present on the classpath. Defaults to `null`. |
 | **`yarn.timeline-service.writer.flush-interval-seconds`** | The setting that controls how often the timeline collector flushes the timeline writer. Defaults to `60`. |
 | **`yarn.timeline-service.app-collector.linger-period.ms`** | Time period till which the application collector will be alive in NM, after the  application master container finishes. Defaults to `1000` (1 second). |

+ 12 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/YarnServiceAPI.md

@@ -1,4 +1,4 @@
-<!---
+# <!---
   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
@@ -330,6 +330,7 @@ Resource determines the amount of resources (vcores, memory, network, etc.) usab
 |profile|Each resource profile has a unique id which is associated with a cluster-level predefined memory, cpus, etc.|false|string||
 |cpus|Amount of vcores allocated to each container (optional but overrides cpus in profile if specified).|false|integer (int32)||
 |memory|Amount of memory allocated to each container (optional but overrides memory in profile if specified). Currently accepts only an integer value and default unit is in MB.|false|string||
+|additional|A map of resource type name to resource type information. Including value (integer), and unit (string). This will be used to specify resource other than cpu and memory. Please refer to example below. |  false | object ||
 
 
 ### Service
@@ -395,8 +396,14 @@ POST URL - http://localhost:8088/ws/v1/services
         "launch_command": "./start_nginx.sh",
         "resource": {
           "cpus": 1,
-          "memory": "256"
-       }
+          "memory": "256",
+          "additional" : {
+            "yarn.io/gpu" : {
+              "value" : 4,
+              "unit" : ""
+            }
+          }     
+        }
       }
     ]
 }
@@ -605,3 +612,5 @@ POST URL - http://localhost:8088:/ws/v1/services/hbase-app-1
   }
 }
 ```
+
+

+ 23 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/queue-navigator.js

@@ -16,7 +16,27 @@
  * limitations under the License.
  */
 
-import Ember from 'ember';
+import Ember from "ember";
 
-export default Ember.Component.extend({
-});
+export default Ember.Component.extend(Ember.TargetActionSupport,{
+  actions: {
+    filterQueuesByPartition(filter) {
+      this.set("filteredPartition", filter);
+      this.sendAction("setFilter", filter);
+    }
+  },
+  didInsertElement: function() {
+    $(".js-filter-queue-by-labels").select2({
+      width: "350px",
+      multiple: false
+    });
+
+    $(".js-filter-queue-by-labels").on("select2:select", e => {
+      this.triggerAction({
+        action: "filterQueuesByPartition",
+        target: this,
+        actionContext: e.params.data.text
+      });
+    });
+  }
+});

+ 187 - 114
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js

@@ -16,19 +16,20 @@
  * limitations under the License.
  */
 
-import Ember from 'ember';
+import Ember from "ember";
+import {PARTITION_LABEL} from '../constants';
 
 const INBETWEEN_HEIGHT = 130;
 
 export default Ember.Component.extend({
   // Map: <queue-name, queue>
-  map : undefined,
+  map: undefined,
 
   // Normalized data for d3
   treeData: undefined,
 
   // folded queues, folded[<queue-name>] == true means <queue-name> is folded
-  foldedQueues: { },
+  foldedQueues: {},
 
   // maxDepth
   maxDepth: 0,
@@ -42,17 +43,23 @@ export default Ember.Component.extend({
   used: undefined,
   max: undefined,
 
+  didUpdateAttrs: function({ oldAttrs, newAttrs }) {
+    if (oldAttrs.filteredPartition.value !== newAttrs.filteredPartition.value) {
+      this.reDraw();
+    }
+  },
   // Init data
   initData: function() {
-    this.map = { };
-    this.treeData = { };
+    this.map = {};
+    this.treeData = {};
     this.maxDepth = 0;
     this.numOfLeafQueue = 0;
 
-    this.get("model")
-      .forEach(function(o) {
+    this.get("model").forEach(
+      function(o) {
         this.map[o.id] = o;
-      }.bind(this));
+      }.bind(this)
+    );
 
     // var selected = this.get("selected");
     this.used = this.get("used");
@@ -81,9 +88,9 @@ export default Ember.Component.extend({
 
   // Init queues
   initQueue: function(queueName, depth, node) {
-    if ((!queueName) || (!this.map[queueName])) {
+    if (!queueName || !this.map[queueName]) {
       // Queue is not existed
-      return;
+      return false;
     }
     if (depth > this.maxDepth) {
       this.maxDepth = this.maxDepth + 1;
@@ -91,6 +98,13 @@ export default Ember.Component.extend({
 
     var queue = this.map[queueName];
 
+    if (
+      this.filteredPartition &&
+      !queue.get("partitions").contains(this.filteredPartition)
+    ) {
+      return false;
+    }
+
     var names = this.getChildrenNamesArray(queue);
 
     node.name = queueName;
@@ -100,14 +114,21 @@ export default Ember.Component.extend({
     if (names.length > 0) {
       node.children = [];
 
-      names.forEach(function(name) {
-        var childQueueData = {};
-        node.children.push(childQueueData);
-        this.initQueue(name, depth + 1, childQueueData);
-      }.bind(this));
+      names.forEach(
+        function(name) {
+          var childQueueData = {};
+          node.children.push(childQueueData);
+          const status = this.initQueue(name, depth + 1, childQueueData);
+          if (!status) {
+            node.children.pop();
+          }
+        }.bind(this)
+      );
     } else {
       this.numOfLeafQueue = this.numOfLeafQueue + 1;
     }
+
+    return true;
   },
 
   update: function(source, root, tree, diagonal) {
@@ -119,141 +140,183 @@ export default Ember.Component.extend({
     var links = tree.links(nodes);
 
     // Normalize for fixed-depth.
-    nodes.forEach(function(d) { d.y = d.depth * 200; });
+    nodes.forEach(function(d) {
+      d.y = d.depth * 200;
+    });
 
     // Update the nodes…
-    var node = this.mainSvg.selectAll("g.node")
-      .data(nodes, function(d) { return d.id || (d.id = ++i); });
+    var node = this.mainSvg.selectAll("g.node").data(nodes, function(d) {
+      return d.id || (d.id = ++i);
+    });
 
     // Enter any new nodes at the parent's previous position.
-    var nodeEnter = node.enter().append("g")
+    var nodeEnter = node
+      .enter()
+      .append("g")
       .attr("class", "node")
-      .attr("transform", function() { return "translate(" + source.y0 + "," + source.x0 + ")"; })
-      .on("click", function(d){
-        if (d.queueData.get("name") !== this.get("selected")) {
-            document.location.href = "#/yarn-queues/" + d.queueData.get("name") + "!";
-        }
-
-        Ember.run.later(this, function () {
-          var treeWidth = this.maxDepth * 200;
-          var treeHeight = this.numOfLeafQueue * INBETWEEN_HEIGHT;
-          var tree = d3.layout.tree().size([treeHeight, treeWidth]);
-          var diagonal = d3.svg.diagonal()
-            .projection(function(d) { return [d.y, d.x]; });
-
-          this.update(this.treeData, this.treeData, tree, diagonal);
-        }, 100);
-
-      }.bind(this))
-    .on("dblclick", function (d) {
-      document.location.href = "#/yarn-queue/" + d.queueData.get("name") + "/apps";
-    });
+      .attr("transform", function() {
+        return `translate(${source.y0 + 50}, ${source.x0})`;
+      })
+      .on(
+        "click",
+        function(d) {
+          if (d.queueData.get("name") !== this.get("selected")) {
+            document.location.href =
+              "#/yarn-queues/" + d.queueData.get("name") + "!";
+          }
+
+          Ember.run.later(
+            this,
+            function() {
+              var treeWidth = this.maxDepth * 200;
+              var treeHeight = this.numOfLeafQueue * INBETWEEN_HEIGHT;
+              var tree = d3.layout.tree().size([treeHeight, treeWidth]);
+              var diagonal = d3.svg.diagonal().projection(function(d) {
+                return [d.y + 50, d.x];
+              });
+
+              this.update(this.treeData, this.treeData, tree, diagonal);
+            },
+            100
+          );
+        }.bind(this)
+      )
+      .on("dblclick", function(d) {
+        document.location.href =
+          "#/yarn-queue/" + d.queueData.get("name") + "/apps";
+      });
 
-    nodeEnter.append("circle")
+    nodeEnter
+      .append("circle")
       .attr("r", 1e-6)
-      .style("fill", function(d) {
-        var maxCap = d.queueData.get(this.max);
-        maxCap = maxCap === undefined ? 100 : maxCap;
-        var usedCap = d.queueData.get(this.used) / maxCap * 100.0;
-        if (usedCap <= 60.0) {
-          return "mediumaquamarine";
-        } else if (usedCap <= 100.0) {
-          return "coral";
-        } else {
-          return "salmon";
-        }
-      }.bind(this));
+      .style(
+        "fill",
+        function(d) {
+          const usedCapacity = getUsedCapacity(d.queueData.get("partitionMap"), this.filteredPartition);
+          if (usedCapacity <= 60.0) {
+            return "#60cea5";
+          } else if (usedCapacity <= 100.0) {
+            return "#ffbc0b";
+          } else {
+            return "#ef6162";
+          }
+        }.bind(this)
+      );
 
     // append percentage
-    nodeEnter.append("text")
-      .attr("x", function() { return 0; })
+    nodeEnter
+      .append("text")
+      .attr("x", function() {
+        return 0;
+      })
       .attr("dy", ".35em")
       .attr("fill", "white")
-      .attr("text-anchor", function() { return "middle"; })
-      .text(function(d) {
-        var maxCap = d.queueData.get(this.max);
-        maxCap = maxCap === undefined ? 100 : maxCap;
-        var usedCap = d.queueData.get(this.used) / maxCap * 100.0;
-        if (usedCap >= 100.0) {
-          return usedCap.toFixed(0) + "%";
-        } else {
-          return usedCap.toFixed(1) + "%";
-        }
-      }.bind(this))
+      .attr("text-anchor", function() {
+        return "middle";
+      })
+      .text(
+        function(d) {
+          const usedCapacity = getUsedCapacity(d.queueData.get("partitionMap"), this.filteredPartition);
+          if (usedCapacity >= 100.0) {
+            return usedCapacity.toFixed(0) + "%";
+          } else {
+            return usedCapacity.toFixed(1) + "%";
+          }
+        }.bind(this)
+      )
       .style("fill-opacity", 1e-6);
 
     // append queue name
-    nodeEnter.append("text")
+    nodeEnter
+      .append("text")
       .attr("x", "0px")
       .attr("dy", "45px")
       .attr("text-anchor", "middle")
-      .text(function(d) { return d.name; })
+      .text(function(d) {
+        return d.name;
+      })
       .style("fill-opacity", 1e-6);
 
     // Transition nodes to their new position.
-    var nodeUpdate = node.transition()
+    var nodeUpdate = node
+      .transition()
       .duration(duration)
-      .attr("transform", function(d) { return "translate(" + d.y + "," + d.x + ")"; });
+      .attr("transform", function(d) {
+        return `translate(${d.y + 50}, ${d.x})`;
+      });
 
-    nodeUpdate.select("circle")
+    nodeUpdate
+      .select("circle")
       .attr("r", 30)
-      .attr("href",
+      .attr("href", function(d) {
+        return "#/yarn-queues/" + d.queueData.get("name");
+      })
+      .style(
+        "stroke-width",
+        function(d) {
+          if (d.queueData.get("name") === this.get("selected")) {
+            return 7;
+          } else {
+            return 2;
+          }
+        }.bind(this)
+      )
+      .style(
+        "stroke",
         function(d) {
-          return "#/yarn-queues/" + d.queueData.get("name");
-        })
-      .style("stroke-width", function(d) {
-        if (d.queueData.get("name") === this.get("selected")) {
-          return 7;
-        } else {
-          return 2;
-        }
-      }.bind(this))
-      .style("stroke", function(d) {
-        if (d.queueData.get("name") === this.get("selected")) {
-          return "gray";
-        } else {
-          return "gray";
-        }
-      }.bind(this));
-
-    nodeUpdate.selectAll("text")
-      .style("fill-opacity", 1);
+          if (d.queueData.get("name") === this.get("selected")) {
+            return "gray";
+          } else {
+            return "gray";
+          }
+        }.bind(this)
+      );
+
+    nodeUpdate.selectAll("text").style("fill-opacity", 1);
 
     // Transition exiting nodes to the parent's new position.
-    var nodeExit = node.exit().transition()
+    var nodeExit = node
+      .exit()
+      .transition()
       .duration(duration)
-      .attr("transform", function() { return "translate(" + source.y + "," + source.x + ")"; })
+      .attr("transform", function() {
+        return `translate(${source.y}, ${source.x})`;
+      })
       .remove();
 
-    nodeExit.select("circle")
-      .attr("r", 1e-6);
+    nodeExit.select("circle").attr("r", 1e-6);
 
-    nodeExit.select("text")
-      .style("fill-opacity", 1e-6);
+    nodeExit.select("text").style("fill-opacity", 1e-6);
 
     // Update the links…
-    var link = this.mainSvg.selectAll("path.link")
-      .data(links, function(d) { return d.target.id; });
+    var link = this.mainSvg.selectAll("path.link").data(links, function(d) {
+      return d.target.id;
+    });
 
     // Enter any new links at the parent's previous position.
-    link.enter().insert("path", "g")
+    link
+      .enter()
+      .insert("path", "g")
       .attr("class", "link")
       .attr("d", function() {
-        var o = {x: source.x0, y: source.y0};
-        return diagonal({source: o, target: o});
+        var o = { x: source.x0, y: source.y0 + 50 };
+        return diagonal({ source: o, target: o });
       });
 
     // Transition links to their new position.
-    link.transition()
+    link
+      .transition()
       .duration(duration)
       .attr("d", diagonal);
 
     // Transition exiting nodes to the parent's new position.
-    link.exit().transition()
+    link
+      .exit()
+      .transition()
       .duration(duration)
       .attr("d", function() {
-        var o = {x: source.x, y: source.y};
-        return diagonal({source: o, target: o});
+        var o = { x: source.x, y: source.y };
+        return diagonal({ source: o, target: o });
       })
       .remove();
 
@@ -267,27 +330,32 @@ export default Ember.Component.extend({
   reDraw: function() {
     this.initData();
 
-    var margin = {top: 20, right: 120, bottom: 20, left: 120};
+    var margin = { top: 20, right: 120, bottom: 20, left: 120 };
     var treeWidth = this.maxDepth * 200;
     var treeHeight = this.numOfLeafQueue * INBETWEEN_HEIGHT;
     var width = treeWidth + margin.left + margin.right;
     var height = treeHeight + margin.top + margin.bottom;
 
     if (this.mainSvg) {
-      this.mainSvg.remove();
+      this.mainSvg.selectAll("*").remove();
+    } else {
+      this.mainSvg = d3
+        .select("#" + this.get("parentId"))
+        .append("svg")
+        .attr("width", width)
+        .attr("height", height)
+        .attr("class", "tree-selector");
     }
 
-    this.mainSvg = d3.select("#" + this.get("parentId")).append("svg")
-      .attr("width", width)
-      .attr("height", height)
-      .attr("class", "tree-selector")
+    this.mainSvg
       .append("g")
       .attr("transform", "translate(" + margin.left + "," + margin.top + ")");
 
     var tree = d3.layout.tree().size([treeHeight, treeWidth]);
 
-    var diagonal = d3.svg.diagonal()
-      .projection(function(d) { return [d.y, d.x]; });
+    var diagonal = d3.svg.diagonal().projection(function(d) {
+      return [d.y + 50, d.x];
+    });
 
     var root = this.treeData;
     root.x0 = height / 2;
@@ -299,6 +367,11 @@ export default Ember.Component.extend({
   },
 
   didInsertElement: function() {
-   this.reDraw();
+    this.reDraw();
   }
 });
+
+
+const getUsedCapacity = (partitionMap, filter=PARTITION_LABEL) => {
+  return partitionMap[filter].absoluteUsedCapacity;
+};

+ 48 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/yarn-queue-partition-capacity-labels.js

@@ -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.
+ */
+
+import Ember from "ember";
+import { PARTITION_LABEL } from "../constants";
+
+export default Ember.Component.extend({
+  didUpdateAttrs: function({ oldAttrs, newAttrs }) {
+    this._super(...arguments);
+    this.set("data", this.initData());
+  },
+
+  init() {
+    this._super(...arguments);
+    this.set("data", this.initData());
+  },
+
+  initData() {
+    const queue = this.get("queue");
+    const partitionMap = this.get("partitionMap");
+    const filteredParition = this.get("filteredPartition") || PARTITION_LABEL;
+    const userLimit = queue.get("userLimit");
+    const userLimitFactor = queue.get("userLimitFactor");
+    const isLeafQueue = queue.get("isLeafQueue");
+
+    return {
+      ...partitionMap[filteredParition],
+      userLimit,
+      userLimitFactor,
+      isLeafQueue
+    };
+  }
+});

+ 3 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/constants.js

@@ -34,4 +34,6 @@ export const Entities = {
   Memory:'memory',
   Resource: 'resource',
   Unit: 'unit'
-}
+}
+
+export const PARTITION_LABEL = 'Default partition';

+ 28 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-queues.js

@@ -17,19 +17,39 @@
  */
 
 import Ember from 'ember';
+import {PARTITION_LABEL} from '../constants';
 
 export default Ember.Controller.extend({
   needReload: true,
   selectedQueue: undefined,
   showLoading: true,
+  filteredPartition: PARTITION_LABEL,
 
-  breadcrumbs: [{
-    text: "Home",
-    routeName: 'application'
-  }, {
-    text: "Queues",
-    routeName: 'yarn-queues',
-    model: 'root'
-  }]
+  breadcrumbs: [
+    {
+      text: "Home",
+      routeName: "application"
+    },
+    {
+      text: "Queues",
+      routeName: "yarn-queues",
+      model: "root"
+    }
+  ],
 
+  actions: {
+    setFilter(partition) {
+      this.set("filteredPartition", partition);
+      const model = this.get('model');
+      const {selectedQueue} = model;
+      // If the selected queue does not have the filtered partition
+      // reset it to root
+      if (!selectedQueue.get('partitions').contains(partition)) {
+        const root = model.queues.get('firstObject');
+        document.location.href = "#/yarn-queues/" + root.get("id") + "!";
+        this.set("model.selectedQueue", root);
+        this.set("model.selected", root.get('id'));
+      }
+    }
+  }
 });

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/initializers/loader.js

@@ -24,7 +24,7 @@ function getTimeLineURL(rmhost) {
   var url = window.location.protocol + '//' +
     (ENV.hosts.localBaseAddress? ENV.hosts.localBaseAddress + '/' : '') + rmhost;
 
-  url += '/conf?name=yarn.timeline-service.webapp.address';
+  url += '/conf?name=yarn.timeline-service.reader.webapp.address';
   Ember.Logger.log("Get Timeline Address URL: " + url);
   return url;
 }

+ 33 - 23
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/capacity-queue.js

@@ -20,24 +20,26 @@ import DS from 'ember-data';
 import Converter from 'yarn-ui/utils/converter';
 
 export default DS.Model.extend({
-  name: DS.attr('string'),
-  children: DS.attr('array'),
-  parent: DS.attr('string'),
-  capacity: DS.attr('number'),
-  maxCapacity: DS.attr('number'),
-  usedCapacity: DS.attr('number'),
-  absCapacity: DS.attr('number'),
-  absMaxCapacity: DS.attr('number'),
-  absUsedCapacity: DS.attr('number'),
-  state: DS.attr('string'),
-  userLimit: DS.attr('number'),
-  userLimitFactor: DS.attr('number'),
-  preemptionDisabled: DS.attr('number'),
-  numPendingApplications: DS.attr('number'),
-  numActiveApplications: DS.attr('number'),
-  users: DS.hasMany('YarnUser'),
-  type: DS.attr('string'),
-  resources: DS.attr('object'),
+  name: DS.attr("string"),
+  children: DS.attr("array"),
+  parent: DS.attr("string"),
+  capacity: DS.attr("number"),
+  partitions: DS.attr("array"),
+  partitionMap: DS.attr("object"),
+  maxCapacity: DS.attr("number"),
+  usedCapacity: DS.attr("number"),
+  absCapacity: DS.attr("number"),
+  absMaxCapacity: DS.attr("number"),
+  absUsedCapacity: DS.attr("number"),
+  state: DS.attr("string"),
+  userLimit: DS.attr("number"),
+  userLimitFactor: DS.attr("number"),
+  preemptionDisabled: DS.attr("number"),
+  numPendingApplications: DS.attr("number"),
+  numActiveApplications: DS.attr("number"),
+  users: DS.hasMany("YarnUser"),
+  type: DS.attr("string"),
+  resources: DS.attr("object"),
 
   isLeafQueue: function() {
     var len = this.get("children.length");
@@ -53,21 +55,29 @@ export default DS.Model.extend({
       {
         label: "Absolute Used",
         style: "primary",
-        value: this.get("name") === "root" ? floatToFixed(this.get("usedCapacity")) : floatToFixed(this.get("absUsedCapacity"))
+        value:
+          this.get("name") === "root"
+            ? floatToFixed(this.get("usedCapacity"))
+            : floatToFixed(this.get("absUsedCapacity"))
       },
       {
         label: "Absolute Capacity",
         style: "primary",
-        value: this.get("name") === "root" ? 100 : floatToFixed(this.get("absCapacity"))
+        value:
+          this.get("name") === "root"
+            ? 100
+            : floatToFixed(this.get("absCapacity"))
       },
       {
         label: "Absolute Max Capacity",
         style: "secondary",
-        value: this.get("name") === "root" ? 100 : floatToFixed(this.get("absMaxCapacity"))
+        value:
+          this.get("name") === "root"
+            ? 100
+            : floatToFixed(this.get("absMaxCapacity"))
       }
     ];
   }.property("absCapacity", "usedCapacity", "absMaxCapacity"),
-
   userUsagesDonutChartData: function() {
     var data = [];
     if (this.get("users")) {
@@ -97,5 +107,5 @@ export default DS.Model.extend({
         value: this.get("numActiveApplications") || 0
       }
     ];
-  }.property("numPendingApplications", "numActiveApplications"),
+  }.property("numPendingApplications", "numActiveApplications")
 });

+ 6 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/capacity-queue.js

@@ -17,6 +17,7 @@
  */
 
 import DS from 'ember-data';
+import {PARTITION_LABEL} from '../../constants';
 
 export default DS.JSONAPISerializer.extend({
 
@@ -73,6 +74,11 @@ export default DS.JSONAPISerializer.extend({
           numPendingApplications: payload.numPendingApplications,
           numActiveApplications: payload.numActiveApplications,
           resources: payload.resources,
+          partitions: payload.capacities.queueCapacitiesByPartition.map(cap => cap.partitionName || PARTITION_LABEL),
+          partitionMap: payload.capacities.queueCapacitiesByPartition.reduce((init, cap) => {
+            init[cap.partitionName || PARTITION_LABEL] = cap;
+            return init;
+          }, {}),
           type: "capacity",
         },
         // Relationships

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.scss

@@ -3,6 +3,7 @@
 @import 'yarn-app.scss';
 @import './compose-box.scss';
 @import 'em-table.scss';
+@import './yarn-queues.scss';
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one

+ 29 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/yarn-queues.scss

@@ -0,0 +1,29 @@
+/**
+ * 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.
+ */
+
+.filter-partitions {
+  padding: 15px;
+  margin-left: auto;
+  label {
+    font-weight: 500;
+  }
+  .filter-queue-by-labels {
+    display: inline-block;
+    max-width: 350px;
+  }
+}

+ 18 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/queue-navigator.hbs

@@ -21,9 +21,25 @@
   <div class="col-md-12 container-fluid">
     <div class="panel panel-default" id="tree-selector-container">
       <div class="panel-heading">
-        Scheduler: {{model.firstObject.type}}
+        {{#if filteredPartition}}
+           {{model.firstObject.type}} scheduler - Showing queues from partition {{lower filteredPartition}}
+        {{else}}
+          {{model.firstObject.type}} scheduler - Showing queues from all partitions
+        {{/if}}
       </div>
-     {{tree-selector model=model parentId="tree-selector-container" selected=selected used=used max=max}}
+       {{#if (eq model.firstObject.type "capacity")}}
+       <div class="flex">
+        <div class="filter-partitions flex-right">
+          <label><i class="glyphicon glyphicon-filter"/> Partitions: </label>
+            <select onchange={{action "filterQueuesByPartition" value="target.value"}} class="form-control js-filter-queue-by-labels">
+              {{#each model.firstObject.partitions as |part|}}
+                <option value={{part}}>{{part}}</option>
+              {{/each}}
+            </select>
+        </div>
+       </div>
+      {{/if}}
+     {{tree-selector model=model parentId="tree-selector-container" selected=selected used=used max=max filteredPartition=filteredPartition}}
     </div>
   </div>
 </div>

+ 54 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue-partition-capacity-labels.hbs

@@ -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.
+}}
+
+<div class="top-1">
+  <span class="yarn-label primary">
+    <span class="label-key">absolute used</span>
+    <span class="label-value">{{data.absoluteUsedCapacity}}%</span>
+  </span>
+  <span class="yarn-label primary">
+    <span class="label-key">absolute capacity</span>
+    <span class="label-value">{{data.absoluteCapacity}}%</span>
+  </span>
+  <span class="yarn-label secondary">
+    <span class="label-key">absolute max capacity</span>
+    <span class="label-value">{{data.absoluteMaxCapacity}}%</span>
+  </span>
+</div>
+<div class="top-1">
+  <span class="yarn-label secondary">
+    <span class="label-key">configured capacity</span>
+    <span class="label-value">{{data.capacity}}%</span>
+  </span>
+  <span class="yarn-label secondary">
+    <span class="label-key">configured max capacity</span>
+    <span class="label-value">{{data.maxCapacity}}%</span>
+  </span>
+</div>
+{{#if data.isLeafQueue}}
+<div class="top-1">
+  <span class="yarn-label secondary">
+    <span class="label-key">user limit</span>
+    <span class="label-value">{{data.userLimit}}%</span>
+  </span>
+  <span class="yarn-label secondary">
+    <span class="label-key">user limit factor</span>
+    <span class="label-value">{{data.userLimitFactor}}</span>
+  </span>
+</div>
+{{/if}}

+ 3 - 31
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue.hbs

@@ -17,7 +17,7 @@
 }}
 
 {{queue-navigator model=model.queues selected=model.selected
-  used="usedCapacity" max="absMaxCapacity"}}
+  used="usedCapacity" max="absMaxCapacity" setFilter=(action setFilter)}}
 
 <div class="yarn-compose-box yarn-queues-container">
   <div>
@@ -31,36 +31,8 @@
         {{em-table-simple-status-cell content=model.selectedQueue.state}}
       </div>
     {{/if}}
-    <div class="top-1">
-      {{#each model.selectedQueue.capacitiesBarChartData as |item|}}
-        <span class="yarn-label {{item.style}}">
-          <span class="label-key"> {{lower item.label}}</span>
-          <span class="label-value">{{item.value}}%</span>
-        </span>
-      {{/each}}
-    </div>
-    <div class="top-1">
-      <span class="yarn-label secondary">
-        <span class="label-key">configured capacity</span>
-        <span class="label-value">{{model.selectedQueue.capacity}}%</span>
-      </span>
-      <span class="yarn-label secondary">
-        <span class="label-key">configured max capacity</span>
-        <span class="label-value">{{model.selectedQueue.maxCapacity}}%</span>
-      </span>
-    </div>
-    {{#if model.selectedQueue.isLeafQueue}}
-      <div class="top-1">
-        <span class="yarn-label secondary">
-          <span class="label-key">user limit</span>
-          <span class="label-value">{{model.selectedQueue.userLimit}}%</span>
-        </span>
-        <span class="yarn-label secondary">
-          <span class="label-key">user limit factor</span>
-          <span class="label-value">{{model.selectedQueue.userLimitFactor}}</span>
-        </span>
-      </div>
-    {{/if}}
+
+    {{yarn-queue-partition-capacity-labels partitionMap=model.selectedQueue.partitionMap queue=model.selectedQueue filteredPartition=filteredPartition}}
   </div>
 
   <h5> Running Apps </h5>

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queues.hbs

@@ -18,9 +18,10 @@
 <div class="queue-page-breadcrumb">
   {{breadcrumb-bar breadcrumbs=breadcrumbs}}
 </div>
+
 <div class="container-fluid">
   {{#if (eq model.queues.firstObject.type "capacity")}}
-    {{yarn-queue.capacity-queue model=model}}
+    {{yarn-queue.capacity-queue model=model setFilter=(action "setFilter") filteredPartition=filteredPartition}}
   {{else if (eq model.queues.firstObject.type "fair")}}
     {{yarn-queue.fair-queue model=model}}
   {{else}}

+ 43 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/integration/components/yarn-queue-partition-capacity-labels-test.js

@@ -0,0 +1,43 @@
+/**
+ * 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 { moduleForComponent, test } from 'ember-qunit';
+import hbs from 'htmlbars-inline-precompile';
+
+moduleForComponent('yarn-queue-partition-capacity-labels', 'Integration | Component | yarn queue partition capacity labels', {
+  integration: true
+});
+
+test('it renders', function(assert) {
+
+  // Set any properties with this.set('myProperty', 'value');
+  // Handle any actions with this.on('myAction', function(val) { ... });" + EOL + EOL +
+
+  this.render(hbs`{{yarn-queue-partition-capacity-labels}}`);
+
+  assert.equal(this.$().text().trim(), '');
+
+  // Template block usage:" + EOL +
+  this.render(hbs`
+    {{#yarn-queue-partition-capacity-labels}}
+      template block text
+    {{/yarn-queue-partition-capacity-labels}}
+  `);
+
+  assert.equal(this.$().text().trim(), 'template block text');
+});

+ 1 - 1
pom.xml

@@ -187,7 +187,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
             <dependency>
               <groupId>org.apache.maven.doxia</groupId>
               <artifactId>doxia-module-markdown</artifactId>
-              <version>1.8-SNAPSHOT</version>
+              <version>1.8</version>
             </dependency>
           </dependencies>
         </plugin>

برخی فایل ها در این مقایسه diff نمایش داده نمی شوند زیرا تعداد فایل ها بسیار زیاد است