Browse Source

Merge branch 'trunk' into HDFS-6581

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
arp 10 năm trước cách đây
mục cha
commit
b1000fbba4
99 tập tin đã thay đổi với 3766 bổ sung653 xóa
  1. 5 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 8 8
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java
  3. 33 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NativeCrc32.java
  4. 12 0
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/NativeCrc32.c
  5. 229 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestNativeCrc32.java
  6. 15 3
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  7. 12 5
      hadoop-hdfs-project/hadoop-hdfs/src/CMakeLists.txt
  8. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
  9. 6 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd
  10. 0 21
      hadoop-hdfs-project/hadoop-hdfs/src/main/conf/blockStoragePolicy-site.xml
  11. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/conf/hdfs-site.xml
  12. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  13. 0 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  14. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  15. 11 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
  16. 25 200
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockStoragePolicy.java
  17. 7 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  18. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
  19. 23 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  20. 20 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  21. 63 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  22. 8 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  23. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java
  24. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
  25. 119 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java
  26. 27 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
  27. 9 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  28. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  29. 17 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  30. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  31. 6 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  32. 0 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryAttributes.java
  33. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  34. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
  35. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  36. 12 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
  37. 65 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetStoragePolicies.java
  38. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
  39. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c
  40. 11 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/test/test_libhdfs_ops.c
  41. 31 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/test/vecsum.c
  42. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/test_libhdfs_threaded.c
  43. 9 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
  44. 14 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
  45. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  46. 66 20
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
  47. 21 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
  48. 9 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestStoragePolicyCommands.java
  49. 14 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
  50. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeleteRace.java
  51. 3 0
      hadoop-mapreduce-project/CHANGES.txt
  52. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryParsing.java
  53. 1 1
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/CopyListingFileStatus.java
  54. 12 9
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java
  55. 1 1
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java
  56. 31 27
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java
  57. 1 1
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestOptionsParser.java
  58. 1 0
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java
  59. 926 71
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/util/TestDistCpUtils.java
  60. 13 0
      hadoop-yarn-project/CHANGES.txt
  61. 194 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineDomain.java
  62. 86 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineDomains.java
  63. 17 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java
  64. 89 27
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
  65. 34 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java
  66. 38 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/timeline/TestTimelineRecords.java
  67. 73 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java
  68. 215 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java
  69. 99 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/MemoryTimelineStore.java
  70. 74 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
  71. 23 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineReader.java
  72. 16 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineWriter.java
  73. 26 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineACLsManager.java
  74. 100 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java
  75. 15 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLeveldbTimelineStore.java
  76. 13 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestMemoryTimelineStore.java
  77. 88 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineStoreTestUtils.java
  78. 59 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineACLsManager.java
  79. 219 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServices.java
  80. 2 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServicesWithSSL.java
  81. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
  82. 2 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/config.h.cmake
  83. 22 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
  84. 17 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java
  85. 13 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationInitEvent.java
  86. 13 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/loghandler/event/LogHandlerAppStartedEvent.java
  87. 6 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/configuration.c
  88. 59 24
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
  89. 8 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
  90. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/proto/yarn_server_nodemanager_recovery.proto
  91. 11 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
  92. 22 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
  93. 6 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
  94. 22 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java
  95. 38 39
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/JAXBContextResolver.java
  96. 18 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
  97. 58 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
  98. 6 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java
  99. 15 15
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRest.apt.vm

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

@@ -543,6 +543,8 @@ Release 2.6.0 - UNRELEASED
     HADOOP-11017. KMS delegation token secret manager should be able to use 
     zookeeper as store. (asuresh via tucu)
 
+    HADOOP-11009. Add Timestamp Preservation to DistCp (Gary Steelman via aw)
+
   OPTIMIZATIONS
 
     HADOOP-10838. Byte array native checksumming. (James Thomas via todd)
@@ -856,6 +858,9 @@ Release 2.6.0 - UNRELEASED
     wrong value if excluded nodes passed are not part of the cluster tree
     (vinayakumarb)
 
+    HADOOP-11064. UnsatisifedLinkError with hadoop 2.4 JARs on hadoop-2.6 due to
+    NativeCRC32 method changes. (cnauroth)
+
 Release 2.5.1 - 2014-09-05
 
   INCOMPATIBLE CHANGES

+ 8 - 8
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java

@@ -133,7 +133,7 @@ extends AbstractDelegationTokenIdentifier>
   
   /** 
    * Add a previously used master key to cache (when NN restarts), 
-   * should be called before activate().
+   * should be called before activate(). 
    * */
   public synchronized void addKey(DelegationKey key) throws IOException {
     if (running) // a safety check
@@ -141,7 +141,7 @@ extends AbstractDelegationTokenIdentifier>
     if (key.getKeyId() > currentId) {
       currentId = key.getKeyId();
     }
-    storeDelegationKey(key);
+    allKeys.put(key.getKeyId(), key);
   }
 
   public synchronized DelegationKey[] getAllKeys() {
@@ -268,6 +268,8 @@ extends AbstractDelegationTokenIdentifier>
   /**
    * This method is intended to be used for recovering persisted delegation
    * tokens
+   * This method must be called before this secret manager is activated (before
+   * startThreads() is called)
    * @param identifier identifier read from persistent storage
    * @param renewDate token renew time
    * @throws IOException
@@ -280,18 +282,17 @@ extends AbstractDelegationTokenIdentifier>
           "Can't add persisted delegation token to a running SecretManager.");
     }
     int keyId = identifier.getMasterKeyId();
-    DelegationKey dKey = getDelegationKey(keyId);
+    DelegationKey dKey = allKeys.get(keyId);
     if (dKey == null) {
       LOG.warn("No KEY found for persisted identifier " + identifier.toString());
       return;
     }
     byte[] password = createPassword(identifier.getBytes(), dKey.getKey());
-    int delegationTokenSeqNum = getDelegationTokenSeqNum();
-    if (identifier.getSequenceNumber() > delegationTokenSeqNum) {
-      setDelegationTokenSeqNum(identifier.getSequenceNumber());
+    if (identifier.getSequenceNumber() > delegationTokenSequenceNumber) {
+      delegationTokenSequenceNumber = identifier.getSequenceNumber();
     }
     if (getTokenInfo(identifier) == null) {
-      storeToken(identifier, new DelegationTokenInformation(renewDate,
+      currentTokens.put(identifier, new DelegationTokenInformation(renewDate,
           password, getTrackingIdIfEnabled(identifier)));
     } else {
       throw new IOException("Same delegation token being added twice.");
@@ -315,7 +316,6 @@ extends AbstractDelegationTokenIdentifier>
         + keyUpdateInterval + tokenMaxLifetime, generateSecret());
     //Log must be invoked outside the lock on 'this'
     logUpdateMasterKey(newKey);
-    storeNewMasterKey(newKey);
     synchronized (this) {
       currentId = newKey.getKeyId();
       currentKey = newKey;

+ 33 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NativeCrc32.java

@@ -21,6 +21,8 @@ import java.nio.ByteBuffer;
 
 import org.apache.hadoop.fs.ChecksumException;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * Wrapper around JNI support code to do checksum computation
  * natively.
@@ -86,6 +88,37 @@ class NativeCrc32 {
         data, dataOffset, dataLength,
         "", 0, false);
   }
+
+  /**
+   * Verify the given buffers of data and checksums, and throw an exception
+   * if any checksum is invalid. The buffers given to this function should
+   * have their position initially at the start of the data, and their limit
+   * set at the end of the data. The position, limit, and mark are not
+   * modified.  This method is retained only for backwards-compatibility with
+   * prior jar versions that need the corresponding JNI function.
+   *
+   * @param bytesPerSum the chunk size (eg 512 bytes)
+   * @param checksumType the DataChecksum type constant
+   * @param sums the DirectByteBuffer pointing at the beginning of the
+   *             stored checksums
+   * @param sumsOffset start offset in sums buffer
+   * @param data the DirectByteBuffer pointing at the beginning of the
+   *             data to check
+   * @param dataOffset start offset in data buffer
+   * @param dataLength length of data buffer
+   * @param fileName the name of the file being verified
+   * @param basePos the position in the file where the data buffer starts 
+   * @throws ChecksumException if there is an invalid checksum
+   * @deprecated use {@link #nativeComputeChunkedSums(int, int, ByteBuffer, int,
+   *   ByteBuffer, int, int, String, long, boolean)} instead
+   */
+  @Deprecated
+  @VisibleForTesting
+  static native void nativeVerifyChunkedSums(
+      int bytesPerSum, int checksumType,
+      ByteBuffer sums, int sumsOffset,
+      ByteBuffer data, int dataOffset, int dataLength,
+      String fileName, long basePos) throws ChecksumException;
   
     private static native void nativeComputeChunkedSums(
       int bytesPerSum, int checksumType,

+ 12 - 0
hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/NativeCrc32.c

@@ -181,6 +181,18 @@ JNIEXPORT void JNICALL Java_org_apache_hadoop_util_NativeCrc32_nativeComputeChun
   }
 }
 
+JNIEXPORT void JNICALL Java_org_apache_hadoop_util_NativeCrc32_nativeVerifyChunkedSums
+  (JNIEnv *env, jclass clazz,
+    jint bytes_per_checksum, jint j_crc_type,
+    jobject j_sums, jint sums_offset,
+    jobject j_data, jint data_offset, jint data_len,
+    jstring j_filename, jlong base_pos)
+{
+  Java_org_apache_hadoop_util_NativeCrc32_nativeComputeChunkedSums(env, clazz,
+    bytes_per_checksum, j_crc_type, j_sums, sums_offset, j_data, data_offset,
+    data_len, j_filename, base_pos, JNI_TRUE);
+}
+
 JNIEXPORT void JNICALL Java_org_apache_hadoop_util_NativeCrc32_nativeComputeChunkedSumsByteArray
   (JNIEnv *env, jclass clazz,
     jint bytes_per_checksum, jint j_crc_type,

+ 229 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestNativeCrc32.java

@@ -0,0 +1,229 @@
+/**
+ * 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.util;
+
+import static org.junit.Assert.*;
+import static org.junit.Assume.*;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ChecksumException;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class TestNativeCrc32 {
+
+  private static final long BASE_POSITION = 0;
+  private static final int IO_BYTES_PER_CHECKSUM_DEFAULT = 512;
+  private static final String IO_BYTES_PER_CHECKSUM_KEY =
+    "io.bytes.per.checksum";
+  private static final int NUM_CHUNKS = 3;
+
+  private final DataChecksum.Type checksumType;
+
+  private int bytesPerChecksum;
+  private String fileName;
+  private ByteBuffer data, checksums;
+  private DataChecksum checksum;
+
+  @Rule
+  public ExpectedException exception = ExpectedException.none();
+
+  @Parameters
+  public static Collection<Object[]> data() {
+    Collection<Object[]> params = new ArrayList<Object[]>(2);
+    params.add(new Object[] { DataChecksum.Type.CRC32 });
+    params.add(new Object[] { DataChecksum.Type.CRC32C });
+    return params;
+  }
+
+  public TestNativeCrc32(DataChecksum.Type checksumType) {
+    this.checksumType = checksumType;
+  }
+
+  @Before
+  public void setup() {
+    assumeTrue(NativeCrc32.isAvailable());
+    assertEquals(
+      "These tests assume they can write a checksum value as a 4-byte int.", 4,
+      checksumType.size);
+    Configuration conf = new Configuration();
+    bytesPerChecksum = conf.getInt(IO_BYTES_PER_CHECKSUM_KEY,
+      IO_BYTES_PER_CHECKSUM_DEFAULT);
+    fileName = this.getClass().getSimpleName();
+    checksum = DataChecksum.newDataChecksum(checksumType, bytesPerChecksum);
+  }
+
+  @Test
+  public void testVerifyChunkedSumsSuccess() throws ChecksumException {
+    allocateDirectByteBuffers();
+    fillDataAndValidChecksums();
+    NativeCrc32.verifyChunkedSums(bytesPerChecksum, checksumType.id,
+      checksums, data, fileName, BASE_POSITION);
+  }
+
+  @Test
+  public void testVerifyChunkedSumsFail() throws ChecksumException {
+    allocateDirectByteBuffers();
+    fillDataAndInvalidChecksums();
+    exception.expect(ChecksumException.class);
+    NativeCrc32.verifyChunkedSums(bytesPerChecksum, checksumType.id,
+      checksums, data, fileName, BASE_POSITION);
+  }
+
+  @Test
+  public void testVerifyChunkedSumsByteArraySuccess() throws ChecksumException {
+    allocateArrayByteBuffers();
+    fillDataAndValidChecksums();
+    NativeCrc32.verifyChunkedSumsByteArray(bytesPerChecksum, checksumType.id,
+      checksums.array(), checksums.position(), data.array(), data.position(),
+      data.remaining(), fileName, BASE_POSITION);
+  }
+
+  @Test
+  public void testVerifyChunkedSumsByteArrayFail() throws ChecksumException {
+    allocateArrayByteBuffers();
+    fillDataAndInvalidChecksums();
+    exception.expect(ChecksumException.class);
+    NativeCrc32.verifyChunkedSumsByteArray(bytesPerChecksum, checksumType.id,
+      checksums.array(), checksums.position(), data.array(), data.position(),
+      data.remaining(), fileName, BASE_POSITION);
+  }
+
+  @Test
+  public void testCalculateChunkedSumsSuccess() throws ChecksumException {
+    allocateDirectByteBuffers();
+    fillDataAndValidChecksums();
+    NativeCrc32.calculateChunkedSums(bytesPerChecksum, checksumType.id,
+      checksums, data);
+  }
+
+  @Test
+  public void testCalculateChunkedSumsFail() throws ChecksumException {
+    allocateDirectByteBuffers();
+    fillDataAndInvalidChecksums();
+    NativeCrc32.calculateChunkedSums(bytesPerChecksum, checksumType.id,
+      checksums, data);
+  }
+
+  @Test
+  public void testCalculateChunkedSumsByteArraySuccess() throws ChecksumException {
+    allocateArrayByteBuffers();
+    fillDataAndValidChecksums();
+    NativeCrc32.calculateChunkedSumsByteArray(bytesPerChecksum, checksumType.id,
+      checksums.array(), checksums.position(), data.array(), data.position(),
+      data.remaining());
+  }
+
+  @Test
+  public void testCalculateChunkedSumsByteArrayFail() throws ChecksumException {
+    allocateArrayByteBuffers();
+    fillDataAndInvalidChecksums();
+    NativeCrc32.calculateChunkedSumsByteArray(bytesPerChecksum, checksumType.id,
+      checksums.array(), checksums.position(), data.array(), data.position(),
+      data.remaining());
+  }
+
+  @Test
+  @SuppressWarnings("deprecation")
+  public void testNativeVerifyChunkedSumsSuccess() throws ChecksumException {
+    allocateDirectByteBuffers();
+    fillDataAndValidChecksums();
+    NativeCrc32.nativeVerifyChunkedSums(bytesPerChecksum, checksumType.id,
+      checksums, checksums.position(), data, data.position(), data.remaining(),
+      fileName, BASE_POSITION);
+  }
+
+  @Test
+  @SuppressWarnings("deprecation")
+  public void testNativeVerifyChunkedSumsFail() throws ChecksumException {
+    allocateDirectByteBuffers();
+    fillDataAndInvalidChecksums();
+    exception.expect(ChecksumException.class);
+    NativeCrc32.nativeVerifyChunkedSums(bytesPerChecksum, checksumType.id,
+      checksums, checksums.position(), data, data.position(), data.remaining(),
+      fileName, BASE_POSITION);
+  }
+
+  /**
+   * Allocates data buffer and checksums buffer as arrays on the heap.
+   */
+  private void allocateArrayByteBuffers() {
+    data = ByteBuffer.wrap(new byte[bytesPerChecksum * NUM_CHUNKS]);
+    checksums = ByteBuffer.wrap(new byte[NUM_CHUNKS * checksumType.size]);
+  }
+
+  /**
+   * Allocates data buffer and checksums buffer as direct byte buffers.
+   */
+  private void allocateDirectByteBuffers() {
+    data = ByteBuffer.allocateDirect(bytesPerChecksum * NUM_CHUNKS);
+    checksums = ByteBuffer.allocateDirect(NUM_CHUNKS * checksumType.size);
+  }
+
+  /**
+   * Fill data buffer with monotonically increasing byte values.  Overflow is
+   * fine, because it's just test data.  Update the checksum with the same byte
+   * values.  After every chunk, write the checksum to the checksums buffer.
+   * After finished writing, flip the buffers to prepare them for reading.
+   */
+  private void fillDataAndValidChecksums() {
+    for (int i = 0; i < NUM_CHUNKS; ++i) {
+      for (int j = 0; j < bytesPerChecksum; ++j) {
+        byte b = (byte)((i * bytesPerChecksum + j) & 0xFF);
+        data.put(b);
+        checksum.update(b);
+      }
+      checksums.putInt((int)checksum.getValue());
+      checksum.reset();
+    }
+    data.flip();
+    checksums.flip();
+  }
+
+  /**
+   * Fill data buffer with monotonically increasing byte values.  Overflow is
+   * fine, because it's just test data.  Update the checksum with different byte
+   * byte values, so that the checksums are incorrect intentionally.  After every
+   * chunk, write the checksum to the checksums buffer.  After finished writing,
+   * flip the buffers to prepare them for reading.
+   */
+  private void fillDataAndInvalidChecksums() {
+    for (int i = 0; i < NUM_CHUNKS; ++i) {
+      for (int j = 0; j < bytesPerChecksum; ++j) {
+        byte b = (byte)((i * bytesPerChecksum + j) & 0xFF);
+        data.put(b);
+        checksum.update((byte)(b + 1));
+      }
+      checksums.putInt((int)checksum.getValue());
+      checksum.reset();
+    }
+    data.flip();
+    checksums.flip();
+  }
+}

+ 15 - 3
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -133,9 +133,6 @@ Trunk (Unreleased)
     HDFS-6609. Use DirectorySnapshottableFeature to represent a snapshottable
     directory. (Jing Zhao via wheat9)
 
-    HDFS-6987. Move CipherSuite xattr information up to the encryption zone
-    root. (Zhe Zhang via wang)
-
   OPTIMIZATIONS
 
   BUG FIXES
@@ -345,6 +342,12 @@ Trunk (Unreleased)
 
     HDFS-7095. TestStorageMover often fails in Jenkins. (jing9)
 
+    HDFS-7081. Add new DistributedFileSystem API for getting all the existing
+    storage policies. (jing9)
+
+    HDFS-7140. Add a tool to list all the existing block storage policies.
+    (jing9)
+
 Release 2.6.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -572,6 +575,13 @@ Release 2.6.0 - UNRELEASED
     HDFS-6948. DN rejects blocks if it has older UC block
     (Eric Payne via kihwal)
 
+    HDFS-6987. Move CipherSuite xattr information up to the encryption zone
+    root. (Zhe Zhang via wang)
+
+    HDFS-7139. Unit test for creating encryption zone on root path. (Zhe Zhang via wang)
+
+    HDFS-7138. Fix hftp to work with encryption. (clamb via wang)
+
   OPTIMIZATIONS
 
     HDFS-6690. Deduplicate xattr names in memory. (wang)
@@ -935,6 +945,8 @@ Release 2.6.0 - UNRELEASED
     HDFS-7130. TestDataTransferKeepalive fails intermittently on Windows.
     (cnauroth)
 
+    HDFS-6534. Fix build on macosx: HDFS parts (Binglin Chang via aw)
+
 Release 2.5.1 - 2014-09-05
 
   INCOMPATIBLE CHANGES

+ 12 - 5
hadoop-hdfs-project/hadoop-hdfs/src/CMakeLists.txt

@@ -211,11 +211,18 @@ if (NOT WIN32)
     add_executable(test_libhdfs_vecsum
         main/native/libhdfs/test/vecsum.c
     )
-    target_link_libraries(test_libhdfs_vecsum
-        hdfs
-        pthread
-        rt
-    )
+    if (${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
+        target_link_libraries(test_libhdfs_vecsum
+            hdfs
+            pthread
+        )
+    else (${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
+        target_link_libraries(test_libhdfs_vecsum
+            hdfs
+            pthread
+            rt
+        )
+    endif (${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
 endif(NOT WIN32)
 
 IF(REQUIRE_LIBWEBHDFS)

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs

@@ -48,6 +48,7 @@ function hadoop_usage
   echo "  secondarynamenode    run the DFS secondary namenode"
   echo "  snapshotDiff         diff two snapshots of a directory or diff the"
   echo "                       current directory contents with a snapshot"
+  echo "  storagepolicies      get all the existing block storage policies"
   echo "  zkfc                 run the ZK Failover Controller daemon"
   echo ""
   echo "Most commands print help when invoked w/o parameters."
@@ -216,6 +217,9 @@ case ${COMMAND} in
   snapshotDiff)
     CLASS=org.apache.hadoop.hdfs.tools.snapshot.SnapshotDiff
   ;;
+  storagepolicies)
+    CLASS=org.apache.hadoop.hdfs.tools.GetStoragePolicies
+  ;;
   zkfc)
     daemon="true"
     CLASS='org.apache.hadoop.hdfs.tools.DFSZKFailoverController'

+ 6 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd

@@ -47,7 +47,7 @@ if "%1" == "--config" (
       goto print_usage
   )
 
-  set hdfscommands=dfs namenode secondarynamenode journalnode zkfc datanode dfsadmin haadmin fsck balancer jmxget oiv oev fetchdt getconf groups snapshotDiff lsSnapshottableDir cacheadmin mover
+  set hdfscommands=dfs namenode secondarynamenode journalnode zkfc datanode dfsadmin haadmin fsck balancer jmxget oiv oev fetchdt getconf groups snapshotDiff lsSnapshottableDir cacheadmin mover storagepolicies
   for %%i in ( %hdfscommands% ) do (
     if %hdfs-command% == %%i set hdfscommand=true
   )
@@ -155,6 +155,10 @@ goto :eof
   set HADOOP_OPTS=%HADOOP_OPTS% %HADOOP_MOVER_OPTS%
   goto :eof
 
+:storagepolicies
+  set CLASS=org.apache.hadoop.hdfs.tools.GetStoragePolicies
+  goto :eof
+
 @rem This changes %1, %2 etc. Hence those cannot be used after calling this.
 :make_command_arguments
   if "%1" == "--config" (
@@ -204,6 +208,7 @@ goto :eof
   @echo 						Use -help to see options
   @echo   cacheadmin           configure the HDFS cache
   @echo   mover                run a utility to move block replicas across storage types
+  @echo   storagepolicies      get all the existing block storage policies
   @echo.
   @echo Most commands print help when invoked w/o parameters.
 

+ 0 - 21
hadoop-hdfs-project/hadoop-hdfs/src/main/conf/blockStoragePolicy-site.xml

@@ -1,21 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-<!--
-  Licensed under the Apache License, Version 2.0 (the "License");
-  you may not use this file except in compliance with the License.
-  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License. See accompanying LICENSE file.
--->
-
-<!-- Put site-specific property overrides in this file. -->
-
-<configuration> 
-
-</configuration>

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/conf/hdfs-site.xml

@@ -16,7 +16,6 @@
 
 <!-- Put site-specific property overrides in this file. -->
 
-<configuration xmlns:xi="http://www.w3.org/2001/XInclude"> 
-  <xi:include href="blockStoragePolicy-site.xml" />
+<configuration>
 
 </configuration>

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -139,6 +139,7 @@ import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.net.TcpPeerServer;
 import org.apache.hadoop.hdfs.protocol.AclException;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveIterator;
@@ -1779,6 +1780,13 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
+  /**
+   * @return All the existing storage policies
+   */
+  public BlockStoragePolicy[] getStoragePolicies() throws IOException {
+    return namenode.getStoragePolicies();
+  }
+
   /**
    * Rename file or directory.
    * @see ClientProtocol#rename(String, String)

+ 0 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -455,14 +455,6 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final Class<BlockPlacementPolicyDefault> DFS_BLOCK_REPLICATOR_CLASSNAME_DEFAULT = BlockPlacementPolicyDefault.class;
   public static final String  DFS_REPLICATION_MAX_KEY = "dfs.replication.max";
   public static final int     DFS_REPLICATION_MAX_DEFAULT = 512;
-  public static final String DFS_BLOCK_STORAGE_POLICIES_KEY
-        = BlockStoragePolicy.DFS_BLOCK_STORAGE_POLICIES_KEY;
-  public static final String DFS_BLOCK_STORAGE_POLICY_KEY_PREFIX
-        = BlockStoragePolicy.DFS_BLOCK_STORAGE_POLICY_KEY_PREFIX;
-  public static final String DFS_BLOCK_STORAGE_POLICY_CREATION_FALLBACK_KEY_PREFIX
-        = BlockStoragePolicy.DFS_BLOCK_STORAGE_POLICY_CREATION_FALLBACK_KEY_PREFIX;
-  public static final String DFS_BLOCK_STORAGE_POLICY_REPLICATION_FALLBACK_KEY_PREFIX
-        = BlockStoragePolicy.DFS_BLOCK_STORAGE_POLICY_REPLICATION_FALLBACK_KEY_PREFIX;
 
   public static final String  DFS_DF_INTERVAL_KEY = "dfs.df.interval";
   public static final int     DFS_DF_INTERVAL_DEFAULT = 60000;

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -62,6 +62,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hdfs.client.HdfsAdmin;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
@@ -504,6 +505,12 @@ public class DistributedFileSystem extends FileSystem {
     }.resolve(this, absF);
   }
 
+  /** Get all the existing storage policies */
+  public BlockStoragePolicy[] getStoragePolicies() throws IOException {
+    statistics.incrementReadOps(1);
+    return dfs.getStoragePolicies();
+  }
+
   /**
    * Move blocks from srcs to trg and delete srcs afterwards.
    * The file block sizes must be the same.

+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java

@@ -325,4 +325,15 @@ public class HdfsAdmin {
       throws IOException {
     return dfs.getInotifyEventStream(lastReadTxid);
   }
+
+  /**
+   * Set the source path to the specified storage policy.
+   *
+   * @param src The source path referring to either a directory or a file.
+   * @param policyName The name of the storage policy.
+   */
+  public void setStoragePolicy(final Path src, final String policyName)
+      throws IOException {
+    dfs.setStoragePolicy(src, policyName);
+  }
 }

+ 25 - 200
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockStoragePolicy.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockStoragePolicy.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hdfs;
+package org.apache.hadoop.hdfs.protocol;
 
 import java.util.Arrays;
 import java.util.EnumSet;
@@ -24,12 +24,10 @@ import java.util.LinkedList;
 import java.util.List;
 
 import com.google.common.annotations.VisibleForTesting;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.XAttr;
-import org.apache.hadoop.fs.XAttr.NameSpace;
+import org.apache.hadoop.hdfs.StorageType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * A block storage policy describes how to select the storage types
@@ -37,67 +35,8 @@ import org.apache.hadoop.fs.XAttr.NameSpace;
  */
 @InterfaceAudience.Private
 public class BlockStoragePolicy {
-  public static final Log LOG = LogFactory.getLog(BlockStoragePolicy.class);
-
-  public static final String DFS_BLOCK_STORAGE_POLICIES_KEY
-      = "dfs.block.storage.policies";
-  public static final String DFS_BLOCK_STORAGE_POLICY_KEY_PREFIX
-      = "dfs.block.storage.policy.";
-  public static final String DFS_BLOCK_STORAGE_POLICY_CREATION_FALLBACK_KEY_PREFIX
-      = "dfs.block.storage.policy.creation-fallback.";
-  public static final String DFS_BLOCK_STORAGE_POLICY_REPLICATION_FALLBACK_KEY_PREFIX
-      = "dfs.block.storage.policy.replication-fallback.";
-  public static final String STORAGE_POLICY_XATTR_NAME = "bsp";
-  /** set the namespace to TRUSTED so that only privilege users can access */
-  public static final NameSpace XAttrNS = NameSpace.TRUSTED;
-
-  public static final int ID_BIT_LENGTH = 4;
-  public static final int ID_MAX = (1 << ID_BIT_LENGTH) - 1;
-  public static final byte ID_UNSPECIFIED = 0;
-
-  private static final Suite DEFAULT_SUITE = createDefaultSuite();
-
-  private static Suite createDefaultSuite() {
-    final BlockStoragePolicy[] policies = new BlockStoragePolicy[1 << ID_BIT_LENGTH];
-    final StorageType[] storageTypes = {StorageType.DISK};
-    final byte defaultPolicyId = 12;
-    policies[defaultPolicyId] = new BlockStoragePolicy(defaultPolicyId, "HOT",
-        storageTypes, StorageType.EMPTY_ARRAY, StorageType.EMPTY_ARRAY);
-    return new Suite(defaultPolicyId, policies);
-  }
-
-  /** A block storage policy suite. */
-  public static class Suite {
-    private final byte defaultPolicyID;
-    private final BlockStoragePolicy[] policies;
-    
-    private Suite(byte defaultPolicyID, BlockStoragePolicy[] policies) {
-      this.defaultPolicyID = defaultPolicyID;
-      this.policies = policies;
-    }
-    
-    /** @return the corresponding policy. */
-    public BlockStoragePolicy getPolicy(byte id) {
-      // id == 0 means policy not specified. 
-      return id == 0? getDefaultPolicy(): policies[id];
-    }
-
-    /** @return the default policy. */
-    public BlockStoragePolicy getDefaultPolicy() {
-      return getPolicy(defaultPolicyID);
-    }
-
-    public BlockStoragePolicy getPolicy(String policyName) {
-      if (policies != null) {
-        for (BlockStoragePolicy policy : policies) {
-          if (policy != null && policy.name.equals(policyName)) {
-            return policy;
-          }
-        }
-      }
-      return null;
-    }
-  }
+  public static final Logger LOG = LoggerFactory.getLogger(BlockStoragePolicy
+      .class);
 
   /** A 4-bit policy ID */
   private final byte id;
@@ -160,7 +99,7 @@ public class BlockStoragePolicy {
   /**
    * Choose the storage types for storing the remaining replicas, given the
    * replication number, the storage types of the chosen replicas and
-   * the unavailable storage types.  It uses fallback storage in case that
+   * the unavailable storage types. It uses fallback storage in case that
    * the desired storage type is unavailable.  
    *
    * @param replication the replication number.
@@ -195,7 +134,7 @@ public class BlockStoragePolicy {
     // remove excess storage types after fallback replacement.
     diff(storageTypes, excess, null);
     if (storageTypes.size() < expectedSize) {
-      LOG.warn("Failed to place enough replicas: expected size is " + expectedSize 
+      LOG.warn("Failed to place enough replicas: expected size is " + expectedSize
           + " but only " + storageTypes.size() + " storage types can be selected "
           + "(replication=" + replication
           + ", selected=" + storageTypes
@@ -207,7 +146,8 @@ public class BlockStoragePolicy {
   }
 
   /**
-   * Compute the list difference t = t - c.
+   * Compute the difference between two lists t and c so that after the diff
+   * computation we have: t = t - c;
    * Further, if e is not null, set e = e + c - t;
    */
   private static void diff(List<StorageType> t, Iterable<StorageType> c,
@@ -242,7 +182,7 @@ public class BlockStoragePolicy {
   public StorageType getCreationFallback(EnumSet<StorageType> unavailables) {
     return getFallback(unavailables, creationFallbacks);
   }
-  
+
   /** @return the fallback {@link StorageType} for replication. */
   public StorageType getReplicationFallback(EnumSet<StorageType> unavailables) {
     return getFallback(unavailables, replicationFallbacks);
@@ -269,7 +209,7 @@ public class BlockStoragePolicy {
     return getClass().getSimpleName() + "{" + name + ":" + id
         + ", storageTypes=" + Arrays.asList(storageTypes)
         + ", creationFallbacks=" + Arrays.asList(creationFallbacks)
-        + ", replicationFallbacks=" + Arrays.asList(replicationFallbacks);
+        + ", replicationFallbacks=" + Arrays.asList(replicationFallbacks) + "}";
   }
 
   public byte getId() {
@@ -280,140 +220,25 @@ public class BlockStoragePolicy {
     return name;
   }
 
-  private static StorageType getFallback(EnumSet<StorageType> unavailables,
-      StorageType[] fallbacks) {
-    for(StorageType fb : fallbacks) {
-      if (!unavailables.contains(fb)) {
-        return fb;
-      }
-    }
-    return null;
-  }
-  
-  private static byte parseID(String idString, String element, Configuration conf) {
-    byte id = 0;
-    try {
-      id = Byte.parseByte(idString);
-    } catch(NumberFormatException nfe) {
-      throwIllegalArgumentException("Failed to parse policy ID \"" + idString
-          + "\" to a " + ID_BIT_LENGTH + "-bit integer", conf);
-    }
-    if (id < 0) {
-      throwIllegalArgumentException("Invalid policy ID: id = " + id
-          + " < 1 in \"" + element + "\"", conf);
-    } else if (id == 0) {
-      throw new IllegalArgumentException("Policy ID 0 is reserved: " + element);
-    } else if (id > ID_MAX) {
-      throwIllegalArgumentException("Invalid policy ID: id = " + id
-          + " > MAX = " + ID_MAX + " in \"" + element + "\"", conf);
-    }
-    return id;
+  public StorageType[] getStorageTypes() {
+    return this.storageTypes;
   }
 
-  private static StorageType[] parseStorageTypes(String[] strings) {
-    if (strings == null || strings.length == 0) {
-      return StorageType.EMPTY_ARRAY;
-    }
-    final StorageType[] types = new StorageType[strings.length];
-    for(int i = 0; i < types.length; i++) {
-      types[i] = StorageType.valueOf(strings[i].trim().toUpperCase());
-    }
-    return types;
-  }
-  
-  private static StorageType[] readStorageTypes(byte id, String keyPrefix,
-      Configuration conf) {
-    final String key = keyPrefix + id;
-    final String[] values = conf.getStrings(key);
-    try {
-      return parseStorageTypes(values);
-    } catch(Exception e) {
-      throw new IllegalArgumentException("Failed to parse " + key
-          + " \"" + conf.get(key), e);
-    }
+  public StorageType[] getCreationFallbacks() {
+    return this.creationFallbacks;
   }
 
-  private static BlockStoragePolicy readBlockStoragePolicy(byte id, String name,
-      Configuration conf) {
-    final StorageType[] storageTypes = readStorageTypes(id, 
-        DFS_BLOCK_STORAGE_POLICY_KEY_PREFIX, conf);
-    if (storageTypes.length == 0) {
-      throw new IllegalArgumentException(
-          DFS_BLOCK_STORAGE_POLICY_KEY_PREFIX + id + " is missing or is empty.");
-    }
-    final StorageType[] creationFallbacks = readStorageTypes(id, 
-        DFS_BLOCK_STORAGE_POLICY_CREATION_FALLBACK_KEY_PREFIX, conf);
-    final StorageType[] replicationFallbacks = readStorageTypes(id, 
-        DFS_BLOCK_STORAGE_POLICY_REPLICATION_FALLBACK_KEY_PREFIX, conf);
-    return new BlockStoragePolicy(id, name, storageTypes, creationFallbacks,
-        replicationFallbacks);
+  public StorageType[] getReplicationFallbacks() {
+    return this.replicationFallbacks;
   }
 
-  /** Read {@link Suite} from conf. */
-  public static Suite readBlockStorageSuite(Configuration conf) {
-    final BlockStoragePolicy[] policies = new BlockStoragePolicy[1 << ID_BIT_LENGTH];
-    final String[] values = conf.getStrings(DFS_BLOCK_STORAGE_POLICIES_KEY);
-    if (values == null) {
-      // conf property is missing, use default suite.
-      return DEFAULT_SUITE;
-    }
-    byte firstID = -1;
-    for(String v : values) {
-      v = v.trim();
-      final int i = v.indexOf(':');
-      if (i < 0) {
-        throwIllegalArgumentException("Failed to parse element \"" + v
-            + "\" (expected format is NAME:ID)", conf);
-      } else if (i == 0) {
-        throwIllegalArgumentException("Policy name is missing in \"" + v + "\"", conf);
-      } else if (i == v.length() - 1) {
-        throwIllegalArgumentException("Policy ID is missing in \"" + v + "\"", conf);
-      }
-      final String name = v.substring(0, i).trim();
-      for(int j = 1; j < policies.length; j++) {
-        if (policies[j] != null && policies[j].name.equals(name)) {
-          throwIllegalArgumentException("Policy name duplication: \""
-              + name + "\" appears more than once", conf);
-        }
-      }
-      
-      final byte id = parseID(v.substring(i + 1).trim(), v, conf);
-      if (policies[id] != null) {
-        throwIllegalArgumentException("Policy duplication: ID " + id
-            + " appears more than once", conf);
-      }
-      policies[id] = readBlockStoragePolicy(id, name, conf);
-      String prefix = "";
-      if (firstID == -1) {
-        firstID = id;
-        prefix = "(default) ";
+  private static StorageType getFallback(EnumSet<StorageType> unavailables,
+      StorageType[] fallbacks) {
+    for(StorageType fb : fallbacks) {
+      if (!unavailables.contains(fb)) {
+        return fb;
       }
-      LOG.info(prefix + policies[id]);
-    }
-    if (firstID == -1) {
-      throwIllegalArgumentException("Empty list is not allowed", conf);
     }
-    return new Suite(firstID, policies);
-  }
-
-  public static String buildXAttrName() {
-    return XAttrNS.toString().toLowerCase() + "." + STORAGE_POLICY_XATTR_NAME;
-  }
-
-  public static XAttr buildXAttr(byte policyId) {
-    final String name = buildXAttrName();
-    return XAttrHelper.buildXAttr(name, new byte[] { policyId });
-  }
-
-  public static boolean isStoragePolicyXAttr(XAttr xattr) {
-    return xattr != null && xattr.getNameSpace() == BlockStoragePolicy.XAttrNS
-        && xattr.getName().equals(BlockStoragePolicy.STORAGE_POLICY_XATTR_NAME);
-  }
-
-  private static void throwIllegalArgumentException(String message,
-      Configuration conf) {
-    throw new IllegalArgumentException(message + " in "
-        + DFS_BLOCK_STORAGE_POLICIES_KEY + " \""
-        + conf.get(DFS_BLOCK_STORAGE_POLICIES_KEY) + "\".");
+    return null;
   }
-}
+}

+ 7 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -43,13 +43,11 @@ import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.inotify.Event;
 import org.apache.hadoop.hdfs.inotify.EventsList;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
 import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
@@ -259,6 +257,13 @@ public interface ClientProtocol {
       FileNotFoundException, SafeModeException, UnresolvedLinkException,
       SnapshotAccessControlException, IOException;
 
+  /**
+   * Get all the available block storage policies.
+   * @return All the in-use block storage policies currently.
+   */
+  @Idempotent
+  public BlockStoragePolicy[] getStoragePolicies() throws IOException;
+
   /**
    * Set the storage policy for a file/directory
    * @param src Path of an existing file/directory. 

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java

@@ -24,8 +24,8 @@ import java.util.Date;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 
 /**
  * Metadata about a snapshottable directory
@@ -62,7 +62,7 @@ public class SnapshottableDirectoryStatus {
       int snapshotNumber, int snapshotQuota, byte[] parentFullPath) {
     this.dirStatus = new HdfsFileStatus(0, true, 0, 0, false, modification_time,
         access_time, permission, owner, group, null, localName, inodeId,
-        childrenNum, null, BlockStoragePolicy.ID_UNSPECIFIED);
+        childrenNum, null, BlockStoragePolicySuite.ID_UNSPECIFIED);
     this.snapshotNumber = snapshotNumber;
     this.snapshotQuota = snapshotQuota;
     this.parentFullPath = parentFullPath;

+ 23 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Options.Rename;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
@@ -119,6 +120,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSna
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesRequestProto;
@@ -1429,6 +1432,26 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
     return VOID_SET_STORAGE_POLICY_RESPONSE;
   }
 
+  @Override
+  public GetStoragePoliciesResponseProto getStoragePolicies(
+      RpcController controller, GetStoragePoliciesRequestProto request)
+      throws ServiceException {
+    try {
+      BlockStoragePolicy[] policies = server.getStoragePolicies();
+      GetStoragePoliciesResponseProto.Builder builder =
+          GetStoragePoliciesResponseProto.newBuilder();
+      if (policies == null) {
+        return builder.build();
+      }
+      for (BlockStoragePolicy policy : policies) {
+        builder.addPolicies(PBHelper.convert(policy));
+      }
+      return builder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
   public GetCurrentEditLogTxidResponseProto getCurrentEditLogTxid(RpcController controller,
       GetCurrentEditLogTxidRequestProto req) throws ServiceException {
     try {

+ 20 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -45,6 +45,7 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.inotify.EventsList;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
@@ -64,9 +65,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
-import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusRequestProto;
@@ -119,6 +118,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSna
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesResponseProto;
@@ -159,13 +160,13 @@ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.SetXAttrRequestProto;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
 import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
@@ -225,6 +226,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
   VOID_GET_DATA_ENCRYPTIONKEY_REQUEST =
       GetDataEncryptionKeyRequestProto.newBuilder().build();
 
+  private final static GetStoragePoliciesRequestProto
+  VOID_GET_STORAGE_POLICIES_REQUEST =
+      GetStoragePoliciesRequestProto.newBuilder().build();
+
   public ClientNamenodeProtocolTranslatorPB(ClientNamenodeProtocolPB proxy) {
     rpcProxy = proxy;
   }
@@ -1440,8 +1445,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
   @Override
   public void setStoragePolicy(String src, String policyName)
-      throws SnapshotAccessControlException, UnresolvedLinkException,
-      FileNotFoundException, QuotaExceededException, IOException {
+      throws IOException {
     SetStoragePolicyRequestProto req = SetStoragePolicyRequestProto
         .newBuilder().setSrc(src).setPolicyName(policyName).build();
     try {
@@ -1451,6 +1455,17 @@ public class ClientNamenodeProtocolTranslatorPB implements
     }
   }
 
+  @Override
+  public BlockStoragePolicy[] getStoragePolicies() throws IOException {
+    try {
+      GetStoragePoliciesResponseProto response = rpcProxy
+          .getStoragePolicies(null, VOID_GET_STORAGE_POLICIES_REQUEST);
+      return PBHelper.convertStoragePolicies(response.getPoliciesList());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
   public long getCurrentEditLogTxid() throws IOException {
     GetCurrentEditLogTxidRequestProto req = GetCurrentEditLogTxidRequestProto
         .getDefaultInstance();

+ 63 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -44,7 +44,7 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.proto.HAServiceProtocolProtos;
-import org.apache.hadoop.hdfs.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.inotify.Event;
@@ -120,6 +120,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterComm
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlocksWithLocationsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointCommandProto;
@@ -174,6 +175,7 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
@@ -280,6 +282,65 @@ public class PBHelper {
     return null;
   }
 
+  public static BlockStoragePolicy[] convertStoragePolicies(
+      List<BlockStoragePolicyProto> policyProtos) {
+    if (policyProtos == null || policyProtos.size() == 0) {
+      return new BlockStoragePolicy[0];
+    }
+    BlockStoragePolicy[] policies = new BlockStoragePolicy[policyProtos.size()];
+    int i = 0;
+    for (BlockStoragePolicyProto proto : policyProtos) {
+      policies[i++] = convert(proto);
+    }
+    return policies;
+  }
+
+  public static BlockStoragePolicy convert(BlockStoragePolicyProto proto) {
+    List<StorageTypeProto> cList = proto.getCreationPolicy()
+        .getStorageTypesList();
+    StorageType[] creationTypes = convertStorageTypes(cList, cList.size());
+    List<StorageTypeProto> cfList = proto.hasCreationFallbackPolicy() ? proto
+        .getCreationFallbackPolicy().getStorageTypesList() : null;
+    StorageType[] creationFallbackTypes = cfList == null ? StorageType
+        .EMPTY_ARRAY : convertStorageTypes(cfList, cfList.size());
+    List<StorageTypeProto> rfList = proto.hasReplicationFallbackPolicy() ?
+        proto.getReplicationFallbackPolicy().getStorageTypesList() : null;
+    StorageType[] replicationFallbackTypes = rfList == null ? StorageType
+        .EMPTY_ARRAY : convertStorageTypes(rfList, rfList.size());
+    return new BlockStoragePolicy((byte) proto.getPolicyId(), proto.getName(),
+        creationTypes, creationFallbackTypes, replicationFallbackTypes);
+  }
+
+  public static BlockStoragePolicyProto convert(BlockStoragePolicy policy) {
+    BlockStoragePolicyProto.Builder builder = BlockStoragePolicyProto
+        .newBuilder().setPolicyId(policy.getId()).setName(policy.getName());
+    // creation storage types
+    StorageTypesProto creationProto = convert(policy.getStorageTypes());
+    Preconditions.checkArgument(creationProto != null);
+    builder.setCreationPolicy(creationProto);
+    // creation fallback
+    StorageTypesProto creationFallbackProto = convert(
+        policy.getCreationFallbacks());
+    if (creationFallbackProto != null) {
+      builder.setCreationFallbackPolicy(creationFallbackProto);
+    }
+    // replication fallback
+    StorageTypesProto replicationFallbackProto = convert(
+        policy.getReplicationFallbacks());
+    if (replicationFallbackProto != null) {
+      builder.setReplicationFallbackPolicy(replicationFallbackProto);
+    }
+    return builder.build();
+  }
+
+  public static StorageTypesProto convert(StorageType[] types) {
+    if (types == null || types.length == 0) {
+      return null;
+    }
+    List<StorageTypeProto> list = convertStorageTypes(types);
+    return StorageTypesProto.newBuilder().addAllStorageTypes(list).build();
+  }
+
   public static StorageInfoProto convert(StorageInfo info) {
     return StorageInfoProto.newBuilder().setClusterID(info.getClusterID())
         .setCTime(info.getCTime()).setLayoutVersion(info.getLayoutVersion())
@@ -1357,7 +1418,7 @@ public class PBHelper {
         fs.hasChildrenNum() ? fs.getChildrenNum() : -1,
         fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) : null,
         fs.hasStoragePolicy() ? (byte) fs.getStoragePolicy()
-            : BlockStoragePolicy.ID_UNSPECIFIED);
+            : BlockStoragePolicySuite.ID_UNSPECIFIED);
   }
 
   public static SnapshottableDirectoryStatus convert(

+ 8 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -42,7 +42,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
@@ -255,7 +255,7 @@ public class BlockManager {
 
   /** for block replicas placement */
   private BlockPlacementPolicy blockplacement;
-  private final BlockStoragePolicy.Suite storagePolicySuite;
+  private final BlockStoragePolicySuite storagePolicySuite;
 
   /** Check whether name system is running before terminating */
   private boolean checkNSRunning = true;
@@ -278,7 +278,7 @@ public class BlockManager {
     blockplacement = BlockPlacementPolicy.getInstance(
         conf, stats, datanodeManager.getNetworkTopology(), 
         datanodeManager.getHost2DatanodeMap());
-    storagePolicySuite = BlockStoragePolicy.readBlockStorageSuite(conf);
+    storagePolicySuite = BlockStoragePolicySuite.createDefaultSuite();
     pendingReplications = new PendingReplicationBlocks(conf.getInt(
       DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY,
       DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_DEFAULT) * 1000L);
@@ -402,6 +402,10 @@ public class BlockManager {
     return storagePolicySuite.getPolicy(policyName);
   }
 
+  public BlockStoragePolicy[] getStoragePolicies() {
+    return storagePolicySuite.getAllPolicies();
+  }
+
   public void setBlockPoolId(String blockPoolId) {
     if (isBlockTokenEnabled()) {
       blockTokenSecretManager.setBlockPoolId(blockPoolId);
@@ -3599,7 +3603,7 @@ public class BlockManager {
     }
     
     private void chooseTargets(BlockPlacementPolicy blockplacement,
-        BlockStoragePolicy.Suite storagePolicySuite,
+        BlockStoragePolicySuite storagePolicySuite,
         Set<Node> excludedNodes) {
       targets = blockplacement.chooseTarget(bc.getName(),
           additionalReplRequired, srcNode, liveReplicaStorages, false,

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

@@ -27,7 +27,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;

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

@@ -23,7 +23,7 @@ import java.util.*;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.StorageType;

+ 119 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.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.hdfs.server.blockmanagement;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import org.apache.hadoop.fs.XAttr;
+import org.apache.hadoop.hdfs.StorageType;
+import org.apache.hadoop.hdfs.XAttrHelper;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+/** A collection of block storage policies. */
+public class BlockStoragePolicySuite {
+  static final Logger LOG = LoggerFactory.getLogger(BlockStoragePolicySuite
+      .class);
+
+  public static final String STORAGE_POLICY_XATTR_NAME
+      = "hsm.block.storage.policy.id";
+  public static final XAttr.NameSpace XAttrNS = XAttr.NameSpace.SYSTEM;
+
+  public static final int ID_BIT_LENGTH = 4;
+  public static final byte ID_UNSPECIFIED = 0;
+
+  @VisibleForTesting
+  public static BlockStoragePolicySuite createDefaultSuite() {
+    final BlockStoragePolicy[] policies =
+        new BlockStoragePolicy[1 << ID_BIT_LENGTH];
+    final byte hotId = 12;
+    policies[hotId] = new BlockStoragePolicy(hotId, "HOT",
+        new StorageType[]{StorageType.DISK}, StorageType.EMPTY_ARRAY,
+        new StorageType[]{StorageType.ARCHIVE});
+    final byte warmId = 8;
+    policies[warmId] = new BlockStoragePolicy(warmId, "WARM",
+        new StorageType[]{StorageType.DISK, StorageType.ARCHIVE},
+        new StorageType[]{StorageType.DISK, StorageType.ARCHIVE},
+        new StorageType[]{StorageType.DISK, StorageType.ARCHIVE});
+    final byte coldId = 4;
+    policies[coldId] = new BlockStoragePolicy(coldId, "COLD",
+        new StorageType[]{StorageType.ARCHIVE}, StorageType.EMPTY_ARRAY,
+        StorageType.EMPTY_ARRAY);
+    return new BlockStoragePolicySuite(hotId, policies);
+  }
+
+  private final byte defaultPolicyID;
+  private final BlockStoragePolicy[] policies;
+
+  public BlockStoragePolicySuite(byte defaultPolicyID,
+      BlockStoragePolicy[] policies) {
+    this.defaultPolicyID = defaultPolicyID;
+    this.policies = policies;
+  }
+
+  /** @return the corresponding policy. */
+  public BlockStoragePolicy getPolicy(byte id) {
+    // id == 0 means policy not specified.
+    return id == 0? getDefaultPolicy(): policies[id];
+  }
+
+  /** @return the default policy. */
+  public BlockStoragePolicy getDefaultPolicy() {
+    return getPolicy(defaultPolicyID);
+  }
+
+  public BlockStoragePolicy getPolicy(String policyName) {
+    if (policies != null) {
+      for (BlockStoragePolicy policy : policies) {
+        if (policy != null && policy.getName().equals(policyName)) {
+          return policy;
+        }
+      }
+    }
+    return null;
+  }
+
+  public BlockStoragePolicy[] getAllPolicies() {
+    List<BlockStoragePolicy> list = Lists.newArrayList();
+    if (policies != null) {
+      for (BlockStoragePolicy policy : policies) {
+        if (policy != null) {
+          list.add(policy);
+        }
+      }
+    }
+    return list.toArray(new BlockStoragePolicy[list.size()]);
+  }
+
+  public static String buildXAttrName() {
+    return XAttrNS.toString().toLowerCase() + "." + STORAGE_POLICY_XATTR_NAME;
+  }
+
+  public static XAttr buildXAttr(byte policyId) {
+    final String name = buildXAttrName();
+    return XAttrHelper.buildXAttr(name, new byte[]{policyId});
+  }
+
+  public static boolean isStoragePolicyXAttr(XAttr xattr) {
+    return xattr != null && xattr.getNameSpace() == XAttrNS
+        && xattr.getName().equals(STORAGE_POLICY_XATTR_NAME);
+  }
+}

+ 27 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.server.balancer.Dispatcher.DDatanode.StorageGroup;
 import org.apache.hadoop.hdfs.server.balancer.ExitStatus;
 import org.apache.hadoop.hdfs.server.balancer.Matcher;
 import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
@@ -102,7 +103,7 @@ public class Mover {
   private final StorageMap storages;
   private final List<Path> targetPaths;
 
-  private final BlockStoragePolicy.Suite blockStoragePolicies;
+  private final BlockStoragePolicy[] blockStoragePolicies;
 
   Mover(NameNodeConnector nnc, Configuration conf) {
     final long movedWinWidth = conf.getLong(
@@ -119,11 +120,13 @@ public class Mover {
         Collections.<String> emptySet(), movedWinWidth, moverThreads, 0,
         maxConcurrentMovesPerNode, conf);
     this.storages = new StorageMap();
-    this.blockStoragePolicies = BlockStoragePolicy.readBlockStorageSuite(conf);
     this.targetPaths = nnc.getTargetPaths();
+    this.blockStoragePolicies = new BlockStoragePolicy[1 <<
+        BlockStoragePolicySuite.ID_BIT_LENGTH];
   }
 
   void init() throws IOException {
+    initStoragePolicies();
     final List<DatanodeStorageReport> reports = dispatcher.init();
     for(DatanodeStorageReport r : reports) {
       final DDatanode dn = dispatcher.newDatanode(r.getDatanodeInfo());
@@ -137,6 +140,14 @@ public class Mover {
     }
   }
 
+  private void initStoragePolicies() throws IOException {
+    BlockStoragePolicy[] policies = dispatcher.getDistributedFileSystem()
+        .getStoragePolicies();
+    for (BlockStoragePolicy policy : policies) {
+      this.blockStoragePolicies[policy.getId()] = policy;
+    }
+  }
+
   private ExitStatus run() {
     try {
       init();
@@ -305,7 +316,7 @@ public class Mover {
           if (!isSnapshotPathInCurrent(fullPath)) {
             // the full path is a snapshot path but it is also included in the
             // current directory tree, thus ignore it.
-            hasRemaining = processFile((HdfsLocatedFileStatus)status);
+            hasRemaining = processFile(fullPath, (HdfsLocatedFileStatus)status);
           }
         } catch (IOException e) {
           LOG.warn("Failed to check the status of " + parent
@@ -317,9 +328,17 @@ public class Mover {
     }
 
     /** @return true if it is necessary to run another round of migration */
-    private boolean processFile(HdfsLocatedFileStatus status) {
-      final BlockStoragePolicy policy = blockStoragePolicies.getPolicy(
-          status.getStoragePolicy());
+    private boolean processFile(String fullPath, HdfsLocatedFileStatus status) {
+      final byte policyId = status.getStoragePolicy();
+      // currently we ignore files with unspecified storage policy
+      if (policyId == BlockStoragePolicySuite.ID_UNSPECIFIED) {
+        return false;
+      }
+      final BlockStoragePolicy policy = blockStoragePolicies[policyId];
+      if (policy == null) {
+        LOG.warn("Failed to get the storage policy of file " + fullPath);
+        return false;
+      }
       final List<StorageType> types = policy.chooseStorageTypes(
           status.getReplication());
 
@@ -368,8 +387,8 @@ public class Mover {
     boolean scheduleMoveReplica(DBlock db, MLocation ml,
                                 List<StorageType> targetTypes) {
       final Source source = storages.getSource(ml);
-      return source == null ? false : scheduleMoveReplica(db,
-          storages.getSource(ml), targetTypes);
+      return source == null ? false : scheduleMoveReplica(db, source,
+          targetTypes);
     }
 
     boolean scheduleMoveReplica(DBlock db, Source source,

+ 9 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -53,7 +53,6 @@ import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.apache.hadoop.hdfs.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -79,6 +78,7 @@ import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
@@ -1042,7 +1042,7 @@ public class FSDirectory implements Closeable {
   private void setDirStoragePolicy(INodeDirectory inode, byte policyId,
       int latestSnapshotId) throws IOException {
     List<XAttr> existingXAttrs = XAttrStorage.readINodeXAttrs(inode);
-    XAttr xAttr = BlockStoragePolicy.buildXAttr(policyId);
+    XAttr xAttr = BlockStoragePolicySuite.buildXAttr(policyId);
     List<XAttr> newXAttrs = setINodeXAttrs(existingXAttrs, Arrays.asList(xAttr),
         EnumSet.of(XAttrSetFlag.CREATE, XAttrSetFlag.REPLACE));
     XAttrStorage.updateINodeXAttrs(inode, newXAttrs, latestSnapshotId);
@@ -1380,7 +1380,7 @@ public class FSDirectory implements Closeable {
   }
 
   private byte getStoragePolicyID(byte inodePolicy, byte parentPolicy) {
-    return inodePolicy != BlockStoragePolicy.ID_UNSPECIFIED ? inodePolicy :
+    return inodePolicy != BlockStoragePolicySuite.ID_UNSPECIFIED ? inodePolicy :
         parentPolicy;
   }
 
@@ -1415,7 +1415,7 @@ public class FSDirectory implements Closeable {
       if (targetNode == null)
         return null;
       byte parentStoragePolicy = isSuperUser ?
-          targetNode.getStoragePolicyID() : BlockStoragePolicy.ID_UNSPECIFIED;
+          targetNode.getStoragePolicyID() : BlockStoragePolicySuite.ID_UNSPECIFIED;
       
       if (!targetNode.isDirectory()) {
         return new DirectoryListing(
@@ -1435,7 +1435,8 @@ public class FSDirectory implements Closeable {
       for (int i=0; i<numOfListing && locationBudget>0; i++) {
         INode cur = contents.get(startChild+i);
         byte curPolicy = isSuperUser && !cur.isSymlink()?
-            cur.getLocalStoragePolicyID(): BlockStoragePolicy.ID_UNSPECIFIED;
+            cur.getLocalStoragePolicyID():
+            BlockStoragePolicySuite.ID_UNSPECIFIED;
         listing[i] = createFileStatus(cur.getLocalNameBytes(), cur, needLocation,
             getStoragePolicyID(curPolicy, parentStoragePolicy), snapshot,
             isRawPath, inodesInPath);
@@ -1489,7 +1490,7 @@ public class FSDirectory implements Closeable {
     for (int i = 0; i < numOfListing; i++) {
       Root sRoot = snapshots.get(i + skipSize).getRoot();
       listing[i] = createFileStatus(sRoot.getLocalNameBytes(), sRoot,
-          BlockStoragePolicy.ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID,
+          BlockStoragePolicySuite.ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID,
           false, null);
     }
     return new DirectoryListing(
@@ -1517,7 +1518,7 @@ public class FSDirectory implements Closeable {
       final INode[] inodes = inodesInPath.getINodes();
       final INode i = inodes[inodes.length - 1];
       byte policyId = includeStoragePolicy && i != null && !i.isSymlink() ?
-          i.getStoragePolicyID() : BlockStoragePolicy.ID_UNSPECIFIED;
+          i.getStoragePolicyID() : BlockStoragePolicySuite.ID_UNSPECIFIED;
       return i == null ? null : createFileStatus(HdfsFileStatus.EMPTY_NAME, i,
           policyId, inodesInPath.getPathSnapshotId(), isRawPath,
           inodesInPath);
@@ -1538,7 +1539,7 @@ public class FSDirectory implements Closeable {
     if (getINode4DotSnapshot(src) != null) {
       return new HdfsFileStatus(0, true, 0, 0, false, 0, 0, null, null, null, null,
           HdfsFileStatus.EMPTY_NAME, -1L, 0, null,
-          BlockStoragePolicy.ID_UNSPECIFIED);
+          BlockStoragePolicySuite.ID_UNSPECIFIED);
     }
     return null;
   }

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

@@ -33,7 +33,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.XAttrSetFlag;
-import org.apache.hadoop.hdfs.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -374,7 +374,7 @@ public class FSEditLogLoader {
         if (toAddRetryCache) {
           HdfsFileStatus stat = fsNamesys.dir.createFileStatus(
               HdfsFileStatus.EMPTY_NAME, newFile,
-              BlockStoragePolicy.ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID,
+              BlockStoragePolicySuite.ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID,
               false, iip);
           fsNamesys.addCacheEntryWithPayload(addCloseOp.rpcClientId,
               addCloseOp.rpcCallId, stat);

+ 17 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -17,8 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.apache.hadoop.crypto.key.KeyProviderCryptoExtension
-    .EncryptedKeyVersion;
+import static org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_DEFAULT;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
@@ -162,7 +161,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.ServiceFailedException;
-import org.apache.hadoop.hdfs.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
@@ -2334,6 +2333,21 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     logAuditEvent(true, "setStoragePolicy", src, null, fileStat);
   }
 
+  /**
+   * @return All the existing block storage policies
+   */
+  BlockStoragePolicy[] getStoragePolicies() throws IOException {
+    checkOperation(OperationCategory.READ);
+    waitForLoadingFSImage();
+    readLock();
+    try {
+      checkOperation(OperationCategory.READ);
+      return blockManager.getStoragePolicies();
+    } finally {
+      readUnlock();
+    }
+  }
+
   long getPreferredBlockSize(String filename) 
       throws IOException, UnresolvedLinkException {
     FSPermissionChecker pc = getPermissionChecker();

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

@@ -29,7 +29,7 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.apache.hadoop.hdfs.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
@@ -695,7 +695,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
 
   /**
    * @return the storage policy directly specified on the INode. Return
-   * {@link BlockStoragePolicy#ID_UNSPECIFIED} if no policy has
+   * {@link BlockStoragePolicySuite#ID_UNSPECIFIED} if no policy has
    * been specified.
    */
   public abstract byte getLocalStoragePolicyID();

+ 6 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java

@@ -28,10 +28,11 @@ import java.util.Map;
 import org.apache.hadoop.fs.PathIsNotDirectoryException;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.apache.hadoop.hdfs.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectorySnapshottableFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
@@ -112,22 +113,22 @@ public class INodeDirectory extends INodeWithAdditionalFields
     ImmutableList<XAttr> xattrs = f == null ? ImmutableList.<XAttr> of() : f
         .getXAttrs();
     for (XAttr xattr : xattrs) {
-      if (BlockStoragePolicy.isStoragePolicyXAttr(xattr)) {
+      if (BlockStoragePolicySuite.isStoragePolicyXAttr(xattr)) {
         return (xattr.getValue())[0];
       }
     }
-    return BlockStoragePolicy.ID_UNSPECIFIED;
+    return BlockStoragePolicySuite.ID_UNSPECIFIED;
   }
 
   @Override
   public byte getStoragePolicyID() {
     byte id = getLocalStoragePolicyID();
-    if (id != BlockStoragePolicy.ID_UNSPECIFIED) {
+    if (id != BlockStoragePolicySuite.ID_UNSPECIFIED) {
       return id;
     }
     // if it is unspecified, check its parent
     return getParent() != null ? getParent().getStoragePolicyID() :
-        BlockStoragePolicy.ID_UNSPECIFIED;
+        BlockStoragePolicySuite.ID_UNSPECIFIED;
   }
 
   void setQuota(long nsQuota, long dsQuota) {

+ 0 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryAttributes.java

@@ -18,12 +18,9 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.apache.hadoop.hdfs.BlockStoragePolicy;
 
 import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableList;
 
 /**
  * The attributes of an inode.

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -28,12 +28,12 @@ import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.apache.hadoop.hdfs.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 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;
@@ -79,7 +79,8 @@ public class INodeFile extends INodeWithAdditionalFields
   static enum HeaderFormat {
     PREFERRED_BLOCK_SIZE(null, 48, 1),
     REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 11, 1),
-    STORAGE_POLICY_ID(REPLICATION.BITS, BlockStoragePolicy.ID_BIT_LENGTH, 0),
+    STORAGE_POLICY_ID(REPLICATION.BITS, BlockStoragePolicySuite.ID_BIT_LENGTH,
+        0),
     LAZY_PERSIST(STORAGE_POLICY_ID.BITS, 1, 0);
 
     private final LongBitFormat BITS;
@@ -393,7 +394,7 @@ public class INodeFile extends INodeWithAdditionalFields
   @Override
   public byte getStoragePolicyID() {
     byte id = getLocalStoragePolicyID();
-    if (id == BlockStoragePolicy.ID_UNSPECIFIED) {
+    if (id == BlockStoragePolicySuite.ID_UNSPECIFIED) {
       return this.getParent() != null ?
           this.getParent().getStoragePolicyID() : id;
     }

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java

@@ -22,8 +22,9 @@ import java.util.List;
 
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.apache.hadoop.hdfs.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.Quota.Counts;
 import org.apache.hadoop.util.GSet;
 import org.apache.hadoop.util.LightWeightGSet;
@@ -125,12 +126,12 @@ public class INodeMap {
 
       @Override
       public byte getStoragePolicyID(){
-        return BlockStoragePolicy.ID_UNSPECIFIED;
+        return BlockStoragePolicySuite.ID_UNSPECIFIED;
       }
 
       @Override
       public byte getLocalStoragePolicyID() {
-        return BlockStoragePolicy.ID_UNSPECIFIED;
+        return BlockStoragePolicySuite.ID_UNSPECIFIED;
       }
     };
       

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -72,6 +72,7 @@ import org.apache.hadoop.hdfs.inotify.EventsList;
 import org.apache.hadoop.hdfs.protocol.AclException;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
@@ -592,6 +593,11 @@ class NameNodeRpcServer implements NamenodeProtocols {
     namesystem.setStoragePolicy(src, policyName);
   }
 
+  @Override
+  public BlockStoragePolicy[] getStoragePolicies() throws IOException {
+    return namesystem.getStoragePolicies();
+  }
+
   @Override // ClientProtocol
   public void setPermission(String src, FsPermission permissions)
       throws IOException {

+ 12 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java

@@ -44,7 +44,7 @@ import org.apache.hadoop.fs.FsStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.shell.Command;
 import org.apache.hadoop.fs.shell.CommandFormat;
-import org.apache.hadoop.hdfs.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -63,6 +63,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.TransferFsImage;
 import org.apache.hadoop.ipc.GenericRefreshProtocol;
@@ -609,15 +610,18 @@ public class DFSAdmin extends FsShell {
           + argv[1]);
     }
     byte storagePolicyId = status.getStoragePolicy();
-    BlockStoragePolicy.Suite suite = BlockStoragePolicy
-        .readBlockStorageSuite(getConf());
-    BlockStoragePolicy policy = suite.getPolicy(storagePolicyId);
-    if (policy != null) {
-      System.out.println("The storage policy of " + argv[1] + ":\n" + policy);
+    if (storagePolicyId == BlockStoragePolicySuite.ID_UNSPECIFIED) {
+      System.out.println("The storage policy of " + argv[1] + " is unspecified");
       return 0;
-    } else {
-      throw new IOException("Cannot identify the storage policy for " + argv[1]);
     }
+    BlockStoragePolicy[] policies = dfs.getStoragePolicies();
+    for (BlockStoragePolicy p : policies) {
+      if (p.getId() == storagePolicyId) {
+        System.out.println("The storage policy of " + argv[1] + ":\n" + p);
+        return 0;
+      }
+    }
+    throw new IOException("Cannot identify the storage policy for " + argv[1]);
   }
 
   /**

+ 65 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetStoragePolicies.java

@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.tools;
+
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import java.io.IOException;
+
+/**
+ * A tool listing all the existing block storage policies. No argument is
+ * required when using this tool.
+ */
+public class GetStoragePolicies extends Configured implements Tool {
+
+  @Override
+  public int run(String[] args) throws Exception {
+    FileSystem fs = FileSystem.get(getConf());
+    if (!(fs instanceof DistributedFileSystem)) {
+      System.err.println("GetStoragePolicies can only be used against HDFS. " +
+          "Please check the default FileSystem setting in your configuration.");
+      return 1;
+    }
+    DistributedFileSystem dfs = (DistributedFileSystem) fs;
+
+    try {
+      BlockStoragePolicy[] policies = dfs.getStoragePolicies();
+      System.out.println("Block Storage Policies:");
+      for (BlockStoragePolicy policy : policies) {
+        if (policy != null) {
+          System.out.println("\t" + policy);
+        }
+      }
+    } catch (IOException e) {
+      String[] content = e.getLocalizedMessage().split("\n");
+      System.err.println("GetStoragePolicies: " + content[0]);
+      return 1;
+    }
+    return 0;
+  }
+
+  public static void main(String[] args) throws Exception {
+    int rc = ToolRunner.run(new GetStoragePolicies(), args);
+    System.exit(rc);
+  }
+}

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java

@@ -21,13 +21,14 @@ import org.apache.hadoop.fs.*;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.*;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.token.Token;
@@ -266,7 +267,7 @@ public class JsonUtil {
             : childrenNumLong.intValue();
     final byte storagePolicy = m.containsKey("storagePolicy") ?
         (byte) (long) (Long) m.get("storagePolicy") :
-          BlockStoragePolicy.ID_UNSPECIFIED;
+          BlockStoragePolicySuite.ID_UNSPECIFIED;
     return new HdfsFileStatus(len, type == PathType.DIRECTORY, replication,
         blockSize, isLazyPersist, mTime, aTime, permission, owner, group,
         symlink, DFSUtil.string2Bytes(localName), fileId, childrenNum, null,

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c

@@ -3215,7 +3215,7 @@ static void hdfsFreeFileInfoEntry(hdfsFileInfo *hdfsFileInfo)
     free(hdfsFileInfo->mName);
     free(hdfsFileInfo->mOwner);
     free(hdfsFileInfo->mGroup);
-    memset(hdfsFileInfo, 0, sizeof(hdfsFileInfo));
+    memset(hdfsFileInfo, 0, sizeof(*hdfsFileInfo));
 }
 
 void hdfsFreeFileInfo(hdfsFileInfo *hdfsFileInfo, int numEntries)

+ 11 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/test/test_libhdfs_ops.c

@@ -122,11 +122,11 @@ int main(int argc, char **argv) {
         currentPos = -1;
         if ((currentPos = hdfsTell(fs, writeFile)) == -1) {
             fprintf(stderr, 
-                    "Failed to get current file position correctly! Got %ld!\n",
+                    "Failed to get current file position correctly! Got %" PRId64 "!\n",
                     currentPos);
             exit(-1);
         }
-        fprintf(stderr, "Current position: %ld\n", currentPos);
+        fprintf(stderr, "Current position: %" PRId64 "\n", currentPos);
 
         if (hdfsFlush(fs, writeFile)) {
             fprintf(stderr, "Failed to 'flush' %s\n", writePath); 
@@ -177,11 +177,11 @@ int main(int argc, char **argv) {
         currentPos = -1;
         if((currentPos = hdfsTell(fs, readFile)) != seekPos) {
             fprintf(stderr, 
-                    "Failed to get current file position correctly! Got %ld!\n", 
+                    "Failed to get current file position correctly! Got %" PRId64 "!\n",
                     currentPos);
             exit(-1);
         }
-        fprintf(stderr, "Current position: %ld\n", currentPos);
+        fprintf(stderr, "Current position: %" PRId64 "\n", currentPos);
 
         if (!hdfsFileUsesDirectRead(readFile)) {
           fprintf(stderr, "Direct read support incorrectly not detected "
@@ -283,9 +283,9 @@ int main(int argc, char **argv) {
         fprintf(stderr, "hdfsGetWorkingDirectory: %s\n", ((resp = hdfsGetWorkingDirectory(fs, buffer2, sizeof(buffer2))) != 0 ? buffer2 : "Failed!"));
         totalResult += (resp ? 0 : 1);
 
-        fprintf(stderr, "hdfsGetDefaultBlockSize: %ld\n", hdfsGetDefaultBlockSize(fs));
-        fprintf(stderr, "hdfsGetCapacity: %ld\n", hdfsGetCapacity(fs));
-        fprintf(stderr, "hdfsGetUsed: %ld\n", hdfsGetUsed(fs));
+        fprintf(stderr, "hdfsGetDefaultBlockSize: %" PRId64 "\n", hdfsGetDefaultBlockSize(fs));
+        fprintf(stderr, "hdfsGetCapacity: %" PRId64 "\n", hdfsGetCapacity(fs));
+        fprintf(stderr, "hdfsGetUsed: %" PRId64 "\n", hdfsGetUsed(fs));
 
         fileInfo = NULL;
         if((fileInfo = hdfsGetPathInfo(fs, slashTmp)) != NULL) {
@@ -293,8 +293,8 @@ int main(int argc, char **argv) {
             fprintf(stderr, "Name: %s, ", fileInfo->mName);
             fprintf(stderr, "Type: %c, ", (char)(fileInfo->mKind));
             fprintf(stderr, "Replication: %d, ", fileInfo->mReplication);
-            fprintf(stderr, "BlockSize: %ld, ", fileInfo->mBlockSize);
-            fprintf(stderr, "Size: %ld, ", fileInfo->mSize);
+            fprintf(stderr, "BlockSize: %" PRId64 ", ", fileInfo->mBlockSize);
+            fprintf(stderr, "Size: %" PRId64 ", ", fileInfo->mSize);
             fprintf(stderr, "LastMod: %s", ctime(&fileInfo->mLastMod)); 
             fprintf(stderr, "Owner: %s, ", fileInfo->mOwner);
             fprintf(stderr, "Group: %s, ", fileInfo->mGroup);
@@ -312,8 +312,8 @@ int main(int argc, char **argv) {
                 fprintf(stderr, "Name: %s, ", fileList[i].mName);
                 fprintf(stderr, "Type: %c, ", (char)fileList[i].mKind);
                 fprintf(stderr, "Replication: %d, ", fileList[i].mReplication);
-                fprintf(stderr, "BlockSize: %ld, ", fileList[i].mBlockSize);
-                fprintf(stderr, "Size: %ld, ", fileList[i].mSize);
+                fprintf(stderr, "BlockSize: %" PRId64 ", ", fileList[i].mBlockSize);
+                fprintf(stderr, "Size: %" PRId64 ", ", fileList[i].mSize);
                 fprintf(stderr, "LastMod: %s", ctime(&fileList[i].mLastMod));
                 fprintf(stderr, "Owner: %s, ", fileList[i].mOwner);
                 fprintf(stderr, "Group: %s, ", fileList[i].mGroup);

+ 31 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/test/vecsum.c

@@ -29,6 +29,12 @@
 #include <time.h>
 #include <unistd.h>
 
+#ifdef __MACH__ // OS X does not have clock_gettime
+#include <mach/clock.h>
+#include <mach/mach.h>
+#include <mach/mach_time.h>
+#endif
+
 #include "config.h"
 #include "hdfs.h"
 
@@ -49,6 +55,29 @@ struct stopwatch {
     struct timespec stop;
 };
 
+
+#ifdef __MACH__
+static int clock_gettime_mono(struct timespec * ts) {
+    static mach_timebase_info_data_t tb;
+    static uint64_t timestart = 0;
+    uint64_t t = 0;
+    if (timestart == 0) {
+        mach_timebase_info(&tb);
+        timestart = mach_absolute_time();
+    }
+    t = mach_absolute_time() - timestart;
+    t *= tb.numer;
+    t /= tb.denom;
+    ts->tv_sec = t / 1000000000ULL;
+    ts->tv_nsec = t - (ts->tv_sec * 1000000000ULL);
+    return 0;
+}
+#else
+static int clock_gettime_mono(struct timespec * ts) {
+    return clock_gettime(CLOCK_MONOTONIC, ts);
+}
+#endif
+
 static struct stopwatch *stopwatch_create(void)
 {
     struct stopwatch *watch;
@@ -58,7 +87,7 @@ static struct stopwatch *stopwatch_create(void)
         fprintf(stderr, "failed to allocate memory for stopwatch\n");
         goto error;
     }
-    if (clock_gettime(CLOCK_MONOTONIC, &watch->start)) {
+    if (clock_gettime_mono(&watch->start)) {
         int err = errno;
         fprintf(stderr, "clock_gettime(CLOCK_MONOTONIC) failed with "
             "error %d (%s)\n", err, strerror(err));
@@ -76,7 +105,7 @@ static void stopwatch_stop(struct stopwatch *watch,
 {
     double elapsed, rate;
 
-    if (clock_gettime(CLOCK_MONOTONIC, &watch->stop)) {
+    if (clock_gettime_mono(&watch->stop)) {
         int err = errno;
         fprintf(stderr, "clock_gettime(CLOCK_MONOTONIC) failed with "
             "error %d (%s)\n", err, strerror(err));

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/test_libhdfs_threaded.c

@@ -84,7 +84,7 @@ static int hdfsSingleNameNodeConnect(struct NativeMiniDfsCluster *cl, hdfsFS *fs
 
 static int doTestGetDefaultBlockSize(hdfsFS fs, const char *path)
 {
-    uint64_t blockSize;
+    int64_t blockSize;
     int ret;
 
     blockSize = hdfsGetDefaultBlockSize(fs);

+ 9 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto

@@ -109,6 +109,13 @@ message SetStoragePolicyRequestProto {
 message SetStoragePolicyResponseProto { // void response
 }
 
+message GetStoragePoliciesRequestProto { // void request
+}
+
+message GetStoragePoliciesResponseProto {
+  repeated BlockStoragePolicyProto policies = 1;
+}
+
 message SetPermissionRequestProto {
   required string src = 1;
   required FsPermissionProto permission = 2;
@@ -700,6 +707,8 @@ service ClientNamenodeProtocol {
       returns(SetReplicationResponseProto);
   rpc setStoragePolicy(SetStoragePolicyRequestProto)
       returns(SetStoragePolicyResponseProto);
+  rpc getStoragePolicies(GetStoragePoliciesRequestProto)
+      returns(GetStoragePoliciesResponseProto);
   rpc setPermission(SetPermissionRequestProto)
       returns(SetPermissionResponseProto);
   rpc setOwner(SetOwnerRequestProto) returns(SetOwnerResponseProto);

+ 14 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto

@@ -169,6 +169,20 @@ message StorageTypesProto {
   repeated StorageTypeProto storageTypes = 1;
 }
 
+/**
+ * Block replica storage policy.
+ */
+message BlockStoragePolicyProto {
+  required uint32 policyId = 1;
+  required string name = 2;
+  // a list of storage types for storing the block replicas when creating a
+  // block.
+  required StorageTypesProto creationPolicy = 3;
+  // A list of storage types for creation fallback storage.
+  optional StorageTypesProto creationFallbackPolicy = 4;
+  optional StorageTypesProto replicationFallbackPolicy = 5;
+}
+
 /**
  * A list of storage IDs. 
  */

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -22,8 +22,7 @@
 <!-- wish to modify from this file into hdfs-site.xml and change them -->
 <!-- there.  If hdfs-site.xml does not already exist, create it.      -->
 
-<configuration xmlns:xi="http://www.w3.org/2001/XInclude"> 
-  <xi:include href="blockStoragePolicy-default.xml" />
+<configuration>
 
 <property>
   <name>hadoop.hdfs.configuration.version</name>

+ 66 - 20
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java

@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.apache.hadoop.hdfs.BlockStoragePolicy.ID_UNSPECIFIED;
+import static org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite.ID_UNSPECIFIED;
 
 import java.io.File;
 import java.io.FileNotFoundException;
@@ -34,6 +34,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.Node;
@@ -44,7 +45,7 @@ import org.junit.Test;
 
 /** Test {@link BlockStoragePolicy} */
 public class TestBlockStoragePolicy {
-  public static final BlockStoragePolicy.Suite POLICY_SUITE;
+  public static final BlockStoragePolicySuite POLICY_SUITE;
   public static final BlockStoragePolicy DEFAULT_STORAGE_POLICY;
   public static final Configuration conf;
 
@@ -52,7 +53,7 @@ public class TestBlockStoragePolicy {
     conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1);
-    POLICY_SUITE = BlockStoragePolicy.readBlockStorageSuite(conf);
+    POLICY_SUITE = BlockStoragePolicySuite.createDefaultSuite();
     DEFAULT_STORAGE_POLICY = POLICY_SUITE.getDefaultPolicy();
   }
 
@@ -72,11 +73,14 @@ public class TestBlockStoragePolicy {
   public void testDefaultPolicies() {
     final Map<Byte, String> expectedPolicyStrings = new HashMap<Byte, String>();
     expectedPolicyStrings.put(COLD,
-        "BlockStoragePolicy{COLD:4, storageTypes=[ARCHIVE], creationFallbacks=[], replicationFallbacks=[]");
+        "BlockStoragePolicy{COLD:4, storageTypes=[ARCHIVE], " +
+            "creationFallbacks=[], replicationFallbacks=[]}");
     expectedPolicyStrings.put(WARM,
-        "BlockStoragePolicy{WARM:8, storageTypes=[DISK, ARCHIVE], creationFallbacks=[DISK, ARCHIVE], replicationFallbacks=[DISK, ARCHIVE]");
+        "BlockStoragePolicy{WARM:8, storageTypes=[DISK, ARCHIVE], " +
+            "creationFallbacks=[DISK, ARCHIVE], replicationFallbacks=[DISK, ARCHIVE]}");
     expectedPolicyStrings.put(HOT,
-        "BlockStoragePolicy{HOT:12, storageTypes=[DISK], creationFallbacks=[], replicationFallbacks=[ARCHIVE]");
+        "BlockStoragePolicy{HOT:12, storageTypes=[DISK], " +
+            "creationFallbacks=[], replicationFallbacks=[ARCHIVE]}");
 
     for(byte i = 1; i < 16; i++) {
       final BlockStoragePolicy policy = POLICY_SUITE.getPolicy(i); 
@@ -948,7 +952,7 @@ public class TestBlockStoragePolicy {
     Assert.assertTrue(typeList.isEmpty());
   }
 
-  private void testIncreaseFileRep(String policyName, byte policyId,
+  private void testChangeFileRep(String policyName, byte policyId,
                                    StorageType[] before,
                                    StorageType[] after) throws Exception {
     final int numDataNodes = 5;
@@ -965,8 +969,6 @@ public class TestBlockStoragePolicy {
       final Path foo = new Path(dir, "foo");
       DFSTestUtil.createFile(fs, foo, FILE_LEN, REPLICATION, 0L);
 
-      // the storage policy of foo should be WARM, and the replicas
-      // should be stored in DISK and ARCHIE
       HdfsFileStatus[] status = fs.getClient().listPaths(foo.toString(),
           HdfsFileStatus.EMPTY_NAME, true).getPartialListing();
       checkDirectoryListing(status, policyId);
@@ -984,7 +986,24 @@ public class TestBlockStoragePolicy {
           HdfsFileStatus.EMPTY_NAME, true).getPartialListing();
       checkDirectoryListing(status, policyId);
       fooStatus = (HdfsLocatedFileStatus) status[0];
-      checkLocatedBlocks(fooStatus, 1, 5, after);
+      checkLocatedBlocks(fooStatus, 1, numDataNodes, after);
+
+      // change the replication factor back to 3
+      fs.setReplication(foo, REPLICATION);
+      Thread.sleep(1000);
+      for (DataNode dn : cluster.getDataNodes()) {
+        DataNodeTestUtils.triggerHeartbeat(dn);
+      }
+      Thread.sleep(1000);
+      for (DataNode dn : cluster.getDataNodes()) {
+        DataNodeTestUtils.triggerBlockReport(dn);
+      }
+      Thread.sleep(1000);
+      status = fs.getClient().listPaths(foo.toString(),
+          HdfsFileStatus.EMPTY_NAME, true).getPartialListing();
+      checkDirectoryListing(status, policyId);
+      fooStatus = (HdfsLocatedFileStatus) status[0];
+      checkLocatedBlocks(fooStatus, 1, REPLICATION, before);
     } finally {
       cluster.shutdown();
     }
@@ -995,11 +1014,12 @@ public class TestBlockStoragePolicy {
    * that file from 3 to 5. Make sure all replications are created in DISKS.
    */
   @Test
-  public void testIncreaseHotFileRep() throws Exception {
-    testIncreaseFileRep("HOT", HOT, new StorageType[]{StorageType.DISK,
-            StorageType.DISK, StorageType.DISK},
+  public void testChangeHotFileRep() throws Exception {
+    testChangeFileRep("HOT", HOT,
         new StorageType[]{StorageType.DISK, StorageType.DISK,
-            StorageType.DISK, StorageType.DISK, StorageType.DISK});
+            StorageType.DISK},
+        new StorageType[]{StorageType.DISK, StorageType.DISK, StorageType.DISK,
+            StorageType.DISK, StorageType.DISK});
   }
 
   /**
@@ -1008,9 +1028,10 @@ public class TestBlockStoragePolicy {
    * and ARCHIVE.
    */
   @Test
-  public void testIncreaseWarmRep() throws Exception {
-    testIncreaseFileRep("WARM", WARM, new StorageType[]{StorageType.DISK,
-        StorageType.ARCHIVE, StorageType.ARCHIVE},
+  public void testChangeWarmRep() throws Exception {
+    testChangeFileRep("WARM", WARM,
+        new StorageType[]{StorageType.DISK, StorageType.ARCHIVE,
+            StorageType.ARCHIVE},
         new StorageType[]{StorageType.DISK, StorageType.ARCHIVE,
             StorageType.ARCHIVE, StorageType.ARCHIVE, StorageType.ARCHIVE});
   }
@@ -1020,9 +1041,10 @@ public class TestBlockStoragePolicy {
    * that file from 3 to 5. Make sure all replicas are created in ARCHIVE.
    */
   @Test
-  public void testIncreaseColdRep() throws Exception {
-    testIncreaseFileRep("COLD", COLD, new StorageType[]{StorageType.ARCHIVE,
-            StorageType.ARCHIVE, StorageType.ARCHIVE},
+  public void testChangeColdRep() throws Exception {
+    testChangeFileRep("COLD", COLD,
+        new StorageType[]{StorageType.ARCHIVE, StorageType.ARCHIVE,
+            StorageType.ARCHIVE},
         new StorageType[]{StorageType.ARCHIVE, StorageType.ARCHIVE,
             StorageType.ARCHIVE, StorageType.ARCHIVE, StorageType.ARCHIVE});
   }
@@ -1072,4 +1094,28 @@ public class TestBlockStoragePolicy {
     System.out.println(Arrays.asList(targets));
     Assert.assertEquals(3, targets.length);
   }
+
+  /**
+   * Test getting all the storage policies from the namenode
+   */
+  @Test
+  public void testGetAllStoragePolicies() throws Exception {
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(0).build();
+    cluster.waitActive();
+    final DistributedFileSystem fs = cluster.getFileSystem();
+    try {
+      BlockStoragePolicy[] policies = fs.getStoragePolicies();
+      Assert.assertEquals(3, policies.length);
+      Assert.assertEquals(POLICY_SUITE.getPolicy(COLD).toString(),
+          policies[0].toString());
+      Assert.assertEquals(POLICY_SUITE.getPolicy(WARM).toString(),
+          policies[1].toString());
+      Assert.assertEquals(POLICY_SUITE.getPolicy(HOT).toString(),
+          policies[2].toString());
+    } finally {
+      IOUtils.cleanup(null, fs);
+      cluster.shutdown();
+    }
+  }
 }

+ 21 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java

@@ -1177,4 +1177,25 @@ public class TestEncryptionZones {
     parser.parse(new InputSource(new StringReader(xml)), new DefaultHandler());
   }
 
+  /**
+   * Test creating encryption zone on the root path
+   */
+  @Test(timeout = 60000)
+  public void testEncryptionZonesOnRootPath() throws Exception {
+    final int len = 8196;
+    final Path rootDir = new Path("/");
+    final Path zoneFile = new Path(rootDir, "file");
+    final Path rawFile = new Path("/.reserved/raw/file");
+    dfsAdmin.createEncryptionZone(rootDir, TEST_KEY);
+    DFSTestUtil.createFile(fs, zoneFile, len, (short) 1, 0xFEED);
+
+    assertEquals("File can be created on the root encryption zone " +
+            "with correct length",
+        len, fs.getFileStatus(zoneFile).getLen());
+    assertEquals("Root dir is encrypted",
+        true, fs.getFileStatus(rootDir).isEncrypted());
+    assertEquals("File is encrypted",
+        true, fs.getFileStatus(zoneFile).isEncrypted());
+    DFSTestUtil.verifyFilesNotEqual(fs, zoneFile, rawFile, len);
+  }
 }

+ 9 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestStoragePolicyCommands.java

@@ -21,6 +21,8 @@ import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -60,6 +62,11 @@ public class TestStoragePolicyCommands {
     final Path bar = new Path(foo, "bar");
     DFSTestUtil.createFile(fs, bar, SIZE, REPL, 0);
 
+    DFSTestUtil.DFSAdminRun("-getStoragePolicy /foo", 0,
+        "The storage policy of " + foo.toString() + " is unspecified", conf);
+    DFSTestUtil.DFSAdminRun("-getStoragePolicy /foo/bar", 0,
+        "The storage policy of " + bar.toString() + " is unspecified", conf);
+
     DFSTestUtil.DFSAdminRun("-setStoragePolicy /foo WARM", 0,
         "Set storage policy WARM on " + foo.toString(), conf);
     DFSTestUtil.DFSAdminRun("-setStoragePolicy /foo/bar COLD", 0,
@@ -67,8 +74,8 @@ public class TestStoragePolicyCommands {
     DFSTestUtil.DFSAdminRun("-setStoragePolicy /fooz WARM", -1,
         "File/Directory does not exist: /fooz", conf);
 
-    final BlockStoragePolicy.Suite suite = BlockStoragePolicy
-        .readBlockStorageSuite(conf);
+    final BlockStoragePolicySuite suite = BlockStoragePolicySuite
+        .createDefaultSuite();
     final BlockStoragePolicy warm = suite.getPolicy("WARM");
     final BlockStoragePolicy cold = suite.getPolicy("COLD");
     DFSTestUtil.DFSAdminRun("-getStoragePolicy /foo", 0,

+ 14 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java

@@ -36,7 +36,7 @@ import org.apache.hadoop.conf.ReconfigurationException;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSOutputStream;
 import org.apache.hadoop.hdfs.DFSTestUtil;
@@ -56,6 +56,7 @@ import org.apache.hadoop.hdfs.server.balancer.Dispatcher;
 import org.apache.hadoop.hdfs.server.balancer.ExitStatus;
 import org.apache.hadoop.hdfs.server.balancer.TestBalancer;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
@@ -87,7 +88,7 @@ public class TestStorageMover {
   private static final short REPL = 3;
   private static final int NUM_DATANODES = 6;
   private static final Configuration DEFAULT_CONF = new HdfsConfiguration();
-  private static final BlockStoragePolicy.Suite DEFAULT_POLICIES;
+  private static final BlockStoragePolicySuite DEFAULT_POLICIES;
   private static final BlockStoragePolicy HOT;
   private static final BlockStoragePolicy WARM;
   private static final BlockStoragePolicy COLD;
@@ -99,7 +100,7 @@ public class TestStorageMover {
         2L);
     DEFAULT_CONF.setLong(DFSConfigKeys.DFS_MOVER_MOVEDWINWIDTH_KEY, 2000L);
 
-    DEFAULT_POLICIES = BlockStoragePolicy.readBlockStorageSuite(DEFAULT_CONF);
+    DEFAULT_POLICIES = BlockStoragePolicySuite.createDefaultSuite();
     HOT = DEFAULT_POLICIES.getPolicy("HOT");
     WARM = DEFAULT_POLICIES.getPolicy("WARM");
     COLD = DEFAULT_POLICIES.getPolicy("COLD");
@@ -192,13 +193,21 @@ public class TestStorageMover {
 
     private MiniDFSCluster cluster;
     private DistributedFileSystem dfs;
-    private final BlockStoragePolicy.Suite policies;
+    private final BlockStoragePolicySuite policies;
 
     MigrationTest(ClusterScheme cScheme, NamespaceScheme nsScheme) {
       this.clusterScheme = cScheme;
       this.nsScheme = nsScheme;
       this.conf = clusterScheme.conf;
-      this.policies = BlockStoragePolicy.readBlockStorageSuite(conf);
+      this.policies = DEFAULT_POLICIES;
+    }
+
+    MigrationTest(ClusterScheme cScheme, NamespaceScheme nsScheme,
+        BlockStoragePolicySuite policies) {
+      this.clusterScheme = cScheme;
+      this.nsScheme = nsScheme;
+      this.conf = clusterScheme.conf;
+      this.policies = policies;
     }
 
     /**

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeleteRace.java

@@ -29,7 +29,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.AppendTestUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;

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

@@ -367,6 +367,9 @@ Release 2.6.0 - UNRELEASED
     MAPREDUCE-6095. Enable DistributedCache for uber-mode Jobs (Gera Shegalov
     via jlowe)
 
+    MAPREDUCE-6104. TestJobHistoryParsing.testPartialJob fails in branch-2
+    (Mit Desai via jlowe)
+
 Release 2.5.1 - 2014-09-05
 
   INCOMPATIBLE CHANGES

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryParsing.java

@@ -810,7 +810,7 @@ public class TestJobHistoryParsing {
   /**
    * Simple test PartialJob
    */
-  @Test(timeout = 1000)
+  @Test(timeout = 3000)
   public void testPartialJob() throws Exception {
     JobId jobId = new JobIdPBImpl();
     jobId.setId(0);

+ 1 - 1
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/CopyListingFileStatus.java

@@ -101,7 +101,7 @@ public final class CopyListingFileStatus extends FileStatus {
    * @return Map<String, byte[]> containing all xAttrs
    */
   public Map<String, byte[]> getXAttrs() {
-    return xAttrs;
+    return xAttrs != null ? xAttrs : Collections.<String, byte[]>emptyMap();
   }
   
   /**

+ 12 - 9
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java

@@ -37,18 +37,21 @@ public enum DistCpOptionSwitch {
   /**
    * Preserves status of file/path in the target.
    * Default behavior with -p, is to preserve replication,
-   * block size, user, group, permission and checksum type on the target file.
-   * Note that when preserving checksum type, block size is also preserved.
+   * block size, user, group, permission, checksum type and timestamps on the 
+   * target file. Note that when preserving checksum type, block size is also 
+   * preserved.
    *
-   * If any of the optional switches are present among rbugpc, then
-   * only the corresponding file attribute is preserved.
+   * @see PRESERVE_STATUS_DEFAULT
    *
+   * If any of the optional switches are present among rbugpcaxt, then
+   * only the corresponding file attribute is preserved.
    */
   PRESERVE_STATUS(DistCpConstants.CONF_LABEL_PRESERVE_STATUS,
-      new Option("p", true, "preserve status (rbugpcax)(replication, " +
-          "block-size, user, group, permission, checksum-type, ACL, XATTR).  " +
-          "If -p is specified with no <arg>, then preserves replication, " +
-          "block size, user, group, permission and checksum type." +
+      new Option("p", true, "preserve status (rbugpcaxt)(replication, " +
+          "block-size, user, group, permission, checksum-type, ACL, XATTR, " +
+          "timestamps). If -p is specified with no <arg>, then preserves " +
+          "replication, block size, user, group, permission, checksum type " +
+          "and timestamps. " +
           "raw.* xattrs are preserved when both the source and destination " +
           "paths are in the /.reserved/raw hierarchy (HDFS only). raw.* xattr" +
           "preservation is independent of the -p flag." +
@@ -166,7 +169,7 @@ public enum DistCpOptionSwitch {
   BANDWIDTH(DistCpConstants.CONF_LABEL_BANDWIDTH_MB,
       new Option("bandwidth", true, "Specify bandwidth per map in MB"));
 
-  static final String PRESERVE_STATUS_DEFAULT = "-prbugpc";
+  public static final String PRESERVE_STATUS_DEFAULT = "-prbugpct";
   private final String confLabel;
   private final Option option;
 

+ 1 - 1
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java

@@ -68,7 +68,7 @@ public class DistCpOptions {
   private boolean targetPathExists = true;
   
   public static enum FileAttribute{
-    REPLICATION, BLOCKSIZE, USER, GROUP, PERMISSION, CHECKSUMTYPE, ACL, XATTR;
+    REPLICATION, BLOCKSIZE, USER, GROUP, PERMISSION, CHECKSUMTYPE, ACL, XATTR, TIMES;
 
     public static FileAttribute getAttribute(char symbol) {
       for (FileAttribute attribute : values()) {

+ 31 - 27
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java

@@ -18,39 +18,39 @@
 
 package org.apache.hadoop.tools.util;
 
-import com.google.common.collect.Maps;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.URI;
+import java.net.UnknownHostException;
+import java.text.DecimalFormat;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Map.Entry;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclUtil;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.tools.CopyListing.AclsNotSupportedException;
 import org.apache.hadoop.tools.CopyListing.XAttrsNotSupportedException;
 import org.apache.hadoop.tools.CopyListingFileStatus;
+import org.apache.hadoop.tools.DistCpOptions;
 import org.apache.hadoop.tools.DistCpOptions.FileAttribute;
 import org.apache.hadoop.tools.mapred.UniformSizeInputFormat;
-import org.apache.hadoop.tools.CopyListing.AclsNotSupportedException;
-import org.apache.hadoop.tools.DistCpOptions;
-import org.apache.hadoop.mapreduce.InputFormat;
 
-import java.io.IOException;
-import java.util.EnumSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.text.DecimalFormat;
-import java.net.URI;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
+import com.google.common.collect.Maps;
 
 /**
  * Utility functions used in DistCp.
@@ -163,7 +163,7 @@ public class DistCpUtils {
   }
 
   /**
-   * Un packs preservation attribute string containing the first character of
+   * Unpacks preservation attribute string containing the first character of
    * each preservation attribute back to a set of attributes to preserve
    * @param attributes - Attribute string
    * @return - Attribute set
@@ -209,7 +209,7 @@ public class DistCpUtils {
       if (!srcAcl.equals(targetAcl)) {
         targetFS.setAcl(path, srcAcl);
       }
-      // setAcl can't preserve sticky bit, so also call setPermission if needed.
+      // setAcl doesn't preserve sticky bit, so also call setPermission if needed.
       if (srcFileStatus.getPermission().getStickyBit() !=
           targetFileStatus.getPermission().getStickyBit()) {
         targetFS.setPermission(path, srcFileStatus.getPermission());
@@ -225,30 +225,28 @@ public class DistCpUtils {
       Map<String, byte[]> srcXAttrs = srcFileStatus.getXAttrs();
       Map<String, byte[]> targetXAttrs = getXAttrs(targetFS, path);
       if (srcXAttrs != null && !srcXAttrs.equals(targetXAttrs)) {
-        Iterator<Entry<String, byte[]>> iter = srcXAttrs.entrySet().iterator();
-        while (iter.hasNext()) {
-          Entry<String, byte[]> entry = iter.next();
-          final String xattrName = entry.getKey();
+        for (Entry<String, byte[]> entry : srcXAttrs.entrySet()) {
+          String xattrName = entry.getKey();
           if (xattrName.startsWith(rawNS) || preserveXAttrs) {
-            targetFS.setXAttr(path, entry.getKey(), entry.getValue());
+            targetFS.setXAttr(path, xattrName, entry.getValue());
           }
         }
       }
     }
 
-    if (attributes.contains(FileAttribute.REPLICATION) && ! targetFileStatus.isDirectory() &&
-        srcFileStatus.getReplication() != targetFileStatus.getReplication()) {
+    if (attributes.contains(FileAttribute.REPLICATION) && !targetFileStatus.isDirectory() &&
+        (srcFileStatus.getReplication() != targetFileStatus.getReplication())) {
       targetFS.setReplication(path, srcFileStatus.getReplication());
     }
 
     if (attributes.contains(FileAttribute.GROUP) &&
-            !group.equals(srcFileStatus.getGroup())) {
+        !group.equals(srcFileStatus.getGroup())) {
       group = srcFileStatus.getGroup();
       chown = true;
     }
 
     if (attributes.contains(FileAttribute.USER) &&
-            !user.equals(srcFileStatus.getOwner())) {
+        !user.equals(srcFileStatus.getOwner())) {
       user = srcFileStatus.getOwner();
       chown = true;
     }
@@ -256,6 +254,12 @@ public class DistCpUtils {
     if (chown) {
       targetFS.setOwner(path, user, group);
     }
+    
+    if (attributes.contains(FileAttribute.TIMES)) {
+      targetFS.setTimes(path, 
+          srcFileStatus.getModificationTime(), 
+          srcFileStatus.getAccessTime());
+    }
   }
 
   /**

+ 1 - 1
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestOptionsParser.java

@@ -497,7 +497,7 @@ public class TestOptionsParser {
       attribIterator.next();
       i++;
     }
-    Assert.assertEquals(i, 6);
+    Assert.assertEquals(i, DistCpOptionSwitch.PRESERVE_STATUS_DEFAULT.length() - 2);
 
     try {
       OptionsParser.parse(new String[] {

+ 1 - 0
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java

@@ -590,6 +590,7 @@ public class TestCopyMapper {
           EnumSet.allOf(DistCpOptions.FileAttribute.class);
       preserveStatus.remove(DistCpOptions.FileAttribute.ACL);
       preserveStatus.remove(DistCpOptions.FileAttribute.XATTR);
+      preserveStatus.remove(DistCpOptions.FileAttribute.TIMES);
 
       context.getConfiguration().set(DistCpConstants.CONF_LABEL_PRESERVE_STATUS,
         DistCpUtils.packAttributes(preserveStatus));

Những thai đổi đã bị hủy bỏ vì nó quá lớn
+ 926 - 71
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/util/TestDistCpUtils.java


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

@@ -94,6 +94,12 @@ Release 2.6.0 - UNRELEASED
     YARN-2569. Added the log handling APIs for the long running services. (Xuan
     Gong via zjshen)
 
+    YARN-2102. Added the concept of a Timeline Domain to handle read/write ACLs
+    on Timeline service event data. (Zhijie Shen via vinodkv)
+
+    YARN-2581. Passed LogAggregationContext to NM via ContainerTokenIdentifier.
+    (Xuan Gong via zjshen)
+
   IMPROVEMENTS
 
     YARN-2197. Add a link to YARN CHANGES.txt in the left side of doc
@@ -434,6 +440,13 @@ Release 2.6.0 - UNRELEASED
     TestFairScheduler.testContinuousScheduling. 
     (Ratandeep Ratti and kasha via kasha)
 
+    YARN-2161. Fix build on macosx: YARN parts (Binglin Chang via aw)
+
+    YARN-2596. TestWorkPreservingRMRestart fails with FairScheduler. (kasha)
+
+    YARN-2546. Made REST API for application creation/submission use numeric and
+    boolean types instead of the string of them. (Varun Vasudev via zjshen)
+
 Release 2.5.1 - 2014-09-05
 
   INCOMPATIBLE CHANGES

+ 194 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineDomain.java

@@ -0,0 +1,194 @@
+/**
+ * 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.api.records.timeline;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * <p>
+ * This class contains the information about a timeline domain, which is used
+ * to a user to host a number of timeline entities, isolating them from others'.
+ * The user can also define the reader and writer users/groups for the the
+ * domain, which is used to control the access to its entities.
+ * </p>
+ * 
+ * <p>
+ * The reader and writer users/groups pattern that the user can supply is the
+ * same as what <code>AccessControlList</code> takes.
+ * </p>
+ * 
+ */
+@XmlRootElement(name = "domain")
+@XmlAccessorType(XmlAccessType.NONE)
+@Public
+@Unstable
+public class TimelineDomain {
+
+  private String id;
+  private String description;
+  private String owner;
+  private String readers;
+  private String writers;
+  private Long createdTime;
+  private Long modifiedTime;
+
+  public TimelineDomain() {
+  }
+
+  /**
+   * Get the domain ID
+   * 
+   * @return the domain ID
+   */
+  @XmlElement(name = "id")
+  public String getId() {
+    return id;
+  }
+
+  /**
+   * Set the domain ID
+   * 
+   * @param id the domain ID
+   */
+  public void setId(String id) {
+    this.id = id;
+  }
+
+  /**
+   * Get the domain description
+   * 
+   * @return the domain description
+   */
+  @XmlElement(name = "description")
+  public String getDescription() {
+    return description;
+  }
+
+  /**
+   * Set the domain description
+   * 
+   * @param description the domain description
+   */
+  public void setDescription(String description) {
+    this.description = description;
+  }
+
+  /**
+   * Get the domain owner
+   * 
+   * @return the domain owner
+   */
+  @XmlElement(name = "owner")
+  public String getOwner() {
+    return owner;
+  }
+
+  /**
+   * Set the domain owner. The user doesn't need to set it, which will
+   * automatically set to the user who puts the domain.
+   * 
+   * @param owner the domain owner
+   */
+  public void setOwner(String owner) {
+    this.owner = owner;
+  }
+
+  /**
+   * Get the reader (and/or reader group) list string
+   * 
+   * @return the reader (and/or reader group) list string
+   */
+  @XmlElement(name = "readers")
+  public String getReaders() {
+    return readers;
+  }
+
+  /**
+   * Set the reader (and/or reader group) list string
+   * 
+   * @param readers the reader (and/or reader group) list string
+   */
+  public void setReaders(String readers) {
+    this.readers = readers;
+  }
+
+  /**
+   * Get the writer (and/or writer group) list string
+   * 
+   * @return the writer (and/or writer group) list string
+   */
+  @XmlElement(name = "writers")
+  public String getWriters() {
+    return writers;
+  }
+
+  /**
+   * Set the writer (and/or writer group) list string
+   * 
+   * @param writers the writer (and/or writer group) list string
+   */
+  public void setWriters(String writers) {
+    this.writers = writers;
+  }
+
+  /**
+   * Get the created time of the domain
+   * 
+   * @return the created time of the domain
+   */
+  @XmlElement(name = "createdtime")
+  public Long getCreatedTime() {
+    return createdTime;
+  }
+
+  /**
+   * Set the created time of the domain
+   * 
+   * @param createdTime the created time of the domain
+   */
+  public void setCreatedTime(Long createdTime) {
+    this.createdTime = createdTime;
+  }
+
+  /**
+   * Get the modified time of the domain
+   * 
+   * @return the modified time of the domain
+   */
+  @XmlElement(name = "modifiedtime")
+  public Long getModifiedTime() {
+    return modifiedTime;
+  }
+
+  /**
+   * Set the modified time of the domain
+   * 
+   * @param modifiedTime the modified time of the domain
+   */
+  public void setModifiedTime(Long modifiedTime) {
+    this.modifiedTime = modifiedTime;
+  }
+
+}

+ 86 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineDomains.java

@@ -0,0 +1,86 @@
+/**
+ * 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.api.records.timeline;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * The class that hosts a list of timeline domains.
+ */
+@XmlRootElement(name = "domains")
+@XmlAccessorType(XmlAccessType.NONE)
+@Public
+@Unstable
+public class TimelineDomains {
+
+  private List<TimelineDomain> domains = new ArrayList<TimelineDomain>();
+
+  public TimelineDomains() {
+  }
+
+  /**
+   * Get a list of domains
+   * 
+   * @return a list of domains
+   */
+  @XmlElement(name = "domains")
+  public List<TimelineDomain> getDomains() {
+    return domains;
+  }
+
+  /**
+   * Add a single domain into the existing domain list
+   * 
+   * @param domain
+   *          a single domain
+   */
+  public void addDomain(TimelineDomain domain) {
+    domains.add(domain);
+  }
+
+  /**
+   * All a list of domains into the existing domain list
+   * 
+   * @param domains
+   *          a list of domains
+   */
+  public void addDomains(List<TimelineDomain> domains) {
+    this.domains.addAll(domains);
+  }
+
+  /**
+   * Set the domain list to the given list of domains
+   * 
+   * @param domains
+   *          a list of domains
+   */
+  public void setDomains(List<TimelineDomain> domains) {
+    this.domains = domains;
+  }
+
+}

+ 17 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -67,6 +68,22 @@ public abstract class TimelineClient extends AbstractService {
   public abstract TimelinePutResponse putEntities(
       TimelineEntity... entities) throws IOException, YarnException;
 
+  /**
+   * <p>
+   * Send the information of a domain to the timeline server. It is a
+   * blocking API. The method will not return until it gets the response from
+   * the timeline server.
+   * </p>
+   * 
+   * @param domain
+   *          an {@link TimelineDomain} object
+   * @throws IOException
+   * @throws YarnException
+   */
+  @Public
+  public abstract void putDomain(
+      TimelineDomain domain) throws IOException, YarnException;
+
   /**
    * <p>
    * Get a delegation token so as to be able to talk to the timeline server in a

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

@@ -50,6 +50,8 @@ import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
 import org.apache.hadoop.security.ssl.SSLFactory;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineDomains;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
@@ -84,11 +86,15 @@ public class TimelineClientImpl extends TimelineClient {
   public final static int DEFAULT_SOCKET_TIMEOUT = 1 * 60 * 1000; // 1 minute
 
   private static Options opts;
+  private static final String ENTITY_DATA_TYPE = "entity";
+  private static final String DOMAIN_DATA_TYPE = "domain";
 
   static {
     opts = new Options();
-    opts.addOption("put", true, "Put the TimelineEntities in a JSON file");
+    opts.addOption("put", true, "Put the timeline entities/domain in a JSON file");
     opts.getOption("put").setArgName("Path to the JSON file");
+    opts.addOption(ENTITY_DATA_TYPE, false, "Specify the JSON file contains the entities");
+    opts.addOption(DOMAIN_DATA_TYPE, false, "Specify the JSON file contains the domain");
     opts.addOption("help", false, "Print usage");
   }
 
@@ -150,9 +156,27 @@ public class TimelineClientImpl extends TimelineClient {
     }
     TimelineEntities entitiesContainer = new TimelineEntities();
     entitiesContainer.addEntities(Arrays.asList(entities));
+    ClientResponse resp = doPosting(entitiesContainer, null);
+    return resp.getEntity(TimelinePutResponse.class);
+  }
+
+
+  @Override
+  public void putDomain(TimelineDomain domain) throws IOException,
+      YarnException {
+    if (!isEnabled) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Nothing will be put because timeline service is not enabled");
+      }
+      return;
+    }
+    doPosting(domain, "domain");
+  }
+
+  private ClientResponse doPosting(Object obj, String path) throws IOException, YarnException {
     ClientResponse resp;
     try {
-      resp = doPostingEntities(entitiesContainer);
+      resp = doPostingObject(obj, path);
     } catch (RuntimeException re) {
       // runtime exception is expected if the client cannot connect the server
       String msg =
@@ -172,7 +196,7 @@ public class TimelineClientImpl extends TimelineClient {
       }
       throw new YarnException(msg);
     }
-    return resp.getEntity(TimelinePutResponse.class);
+    return resp;
   }
 
   @Override
@@ -184,11 +208,22 @@ public class TimelineClientImpl extends TimelineClient {
 
   @Private
   @VisibleForTesting
-  public ClientResponse doPostingEntities(TimelineEntities entities) {
+  public ClientResponse doPostingObject(Object object, String path) {
     WebResource webResource = client.resource(resURI);
-    return webResource.accept(MediaType.APPLICATION_JSON)
-        .type(MediaType.APPLICATION_JSON)
-        .post(ClientResponse.class, entities);
+    if (path != null) {
+      webResource.path(path);
+    }
+    if (path == null) {
+      return webResource.accept(MediaType.APPLICATION_JSON)
+          .type(MediaType.APPLICATION_JSON)
+          .post(ClientResponse.class, object);
+    } else if (path.equals("domain")) {
+      return webResource.path(path).accept(MediaType.APPLICATION_JSON)
+          .type(MediaType.APPLICATION_JSON)
+          .put(ClientResponse.class, object);
+    } else {
+      throw new YarnRuntimeException("Unknown resource type");
+    }
   }
 
   private static class PseudoAuthenticatedURLConnectionFactory
@@ -334,8 +369,13 @@ public class TimelineClientImpl extends TimelineClient {
     if (cliParser.hasOption("put")) {
       String path = cliParser.getOptionValue("put");
       if (path != null && path.length() > 0) {
-        putTimelineEntitiesInJSONFile(path);
-        return;
+        if (cliParser.hasOption(ENTITY_DATA_TYPE)) {
+          putTimelineDataInJSONFile(path, ENTITY_DATA_TYPE);
+          return;
+        } else if (cliParser.hasOption(DOMAIN_DATA_TYPE)) {
+          putTimelineDataInJSONFile(path, DOMAIN_DATA_TYPE);
+          return;
+        }
       }
     }
     printUsage();
@@ -345,22 +385,28 @@ public class TimelineClientImpl extends TimelineClient {
    * Put timeline data in a JSON file via command line.
    * 
    * @param path
-   *          path to the {@link TimelineEntities} JSON file
+   *          path to the timeline data JSON file
+   * @param type
+   *          the type of the timeline data in the JSON file
    */
-  private static void putTimelineEntitiesInJSONFile(String path) {
+  private static void putTimelineDataInJSONFile(String path, String type) {
     File jsonFile = new File(path);
     if (!jsonFile.exists()) {
-      System.out.println("Error: File [" + jsonFile.getAbsolutePath()
-          + "] doesn't exist");
+      LOG.error("File [" + jsonFile.getAbsolutePath() + "] doesn't exist");
       return;
     }
     ObjectMapper mapper = new ObjectMapper();
     YarnJacksonJaxbJsonProvider.configObjectMapper(mapper);
     TimelineEntities entities = null;
+    TimelineDomains domains = null;
     try {
-      entities = mapper.readValue(jsonFile, TimelineEntities.class);
+      if (type.equals(ENTITY_DATA_TYPE)) {
+        entities = mapper.readValue(jsonFile, TimelineEntities.class);
+      } else if (type.equals(DOMAIN_DATA_TYPE)){
+        domains = mapper.readValue(jsonFile, TimelineDomains.class);
+      }
     } catch (Exception e) {
-      System.err.println("Error: " + e.getMessage());
+      LOG.error("Error when reading  " + e.getMessage());
       e.printStackTrace(System.err);
       return;
     }
@@ -376,21 +422,37 @@ public class TimelineClientImpl extends TimelineClient {
                 UserGroupInformation.getCurrentUser().getUserName());
         UserGroupInformation.getCurrentUser().addToken(token);
       }
-      TimelinePutResponse response = client.putEntities(
-          entities.getEntities().toArray(
-              new TimelineEntity[entities.getEntities().size()]));
-      if (response.getErrors().size() == 0) {
-        System.out.println("Timeline data is successfully put");
-      } else {
-        for (TimelinePutResponse.TimelinePutError error : response.getErrors()) {
-          System.out.println("TimelineEntity [" + error.getEntityType() + ":" +
-              error.getEntityId() + "] is not successfully put. Error code: " +
-              error.getErrorCode());
+      if (type.equals(ENTITY_DATA_TYPE)) {
+        TimelinePutResponse response = client.putEntities(
+            entities.getEntities().toArray(
+                new TimelineEntity[entities.getEntities().size()]));
+        if (response.getErrors().size() == 0) {
+          LOG.info("Timeline entities are successfully put");
+        } else {
+          for (TimelinePutResponse.TimelinePutError error : response.getErrors()) {
+            LOG.error("TimelineEntity [" + error.getEntityType() + ":" +
+                error.getEntityId() + "] is not successfully put. Error code: " +
+                error.getErrorCode());
+          }
+        }
+      } else if (type.equals(DOMAIN_DATA_TYPE)) {
+        boolean hasError = false;
+        for (TimelineDomain domain : domains.getDomains()) {
+          try {
+            client.putDomain(domain);
+          } catch (Exception e) {
+            LOG.error("Error when putting domain " + domain.getId(), e);
+            hasError = true;
+          }
+        }
+        if (!hasError) {
+          LOG.info("Timeline domains are successfully put");
         }
       }
+    } catch(RuntimeException e) {
+      LOG.error("Error when putting the timeline data", e);
     } catch (Exception e) {
-      System.err.println("Error: " + e.getMessage());
-      e.printStackTrace(System.err);
+      LOG.error("Error when putting the timeline data", e);
     } finally {
       client.stop();
     }

+ 34 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java

@@ -34,8 +34,11 @@ import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.impl.pb.LogAggregationContextPBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.LogAggregationContextProto;
 
 /**
  * TokenIdentifier for a container. Encodes {@link ContainerId},
@@ -59,10 +62,19 @@ public class ContainerTokenIdentifier extends TokenIdentifier {
   private long rmIdentifier;
   private Priority priority;
   private long creationTime;
+  private LogAggregationContext logAggregationContext;
 
   public ContainerTokenIdentifier(ContainerId containerID,
       String hostName, String appSubmitter, Resource r, long expiryTimeStamp,
       int masterKeyId, long rmIdentifier, Priority priority, long creationTime) {
+    this(containerID, hostName, appSubmitter, r, expiryTimeStamp, masterKeyId,
+        rmIdentifier, priority, creationTime, null);
+  }
+
+  public ContainerTokenIdentifier(ContainerId containerID, String hostName,
+      String appSubmitter, Resource r, long expiryTimeStamp, int masterKeyId,
+      long rmIdentifier, Priority priority, long creationTime,
+      LogAggregationContext logAggregationContext) {
     this.containerId = containerID;
     this.nmHostAddr = hostName;
     this.appSubmitter = appSubmitter;
@@ -72,6 +84,7 @@ public class ContainerTokenIdentifier extends TokenIdentifier {
     this.rmIdentifier = rmIdentifier;
     this.priority = priority;
     this.creationTime = creationTime;
+    this.logAggregationContext = logAggregationContext;
   }
 
   /**
@@ -119,6 +132,10 @@ public class ContainerTokenIdentifier extends TokenIdentifier {
     return this.rmIdentifier;
   }
 
+  public LogAggregationContext getLogAggregationContext() {
+    return this.logAggregationContext;
+  }
+
   @Override
   public void write(DataOutput out) throws IOException {
     LOG.debug("Writing ContainerTokenIdentifier to RPC layer: " + this);
@@ -138,6 +155,15 @@ public class ContainerTokenIdentifier extends TokenIdentifier {
     out.writeLong(this.rmIdentifier);
     out.writeInt(this.priority.getPriority());
     out.writeLong(this.creationTime);
+    if (this.logAggregationContext == null) {
+      out.writeInt(-1);
+    } else {
+      byte[] logAggregationContext =
+          ((LogAggregationContextPBImpl) this.logAggregationContext).getProto()
+            .toByteArray();
+      out.writeInt(logAggregationContext.length);
+      out.write(logAggregationContext);
+    }
   }
 
   @Override
@@ -158,6 +184,14 @@ public class ContainerTokenIdentifier extends TokenIdentifier {
     this.rmIdentifier = in.readLong();
     this.priority = Priority.newInstance(in.readInt());
     this.creationTime = in.readLong();
+    int size = in.readInt();
+    if (size != -1) {
+      byte[] bytes = new byte[size];
+      in.readFully(bytes);
+      this.logAggregationContext =
+          new LogAggregationContextPBImpl(
+            LogAggregationContextProto.parseFrom(bytes));
+    }
   }
 
   @Override

+ 38 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/timeline/TestTimelineRecords.java

@@ -161,4 +161,42 @@ public class TestTimelineRecords {
     Assert.assertEquals(error2.getErrorCode(), e.getErrorCode());
   }
 
+  @Test
+  public void testTimelineDomain() throws Exception {
+    TimelineDomains domains = new TimelineDomains();
+
+    TimelineDomain domain = null;
+    for (int i = 0; i < 2; ++i) {
+      domain = new TimelineDomain();
+      domain.setId("test id " + (i + 1));
+      domain.setDescription("test description " + (i + 1));
+      domain.setOwner("test owner " + (i + 1));
+      domain.setReaders("test_reader_user_" + (i + 1) +
+          " test_reader_group+" + (i + 1));
+      domain.setWriters("test_writer_user_" + (i + 1) +
+          " test_writer_group+" + (i + 1));
+      domain.setCreatedTime(0L);
+      domain.setModifiedTime(1L);
+      domains.addDomain(domain);
+    }
+    LOG.info("Domain in JSON:");
+    LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(domains, true));
+
+    Assert.assertEquals(2, domains.getDomains().size());
+
+    for (int i = 0; i < domains.getDomains().size(); ++i) {
+      domain = domains.getDomains().get(i);
+      Assert.assertEquals("test id " + (i + 1), domain.getId());
+      Assert.assertEquals("test description " + (i + 1),
+          domain.getDescription());
+      Assert.assertEquals("test owner " + (i + 1), domain.getOwner());
+      Assert.assertEquals("test_reader_user_" + (i + 1) +
+          " test_reader_group+" + (i + 1), domain.getReaders());
+      Assert.assertEquals("test_writer_user_" + (i + 1) +
+          " test_writer_group+" + (i + 1), domain.getWriters());
+      Assert.assertEquals(new Long(0L), domain.getCreatedTime());
+      Assert.assertEquals(new Long(1L), domain.getModifiedTime());
+    }
+  }
+
 }

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

@@ -27,16 +27,16 @@ import static org.mockito.Mockito.when;
 
 import java.net.ConnectException;
 
-import org.junit.Assert;
-
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.client.api.TimelineClient;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.junit.After;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -63,7 +63,7 @@ public class TestTimelineClient {
 
   @Test
   public void testPostEntities() throws Exception {
-    mockClientResponse(client, ClientResponse.Status.OK, false, false);
+    mockEntityClientResponse(client, ClientResponse.Status.OK, false, false);
     try {
       TimelinePutResponse response = client.putEntities(generateEntity());
       Assert.assertEquals(0, response.getErrors().size());
@@ -74,7 +74,7 @@ public class TestTimelineClient {
 
   @Test
   public void testPostEntitiesWithError() throws Exception {
-    mockClientResponse(client, ClientResponse.Status.OK, true, false);
+    mockEntityClientResponse(client, ClientResponse.Status.OK, true, false);
     try {
       TimelinePutResponse response = client.putEntities(generateEntity());
       Assert.assertEquals(1, response.getErrors().size());
@@ -91,7 +91,7 @@ public class TestTimelineClient {
 
   @Test
   public void testPostEntitiesNoResponse() throws Exception {
-    mockClientResponse(
+    mockEntityClientResponse(
         client, ClientResponse.Status.INTERNAL_SERVER_ERROR, false, false);
     try {
       client.putEntities(generateEntity());
@@ -104,7 +104,7 @@ public class TestTimelineClient {
 
   @Test
   public void testPostEntitiesConnectionRefused() throws Exception {
-    mockClientResponse(client, null, false, true);
+    mockEntityClientResponse(client, null, false, true);
     try {
       client.putEntities(generateEntity());
       Assert.fail("RuntimeException is expected");
@@ -118,7 +118,7 @@ public class TestTimelineClient {
     YarnConfiguration conf = new YarnConfiguration();
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, false);
     TimelineClientImpl client = createTimelineClient(conf);
-    mockClientResponse(
+    mockEntityClientResponse(
         client, ClientResponse.Status.INTERNAL_SERVER_ERROR, false, false);
     try {
       TimelinePutResponse response = client.putEntities(generateEntity());
@@ -137,7 +137,7 @@ public class TestTimelineClient {
     // Make sure default value is pickup up
     conf.unset(YarnConfiguration.TIMELINE_SERVICE_ENABLED);
     TimelineClientImpl client = createTimelineClient(conf);
-    mockClientResponse(client, ClientResponse.Status.INTERNAL_SERVER_ERROR,
+    mockEntityClientResponse(client, ClientResponse.Status.INTERNAL_SERVER_ERROR,
         false, false);
     try {
       TimelinePutResponse response = client.putEntities(generateEntity());
@@ -148,16 +148,50 @@ public class TestTimelineClient {
     }
   }
 
-  private static ClientResponse mockClientResponse(TimelineClientImpl client,
-      ClientResponse.Status status, boolean hasError, boolean hasRuntimeError) {
+  @Test
+  public void testPutDomain() throws Exception {
+    mockDomainClientResponse(client, ClientResponse.Status.OK, false);
+    try {
+      client.putDomain(generateDomain());
+    } catch (YarnException e) {
+      Assert.fail("Exception is not expected");
+    }
+  }
+
+  @Test
+  public void testPutDomainNoResponse() throws Exception {
+    mockDomainClientResponse(client, ClientResponse.Status.FORBIDDEN, false);
+    try {
+      client.putDomain(generateDomain());
+      Assert.fail("Exception is expected");
+    } catch (YarnException e) {
+      Assert.assertTrue(e.getMessage().contains(
+          "Failed to get the response from the timeline server."));
+    }
+  }
+
+  @Test
+  public void testPutDomainConnectionRefused() throws Exception {
+    mockDomainClientResponse(client, null, true);
+    try {
+      client.putDomain(generateDomain());
+      Assert.fail("RuntimeException is expected");
+    } catch (RuntimeException re) {
+      Assert.assertTrue(re instanceof ClientHandlerException);
+    }
+  }
+
+  private static ClientResponse mockEntityClientResponse(
+      TimelineClientImpl client, ClientResponse.Status status,
+      boolean hasError, boolean hasRuntimeError) {
     ClientResponse response = mock(ClientResponse.class);
     if (hasRuntimeError) {
       doThrow(new ClientHandlerException(new ConnectException())).when(client)
-          .doPostingEntities(any(TimelineEntities.class));
+          .doPostingObject(any(TimelineEntities.class), any(String.class));
       return response;
     }
     doReturn(response).when(client)
-        .doPostingEntities(any(TimelineEntities.class));
+        .doPostingObject(any(TimelineEntities.class), any(String.class));
     when(response.getClientResponseStatus()).thenReturn(status);
     TimelinePutResponse.TimelinePutError error =
         new TimelinePutResponse.TimelinePutError();
@@ -172,6 +206,21 @@ public class TestTimelineClient {
     return response;
   }
 
+  private static ClientResponse mockDomainClientResponse(
+      TimelineClientImpl client, ClientResponse.Status status,
+      boolean hasRuntimeError) {
+    ClientResponse response = mock(ClientResponse.class);
+    if (hasRuntimeError) {
+      doThrow(new ClientHandlerException(new ConnectException())).when(client)
+          .doPostingObject(any(TimelineDomain.class), any(String.class));
+      return response;
+    }
+    doReturn(response).when(client)
+        .doPostingObject(any(TimelineDomain.class), any(String.class));
+    when(response.getClientResponseStatus()).thenReturn(status);
+    return response;
+  }
+
   private static TimelineEntity generateEntity() {
     TimelineEntity entity = new TimelineEntity();
     entity.setEntityId("entity id");
@@ -194,6 +243,18 @@ public class TestTimelineClient {
     return entity;
   }
 
+  public static TimelineDomain generateDomain() {
+    TimelineDomain domain = new TimelineDomain();
+    domain.setId("namesapce id");
+    domain.setDescription("domain description");
+    domain.setOwner("domain owner");
+    domain.setReaders("domain_reader");
+    domain.setWriters("domain_writer");
+    domain.setCreatedTime(0L);
+    domain.setModifiedTime(1L);
+    return domain;
+  }
+
   private static TimelineClientImpl createTimelineClient(
       YarnConfiguration conf) {
     TimelineClientImpl client =

+ 215 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java

@@ -58,6 +58,8 @@ import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents.EventsOfOneEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineDomains;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse.TimelinePutError;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -145,6 +147,14 @@ public class LeveldbTimelineStore extends AbstractService
   private static final byte[] INVISIBLE_REVERSE_RELATED_ENTITIES_COLUMN =
       "z".getBytes();
 
+  private static final byte[] DOMAIN_ENTRY_PREFIX = "d".getBytes();
+  private static final byte[] OWNER_LOOKUP_PREFIX = "o".getBytes();
+  private static final byte[] DESCRIPTION_COLUMN = "d".getBytes();
+  private static final byte[] OWNER_COLUMN = "o".getBytes();
+  private static final byte[] READER_COLUMN = "r".getBytes();
+  private static final byte[] WRITER_COLUMN = "w".getBytes();
+  private static final byte[] TIMESTAMP_COLUMN = "t".getBytes();
+
   private static final byte[] EMPTY_BYTES = new byte[0];
   
   private static final String TIMELINE_STORE_VERSION_KEY = "timeline-store-version";
@@ -1558,5 +1568,209 @@ public class LeveldbTimelineStore extends AbstractService
       throw new IOException(incompatibleMessage);
     }
   }
-  
+
+  //TODO: make data retention work with the domain data as well
+  @Override
+  public void put(TimelineDomain domain) throws IOException {
+    WriteBatch writeBatch = null;
+    try {
+      writeBatch = db.createWriteBatch();
+      if (domain.getId() == null || domain.getId().length() == 0) {
+        throw new IllegalArgumentException("Domain doesn't have an ID");
+      }
+      if (domain.getOwner() == null || domain.getOwner().length() == 0) {
+        throw new IllegalArgumentException("Domain doesn't have an owner.");
+      }
+
+      // Write description
+      byte[] domainEntryKey = createDomainEntryKey(
+          domain.getId(), DESCRIPTION_COLUMN);
+      byte[] ownerLookupEntryKey = createOwnerLookupKey(
+          domain.getOwner(), domain.getId(), DESCRIPTION_COLUMN);
+      if (domain.getDescription() != null) {
+        writeBatch.put(domainEntryKey, domain.getDescription().getBytes());
+        writeBatch.put(ownerLookupEntryKey, domain.getDescription().getBytes());
+      } else {
+        writeBatch.put(domainEntryKey, EMPTY_BYTES);
+        writeBatch.put(ownerLookupEntryKey, EMPTY_BYTES);
+      }
+
+      // Write owner
+      domainEntryKey = createDomainEntryKey(domain.getId(), OWNER_COLUMN);
+      ownerLookupEntryKey = createOwnerLookupKey(
+          domain.getOwner(), domain.getId(), OWNER_COLUMN);
+      // Null check for owner is done before
+      writeBatch.put(domainEntryKey, domain.getOwner().getBytes());
+      writeBatch.put(ownerLookupEntryKey, domain.getOwner().getBytes());
+
+      // Write readers
+      domainEntryKey = createDomainEntryKey(domain.getId(), READER_COLUMN);
+      ownerLookupEntryKey = createOwnerLookupKey(
+          domain.getOwner(), domain.getId(), READER_COLUMN);
+      if (domain.getReaders() != null && domain.getReaders().length() > 0) {
+        writeBatch.put(domainEntryKey, domain.getReaders().getBytes());
+        writeBatch.put(ownerLookupEntryKey, domain.getReaders().getBytes());
+      } else {
+        writeBatch.put(domainEntryKey, EMPTY_BYTES);
+        writeBatch.put(ownerLookupEntryKey, EMPTY_BYTES);
+      }
+
+      // Write writers
+      domainEntryKey = createDomainEntryKey(domain.getId(), WRITER_COLUMN);
+      ownerLookupEntryKey = createOwnerLookupKey(
+          domain.getOwner(), domain.getId(), WRITER_COLUMN);
+      if (domain.getWriters() != null && domain.getWriters().length() > 0) {
+        writeBatch.put(domainEntryKey, domain.getWriters().getBytes());
+        writeBatch.put(ownerLookupEntryKey, domain.getWriters().getBytes());
+      } else {
+        writeBatch.put(domainEntryKey, EMPTY_BYTES);
+        writeBatch.put(ownerLookupEntryKey, EMPTY_BYTES);
+      }
+
+      // Write creation time and modification time
+      // We put both timestamps together because they are always retrieved
+      // together, and store them in the same way as we did for the entity's
+      // start time and insert time.
+      domainEntryKey = createDomainEntryKey(domain.getId(), TIMESTAMP_COLUMN);
+      ownerLookupEntryKey = createOwnerLookupKey(
+          domain.getOwner(), domain.getId(), TIMESTAMP_COLUMN);
+      long currentTimestamp = System.currentTimeMillis();
+      byte[] timestamps = db.get(domainEntryKey);
+      if (timestamps == null) {
+        timestamps = new byte[16];
+        writeReverseOrderedLong(currentTimestamp, timestamps, 0);
+        writeReverseOrderedLong(currentTimestamp, timestamps, 8);
+      } else {
+        writeReverseOrderedLong(currentTimestamp, timestamps, 8);
+      }
+      writeBatch.put(domainEntryKey, timestamps);
+      writeBatch.put(ownerLookupEntryKey, timestamps);
+      db.write(writeBatch);
+    } finally {
+      IOUtils.cleanup(LOG, writeBatch);
+    }
+  }
+
+  /**
+   * Creates a domain entity key with column name suffix,
+   * of the form DOMAIN_ENTRY_PREFIX + domain id + column name.
+   */
+  private static byte[] createDomainEntryKey(String domainId,
+      byte[] columnName) throws IOException {
+    return KeyBuilder.newInstance().add(DOMAIN_ENTRY_PREFIX)
+        .add(domainId).add(columnName).getBytes();
+  }
+
+  /**
+   * Creates an owner lookup key with column name suffix,
+   * of the form OWNER_LOOKUP_PREFIX + owner + domain id + column name.
+   */
+  private static byte[] createOwnerLookupKey(
+      String owner, String domainId, byte[] columnName) throws IOException {
+    return KeyBuilder.newInstance().add(OWNER_LOOKUP_PREFIX)
+        .add(owner).add(domainId).add(columnName).getBytes();
+  }
+
+  @Override
+  public TimelineDomain getDomain(String domainId)
+      throws IOException {
+    DBIterator iterator = null;
+    try {
+      byte[] prefix = KeyBuilder.newInstance()
+          .add(DOMAIN_ENTRY_PREFIX).add(domainId).getBytesForLookup();
+      iterator = db.iterator();
+      iterator.seek(prefix);
+      return getTimelineDomain(iterator, domainId, prefix);
+    } finally {
+      IOUtils.cleanup(LOG, iterator);
+    }
+  }
+
+  @Override
+  public TimelineDomains getDomains(String owner)
+      throws IOException {
+    DBIterator iterator = null;
+    try {
+      byte[] prefix = KeyBuilder.newInstance()
+          .add(OWNER_LOOKUP_PREFIX).add(owner).getBytesForLookup();
+      List<TimelineDomain> domains = new ArrayList<TimelineDomain>();
+      for (iterator = db.iterator(), iterator.seek(prefix);
+          iterator.hasNext();) {
+        byte[] key = iterator.peekNext().getKey();
+        if (!prefixMatches(prefix, prefix.length, key)) {
+          break;
+        }
+        // Iterator to parse the rows of an individual domain
+        KeyParser kp = new KeyParser(key, prefix.length);
+        String domainId = kp.getNextString();
+        byte[] prefixExt = KeyBuilder.newInstance().add(OWNER_LOOKUP_PREFIX)
+            .add(owner).add(domainId).getBytesForLookup();
+        TimelineDomain domainToReturn =
+            getTimelineDomain(iterator, domainId, prefixExt);
+        if (domainToReturn != null) {
+          domains.add(domainToReturn);
+        }
+      }
+      // Sort the domains to return
+      Collections.sort(domains, new Comparator<TimelineDomain>() {
+        @Override
+        public int compare(
+            TimelineDomain domain1, TimelineDomain domain2) {
+           int result = domain2.getCreatedTime().compareTo(
+               domain1.getCreatedTime());
+           if (result == 0) {
+             return domain2.getModifiedTime().compareTo(
+                 domain1.getModifiedTime());
+           } else {
+             return result;
+           }
+        }
+      });
+      TimelineDomains domainsToReturn = new TimelineDomains();
+      domainsToReturn.addDomains(domains);
+      return domainsToReturn;
+    } finally {
+      IOUtils.cleanup(LOG, iterator);
+    }
+  }
+
+  private static TimelineDomain getTimelineDomain(
+      DBIterator iterator, String domainId, byte[] prefix) throws IOException {
+    // Iterate over all the rows whose key starts with prefix to retrieve the
+    // domain information.
+    TimelineDomain domain = new TimelineDomain();
+    domain.setId(domainId);
+    boolean noRows = true;
+    for (; iterator.hasNext(); iterator.next()) {
+      byte[] key = iterator.peekNext().getKey();
+      if (!prefixMatches(prefix, prefix.length, key)) {
+        break;
+      }
+      if (noRows) {
+        noRows = false;
+      }
+      byte[] value = iterator.peekNext().getValue();
+      if (value != null && value.length > 0) {
+        if (key[prefix.length] == DESCRIPTION_COLUMN[0]) {
+          domain.setDescription(new String(value));
+        } else if (key[prefix.length] == OWNER_COLUMN[0]) {
+          domain.setOwner(new String(value));
+        } else if (key[prefix.length] == READER_COLUMN[0]) {
+          domain.setReaders(new String(value));
+        } else if (key[prefix.length] == WRITER_COLUMN[0]) {
+          domain.setWriters(new String(value));
+        } else if (key[prefix.length] == TIMESTAMP_COLUMN[0]) {
+          domain.setCreatedTime(readReverseOrderedLong(value, 0));
+          domain.setModifiedTime(readReverseOrderedLong(value, 8));
+        } else {
+          LOG.error("Unrecognized domain column: " + key[prefix.length]);
+        }
+      }
+    }
+    if (noRows) {
+      return null;
+    } else {
+      return domain;
+    }
+  }
 }

+ 99 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/MemoryTimelineStore.java

@@ -18,11 +18,14 @@
 
 package org.apache.hadoop.yarn.server.timeline;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Comparator;
 import java.util.EnumSet;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -40,6 +43,8 @@ import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents.EventsOfOneEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineDomains;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse.TimelinePutError;
 
@@ -61,6 +66,10 @@ public class MemoryTimelineStore
       new HashMap<EntityIdentifier, TimelineEntity>();
   private Map<EntityIdentifier, Long> entityInsertTimes =
       new HashMap<EntityIdentifier, Long>();
+  private Map<String, TimelineDomain> domainsById =
+      new HashMap<String, TimelineDomain>();
+  private Map<String, Set<TimelineDomain>> domainsByOwner =
+      new HashMap<String, Set<TimelineDomain>>();
 
   public MemoryTimelineStore() {
     super(MemoryTimelineStore.class.getName());
@@ -210,6 +219,58 @@ public class MemoryTimelineStore
     return allEvents;
   }
 
+  @Override
+  public TimelineDomain getDomain(String domainId)
+      throws IOException {
+    TimelineDomain domain = domainsById.get(domainId);
+    if (domain == null) {
+      return null;
+    } else {
+      return createTimelineDomain(
+          domain.getId(),
+          domain.getDescription(),
+          domain.getOwner(),
+          domain.getReaders(),
+          domain.getWriters(),
+          domain.getCreatedTime(),
+          domain.getModifiedTime());
+    }
+  }
+
+  @Override
+  public TimelineDomains getDomains(String owner)
+      throws IOException {
+    List<TimelineDomain> domains = new ArrayList<TimelineDomain>();
+    for (TimelineDomain domain : domainsByOwner.get(owner)) {
+      TimelineDomain domainToReturn = createTimelineDomain(
+          domain.getId(),
+          domain.getDescription(),
+          domain.getOwner(),
+          domain.getReaders(),
+          domain.getWriters(),
+          domain.getCreatedTime(),
+          domain.getModifiedTime());
+      domains.add(domainToReturn);
+    }
+    Collections.sort(domains, new Comparator<TimelineDomain>() {
+      @Override
+      public int compare(
+          TimelineDomain domain1, TimelineDomain domain2) {
+         int result = domain2.getCreatedTime().compareTo(
+             domain1.getCreatedTime());
+         if (result == 0) {
+           return domain2.getModifiedTime().compareTo(
+               domain1.getModifiedTime());
+         } else {
+           return result;
+         }
+      }
+    });
+    TimelineDomains domainsToReturn = new TimelineDomains();
+    domainsToReturn.addDomains(domains);
+    return domainsToReturn;
+  }
+
   @Override
   public synchronized TimelinePutResponse put(TimelineEntities data) {
     TimelinePutResponse response = new TimelinePutResponse();
@@ -308,6 +369,44 @@ public class MemoryTimelineStore
     return response;
   }
 
+  public void put(TimelineDomain domain) throws IOException {
+    TimelineDomain domainToReplace =
+        domainsById.get(domain.getId());
+    long currentTimestamp = System.currentTimeMillis();
+    TimelineDomain domainToStore = createTimelineDomain(
+        domain.getId(), domain.getDescription(), domain.getOwner(),
+        domain.getReaders(), domain.getWriters(),
+        (domainToReplace == null ?
+            currentTimestamp : domainToReplace.getCreatedTime()),
+        currentTimestamp);
+    domainsById.put(domainToStore.getId(), domainToStore);
+    Set<TimelineDomain> domainsByOneOwner =
+        domainsByOwner.get(domainToStore.getOwner());
+    if (domainsByOneOwner == null) {
+      domainsByOneOwner = new HashSet<TimelineDomain>();
+      domainsByOwner.put(domainToStore.getOwner(), domainsByOneOwner);
+    }
+    if (domainToReplace != null) {
+      domainsByOneOwner.remove(domainToReplace);
+    }
+    domainsByOneOwner.add(domainToStore);
+  }
+
+  private static TimelineDomain createTimelineDomain(
+      String id, String description, String owner,
+      String readers, String writers,
+      Long createdTime, Long modifiedTime) {
+    TimelineDomain domainToStore = new TimelineDomain();
+    domainToStore.setId(id);
+    domainToStore.setDescription(description);
+    domainToStore.setOwner(owner);
+    domainToStore.setReaders(readers);
+    domainToStore.setWriters(writers);
+    domainToStore.setCreatedTime(createdTime);
+    domainToStore.setModifiedTime(modifiedTime);
+    return domainToStore;
+  }
+
   private static TimelineEntity maskFields(
       TimelineEntity entity, EnumSet<Field> fields) {
     // Conceal the fields that are not going to be exposed

+ 74 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java

@@ -34,6 +34,8 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineDomains;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.timeline.TimelineReader.Field;
@@ -286,6 +288,78 @@ public class TimelineDataManager {
     return response;
   }
 
+  /**
+   * Add or update an domain. If the domain already exists, only the owner
+   * and the admin can update it.
+   */
+  public void putDomain(TimelineDomain domain,
+      UserGroupInformation callerUGI) throws YarnException, IOException {
+    TimelineDomain existingDomain =
+        store.getDomain(domain.getId());
+    if (existingDomain != null) {
+      if (!timelineACLsManager.checkAccess(callerUGI, existingDomain)) {
+        throw new YarnException(callerUGI.getShortUserName() +
+            " is not allowed to override an existing domain " +
+            existingDomain.getId());
+      }
+      // Set it again in case ACLs are not enabled: The domain can be
+      // modified by every body, but the owner is not changed.
+      domain.setOwner(existingDomain.getOwner());
+    }
+    store.put(domain);
+  }
+
+  /**
+   * Get a single domain of the particular ID. If callerUGI is not the owner
+   * or the admin of the domain, we need to hide the details from him, and
+   * only allow him to see the ID.
+   */
+  public TimelineDomain getDomain(String domainId,
+      UserGroupInformation callerUGI) throws YarnException, IOException {
+    TimelineDomain domain = store.getDomain(domainId);
+    if (domain != null) {
+      if (timelineACLsManager.checkAccess(callerUGI, domain)) {
+        return domain;
+      } else {
+        hideDomainDetails(domain);
+        return domain;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Get all the domains that belong to the given owner. If callerUGI is not
+   * the owner or the admin of the domain, we need to hide the details from
+   * him, and only allow him to see the ID.
+   */
+  public TimelineDomains getDomains(String owner,
+      UserGroupInformation callerUGI) throws YarnException, IOException {
+    TimelineDomains domains = store.getDomains(owner);
+    boolean hasAccess = true;
+    boolean isChecked = false;
+    for (TimelineDomain domain : domains.getDomains()) {
+      // The owner for each domain is the same, just need to check on
+      if (!isChecked) {
+        hasAccess = timelineACLsManager.checkAccess(callerUGI, domain);
+        isChecked = true;
+      }
+      if (!hasAccess) {
+        hideDomainDetails(domain);
+      }
+    }
+    return domains;
+  }
+
+  private static void hideDomainDetails(TimelineDomain domain) {
+    domain.setDescription(null);
+    domain.setOwner(null);
+    domain.setReaders(null);
+    domain.setWriters(null);
+    domain.setCreatedTime(null);
+    domain.setModifiedTime(null);
+  }
+
   private static boolean extendFields(EnumSet<Field> fieldEnums) {
     boolean modified = false;
     if (fieldEnums != null && !fieldEnums.contains(Field.PRIMARY_FILTERS)) {

+ 23 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineReader.java

@@ -29,6 +29,8 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineDomains;
 
 /**
  * This interface is for retrieving timeline information.
@@ -152,4 +154,25 @@ public interface TimelineReader {
   TimelineEvents getEntityTimelines(String entityType,
       SortedSet<String> entityIds, Long limit, Long windowStart,
       Long windowEnd, Set<String> eventTypes) throws IOException;
+
+  /**
+   * This method retrieves the domain information for a given ID.
+   * 
+   * @return a {@link TimelineDomain} object.
+   * @throws IOException
+   */
+  TimelineDomain getDomain(
+      String domainId) throws IOException;
+
+  /**
+   * This method retrieves all the domains that belong to a given owner.
+   * The domains are sorted according to the created time firstly and the
+   * modified time secondly in descending order.
+   * 
+   * @param owner
+   *          the domain owner
+   * @return an {@link TimelineDomains} object.
+   * @throws IOException
+   */
+  TimelineDomains getDomains(String owner) throws IOException;
 }

+ 16 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineWriter.java

@@ -18,13 +18,14 @@
 
 package org.apache.hadoop.yarn.server.timeline;
 
+import java.io.IOException;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 
-import java.io.IOException;
-
 /**
  * This interface is for storing timeline information.
  */
@@ -37,10 +38,21 @@ public interface TimelineWriter {
    * individual put request objects will be reported in the response.
    * 
    * @param data
-   *          An {@link TimelineEntities} object.
-   * @return An {@link TimelinePutResponse} object.
+   *          a {@link TimelineEntities} object.
+   * @return a {@link TimelinePutResponse} object.
    * @throws IOException
    */
   TimelinePutResponse put(TimelineEntities data) throws IOException;
 
+  /**
+   * Store domain information to the timeline store. If A domain of the
+   * same ID already exists in the timeline store, it will be COMPLETELY updated
+   * with the given domain.
+   * 
+   * @param domain
+   *          a {@link TimelineDomain} object
+   * @throws IOException
+   */
+   void put(TimelineDomain domain) throws IOException;
+
 }

+ 26 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineACLsManager.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.AdminACLsManager;
 import org.apache.hadoop.yarn.server.timeline.EntityIdentifier;
@@ -81,6 +82,31 @@ public class TimelineACLsManager {
     return false;
   }
 
+  public boolean checkAccess(UserGroupInformation callerUGI,
+      TimelineDomain domain) throws YarnException, IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Verifying the access of "
+          + (callerUGI == null ? null : callerUGI.getShortUserName())
+          + " on the timeline domain " + domain);
+    }
+
+    if (!adminAclsManager.areACLsEnabled()) {
+      return true;
+    }
+
+    String owner = domain.getOwner();
+    if (owner == null || owner.length() == 0) {
+      throw new YarnException("Owner information of the timeline domain "
+          + domain.getId() + " is corrupted.");
+    }
+    if (callerUGI != null
+        && (adminAclsManager.isAdmin(callerUGI) ||
+            callerUGI.getShortUserName().equals(owner))) {
+      return true;
+    }
+    return false;
+  }
+
   @Private
   @VisibleForTesting
   public AdminACLsManager

+ 100 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.timeline.webapp;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.EnumSet;
@@ -32,6 +33,7 @@ import javax.servlet.http.HttpServletResponse;
 import javax.ws.rs.Consumes;
 import javax.ws.rs.GET;
 import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
 import javax.ws.rs.Path;
 import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
@@ -40,6 +42,7 @@ import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
+import javax.ws.rs.core.Response.Status;
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlElement;
@@ -53,7 +56,10 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineDomains;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.timeline.EntityIdentifier;
 import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper;
 import org.apache.hadoop.yarn.server.timeline.NameValuePair;
@@ -259,6 +265,100 @@ public class TimelineWebServices {
     }
   }
 
+  /**
+   * Store the given domain into the timeline store, and return the errors
+   * that happen during storing.
+   */
+  @PUT
+  @Path("/domain")
+  @Consumes({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */})
+  public Response putDomain(
+      @Context HttpServletRequest req,
+      @Context HttpServletResponse res,
+      TimelineDomain domain) {
+    init(res);
+    UserGroupInformation callerUGI = getUser(req);
+    if (callerUGI == null) {
+      String msg = "The owner of the posted timeline domain is not set";
+      LOG.error(msg);
+      throw new ForbiddenException(msg);
+    }
+    domain.setOwner(callerUGI.getShortUserName());
+    try {
+      timelineDataManager.putDomain(domain, callerUGI);
+    } catch (YarnException e) {
+      // The user doesn't have the access to override the existing domain.
+      LOG.error(e.getMessage(), e);
+      throw new ForbiddenException(e);
+    } catch (IOException e) {
+      LOG.error("Error putting domain", e);
+      throw new WebApplicationException(e,
+          Response.Status.INTERNAL_SERVER_ERROR);
+    }
+    return Response.status(Status.OK).build();
+  }
+
+  /**
+   * Return a single domain of the given domain Id.
+   */
+  @GET
+  @Path("/domain/{domainId}")
+  @Produces({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */})
+  public TimelineDomain getDomain(
+      @Context HttpServletRequest req,
+      @Context HttpServletResponse res,
+      @PathParam("domainId") String domainId) {
+    init(res);
+    domainId = parseStr(domainId);
+    if (domainId == null || domainId.length() == 0) {
+      throw new BadRequestException("Domain ID is not specified.");
+    }
+    TimelineDomain domain = null;
+    try {
+      domain = timelineDataManager.getDomain(
+          parseStr(domainId), getUser(req));
+    } catch (Exception e) {
+      LOG.error("Error getting domain", e);
+      throw new WebApplicationException(e,
+          Response.Status.INTERNAL_SERVER_ERROR);
+    }
+    if (domain == null) {
+      throw new NotFoundException("Timeline domain ["
+          + domainId + "] is not found");
+    }
+    return domain;
+  }
+
+  /**
+   * Return a list of domains of the given owner.
+   */
+  @GET
+  @Path("/domain")
+  @Produces({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */})
+  public TimelineDomains getDomains(
+      @Context HttpServletRequest req,
+      @Context HttpServletResponse res,
+      @QueryParam("owner") String owner) {
+    init(res);
+    owner = parseStr(owner);
+    UserGroupInformation callerUGI = getUser(req);
+    if (owner == null || owner.length() == 0) {
+      if (callerUGI == null) {
+        throw new BadRequestException("Domain owner is not specified.");
+      } else {
+        // By default it's going to list the caller's domains
+        owner = callerUGI.getShortUserName();
+      }
+    }
+    try {
+      return timelineDataManager.getDomains(owner, callerUGI);
+    } catch (Exception e) {
+      LOG.error("Error getting domains", e);
+      throw new WebApplicationException(e,
+          Response.Status.INTERNAL_SERVER_ERROR);
+    }
+  }
+
   private void init(HttpServletResponse response) {
     response.setContentType(null);
   }

+ 15 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLeveldbTimelineStore.java

@@ -69,8 +69,9 @@ public class TestLeveldbTimelineStore extends TimelineStoreTestUtils {
     store = new LeveldbTimelineStore();
     store.init(config);
     store.start();
-    loadTestData();
-    loadVerificationData();
+    loadTestEntityData();
+    loadVerificationEntityData();
+    loadTestDomainData();
   }
 
   @After
@@ -93,7 +94,7 @@ public class TestLeveldbTimelineStore extends TimelineStoreTestUtils {
     super.testGetSingleEntity();
     ((LeveldbTimelineStore)store).clearStartTimeCache();
     super.testGetSingleEntity();
-    loadTestData();
+    loadTestEntityData();
   }
 
   @Test
@@ -257,7 +258,7 @@ public class TestLeveldbTimelineStore extends TimelineStoreTestUtils {
     assertEquals(0, getEntities("type_2").size());
     assertEquals(0, getEntitiesFromTsWithPrimaryFilter("type_1", userFilter,
         l).size());
-    loadTestData();
+    loadTestEntityData();
     assertEquals(0, getEntitiesFromTs("type_1", l).size());
     assertEquals(0, getEntitiesFromTs("type_2", l).size());
     assertEquals(0, getEntitiesFromTsWithPrimaryFilter("type_1", userFilter,
@@ -309,4 +310,14 @@ public class TestLeveldbTimelineStore extends TimelineStoreTestUtils {
     store.start();
   }
 
+  @Test
+  public void testGetDomain() throws IOException {
+    super.testGetDomain();
+  }
+
+  @Test
+  public void testGetDomains() throws IOException {
+    super.testGetDomains();
+  }
+
 }

+ 13 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestMemoryTimelineStore.java

@@ -34,8 +34,9 @@ public class TestMemoryTimelineStore extends TimelineStoreTestUtils {
     store = new MemoryTimelineStore();
     store.init(new YarnConfiguration());
     store.start();
-    loadTestData();
-    loadVerificationData();
+    loadTestEntityData();
+    loadVerificationEntityData();
+    loadTestDomainData();
   }
 
   @After
@@ -82,4 +83,14 @@ public class TestMemoryTimelineStore extends TimelineStoreTestUtils {
     super.testGetEvents();
   }
 
+  @Test
+  public void testGetDomain() throws IOException {
+    super.testGetDomain();
+  }
+
+  @Test
+  public void testGetDomains() throws IOException {
+    super.testGetDomains();
+  }
+
 }

+ 88 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineStoreTestUtils.java

@@ -38,11 +38,11 @@ import java.util.TreeSet;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
-import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents.EventsOfOneEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineDomains;
+import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse.TimelinePutError;
-import org.apache.hadoop.yarn.server.timeline.NameValuePair;
-import org.apache.hadoop.yarn.server.timeline.TimelineStore;
 import org.apache.hadoop.yarn.server.timeline.TimelineReader.Field;
 
 public class TimelineStoreTestUtils {
@@ -88,9 +88,9 @@ public class TimelineStoreTestUtils {
   protected long beforeTs;
 
   /**
-   * Load test data into the given store
+   * Load test entity data into the given store
    */
-  protected void loadTestData() throws IOException {
+  protected void loadTestEntityData() throws IOException {
     beforeTs = System.currentTimeMillis()-1;
     TimelineEntities entities = new TimelineEntities();
     Map<String, Set<Object>> primaryFilters =
@@ -184,9 +184,9 @@ public class TimelineStoreTestUtils {
   }
 
   /**
-   * Load verification data
+   * Load verification entity data
    */
-  protected void loadVerificationData() throws Exception {
+  protected void loadVerificationEntityData() throws Exception {
     userFilter = new NameValuePair("user", "username");
     numericFilter1 = new NameValuePair("appname", Integer.MAX_VALUE);
     numericFilter2 = new NameValuePair("long", (long)Integer.MAX_VALUE + 1l);
@@ -263,6 +263,51 @@ public class TimelineStoreTestUtils {
     events2.add(ev4);
   }
 
+  private TimelineDomain domain1;
+  private TimelineDomain domain2;
+  private TimelineDomain domain3;
+  private long elapsedTime;
+
+  protected void loadTestDomainData() throws IOException {
+    domain1 = new TimelineDomain();
+    domain1.setId("domain_id_1");
+    domain1.setDescription("description_1");
+    domain1.setOwner("owner_1");
+    domain1.setReaders("reader_user_1 reader_group_1");
+    domain1.setWriters("writer_user_1 writer_group_1");
+    store.put(domain1);
+
+    domain2 = new TimelineDomain();
+    domain2.setId("domain_id_2");
+    domain2.setDescription("description_2");
+    domain2.setOwner("owner_2");
+    domain2.setReaders("reader_user_2 reader_group_2");
+    domain2.setWriters("writer_user_2writer_group_2");
+    store.put(domain2);
+
+    // Wait a second before updating the domain information
+    elapsedTime = 1000;
+    try {
+      Thread.sleep(elapsedTime);
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
+
+    domain2.setDescription("description_3");
+    domain2.setOwner("owner_3");
+    domain2.setReaders("reader_user_3 reader_group_3");
+    domain2.setWriters("writer_user_3 writer_group_3");
+    store.put(domain2);
+
+    domain3 = new TimelineDomain();
+    domain3.setId("domain_id_4");
+    domain3.setDescription("description_4");
+    domain3.setOwner("owner_1");
+    domain3.setReaders("reader_user_4 reader_group_4");
+    domain3.setWriters("writer_user_4 writer_group_4");
+    store.put(domain3);
+  }
+
   public void testGetSingleEntity() throws IOException {
     // test getting entity info
     verifyEntityInfo(null, null, null, null, null, null,
@@ -519,7 +564,7 @@ public class TimelineStoreTestUtils {
     assertEquals(2, getEntitiesWithPrimaryFilter("type_1", userFilter).size());
     // check insert time is not overwritten
     long beforeTs = this.beforeTs;
-    loadTestData();
+    loadTestEntityData();
     assertEquals(0, getEntitiesFromTs("type_1", beforeTs).size());
     assertEquals(0, getEntitiesFromTs("type_2", beforeTs).size());
     assertEquals(0, getEntitiesFromTsWithPrimaryFilter("type_1", userFilter,
@@ -788,4 +833,39 @@ public class TimelineStoreTestUtils {
     return event;
   }
 
+  public void testGetDomain() throws IOException {
+    TimelineDomain actualDomain1 =
+        store.getDomain(domain1.getId());
+    verifyDomainInfo(domain1, actualDomain1);
+    assertTrue(actualDomain1.getCreatedTime() > 0);
+    assertTrue(actualDomain1.getModifiedTime() > 0);
+    assertEquals(
+        actualDomain1.getCreatedTime(), actualDomain1.getModifiedTime());
+
+    TimelineDomain actualDomain2 =
+        store.getDomain(domain2.getId());
+    verifyDomainInfo(domain2, actualDomain2);
+    assertEquals("domain_id_2", actualDomain2.getId());
+    assertTrue(actualDomain2.getCreatedTime() > 0);
+    assertTrue(actualDomain2.getModifiedTime() > 0);
+    assertTrue(
+        actualDomain2.getCreatedTime() < actualDomain2.getModifiedTime());
+  }
+
+  public void testGetDomains() throws IOException {
+    TimelineDomains actualDomains =
+        store.getDomains("owner_1");
+    assertEquals(2, actualDomains.getDomains().size());
+    verifyDomainInfo(domain3, actualDomains.getDomains().get(0));
+    verifyDomainInfo(domain1, actualDomains.getDomains().get(1));
+  }
+
+  private static void verifyDomainInfo(
+      TimelineDomain expected, TimelineDomain actual) {
+    assertEquals(expected.getId(), actual.getId());
+    assertEquals(expected.getDescription(), actual.getDescription());
+    assertEquals(expected.getOwner(), actual.getOwner());
+    assertEquals(expected.getReaders(), actual.getReaders());
+    assertEquals(expected.getWriters(), actual.getWriters());
+  }
 }

+ 59 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineACLsManager.java

@@ -21,17 +21,17 @@ package org.apache.hadoop.yarn.server.timeline.security;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.timeline.TimelineStore;
-import org.apache.hadoop.yarn.server.timeline.security.TimelineACLsManager;
 import org.junit.Assert;
 import org.junit.Test;
 
 public class TestTimelineACLsManager {
 
   @Test
-  public void testYarnACLsNotEnabled() throws Exception {
+  public void testYarnACLsNotEnabledForEntity() throws Exception {
     Configuration conf = new YarnConfiguration();
     conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, false);
     TimelineACLsManager timelineACLsManager =
@@ -47,7 +47,7 @@ public class TestTimelineACLsManager {
   }
 
   @Test
-  public void testYarnACLsEnabled() throws Exception {
+  public void testYarnACLsEnabledForEntity() throws Exception {
     Configuration conf = new YarnConfiguration();
     conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
     conf.set(YarnConfiguration.YARN_ADMIN_ACL, "admin");
@@ -72,7 +72,7 @@ public class TestTimelineACLsManager {
   }
 
   @Test
-  public void testCorruptedOwnerInfo() throws Exception {
+  public void testCorruptedOwnerInfoForEntity() throws Exception {
     Configuration conf = new YarnConfiguration();
     conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
     conf.set(YarnConfiguration.YARN_ADMIN_ACL, "owner");
@@ -89,4 +89,59 @@ public class TestTimelineACLsManager {
     }
   }
 
+  @Test
+  public void testYarnACLsNotEnabledForDomain() throws Exception {
+    Configuration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, false);
+    TimelineACLsManager timelineACLsManager =
+        new TimelineACLsManager(conf);
+    TimelineDomain domain = new TimelineDomain();
+    domain.setOwner("owner");
+    Assert.assertTrue(
+        "Always true when ACLs are not enabled",
+        timelineACLsManager.checkAccess(
+            UserGroupInformation.createRemoteUser("user"), domain));
+  }
+
+  @Test
+  public void testYarnACLsEnabledForDomain() throws Exception {
+    Configuration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
+    conf.set(YarnConfiguration.YARN_ADMIN_ACL, "admin");
+    TimelineACLsManager timelineACLsManager =
+        new TimelineACLsManager(conf);
+    TimelineDomain domain = new TimelineDomain();
+    domain.setOwner("owner");
+    Assert.assertTrue(
+        "Owner should be allowed to access",
+        timelineACLsManager.checkAccess(
+            UserGroupInformation.createRemoteUser("owner"), domain));
+    Assert.assertFalse(
+        "Other shouldn't be allowed to access",
+        timelineACLsManager.checkAccess(
+            UserGroupInformation.createRemoteUser("other"), domain));
+    Assert.assertTrue(
+        "Admin should be allowed to access",
+        timelineACLsManager.checkAccess(
+            UserGroupInformation.createRemoteUser("admin"), domain));
+  }
+
+  @Test
+  public void testCorruptedOwnerInfoForDomain() throws Exception {
+    Configuration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
+    conf.set(YarnConfiguration.YARN_ADMIN_ACL, "owner");
+    TimelineACLsManager timelineACLsManager =
+        new TimelineACLsManager(conf);
+    TimelineDomain domain = new TimelineDomain();
+    try {
+      timelineACLsManager.checkAccess(
+          UserGroupInformation.createRemoteUser("owner"), domain);
+      Assert.fail("Exception is expected");
+    } catch (YarnException e) {
+      Assert.assertTrue("It's not the exact expected exception", e.getMessage()
+          .contains("is corrupted."));
+    }
+  }
+
 }

+ 219 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServices.java

@@ -36,6 +36,7 @@ import javax.servlet.FilterConfig;
 import javax.servlet.ServletContext;
 import javax.servlet.ServletException;
 import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response.Status;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
@@ -44,6 +45,8 @@ import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineDomains;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse.TimelinePutError;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -69,7 +72,6 @@ import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
 import com.sun.jersey.test.framework.JerseyTest;
 import com.sun.jersey.test.framework.WebAppDescriptor;
 
-
 public class TestTimelineWebServices extends JerseyTest {
 
   private static TimelineStore store;
@@ -85,7 +87,7 @@ public class TestTimelineWebServices extends JerseyTest {
       bind(YarnJacksonJaxbJsonProvider.class);
       bind(TimelineWebServices.class);
       bind(GenericExceptionHandler.class);
-      try{
+      try {
         store = mockTimelineStore();
       } catch (Exception e) {
         Assert.fail();
@@ -100,7 +102,8 @@ public class TestTimelineWebServices extends JerseyTest {
           new TimelineDataManager(store, timelineACLsManager);
       bind(TimelineDataManager.class).toInstance(timelineDataManager);
       serve("/*").with(GuiceContainer.class);
-      TimelineAuthenticationFilter taFilter = new TimelineAuthenticationFilter();
+      TimelineAuthenticationFilter taFilter =
+          new TimelineAuthenticationFilter();
       FilterConfig filterConfig = mock(FilterConfig.class);
       when(filterConfig.getInitParameter(AuthenticationFilter.CONFIG_PREFIX))
           .thenReturn(null);
@@ -159,7 +162,8 @@ public class TestTimelineWebServices extends JerseyTest {
         .filterClass(com.google.inject.servlet.GuiceFilter.class)
         .contextPath("jersey-guice-filter")
         .servletPath("/")
-        .clientConfig(new DefaultClientConfig(YarnJacksonJaxbJsonProvider.class))
+        .clientConfig(
+            new DefaultClientConfig(YarnJacksonJaxbJsonProvider.class))
         .build());
   }
 
@@ -277,7 +281,7 @@ public class TestTimelineWebServices extends JerseyTest {
     WebResource r = resource();
     ClientResponse response = r.path("ws").path("v1").path("timeline")
         .path("type_1").queryParam("primaryFilter",
-            "long:" + Long.toString((long)Integer.MAX_VALUE + 1l))
+            "long:" + Long.toString((long) Integer.MAX_VALUE + 1l))
         .accept(MediaType.APPLICATION_JSON)
         .get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
@@ -406,7 +410,8 @@ public class TestTimelineWebServices extends JerseyTest {
     TimelineEntities entities = new TimelineEntities();
     TimelineEntity entity = new TimelineEntity();
     Map<String, Set<Object>> filters = new HashMap<String, Set<Object>>();
-    filters.put(TimelineStore.SystemFilter.ENTITY_OWNER.toString(), new HashSet<Object>());
+    filters.put(TimelineStore.SystemFilter.ENTITY_OWNER.toString(),
+        new HashSet<Object>());
     entity.setPrimaryFilters(filters);
     entity.setEntityId("test id 6");
     entity.setEntityType("test type 6");
@@ -418,13 +423,15 @@ public class TestTimelineWebServices extends JerseyTest {
         .accept(MediaType.APPLICATION_JSON)
         .type(MediaType.APPLICATION_JSON)
         .post(ClientResponse.class, entities);
-    TimelinePutResponse putResposne = response.getEntity(TimelinePutResponse.class);
+    TimelinePutResponse putResposne =
+        response.getEntity(TimelinePutResponse.class);
     Assert.assertEquals(1, putResposne.getErrors().size());
     List<TimelinePutError> errors = putResposne.getErrors();
-    Assert.assertEquals(TimelinePutResponse.TimelinePutError.SYSTEM_FILTER_CONFLICT,
-      errors.get(0).getErrorCode());
+    Assert.assertEquals(
+        TimelinePutResponse.TimelinePutError.SYSTEM_FILTER_CONFLICT,
+        errors.get(0).getErrorCode());
   }
-  
+
   @Test
   public void testPostEntities() throws Exception {
     TimelineEntities entities = new TimelineEntities();
@@ -449,7 +456,8 @@ public class TestTimelineWebServices extends JerseyTest {
         .type(MediaType.APPLICATION_JSON)
         .post(ClientResponse.class, entities);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    TimelinePutResponse putResposne = response.getEntity(TimelinePutResponse.class);
+    TimelinePutResponse putResposne =
+        response.getEntity(TimelinePutResponse.class);
     Assert.assertNotNull(putResposne);
     Assert.assertEquals(0, putResposne.getErrors().size());
     // verify the entity exists in the store
@@ -482,7 +490,8 @@ public class TestTimelineWebServices extends JerseyTest {
           .type(MediaType.APPLICATION_JSON)
           .post(ClientResponse.class, entities);
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-      TimelinePutResponse putResponse = response.getEntity(TimelinePutResponse.class);
+      TimelinePutResponse putResponse =
+          response.getEntity(TimelinePutResponse.class);
       Assert.assertNotNull(putResponse);
       Assert.assertEquals(0, putResponse.getErrors().size());
 
@@ -668,4 +677,202 @@ public class TestTimelineWebServices extends JerseyTest {
     }
   }
 
+  @Test
+  public void testGetDomain() throws Exception {
+    WebResource r = resource();
+    ClientResponse response = r.path("ws").path("v1").path("timeline")
+        .path("domain").path("domain_id_1")
+        .accept(MediaType.APPLICATION_JSON)
+        .get(ClientResponse.class);
+    Assert.assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    TimelineDomain domain = response.getEntity(TimelineDomain.class);
+    verifyDomain(domain, "domain_id_1", true);
+  }
+
+  @Test
+  public void testGetDomainYarnACLsEnabled() {
+    AdminACLsManager oldAdminACLsManager =
+        timelineACLsManager.setAdminACLsManager(adminACLsManager);
+    try {
+      WebResource r = resource();
+      ClientResponse response = r.path("ws").path("v1").path("timeline")
+          .path("domain").path("domain_id_1")
+          .queryParam("user.name", "owner_1")
+          .accept(MediaType.APPLICATION_JSON)
+          .get(ClientResponse.class);
+      Assert.assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      TimelineDomain domain = response.getEntity(TimelineDomain.class);
+      verifyDomain(domain, "domain_id_1", true);
+
+      response = r.path("ws").path("v1").path("timeline")
+          .path("domain").path("domain_id_1")
+          .queryParam("user.name", "tester")
+          .accept(MediaType.APPLICATION_JSON)
+          .get(ClientResponse.class);
+      Assert.assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      domain = response.getEntity(TimelineDomain.class);
+      verifyDomain(domain, "domain_id_1", false);
+    } finally {
+      timelineACLsManager.setAdminACLsManager(oldAdminACLsManager);
+    }
+  }
+
+  @Test
+  public void testGetDomains() throws Exception {
+    WebResource r = resource();
+    ClientResponse response = r.path("ws").path("v1").path("timeline")
+        .path("domain")
+        .queryParam("owner", "owner_1")
+        .accept(MediaType.APPLICATION_JSON)
+        .get(ClientResponse.class);
+    Assert.assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    TimelineDomains domains = response.getEntity(TimelineDomains.class);
+    Assert.assertEquals(2, domains.getDomains().size());
+    for (int i = 0; i < domains.getDomains().size(); ++i) {
+      verifyDomain(domains.getDomains().get(i),
+          i == 0 ? "domain_id_4" : "domain_id_1", true);
+    }
+  }
+
+  @Test
+  public void testGetDomainsYarnACLsEnabled() throws Exception {
+    AdminACLsManager oldAdminACLsManager =
+        timelineACLsManager.setAdminACLsManager(adminACLsManager);
+    try {
+      WebResource r = resource();
+      ClientResponse response = r.path("ws").path("v1").path("timeline")
+          .path("domain")
+          .queryParam("user.name", "owner_1")
+          .accept(MediaType.APPLICATION_JSON)
+          .get(ClientResponse.class);
+      Assert.assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      TimelineDomains domains = response.getEntity(TimelineDomains.class);
+      Assert.assertEquals(2, domains.getDomains().size());
+      for (int i = 0; i < domains.getDomains().size(); ++i) {
+        verifyDomain(domains.getDomains().get(i),
+            i == 0 ? "domain_id_4" : "domain_id_1", true);
+      }
+
+      response = r.path("ws").path("v1").path("timeline")
+          .path("domain")
+          .queryParam("owner", "owner_1")
+          .queryParam("user.name", "tester")
+          .accept(MediaType.APPLICATION_JSON)
+          .get(ClientResponse.class);
+      Assert.assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+      domains = response.getEntity(TimelineDomains.class);
+      Assert.assertEquals(2, domains.getDomains().size());
+      for (int i = 0; i < domains.getDomains().size(); ++i) {
+        verifyDomain(domains.getDomains().get(i),
+            i == 0 ? "domain_id_4" : "domain_id_1", false);
+      }
+    } finally {
+      timelineACLsManager.setAdminACLsManager(oldAdminACLsManager);
+    }
+  }
+
+  @Test
+  public void testPutDomain() throws Exception {
+    TimelineDomain domain = new TimelineDomain();
+    domain.setId("test_domain_id");
+    WebResource r = resource();
+    // No owner, will be rejected
+    ClientResponse response = r.path("ws").path("v1")
+        .path("timeline").path("domain")
+        .accept(MediaType.APPLICATION_JSON)
+        .type(MediaType.APPLICATION_JSON)
+        .put(ClientResponse.class, domain);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(ClientResponse.Status.FORBIDDEN,
+        response.getClientResponseStatus());
+
+    response = r.path("ws").path("v1")
+        .path("timeline").path("domain")
+        .queryParam("user.name", "tester")
+        .accept(MediaType.APPLICATION_JSON)
+        .type(MediaType.APPLICATION_JSON)
+        .put(ClientResponse.class, domain);
+    assertEquals(Status.OK.getStatusCode(), response.getStatus());
+    
+    // Verify the domain exists
+    response = r.path("ws").path("v1").path("timeline")
+        .path("domain").path("test_domain_id")
+        .accept(MediaType.APPLICATION_JSON)
+        .get(ClientResponse.class);
+    Assert.assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    domain = response.getEntity(TimelineDomain.class);
+    Assert.assertNotNull(domain);
+    Assert.assertEquals("test_domain_id", domain.getId());
+    Assert.assertEquals("tester", domain.getOwner());
+    Assert.assertEquals(null, domain.getDescription());
+
+    // Update the domain
+    domain.setDescription("test_description");
+    response = r.path("ws").path("v1")
+        .path("timeline").path("domain")
+        .queryParam("user.name", "tester")
+        .accept(MediaType.APPLICATION_JSON)
+        .type(MediaType.APPLICATION_JSON)
+        .put(ClientResponse.class, domain);
+    assertEquals(Status.OK.getStatusCode(), response.getStatus());
+
+    // Verify the domain is updated
+    response = r.path("ws").path("v1").path("timeline")
+        .path("domain").path("test_domain_id")
+        .accept(MediaType.APPLICATION_JSON)
+        .get(ClientResponse.class);
+    Assert.assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    domain = response.getEntity(TimelineDomain.class);
+    Assert.assertNotNull(domain);
+    Assert.assertEquals("test_domain_id", domain.getId());
+    Assert.assertEquals("test_description", domain.getDescription());
+  }
+
+  @Test
+  public void testPutDomainYarnACLsEnabled() throws Exception {
+    AdminACLsManager oldAdminACLsManager =
+        timelineACLsManager.setAdminACLsManager(adminACLsManager);
+    try {
+      TimelineDomain domain = new TimelineDomain();
+      domain.setId("test_domain_id_acl");
+      WebResource r = resource();
+      ClientResponse response = r.path("ws").path("v1")
+          .path("timeline").path("domain")
+          .queryParam("user.name", "tester")
+          .accept(MediaType.APPLICATION_JSON)
+          .type(MediaType.APPLICATION_JSON)
+          .put(ClientResponse.class, domain);
+      assertEquals(Status.OK.getStatusCode(), response.getStatus());
+
+      // Update the domain by another user
+      response = r.path("ws").path("v1")
+          .path("timeline").path("domain")
+          .queryParam("user.name", "other")
+          .accept(MediaType.APPLICATION_JSON)
+          .type(MediaType.APPLICATION_JSON)
+          .put(ClientResponse.class, domain);
+      assertEquals(Status.FORBIDDEN.getStatusCode(), response.getStatus());
+    } finally {
+      timelineACLsManager.setAdminACLsManager(oldAdminACLsManager);
+    }
+  }
+
+  private static void verifyDomain(TimelineDomain domain,
+      String domainId, boolean hasAccess) {
+    Assert.assertNotNull(domain);
+    Assert.assertEquals(domainId, domain.getId());
+    // The specific values have been verified in TestMemoryTimelineStore
+    Assert.assertTrue(hasAccess && domain.getDescription() != null ||
+        !hasAccess && domain.getDescription() == null);
+    Assert.assertTrue(hasAccess && domain.getOwner() != null ||
+        !hasAccess && domain.getOwner() == null);
+    Assert.assertTrue(hasAccess && domain.getReaders() != null ||
+        !hasAccess && domain.getReaders() == null);
+    Assert.assertTrue(hasAccess && domain.getWriters() != null ||
+        !hasAccess && domain.getWriters() == null);
+    Assert.assertTrue(hasAccess && domain.getCreatedTime() != null ||
+        !hasAccess && domain.getCreatedTime() == null);
+    Assert.assertTrue(hasAccess && domain.getModifiedTime() != null ||
+        !hasAccess && domain.getModifiedTime() == null);
+  }
 }

+ 2 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServicesWithSSL.java

@@ -24,7 +24,6 @@ import java.util.EnumSet;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
-import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
@@ -124,8 +123,8 @@ public class TestTimelineWebServicesWithSSL {
     private ClientResponse resp;
 
     @Override
-    public ClientResponse doPostingEntities(TimelineEntities entities) {
-      resp = super.doPostingEntities(entities);
+    public ClientResponse doPostingObject(Object obj, String path) {
+      resp = super.doPostingObject(obj, path);
       return resp;
     }
 

+ 3 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt

@@ -20,6 +20,9 @@ set(CMAKE_BUILD_TYPE, Release)
 
 include(../../../../../hadoop-common-project/hadoop-common/src/JNIFlags.cmake NO_POLICY_SCOPE)
 
+include(CheckFunctionExists)
+CHECK_FUNCTION_EXISTS(fcloseall HAVE_FCLOSEALL)
+
 function(output_directory TGT DIR)
     SET_TARGET_PROPERTIES(${TGT} PROPERTIES
         RUNTIME_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")

+ 2 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/config.h.cmake

@@ -20,4 +20,6 @@
 
 #cmakedefine HADOOP_CONF_DIR "@HADOOP_CONF_DIR@"
 
+#cmakedefine HAVE_FCLOSEALL "@HAVE_FCLOSEALL@"
+
 #endif

+ 22 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java

@@ -72,9 +72,11 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.SerializedException;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.LogAggregationContextPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
@@ -275,11 +277,17 @@ public class ContainerManagerImpl extends CompositeService implements
           aclProto.getAcl());
     }
 
+    LogAggregationContext logAggregationContext = null;
+    if (p.getLogAggregationContext() != null) {
+      logAggregationContext =
+          new LogAggregationContextPBImpl(p.getLogAggregationContext());
+    }
+
     LOG.info("Recovering application " + appId);
     ApplicationImpl app = new ApplicationImpl(dispatcher, p.getUser(), appId,
         creds, context);
     context.getApplications().put(appId, app);
-    app.handle(new ApplicationInitEvent(appId, acls));
+    app.handle(new ApplicationInitEvent(appId, acls, logAggregationContext));
   }
 
   @SuppressWarnings("unchecked")
@@ -719,13 +727,19 @@ public class ContainerManagerImpl extends CompositeService implements
 
   private ContainerManagerApplicationProto buildAppProto(ApplicationId appId,
       String user, Credentials credentials,
-      Map<ApplicationAccessType, String> appAcls) {
+      Map<ApplicationAccessType, String> appAcls,
+      LogAggregationContext logAggregationContext) {
 
     ContainerManagerApplicationProto.Builder builder =
         ContainerManagerApplicationProto.newBuilder();
     builder.setId(((ApplicationIdPBImpl) appId).getProto());
     builder.setUser(user);
 
+    if (logAggregationContext != null) {
+      builder.setLogAggregationContext((
+          (LogAggregationContextPBImpl)logAggregationContext).getProto());
+    }
+
     builder.clearCredentials();
     if (credentials != null) {
       DataOutputBuffer dob = new DataOutputBuffer();
@@ -826,12 +840,16 @@ public class ContainerManagerImpl extends CompositeService implements
         if (null == context.getApplications().putIfAbsent(applicationID,
           application)) {
           LOG.info("Creating a new application reference for app " + applicationID);
+          LogAggregationContext logAggregationContext =
+              containerTokenIdentifier.getLogAggregationContext();
           Map<ApplicationAccessType, String> appAcls =
               container.getLaunchContext().getApplicationACLs();
           context.getNMStateStore().storeApplication(applicationID,
-              buildAppProto(applicationID, user, credentials, appAcls));
+              buildAppProto(applicationID, user, credentials, appAcls,
+                logAggregationContext));
           dispatcher.getEventHandler().handle(
-            new ApplicationInitEvent(applicationID, appAcls));
+            new ApplicationInitEvent(applicationID, appAcls,
+              logAggregationContext));
         }
 
         this.context.getNMStateStore().storeContainer(containerId, request);

+ 17 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java

@@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.logaggregation.ContainerLogsRetentionPolicy;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
@@ -54,6 +55,8 @@ import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
 import org.apache.hadoop.yarn.state.StateMachineFactory;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * The state machine for the representation of an Application
  * within the NodeManager.
@@ -72,6 +75,8 @@ public class ApplicationImpl implements Application {
 
   private static final Log LOG = LogFactory.getLog(Application.class);
 
+  private LogAggregationContext logAggregationContext;
+
   Map<ContainerId, Container> containers =
       new HashMap<ContainerId, Container>();
 
@@ -234,10 +239,11 @@ public class ApplicationImpl implements Application {
       app.applicationACLs = initEvent.getApplicationACLs();
       app.aclsManager.addApplication(app.getAppId(), app.applicationACLs);
       // Inform the logAggregator
+      app.logAggregationContext = initEvent.getLogAggregationContext();
       app.dispatcher.getEventHandler().handle(
           new LogHandlerAppStartedEvent(app.appId, app.user,
               app.credentials, ContainerLogsRetentionPolicy.ALL_CONTAINERS,
-              app.applicationACLs)); 
+              app.applicationACLs, app.logAggregationContext)); 
     }
   }
 
@@ -467,4 +473,14 @@ public class ApplicationImpl implements Application {
   public String toString() {
     return appId.toString();
   }
+
+  @VisibleForTesting
+  public LogAggregationContext getLogAggregationContext() {
+    try {
+      this.readLock.lock();
+      return this.logAggregationContext;
+    } finally {
+      this.readLock.unlock();
+    }
+  }
 }

+ 13 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationInitEvent.java

@@ -22,18 +22,31 @@ import java.util.Map;
 
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 
 public class ApplicationInitEvent extends ApplicationEvent {
 
   private final Map<ApplicationAccessType, String> applicationACLs;
+  private final LogAggregationContext logAggregationContext;  
 
   public ApplicationInitEvent(ApplicationId appId,
       Map<ApplicationAccessType, String> acls) {
+    this(appId, acls, null);
+  }
+
+  public ApplicationInitEvent(ApplicationId appId,
+      Map<ApplicationAccessType, String> acls,
+      LogAggregationContext logAggregationContext) {
     super(appId, ApplicationEventType.INIT_APPLICATION);
     this.applicationACLs = acls;
+    this.logAggregationContext = logAggregationContext;
   }
 
   public Map<ApplicationAccessType, String> getApplicationACLs() {
     return this.applicationACLs;
   }
+
+  public LogAggregationContext getLogAggregationContext() {
+    return this.logAggregationContext;
+  }
 }

+ 13 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/loghandler/event/LogHandlerAppStartedEvent.java

@@ -23,6 +23,7 @@ import java.util.Map;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.logaggregation.ContainerLogsRetentionPolicy;
 
 public class LogHandlerAppStartedEvent extends LogHandlerEvent {
@@ -32,16 +33,25 @@ public class LogHandlerAppStartedEvent extends LogHandlerEvent {
   private final String user;
   private final Credentials credentials;
   private final Map<ApplicationAccessType, String> appAcls;
+  private final LogAggregationContext logAggregationContext;
 
   public LogHandlerAppStartedEvent(ApplicationId appId, String user,
       Credentials credentials, ContainerLogsRetentionPolicy retentionPolicy,
       Map<ApplicationAccessType, String> appAcls) {
+    this(appId, user, credentials, retentionPolicy, appAcls, null);
+  }
+
+  public LogHandlerAppStartedEvent(ApplicationId appId, String user,
+      Credentials credentials, ContainerLogsRetentionPolicy retentionPolicy,
+      Map<ApplicationAccessType, String> appAcls,
+      LogAggregationContext logAggregationContext) {
     super(LogHandlerEventType.APPLICATION_STARTED);
     this.applicationId = appId;
     this.user = user;
     this.credentials = credentials;
     this.retentionPolicy = retentionPolicy;
     this.appAcls = appAcls;
+    this.logAggregationContext = logAggregationContext;
   }
 
   public ApplicationId getApplicationId() {
@@ -64,4 +74,7 @@ public class LogHandlerAppStartedEvent extends LogHandlerEvent {
     return this.appAcls;
   }
 
+  public LogAggregationContext getLogAggregationContext() {
+    return this.logAggregationContext;
+  }
 }

+ 6 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/configuration.c

@@ -103,7 +103,12 @@ char *resolve_config_path(const char* file_name, const char *root) {
     real_fname = buffer;
   }
 
-  return (real_fname == NULL) ? NULL : realpath(real_fname, NULL);
+  char * ret = (real_fname == NULL) ? NULL : realpath(real_fname, NULL);
+#ifdef DEBUG
+  fprintf(stderr, "resolve_config_path(file_name=%s,root=%s)=%s\n",
+          file_name, root ? root : "null", ret ? ret : "null");
+#endif
+  return ret;
 }
 
 /**

+ 59 - 24
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c

@@ -19,6 +19,7 @@
 #include "configuration.h"
 #include "container-executor.h"
 
+#include <libgen.h>
 #include <dirent.h>
 #include <fcntl.h>
 #include <fts.h>
@@ -450,39 +451,59 @@ char *get_tmp_directory(const char *work_dir) {
  * with the desired permissions.
  */
 int mkdirs(const char* path, mode_t perm) {
-  char *buffer = strdup(path);
-  char *token;
-  int cwd = open("/", O_RDONLY);
-  if (cwd == -1) {
-    fprintf(LOGFILE, "Can't open / in %s - %s\n", path, strerror(errno));
-    free(buffer);
+  struct stat sb;
+  char * npath;
+  char * p;
+  if (stat(path, &sb) == 0) {
+    if (S_ISDIR (sb.st_mode)) {
+      return 0;
+    } else {
+      fprintf(LOGFILE, "Path %s is file not dir\n", path);
+      return -1;
+    }
+  }
+  npath = strdup(path);
+  if (npath == NULL) {
+    fprintf(LOGFILE, "Not enough memory to copy path string");
     return -1;
   }
-  for(token = strtok(buffer, "/"); token != NULL; token = strtok(NULL, "/")) {
-    if (mkdirat(cwd, token, perm) != 0) {
-      if (errno != EEXIST) {
-        fprintf(LOGFILE, "Can't create directory %s in %s - %s\n", 
-                token, path, strerror(errno));
-        close(cwd);
-        free(buffer);
+  /* Skip leading slashes. */
+  p = npath;
+  while (*p == '/') {
+    p++;
+  }
+
+  while (NULL != (p = strchr(p, '/'))) {
+    *p = '\0';
+    if (stat(npath, &sb) != 0) {
+      if (mkdir(npath, perm) != 0) {
+        fprintf(LOGFILE, "Can't create directory %s in %s - %s\n", npath,
+                path, strerror(errno));
+        free(npath);
         return -1;
       }
-    }
-    int new_dir = openat(cwd, token, O_RDONLY);
-    close(cwd);
-    cwd = new_dir;
-    if (cwd == -1) {
-      fprintf(LOGFILE, "Can't open %s in %s - %s\n", token, path, 
-              strerror(errno));
-      free(buffer);
+    } else if (!S_ISDIR (sb.st_mode)) {
+      fprintf(LOGFILE, "Path %s is file not dir\n", npath);
+      free(npath);
       return -1;
     }
+    *p++ = '/'; /* restore slash */
+    while (*p == '/')
+      p++;
+  }
+
+  /* Create the final directory component. */
+  if (mkdir(npath, perm) != 0) {
+    fprintf(LOGFILE, "Can't create directory %s - %s\n", npath,
+            strerror(errno));
+    free(npath);
+    return -1;
   }
-  free(buffer);
-  close(cwd);
+  free(npath);
   return 0;
 }
 
+
 /**
  * Function to prepare the container directories.
  * It creates the container work and log directories.
@@ -582,7 +603,7 @@ int is_whitelisted(const char *user) {
   char **users = whitelist;
   if (whitelist != NULL) {
     for(; *users; ++users) {
-      if (strncmp(*users, user, LOGIN_NAME_MAX) == 0) {
+      if (strncmp(*users, user, sysconf(_SC_LOGIN_NAME_MAX)) == 0) {
         free_values(whitelist);
         return 1;
       }
@@ -1068,7 +1089,16 @@ int launch_container_as_user(const char *user, const char *app_id,
     goto cleanup;
   }
 
+#if HAVE_FCLOSEALL
   fcloseall();
+#else
+  // only those fds are opened assuming no bug
+  fclose(LOGFILE);
+  fclose(ERRORFILE);
+  fclose(stdin);
+  fclose(stdout);
+  fclose(stderr);
+#endif
   umask(0027);
   if (chdir(work_dir) != 0) {
     fprintf(LOGFILE, "Can't change directory to %s -%s\n", work_dir,
@@ -1351,6 +1381,10 @@ void chown_dir_contents(const char *dir_path, uid_t uid, gid_t gid) {
  * hierarchy: the top directory of the hierarchy for the NM
  */
 int mount_cgroup(const char *pair, const char *hierarchy) {
+#ifndef __linux
+  fprintf(LOGFILE, "Failed to mount cgroup controller, not supported\n");
+  return -1;
+#else
   char *controller = malloc(strlen(pair));
   char *mount_path = malloc(strlen(pair));
   char hier_path[PATH_MAX];
@@ -1387,5 +1421,6 @@ int mount_cgroup(const char *pair, const char *hierarchy) {
   free(mount_path);
 
   return result;
+#endif
 }
 

+ 8 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c

@@ -98,7 +98,7 @@ int write_config_file(char *file_name) {
   }
   fprintf(file, "banned.users=bannedUser\n");
   fprintf(file, "min.user.id=500\n");
-  fprintf(file, "allowed.system.users=allowedUser,bin\n");
+  fprintf(file, "allowed.system.users=allowedUser,daemon\n");
   fclose(file);
   return 0;
 }
@@ -222,20 +222,20 @@ void test_check_user() {
     printf("FAIL: failed check for system user root\n");
     exit(1);
   }
-  if (check_user("bin") == NULL) {
-    printf("FAIL: failed check for whitelisted system user bin\n");
+  if (check_user("daemon") == NULL) {
+    printf("FAIL: failed check for whitelisted system user daemon\n");
     exit(1);
   }
 }
 
 void test_resolve_config_path() {
   printf("\nTesting resolve_config_path\n");
-  if (strcmp(resolve_config_path("/etc/passwd", NULL), "/etc/passwd") != 0) {
-    printf("FAIL: failed to resolve config_name on an absolute path name: /etc/passwd\n");
+  if (strcmp(resolve_config_path("/bin/ls", NULL), "/bin/ls") != 0) {
+    printf("FAIL: failed to resolve config_name on an absolute path name: /bin/ls\n");
     exit(1);
   }
-  if (strcmp(resolve_config_path("../etc/passwd", "/etc/passwd"), "/etc/passwd") != 0) {
-    printf("FAIL: failed to resolve config_name on a relative path name: ../etc/passwd (relative to /etc/passwd)");
+  if (strcmp(resolve_config_path("../bin/ls", "/bin/ls"), "/bin/ls") != 0) {
+    printf("FAIL: failed to resolve config_name on a relative path name: ../bin/ls (relative to /bin/ls)");
     exit(1);
   }
 }
@@ -476,7 +476,7 @@ void test_signal_container_group() {
     printf("FAIL: fork failed\n");
     exit(1);
   } else if (child == 0) {
-    setpgrp();
+    setpgid(0,0);
     if (change_user(user_detail->pw_uid, user_detail->pw_gid) != 0) {
       exit(1);
     }

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/proto/yarn_server_nodemanager_recovery.proto

@@ -29,6 +29,7 @@ message ContainerManagerApplicationProto {
   optional string user = 2;
   optional bytes credentials = 3;
   repeated ApplicationACLMapProto acls = 4;
+  optional LogAggregationContextProto log_aggregation_context = 5;
 }
 
 message DeletionServiceDeleteTaskProto {

+ 11 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java

@@ -55,6 +55,7 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -795,11 +796,20 @@ public class TestContainerManager extends BaseContainerManagerTest {
       NodeId nodeId, String user,
       NMContainerTokenSecretManager containerTokenSecretManager)
       throws IOException {
+    return createContainerToken(cId, rmIdentifier, nodeId, user,
+      containerTokenSecretManager, null);
+  }
+
+  public static Token createContainerToken(ContainerId cId, long rmIdentifier,
+      NodeId nodeId, String user,
+      NMContainerTokenSecretManager containerTokenSecretManager,
+      LogAggregationContext logAggregationContext)
+      throws IOException {
     Resource r = BuilderUtils.newResource(1024, 1);
     ContainerTokenIdentifier containerTokenIdentifier =
         new ContainerTokenIdentifier(cId, nodeId.toString(), user, r,
           System.currentTimeMillis() + 100000L, 123, rmIdentifier,
-          Priority.newInstance(0), 0);
+          Priority.newInstance(0), 0, logAggregationContext);
     Token containerToken =
         BuilderUtils
           .newContainerToken(nodeId, containerTokenSecretManager

+ 22 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java

@@ -45,6 +45,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.security.NMTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
@@ -58,6 +59,7 @@ import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncher;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncherEvent;
@@ -126,8 +128,12 @@ public class TestContainerManagerRecovery {
     ContainerLaunchContext clc = ContainerLaunchContext.newInstance(
         localResources, containerEnv, containerCmds, serviceData,
         containerTokens, acls);
+    // create the logAggregationContext
+    LogAggregationContext logAggregationContext =
+        LogAggregationContext.newInstance("includePattern", "excludePattern",
+          1000);
     StartContainersResponse startResponse = startContainer(context, cm, cid,
-        clc);
+        clc, logAggregationContext);
     assertTrue(startResponse.getFailedRequests().isEmpty());
     assertEquals(1, context.getApplications().size());
     Application app = context.getApplications().get(appId);
@@ -157,6 +163,18 @@ public class TestContainerManagerRecovery {
     assertEquals(1, context.getApplications().size());
     app = context.getApplications().get(appId);
     assertNotNull(app);
+
+    // check whether LogAggregationContext is recovered correctly
+    LogAggregationContext recovered =
+        ((ApplicationImpl) app).getLogAggregationContext();
+    assertNotNull(recovered);
+    assertEquals(logAggregationContext.getRollingIntervalSeconds(),
+      recovered.getRollingIntervalSeconds());
+    assertEquals(logAggregationContext.getIncludePattern(),
+      recovered.getIncludePattern());
+    assertEquals(logAggregationContext.getExcludePattern(),
+      recovered.getExcludePattern());
+
     waitForAppState(app, ApplicationState.INITING);
     assertTrue(context.getApplicationACLsManager().checkAccess(
         UserGroupInformation.createRemoteUser(modUser),
@@ -224,13 +242,14 @@ public class TestContainerManagerRecovery {
 
   private StartContainersResponse startContainer(Context context,
       final ContainerManagerImpl cm, ContainerId cid,
-      ContainerLaunchContext clc) throws Exception {
+      ContainerLaunchContext clc, LogAggregationContext logAggregationContext)
+          throws Exception {
     UserGroupInformation user = UserGroupInformation.createRemoteUser(
         cid.getApplicationAttemptId().toString());
     StartContainerRequest scReq = StartContainerRequest.newInstance(
         clc, TestContainerManager.createContainerToken(cid, 0,
             context.getNodeId(), user.getShortUserName(),
-            context.getContainerTokenSecretManager()));
+            context.getContainerTokenSecretManager(), logAggregationContext));
     final List<StartContainerRequest> scReqList =
         new ArrayList<StartContainerRequest>();
     scReqList.add(scReq);

+ 6 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
@@ -88,9 +89,10 @@ public class SchedulerApplicationAttempt {
   protected final Resource currentReservation = Resource.newInstance(0, 0);
   private Resource resourceLimit = Resource.newInstance(0, 0);
   protected Resource currentConsumption = Resource.newInstance(0, 0);
-  private Resource amResource;
+  private Resource amResource = Resources.none();
   private boolean unmanagedAM = true;
   private boolean amRunning = false;
+  private LogAggregationContext logAggregationContext;
 
   protected List<RMContainer> newlyAllocatedContainers = 
       new ArrayList<RMContainer>();
@@ -138,6 +140,8 @@ public class SchedulerApplicationAttempt {
               .getApplicationSubmissionContext();
       if (appSubmissionContext != null) {
         unmanagedAM = appSubmissionContext.getUnmanagedAM();
+        this.logAggregationContext =
+            appSubmissionContext.getLogAggregationContext();
       }
     }
   }
@@ -444,7 +448,7 @@ public class SchedulerApplicationAttempt {
         container.setContainerToken(rmContext.getContainerTokenSecretManager()
           .createContainerToken(container.getId(), container.getNodeId(),
             getUser(), container.getResource(), container.getPriority(),
-            rmContainer.getCreationTime()));
+            rmContainer.getCreationTime(), this.logAggregationContext));
         NMToken nmToken =
             rmContext.getNMTokenSecretManager().createAndGetNMToken(getUser(),
               getApplicationAttemptId(), container);

+ 22 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java

@@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -177,6 +178,25 @@ public class RMContainerTokenSecretManager extends
   public Token createContainerToken(ContainerId containerId, NodeId nodeId,
       String appSubmitter, Resource capability, Priority priority,
       long createTime) {
+    return createContainerToken(containerId, nodeId, appSubmitter, capability,
+      priority, createTime, null);
+  }
+
+  /**
+   * Helper function for creating ContainerTokens
+   * 
+   * @param containerId
+   * @param nodeId
+   * @param appSubmitter
+   * @param capability
+   * @param priority
+   * @param createTime
+   * @param logAggregationContext
+   * @return the container-token
+   */
+  public Token createContainerToken(ContainerId containerId, NodeId nodeId,
+      String appSubmitter, Resource capability, Priority priority,
+      long createTime, LogAggregationContext logAggregationContext) {
     byte[] password;
     ContainerTokenIdentifier tokenIdentifier;
     long expiryTimeStamp =
@@ -189,7 +209,8 @@ public class RMContainerTokenSecretManager extends
           new ContainerTokenIdentifier(containerId, nodeId.toString(),
             appSubmitter, capability, expiryTimeStamp, this.currentMasterKey
               .getMasterKey().getKeyId(),
-            ResourceManager.getClusterTimeStamp(), priority, createTime);
+            ResourceManager.getClusterTimeStamp(), priority, createTime,
+            logAggregationContext);
       password = this.createPassword(tokenIdentifier);
 
     } finally {

+ 38 - 39
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/JAXBContextResolver.java

@@ -22,62 +22,61 @@ import com.google.inject.Singleton;
 import com.sun.jersey.api.json.JSONConfiguration;
 import com.sun.jersey.api.json.JSONJAXBContext;
 
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.Set;
+import java.util.*;
 
 import javax.ws.rs.ext.ContextResolver;
 import javax.ws.rs.ext.Provider;
 import javax.xml.bind.JAXBContext;
 
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.UserInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptsInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationStatisticsInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppsInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerQueueInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerQueueInfoList;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterMetricsInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.StatisticsItemInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.FifoSchedulerInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodesInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ResourceInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerTypeInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.UserMetricsInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.UsersInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.*;
 import org.apache.hadoop.yarn.webapp.RemoteExceptionData;
 
 @Singleton
 @Provider
 public class JAXBContextResolver implements ContextResolver<JAXBContext> {
 
-  private JAXBContext context;
-  private final Set<Class> types;
-
-  // you have to specify all the dao classes here
-  private final Class[] cTypes = { AppInfo.class, AppAttemptInfo.class,
-      AppAttemptsInfo.class, ClusterInfo.class,
-      CapacitySchedulerQueueInfo.class, FifoSchedulerInfo.class,
-      SchedulerTypeInfo.class, NodeInfo.class, UserMetricsInfo.class,
-      CapacitySchedulerInfo.class, ClusterMetricsInfo.class,
-      SchedulerInfo.class, AppsInfo.class, NodesInfo.class,
-      RemoteExceptionData.class, CapacitySchedulerQueueInfoList.class,
-      ResourceInfo.class, UsersInfo.class, UserInfo.class,
-      ApplicationStatisticsInfo.class, StatisticsItemInfo.class};
+  private final Map<Class, JAXBContext> typesContextMap;
 
   public JAXBContextResolver() throws Exception {
-    this.types = new HashSet<Class>(Arrays.asList(cTypes));
-    this.context = new JSONJAXBContext(JSONConfiguration.natural()
-        .rootUnwrapping(false).build(), cTypes);
+
+    JAXBContext context;
+    JAXBContext unWrappedRootContext;
+
+    // you have to specify all the dao classes here
+    final Class[] cTypes =
+        { AppInfo.class, AppAttemptInfo.class, AppAttemptsInfo.class,
+            ClusterInfo.class, CapacitySchedulerQueueInfo.class,
+            FifoSchedulerInfo.class, SchedulerTypeInfo.class, NodeInfo.class,
+            UserMetricsInfo.class, CapacitySchedulerInfo.class,
+            ClusterMetricsInfo.class, SchedulerInfo.class, AppsInfo.class,
+            NodesInfo.class, RemoteExceptionData.class,
+            CapacitySchedulerQueueInfoList.class, ResourceInfo.class,
+            UsersInfo.class, UserInfo.class, ApplicationStatisticsInfo.class,
+            StatisticsItemInfo.class };
+    // these dao classes need root unwrapping
+    final Class[] rootUnwrappedTypes =
+        { NewApplication.class, ApplicationSubmissionContextInfo.class,
+            ContainerLaunchContextInfo.class, LocalResourceInfo.class,
+            DelegationToken.class };
+
+    this.typesContextMap = new HashMap<Class, JAXBContext>();
+    context =
+        new JSONJAXBContext(JSONConfiguration.natural().rootUnwrapping(false)
+          .build(), cTypes);
+    unWrappedRootContext =
+        new JSONJAXBContext(JSONConfiguration.natural().rootUnwrapping(true)
+          .build(), rootUnwrappedTypes);
+    for (Class type : cTypes) {
+      typesContextMap.put(type, context);
+    }
+    for (Class type : rootUnwrappedTypes) {
+      typesContextMap.put(type, unWrappedRootContext);
+    }
   }
 
   @Override
   public JAXBContext getContext(Class<?> objectType) {
-    return (types.contains(objectType)) ? context : null;
+    return typesContextMap.get(objectType);
   }
 }

+ 18 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java

@@ -49,6 +49,7 @@ import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 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.LogAggregationContext;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -278,7 +279,7 @@ public class MockRM extends ResourceManager {
       boolean waitForAccepted, boolean keepContainers) throws Exception {
     return submitApp(masterMemory, name, user, acls, unmanaged, queue,
         maxAppAttempts, ts, appType, waitForAccepted, keepContainers,
-        false, null, 0);
+        false, null, 0, null);
   }
 
   public RMApp submitApp(int masterMemory, long attemptFailuresValidityInterval)
@@ -287,7 +288,7 @@ public class MockRM extends ResourceManager {
       .getShortUserName(), null, false, null,
       super.getConfig().getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
       YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS), null, null, true, false,
-      false, null, attemptFailuresValidityInterval);
+      false, null, attemptFailuresValidityInterval, null);
   }
 
   public RMApp submitApp(int masterMemory, String name, String user,
@@ -297,14 +298,24 @@ public class MockRM extends ResourceManager {
       ApplicationId applicationId) throws Exception {
     return submitApp(masterMemory, name, user, acls, unmanaged, queue,
       maxAppAttempts, ts, appType, waitForAccepted, keepContainers,
-      isAppIdProvided, applicationId, 0);
+      isAppIdProvided, applicationId, 0, null);
   }
 
+  public RMApp submitApp(int masterMemory,
+      LogAggregationContext logAggregationContext) throws Exception {
+    return submitApp(masterMemory, "", UserGroupInformation.getCurrentUser()
+      .getShortUserName(), null, false, null,
+      super.getConfig().getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
+      YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS), null, null, true, false,
+      false, null, 0, logAggregationContext);
+   }
+
   public RMApp submitApp(int masterMemory, String name, String user,
       Map<ApplicationAccessType, String> acls, boolean unmanaged, String queue,
       int maxAppAttempts, Credentials ts, String appType,
       boolean waitForAccepted, boolean keepContainers, boolean isAppIdProvided,
-      ApplicationId applicationId, long attemptFailuresValidityInterval)
+      ApplicationId applicationId, long attemptFailuresValidityInterval,
+      LogAggregationContext logAggregationContext)
       throws Exception {
     ApplicationId appId = isAppIdProvided ? applicationId : null;
     ApplicationClientProtocol client = getClientRMService();
@@ -342,6 +353,9 @@ public class MockRM extends ResourceManager {
     }
     sub.setAMContainerSpec(clc);
     sub.setAttemptFailuresValidityInterval(attemptFailuresValidityInterval);
+    if (logAggregationContext != null) {
+      sub.setLogAggregationContext(logAggregationContext);
+    }
     req.setApplicationSubmissionContext(sub);
     UserGroupInformation fakeUser =
       UserGroupInformation.createUserForTesting(user, new String[] {"someGroup"});

+ 58 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java

@@ -28,12 +28,14 @@ import org.apache.hadoop.security.SecurityUtilTestHelper;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 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.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
@@ -47,6 +49,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -195,6 +198,58 @@ public class TestContainerAllocation {
     Assert.assertEquals(1, containers.size());
   }
 
+  // This is to test whether LogAggregationContext is passed into
+  // container tokens correctly
+  @Test
+  public void testLogAggregationContextPassedIntoContainerToken()
+      throws Exception {
+    MockRM rm1 = new MockRM(conf);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("127.0.0.1:1234", 8000);
+    MockNM nm2 = rm1.registerNode("127.0.0.1:2345", 8000);
+    // LogAggregationContext is set as null
+    Assert
+      .assertNull(getLogAggregationContextFromContainerToken(rm1, nm1, null));
+
+    // create a not-null LogAggregationContext
+    final int interval = 2000;
+    LogAggregationContext logAggregationContext =
+        LogAggregationContext.newInstance(
+          "includePattern", "excludePattern", interval);
+    LogAggregationContext returned =
+        getLogAggregationContextFromContainerToken(rm1, nm2,
+          logAggregationContext);
+    Assert.assertEquals("includePattern", returned.getIncludePattern());
+    Assert.assertEquals("excludePattern", returned.getExcludePattern());
+    Assert.assertEquals(interval, returned.getRollingIntervalSeconds());
+    rm1.stop();
+  }
+
+  private LogAggregationContext getLogAggregationContextFromContainerToken(
+      MockRM rm1, MockNM nm1, LogAggregationContext logAggregationContext)
+      throws Exception {
+    RMApp app2 = rm1.submitApp(200, logAggregationContext);
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm1);
+    nm1.nodeHeartbeat(true);
+    // request a container.
+    am2.allocate("127.0.0.1", 512, 1, new ArrayList<ContainerId>());
+    ContainerId containerId =
+        ContainerId.newInstance(am2.getApplicationAttemptId(), 2);
+    rm1.waitForState(nm1, containerId, RMContainerState.ALLOCATED);
+
+    // acquire the container.
+    List<Container> containers =
+        am2.allocate(new ArrayList<ResourceRequest>(),
+          new ArrayList<ContainerId>()).getAllocatedContainers();
+    Assert.assertEquals(containerId, containers.get(0).getId());
+    // container token is generated.
+    Assert.assertNotNull(containers.get(0).getContainerToken());
+    ContainerTokenIdentifier token =
+        BuilderUtils.newContainerTokenIdentifier(containers.get(0)
+          .getContainerToken());
+    return token.getLogAggregationContext();
+  }
+
   private volatile int numRetries = 0;
   private class TestRMSecretManagerService extends RMSecretManagerService {
 
@@ -210,10 +265,11 @@ public class TestContainerAllocation {
         @Override
         public Token createContainerToken(ContainerId containerId,
             NodeId nodeId, String appSubmitter, Resource capability,
-            Priority priority, long createTime) {
+            Priority priority, long createTime,
+            LogAggregationContext logAggregationContext) {
           numRetries++;
           return super.createContainerToken(containerId, nodeId, appSubmitter,
-            capability, priority, createTime);
+            capability, priority, createTime, logAggregationContext);
         }
       };
     }

+ 6 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java

@@ -43,6 +43,8 @@ import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.ParserConfigurationException;
 
+import com.sun.jersey.api.client.config.ClientConfig;
+import com.sun.jersey.api.client.config.DefaultClientConfig;
 import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
@@ -214,6 +216,7 @@ public class TestRMWebServicesAppsModification extends JerseyTest {
       "org.apache.hadoop.yarn.server.resourcemanager.webapp")
       .contextListenerClass(GuiceServletConfig.class)
       .filterClass(com.google.inject.servlet.GuiceFilter.class)
+      .clientConfig(new DefaultClientConfig(JAXBContextResolver.class))
       .contextPath("jersey-guice-filter").servletPath("/").build());
     switch (run) {
     case 0:
@@ -550,10 +553,10 @@ public class TestRMWebServicesAppsModification extends JerseyTest {
     }
   }
 
-  // Simple test - just post to /apps/id and validate the response
+  // Simple test - just post to /apps/new-application and validate the response
   @Test
   public void testGetNewApplication() throws Exception {
-    // client().addFilter(new LoggingFilter(System.out));
+    client().addFilter(new LoggingFilter(System.out));
     rm.start();
     String mediaTypes[] =
         { MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML };
@@ -653,7 +656,7 @@ public class TestRMWebServicesAppsModification extends JerseyTest {
     // create a test app and submit it via rest(after getting an app-id) then
     // get the app details from the rmcontext and check that everything matches
 
-    // client().addFilter(new LoggingFilter(System.out));
+    client().addFilter(new LoggingFilter(System.out));
     String lrKey = "example";
     String queueName = "testqueue";
     String appName = "test";

+ 15 - 15
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRest.apt.vm

@@ -2083,12 +2083,12 @@ _01_000001</amContainerLogs>
 
 +---+
 {
-  "application-id":"application_1404198295326_0001",
+  "application-id":"application_1404198295326_0003",
   "maximum-resource-capability":
-   {
-      "memory":"8192",
-      "vCores":"32"
-   }
+    {
+      "memory":8192,
+      "vCores":32
+    }
 }
 +---+
 
@@ -2257,8 +2257,8 @@ _01_000001</amContainerLogs>
               "resource":"hdfs://hdfs-namenode:9000/user/testuser/DistributedShell/demo-app/AppMaster.jar",
               "type":"FILE",
               "visibility":"APPLICATION",
-              "size": "43004",
-              "timestamp": "1405452071209"
+              "size": 43004,
+              "timestamp": 1405452071209
             }
           }
         ]
@@ -2290,15 +2290,15 @@ _01_000001</amContainerLogs>
         ]
       }
     },
-    "unmanaged-AM":"false",
-    "max-app-attempts":"2",
+    "unmanaged-AM":false,
+    "max-app-attempts":2,
     "resource":
     {
-      "memory":"1024",
-      "vCores":"1"
+      "memory":1024,
+      "vCores":1
     },
     "application-type":"YARN",
-    "keep-containers-across-application-attempts":"false"
+    "keep-containers-across-application-attempts":false
   }
 
 +---+
@@ -2797,8 +2797,8 @@ Server: Jetty(6.1.26)
     "renewer":"test-renewer",
     "owner":"client@EXAMPLE.COM",
     "kind":"RM_DELEGATION_TOKEN",
-    "expiration-time":"1405153616489",
-    "max-validity":"1405672016489"
+    "expiration-time":1405153616489,
+    "max-validity":1405672016489
   }
 +---+
 
@@ -2869,7 +2869,7 @@ Server: Jetty(6.1.26)
 
 +---+
   {
-    "expiration-time":"1404112520402"
+    "expiration-time":1404112520402
   }
 +---+
 

Một số tệp đã không được hiển thị bởi vì quá nhiều tập tin thay đổi trong này khác