You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by st...@apache.org on 2017/08/09 17:36:21 UTC

[01/51] [abbrv] hadoop git commit: HDFS-6984. Serialize FileStatus via protobuf.

Repository: hadoop
Updated Branches:
  refs/heads/HADOOP-13345 b4c2ab278 -> b114f2488


http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
index 4c92249..3b3368d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
@@ -31,7 +31,6 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
@@ -47,6 +46,7 @@ import org.apache.hadoop.security.AccessControlException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.EnumSet;
 
 import static org.apache.hadoop.util.Time.now;
 
@@ -384,7 +384,6 @@ class FSDirStatAndListingOp {
    * @param child for a directory listing of the iip, else null
    * @param storagePolicy for the path or closest ancestor
    * @param needLocation if block locations need to be included or not
-   * @param includeStoragePolicy if storage policy should be returned
    * @return a file status
    * @throws java.io.IOException if any error occurs
    */
@@ -439,7 +438,19 @@ class FSDirStatAndListingOp {
     int childrenNum = node.isDirectory() ?
         node.asDirectory().getChildrenNum(snapshot) : 0;
 
+    EnumSet<HdfsFileStatus.Flags> flags =
+        EnumSet.noneOf(HdfsFileStatus.Flags.class);
     INodeAttributes nodeAttrs = fsd.getAttributes(iip);
+    boolean hasAcl = nodeAttrs.getAclFeature() != null;
+    if (hasAcl) {
+      flags.add(HdfsFileStatus.Flags.HAS_ACL);
+    }
+    if (isEncrypted) {
+      flags.add(HdfsFileStatus.Flags.HAS_CRYPT);
+    }
+    if (isErasureCoded) {
+      flags.add(HdfsFileStatus.Flags.HAS_EC);
+    }
     return createFileStatus(
         size,
         node.isDirectory(),
@@ -447,7 +458,8 @@ class FSDirStatAndListingOp {
         blocksize,
         node.getModificationTime(snapshot),
         node.getAccessTime(snapshot),
-        getPermissionForFileStatus(nodeAttrs, isEncrypted, isErasureCoded),
+        nodeAttrs.getFsPermission(),
+        flags,
         nodeAttrs.getUserName(),
         nodeAttrs.getGroupName(),
         node.isSymlink() ? node.asSymlink().getSymlink() : null,
@@ -460,42 +472,22 @@ class FSDirStatAndListingOp {
         loc);
   }
 
-  private static HdfsFileStatus createFileStatus(long length, boolean isdir,
-      int replication, long blocksize, long mtime,
-      long atime, FsPermission permission, String owner, String group,
-      byte[] symlink, byte[] path, long fileId, int childrenNum,
-      FileEncryptionInfo feInfo, byte storagePolicy,
+  private static HdfsFileStatus createFileStatus(
+      long length, boolean isdir,
+      int replication, long blocksize, long mtime, long atime,
+      FsPermission permission, EnumSet<HdfsFileStatus.Flags> flags,
+      String owner, String group, byte[] symlink, byte[] path, long fileId,
+      int childrenNum, FileEncryptionInfo feInfo, byte storagePolicy,
       ErasureCodingPolicy ecPolicy, LocatedBlocks locations) {
     if (locations == null) {
       return new HdfsFileStatus(length, isdir, replication, blocksize,
-          mtime, atime, permission, owner, group, symlink, path, fileId,
-          childrenNum, feInfo, storagePolicy, ecPolicy);
+          mtime, atime, permission, flags, owner, group, symlink, path,
+          fileId, childrenNum, feInfo, storagePolicy, ecPolicy);
     } else {
       return new HdfsLocatedFileStatus(length, isdir, replication, blocksize,
-          mtime, atime, permission, owner, group, symlink, path, fileId,
-          locations, childrenNum, feInfo, storagePolicy, ecPolicy);
-    }
-  }
-
-  /**
-   * Returns an inode's FsPermission for use in an outbound FileStatus.  If the
-   * inode has an ACL or is for an encrypted file/dir, then this method will
-   * return an FsPermissionExtension.
-   *
-   * @param node INode to check
-   * @param isEncrypted boolean true if the file/dir is encrypted
-   * @return FsPermission from inode, with ACL bit on if the inode has an ACL
-   * and encrypted bit on if it represents an encrypted file/dir.
-   */
-  private static FsPermission getPermissionForFileStatus(
-      INodeAttributes node, boolean isEncrypted, boolean isErasureCoded) {
-    FsPermission perm = node.getFsPermission();
-    boolean hasAcl = node.getAclFeature() != null;
-    if (hasAcl || isEncrypted || isErasureCoded) {
-      perm = new FsPermissionExtension(perm, hasAcl,
-          isEncrypted, isErasureCoded);
+          mtime, atime, permission, flags, owner, group, symlink, path,
+          fileId, locations, childrenNum, feInfo, storagePolicy, ecPolicy);
     }
-    return perm;
   }
 
   private static ContentSummary getContentSummaryInt(FSDirectory fsd,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index 07dc5c1..87b1156 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -72,12 +72,13 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.concurrent.ForkJoinPool;
-import java.util.concurrent.RecursiveAction;
+import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
 import java.util.SortedSet;
 import java.util.TreeSet;
+import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.RecursiveAction;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeys.FS_PROTECTED_DIRECTORIES;
@@ -135,11 +136,13 @@ public class FSDirectory implements Closeable {
 
   public final static HdfsFileStatus DOT_RESERVED_STATUS =
       new HdfsFileStatus(0, true, 0, 0, 0, 0, new FsPermission((short) 01770),
-          null, null, null, HdfsFileStatus.EMPTY_NAME, -1L, 0, null,
+          EnumSet.noneOf(HdfsFileStatus.Flags.class), null, null, null,
+          HdfsFileStatus.EMPTY_NAME, -1L, 0, null,
           HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null);
 
   public final static HdfsFileStatus DOT_SNAPSHOT_DIR_STATUS =
-      new HdfsFileStatus(0, true, 0, 0, 0, 0, null, null, null, null,
+      new HdfsFileStatus(0, true, 0, 0, 0, 0, null,
+          EnumSet.noneOf(HdfsFileStatus.Flags.class), null, null, null,
           HdfsFileStatus.EMPTY_NAME, -1L, 0, null,
           HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null);
 
@@ -383,12 +386,15 @@ public class FSDirectory implements Closeable {
    */
   void createReservedStatuses(long cTime) {
     HdfsFileStatus inodes = new HdfsFileStatus(0, true, 0, 0, cTime, cTime,
-        new FsPermission((short) 0770), null, supergroup, null,
+        new FsPermission((short) 0770),
+        EnumSet.noneOf(HdfsFileStatus.Flags.class), null, supergroup, null,
         DOT_INODES, -1L, 0, null,
         HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null);
     HdfsFileStatus raw = new HdfsFileStatus(0, true, 0, 0, cTime, cTime,
-        new FsPermission((short) 0770), null, supergroup, null, RAW, -1L,
-        0, null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null);
+        new FsPermission((short) 0770),
+        EnumSet.noneOf(HdfsFileStatus.Flags.class), null, supergroup, null,
+        RAW, -1L, 0, null,
+        HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null);
     reservedStatuses = new HdfsFileStatus[] {inodes, raw};
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index f0ebcbb..5849712 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -177,6 +177,7 @@ import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.ServiceFailedException;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.UnknownCryptoProtocolVersionException;
@@ -371,9 +372,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
     FileStatus status = null;
     if (stat != null) {
-      Path symlink = stat.isSymlink() ? new Path(stat.getSymlink()) : null;
+      Path symlink = stat.isSymlink()
+          ? new Path(DFSUtilClient.bytes2String(stat.getSymlinkInBytes()))
+          : null;
       Path path = new Path(src);
-      status = new FileStatus(stat.getLen(), stat.isDir(),
+      status = new FileStatus(stat.getLen(), stat.isDirectory(),
           stat.getReplication(), stat.getBlockSize(),
           stat.getModificationTime(),
           stat.getAccessTime(), stat.getPermission(), stat.getOwner(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 9cd58cb..52b422c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -80,6 +80,7 @@ import org.apache.hadoop.ha.protocolPB.HAServiceProtocolPB;
 import org.apache.hadoop.ha.protocolPB.HAServiceProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.inotify.EventBatch;
 import org.apache.hadoop.hdfs.inotify.EventBatchList;
@@ -1430,7 +1431,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     } else if (!stat.isSymlink()) {
       throw new IOException("Path " + path + " is not a symbolic link");
     }
-    return stat.getSymlink();
+    return DFSUtilClient.bytes2String(stat.getSymlinkInBytes());
   }
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
index ffc203f..4b479e0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
@@ -25,6 +25,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -35,6 +36,7 @@ import javax.management.ObjectName;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.protocol.SnapshotInfo;
@@ -345,8 +347,9 @@ public class SnapshotManager implements SnapshotStatsMXBean {
       if (userName == null || userName.equals(dir.getUserName())) {
         SnapshottableDirectoryStatus status = new SnapshottableDirectoryStatus(
             dir.getModificationTime(), dir.getAccessTime(),
-            dir.getFsPermission(), dir.getUserName(), dir.getGroupName(),
-            dir.getLocalNameBytes(), dir.getId(), 
+            dir.getFsPermission(), EnumSet.noneOf(HdfsFileStatus.Flags.class),
+            dir.getUserName(), dir.getGroupName(),
+            dir.getLocalNameBytes(), dir.getId(),
             dir.getChildrenNum(Snapshot.CURRENT_STATE_ID),
             dir.getDirectorySnapshottableFeature().getNumSnapshots(),
             dir.getDirectorySnapshottableFeature().getSnapshotQuota(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
index a8861a8..94752f5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
@@ -17,10 +17,18 @@
  */
 package org.apache.hadoop.hdfs.web;
 
-import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileChecksum;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.XAttr;
+import org.apache.hadoop.fs.XAttrCodec;
 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.DFSUtilClient;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.*;
 import org.apache.hadoop.ipc.RemoteException;
@@ -110,21 +118,20 @@ public class JsonUtil {
     m.put("pathSuffix", status.getLocalName());
     m.put("type", WebHdfsConstants.PathType.valueOf(status));
     if (status.isSymlink()) {
-      m.put("symlink", status.getSymlink());
+      m.put("symlink", DFSUtilClient.bytes2String(status.getSymlinkInBytes()));
     }
-
     m.put("length", status.getLen());
     m.put("owner", status.getOwner());
     m.put("group", status.getGroup());
     FsPermission perm = status.getPermission();
     m.put("permission", toString(perm));
-    if (perm.getAclBit()) {
+    if (status.hasAcl()) {
       m.put("aclBit", true);
     }
-    if (perm.getEncryptedBit()) {
+    if (status.isEncrypted()) {
       m.put("encBit", true);
     }
-    if (perm.getErasureCodedBit()) {
+    if (status.isErasureCoded()) {
       m.put("ecBit", true);
     }
     m.put("accessTime", status.getAccessTime());
@@ -373,15 +380,6 @@ public class JsonUtil {
     FsPermission perm = status.getPermission();
     if (perm != null) {
       m.put("permission", toString(perm));
-      if (perm.getAclBit()) {
-        m.put("aclBit", true);
-      }
-      if (perm.getEncryptedBit()) {
-        m.put("encBit", true);
-      }
-      if (perm.getErasureCodedBit()) {
-        m.put("ecBit", true);
-      }
     }
     final Map<String, Map<String, Object>> finalMap =
         new TreeMap<String, Map<String, Object>>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
index 7a71df8..5b16f4c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
@@ -259,12 +259,14 @@ public class TestDFSClientRetries {
     
     Mockito.doReturn(
             new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
-                (short) 777), "owner", "group", new byte[0], new byte[0],
+                (short) 777), EnumSet.noneOf(HdfsFileStatus.Flags.class),
+                "owner", "group", new byte[0], new byte[0],
                 1010, 0, null, (byte) 0, null)).when(mockNN).getFileInfo(anyString());
     
     Mockito.doReturn(
             new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
-                (short) 777), "owner", "group", new byte[0], new byte[0],
+                (short) 777), EnumSet.noneOf(HdfsFileStatus.Flags.class),
+                "owner", "group", new byte[0], new byte[0],
                 1010, 0, null, (byte) 0, null))
         .when(mockNN)
         .create(anyString(), (FsPermission) anyObject(), anyString(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
index 8eb3b7b..bf02db3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
@@ -891,7 +891,8 @@ public class TestEncryptionZones {
       CipherSuite suite, CryptoProtocolVersion version) throws Exception {
     Mockito.doReturn(
         new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
-            (short) 777), "owner", "group", new byte[0], new byte[0],
+            (short) 777), EnumSet.noneOf(HdfsFileStatus.Flags.class),
+            "owner", "group", new byte[0], new byte[0],
             1010, 0, new FileEncryptionInfo(suite,
             version, new byte[suite.getAlgorithmBlockSize()],
             new byte[suite.getAlgorithmBlockSize()],

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusSerialization.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusSerialization.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusSerialization.java
new file mode 100644
index 0000000..e5d0566
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusSerialization.java
@@ -0,0 +1,153 @@
+/**
+ * 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;
+
+import java.net.URI;
+
+import org.apache.hadoop.fs.FSProtos.FileStatusProto;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto.FileType;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+
+import com.google.protobuf.ByteString;
+
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Verify compatible FileStatus/HdfsFileStatus serialization.
+ */
+public class TestFileStatusSerialization {
+
+  private static void checkFields(FileStatus expected, FileStatus actual) {
+    assertEquals(expected.getPath(), actual.getPath());
+    assertEquals(expected.isDirectory(), actual.isDirectory());
+    assertEquals(expected.getLen(), actual.getLen());
+    assertEquals(expected.getPermission(), actual.getPermission());
+    assertEquals(expected.getOwner(), actual.getOwner());
+    assertEquals(expected.getGroup(), actual.getGroup());
+    assertEquals(expected.getModificationTime(), actual.getModificationTime());
+    assertEquals(expected.getAccessTime(), actual.getAccessTime());
+    assertEquals(expected.getReplication(), actual.getReplication());
+    assertEquals(expected.getBlockSize(), actual.getBlockSize());
+  }
+
+  /**
+   * Test API backwards-compatibility with 2.x applications w.r.t. FsPermission.
+   */
+  @Test
+  @SuppressWarnings("deprecation")
+  public void testFsPermissionCompatibility() throws Exception {
+    final int flagmask = 0x8;
+    // flags compatible with 2.x; fixed as constant in this test to ensure
+    // compatibility is maintained. New flags are not part of the contract this
+    // test verifies.
+    for (int i = 0; i < flagmask; ++i) {
+      FsPermission perm = FsPermission.createImmutable((short) 0013);
+      HdfsFileStatusProto.Builder hspb = HdfsFileStatusProto.newBuilder()
+          .setFileType(FileType.IS_FILE)
+          .setPath(ByteString.copyFromUtf8("hdfs://foobar/dingos/zot"))
+          .setLength(4344)
+          .setPermission(PBHelperClient.convert(perm))
+          .setOwner("hadoop")
+          .setGroup("unqbbc")
+          .setModificationTime(12345678L)
+          .setAccessTime(87654321L)
+          .setBlockReplication(10)
+          .setBlocksize(1L << 33)
+          .setFlags(i);
+      HdfsFileStatus stat = PBHelperClient.convert(hspb.build());
+      stat.makeQualified(new URI("hdfs://foobar"), new Path("/dingos"));
+      assertEquals(new Path("hdfs://foobar/dingos/zot"), stat.getPath());
+
+      // verify deprecated FsPermissionExtension methods
+      FsPermission sp = stat.getPermission();
+      assertEquals(sp.getAclBit(), stat.hasAcl());
+      assertEquals(sp.getEncryptedBit(), stat.isEncrypted());
+      assertEquals(sp.getErasureCodedBit(), stat.isErasureCoded());
+
+      // verify Writable contract
+      DataOutputBuffer dob = new DataOutputBuffer();
+      stat.write(dob);
+      DataInputBuffer dib = new DataInputBuffer();
+      dib.reset(dob.getData(), 0, dob.getLength());
+      FileStatus fstat = new FileStatus();
+      fstat.readFields(dib);
+      checkFields(stat, fstat);
+
+      // FsPermisisonExtension used for HdfsFileStatus, not FileStatus,
+      // attribute flags should still be preserved
+      assertEquals(sp.getAclBit(), fstat.hasAcl());
+      assertEquals(sp.getEncryptedBit(), fstat.isEncrypted());
+      assertEquals(sp.getErasureCodedBit(), fstat.isErasureCoded());
+    }
+  }
+  // param for LocatedFileStatus, HttpFileStatus
+
+  @Test
+  public void testCrossSerializationProto() throws Exception {
+    FsPermission perm = FsPermission.getFileDefault();
+    for (FileType t : FileType.values()) {
+      HdfsFileStatusProto.Builder hspb = HdfsFileStatusProto.newBuilder()
+          .setFileType(t)
+          .setPath(ByteString.copyFromUtf8("hdfs://foobar/dingos"))
+          .setLength(4344)
+          .setPermission(PBHelperClient.convert(perm))
+          .setOwner("hadoop")
+          .setGroup("unqbbc")
+          .setModificationTime(12345678L)
+          .setAccessTime(87654321L)
+          .setBlockReplication(10)
+          .setBlocksize(1L << 33);
+      if (FileType.IS_SYMLINK.equals(t)) {
+        hspb.setSymlink(ByteString.copyFromUtf8("hdfs://yaks/dingos"));
+      }
+      if (FileType.IS_FILE.equals(t)) {
+        hspb.setFileId(4544);
+      }
+      HdfsFileStatusProto hsp = hspb.build();
+      byte[] src = hsp.toByteArray();
+      FileStatusProto fsp = FileStatusProto.parseFrom(src);
+      assertEquals(hsp.getPath().toStringUtf8(), fsp.getPath());
+      assertEquals(hsp.getLength(), fsp.getLength());
+      assertEquals(hsp.getPermission().getPerm(),
+                   fsp.getPermission().getPerm());
+      assertEquals(hsp.getOwner(), fsp.getOwner());
+      assertEquals(hsp.getGroup(), fsp.getGroup());
+      assertEquals(hsp.getModificationTime(), fsp.getModificationTime());
+      assertEquals(hsp.getAccessTime(), fsp.getAccessTime());
+      assertEquals(hsp.getSymlink().toStringUtf8(), fsp.getSymlink());
+      assertEquals(hsp.getBlockReplication(), fsp.getBlockReplication());
+      assertEquals(hsp.getBlocksize(), fsp.getBlockSize());
+      assertEquals(hsp.getFileType().ordinal(), fsp.getFileType().ordinal());
+
+      // verify unknown fields preserved
+      byte[] dst = fsp.toByteArray();
+      HdfsFileStatusProto hsp2 = HdfsFileStatusProto.parseFrom(dst);
+      assertEquals(hsp, hsp2);
+      checkFields(PBHelperClient.convert(hsp), PBHelperClient.convert(hsp2));
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
index 16cdf9b..161e227 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
@@ -30,6 +30,7 @@ import java.io.DataOutputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
+import java.util.EnumSet;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -353,12 +354,14 @@ public class TestLease {
 
     Mockito.doReturn(
         new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
-            (short) 777), "owner", "group", new byte[0], new byte[0],
+            (short) 777), EnumSet.noneOf(HdfsFileStatus.Flags.class),
+            "owner", "group", new byte[0], new byte[0],
             1010, 0, null, (byte) 0, null)).when(mcp).getFileInfo(anyString());
     Mockito
         .doReturn(
             new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
-                (short) 777), "owner", "group", new byte[0], new byte[0],
+                (short) 777), EnumSet.noneOf(HdfsFileStatus.Flags.class),
+                "owner", "group", new byte[0], new byte[0],
                 1010, 0, null, (byte) 0, null))
         .when(mcp)
         .create(anyString(), (FsPermission) anyObject(), anyString(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/AclTestHelpers.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/AclTestHelpers.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/AclTestHelpers.java
index 52a6858..646e800 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/AclTestHelpers.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/AclTestHelpers.java
@@ -21,6 +21,7 @@ import static org.junit.Assert.*;
 
 import java.io.IOException;
 
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.AclEntry;
@@ -141,6 +142,11 @@ public final class AclTestHelpers {
     }
   }
 
+  public static void assertPermission(FileSystem fs, Path pathToCheck,
+      short perm) throws IOException {
+    assertPermission(fs, pathToCheck, perm, (perm & (1 << 12)) != 0);
+  }
+
   /**
    * Asserts the value of the FsPermission bits on the inode of a specific path.
    *
@@ -150,10 +156,11 @@ public final class AclTestHelpers {
    * @throws IOException thrown if there is an I/O error
    */
   public static void assertPermission(FileSystem fs, Path pathToCheck,
-      short perm) throws IOException {
+      short perm, boolean hasAcl) throws IOException {
     short filteredPerm = (short)(perm & 01777);
-    FsPermission fsPermission = fs.getFileStatus(pathToCheck).getPermission();
+    FileStatus stat = fs.getFileStatus(pathToCheck);
+    FsPermission fsPermission = stat.getPermission();
     assertEquals(filteredPerm, fsPermission.toShort());
-    assertEquals(((perm & (1 << 12)) != 0), fsPermission.getAclBit());
+    assertEquals(hasAcl, stat.hasAcl());
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
index 6c755e7..38c17b7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
@@ -48,6 +48,7 @@ import java.nio.channels.FileChannel;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -1355,7 +1356,8 @@ public class TestFsck {
     byte storagePolicy = 0;
 
     HdfsFileStatus file = new HdfsFileStatus(length, isDir, blockReplication,
-        blockSize, modTime, accessTime, perms, owner, group, symlink,
+        blockSize, modTime, accessTime, perms,
+        EnumSet.noneOf(HdfsFileStatus.Flags.class), owner, group, symlink,
         path, fileId, numChildren, null, storagePolicy, null);
     Result replRes = new ReplicationResult(conf);
     Result ecRes = new ErasureCodingResult(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
index 5a3d451..edb79d3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
@@ -23,6 +23,7 @@ import static org.apache.hadoop.fs.permission.FsAction.*;
 import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.*;
 
 import java.io.IOException;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -37,6 +38,7 @@ 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.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -51,10 +53,12 @@ import com.google.common.collect.Lists;
 
 public class TestJsonUtil {
   static FileStatus toFileStatus(HdfsFileStatus f, String parent) {
-    return new FileStatus(f.getLen(), f.isDir(), f.getReplication(),
+    return new FileStatus(f.getLen(), f.isDirectory(), f.getReplication(),
         f.getBlockSize(), f.getModificationTime(), f.getAccessTime(),
         f.getPermission(), f.getOwner(), f.getGroup(),
-        f.isSymlink() ? new Path(f.getSymlink()) : null,
+        f.isSymlink()
+          ? new Path(DFSUtilClient.bytes2String(f.getSymlinkInBytes()))
+          : null,
         new Path(f.getFullName(parent)));
   }
 
@@ -63,7 +67,8 @@ public class TestJsonUtil {
     final long now = Time.now();
     final String parent = "/dir";
     final HdfsFileStatus status = new HdfsFileStatus(1001L, false, 3, 1L << 26,
-        now, now + 10, new FsPermission((short) 0644), "user", "group",
+        now, now + 10, new FsPermission((short) 0644),
+        EnumSet.noneOf(HdfsFileStatus.Flags.class), "user", "group",
         DFSUtil.string2Bytes("bar"), DFSUtil.string2Bytes("foo"),
         HdfsConstants.GRANDFATHER_INODE_ID, 0, null, (byte) 0, null);
     final FileStatus fstatus = toFileStatus(status, parent);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[22/51] [abbrv] hadoop git commit: YARN-6951. Fix debug log when Resource Handler chain is enabled. Contributed by Yang Wang.

Posted by st...@apache.org.
YARN-6951. Fix debug log when Resource Handler chain is enabled. Contributed by Yang Wang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/46b7054f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/46b7054f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/46b7054f

Branch: refs/heads/HADOOP-13345
Commit: 46b7054fa7eae9129c21c9f3dc70acff46bfdc41
Parents: d91b7a8
Author: Sunil G <su...@apache.org>
Authored: Mon Aug 7 13:15:46 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Mon Aug 7 13:15:46 2017 +0530

----------------------------------------------------------------------
 .../hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/46b7054f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
index 2aaa835..b3e13b4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
@@ -307,7 +307,7 @@ public class LinuxContainerExecutor extends ContainerExecutor {
           .getConfiguredResourceHandlerChain(conf);
       if (LOG.isDebugEnabled()) {
         LOG.debug("Resource handler chain enabled = " + (resourceHandlerChain
-            == null));
+            != null));
       }
       if (resourceHandlerChain != null) {
         LOG.debug("Bootstrapping resource handler chain");


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[24/51] [abbrv] hadoop git commit: YARN-6873. Moving logging APIs over to slf4j in hadoop-yarn-server-applicationhistoryservice. Contributed by Yeliang Cang.

Posted by st...@apache.org.
YARN-6873. Moving logging APIs over to slf4j in hadoop-yarn-server-applicationhistoryservice. Contributed by Yeliang Cang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/839e077f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/839e077f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/839e077f

Branch: refs/heads/HADOOP-13345
Commit: 839e077faf4019d6efdcd89d95930023cd0b0a08
Parents: a4eb701
Author: Akira Ajisaka <aa...@apache.org>
Authored: Mon Aug 7 18:56:00 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Mon Aug 7 18:56:00 2017 +0900

----------------------------------------------------------------------
 .../ApplicationHistoryClientService.java        |  8 ++---
 .../ApplicationHistoryManagerImpl.java          |  8 ++---
 ...pplicationHistoryManagerOnTimelineStore.java |  8 ++---
 .../ApplicationHistoryServer.java               | 10 +++---
 .../FileSystemApplicationHistoryStore.java      | 22 ++++++------
 .../webapp/AHSWebServices.java                  |  7 ++--
 .../webapp/NavBlock.java                        |  8 ++---
 .../timeline/KeyValueBasedTimelineStore.java    |  8 ++---
 .../server/timeline/LeveldbTimelineStore.java   | 35 ++++++++++----------
 .../yarn/server/timeline/RollingLevelDB.java    | 15 +++++----
 .../timeline/RollingLevelDBTimelineStore.java   | 22 ++++++------
 .../server/timeline/TimelineDataManager.java    |  7 ++--
 .../recovery/LeveldbTimelineStateStore.java     | 30 ++++++++---------
 .../timeline/security/TimelineACLsManager.java  |  7 ++--
 ...lineDelegationTokenSecretManagerService.java |  8 ++---
 .../timeline/webapp/TimelineWebServices.java    |  7 ++--
 .../TestFileSystemApplicationHistoryStore.java  |  8 ++---
 .../timeline/TestLeveldbTimelineStore.java      |  2 +-
 18 files changed, 111 insertions(+), 109 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/839e077f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryClientService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryClientService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryClientService.java
index 73d5d39..7d57048 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryClientService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryClientService.java
@@ -22,8 +22,6 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
@@ -61,11 +59,13 @@ import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.server.timeline.security.authorize.TimelinePolicyProvider;
 
 import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class ApplicationHistoryClientService extends AbstractService implements
     ApplicationHistoryProtocol {
-  private static final Log LOG = LogFactory
-    .getLog(ApplicationHistoryClientService.class);
+  private static final Logger LOG =
+          LoggerFactory.getLogger(ApplicationHistoryClientService.class);
   private ApplicationHistoryManager history;
   private Server server;
   private InetSocketAddress bindAddress;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/839e077f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerImpl.java
index 130bb32..b8931d8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerImpl.java
@@ -23,8 +23,6 @@ import java.util.HashMap;
 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.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
@@ -42,11 +40,13 @@ import org.apache.hadoop.yarn.server.applicationhistoryservice.records.Container
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class ApplicationHistoryManagerImpl extends AbstractService implements
     ApplicationHistoryManager {
-  private static final Log LOG = LogFactory
-    .getLog(ApplicationHistoryManagerImpl.class);
+  private static final Logger LOG =
+          LoggerFactory.getLogger(ApplicationHistoryManagerImpl.class);
   private static final String UNAVAILABLE = "N/A";
 
   private ApplicationHistoryStore historyStore;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/839e077f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
index 5404338..9240ed8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
@@ -28,8 +28,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AuthorizationException;
@@ -69,12 +67,14 @@ import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
     implements
     ApplicationHistoryManager {
-  private static final Log LOG = LogFactory
-      .getLog(ApplicationHistoryManagerOnTimelineStore.class);
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ApplicationHistoryManagerOnTimelineStore.class);
 
   @VisibleForTesting
   static final String UNAVAILABLE = "N/A";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/839e077f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
index 6e6e98b..85e5f2d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
@@ -22,8 +22,6 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.http.HttpServer2;
@@ -60,6 +58,8 @@ import org.eclipse.jetty.servlet.FilterHolder;
 import org.eclipse.jetty.webapp.WebAppContext;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * History server that keeps track of all types of history in the cluster.
@@ -68,8 +68,8 @@ import com.google.common.annotations.VisibleForTesting;
 public class ApplicationHistoryServer extends CompositeService {
 
   public static final int SHUTDOWN_HOOK_PRIORITY = 30;
-  private static final Log LOG = LogFactory
-    .getLog(ApplicationHistoryServer.class);
+  private static final Logger LOG = LoggerFactory
+      .getLogger(ApplicationHistoryServer.class);
 
   private ApplicationHistoryClientService ahsClientService;
   private ApplicationACLsManager aclsManager;
@@ -178,7 +178,7 @@ public class ApplicationHistoryServer extends CompositeService {
       appHistoryServer.init(conf);
       appHistoryServer.start();
     } catch (Throwable t) {
-      LOG.fatal("Error starting ApplicationHistoryServer", t);
+      LOG.error("Error starting ApplicationHistoryServer", t);
       ExitUtil.terminate(-1, "Error starting ApplicationHistoryServer");
     }
     return appHistoryServer;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/839e077f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/FileSystemApplicationHistoryStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/FileSystemApplicationHistoryStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/FileSystemApplicationHistoryStore.java
index be7bc6d..fa2da44 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/FileSystemApplicationHistoryStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/FileSystemApplicationHistoryStore.java
@@ -30,8 +30,6 @@ import java.util.Map.Entry;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -74,6 +72,8 @@ import org.apache.hadoop.yarn.server.applicationhistoryservice.records.impl.pb.C
 import org.apache.hadoop.yarn.util.ConverterUtils;
 
 import com.google.protobuf.InvalidProtocolBufferException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * File system implementation of {@link ApplicationHistoryStore}. In this
@@ -89,8 +89,8 @@ import com.google.protobuf.InvalidProtocolBufferException;
 public class FileSystemApplicationHistoryStore extends AbstractService
     implements ApplicationHistoryStore {
 
-  private static final Log LOG = LogFactory
-    .getLog(FileSystemApplicationHistoryStore.class);
+  private static final Logger LOG = LoggerFactory
+      .getLogger(FileSystemApplicationHistoryStore.class);
 
   private static final String ROOT_DIR_NAME = "ApplicationHistoryDataRoot";
   private static final int MIN_BLOCK_SIZE = 256 * 1024;
@@ -141,7 +141,7 @@ public class FileSystemApplicationHistoryStore extends AbstractService
       }
       outstandingWriters.clear();
     } finally {
-      IOUtils.cleanup(LOG, fs);
+      IOUtils.cleanupWithLogger(LOG, fs);
     }
     super.serviceStop();
   }
@@ -711,12 +711,12 @@ public class FileSystemApplicationHistoryStore extends AbstractService
     }
 
     public void reset() throws IOException {
-      IOUtils.cleanup(LOG, scanner);
+      IOUtils.cleanupWithLogger(LOG, scanner);
       scanner = reader.createScanner();
     }
 
     public void close() {
-      IOUtils.cleanup(LOG, scanner, reader, fsdis);
+      IOUtils.cleanupWithLogger(LOG, scanner, reader, fsdis);
     }
 
   }
@@ -740,13 +740,13 @@ public class FileSystemApplicationHistoryStore extends AbstractService
                 YarnConfiguration.DEFAULT_FS_APPLICATION_HISTORY_STORE_COMPRESSION_TYPE), null,
                 getConfig());
       } catch (IOException e) {
-        IOUtils.cleanup(LOG, fsdos);
+        IOUtils.cleanupWithLogger(LOG, fsdos);
         throw e;
       }
     }
 
     public synchronized void close() {
-      IOUtils.cleanup(LOG, writer, fsdos);
+      IOUtils.cleanupWithLogger(LOG, writer, fsdos);
     }
 
     public synchronized void writeHistoryData(HistoryDataKey key, byte[] value)
@@ -756,13 +756,13 @@ public class FileSystemApplicationHistoryStore extends AbstractService
         dos = writer.prepareAppendKey(-1);
         key.write(dos);
       } finally {
-        IOUtils.cleanup(LOG, dos);
+        IOUtils.cleanupWithLogger(LOG, dos);
       }
       try {
         dos = writer.prepareAppendValue(value.length);
         dos.write(value);
       } finally {
-        IOUtils.cleanup(LOG, dos);
+        IOUtils.cleanupWithLogger(LOG, dos);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/839e077f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
index 6195199..13410a8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
@@ -42,8 +42,6 @@ import javax.ws.rs.core.Response;
 import javax.ws.rs.core.StreamingOutput;
 import javax.ws.rs.core.Response.ResponseBuilder;
 import javax.ws.rs.core.Response.Status;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
@@ -80,12 +78,15 @@ import com.google.inject.Inject;
 import com.google.inject.Singleton;
 import com.sun.jersey.api.client.ClientHandlerException;
 import com.sun.jersey.api.client.UniformInterfaceException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @Singleton
 @Path("/ws/v1/applicationhistory")
 public class AHSWebServices extends WebServices {
 
-  private static final Log LOG = LogFactory.getLog(AHSWebServices.class);
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AHSWebServices.class);
   private static final String NM_DOWNLOAD_URI_STR =
       "/ws/v1/node/containers";
   private static final Joiner JOINER = Joiner.on("");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/839e077f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/NavBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/NavBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/NavBlock.java
index 3ee4dd1..915af4a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/NavBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/NavBlock.java
@@ -18,21 +18,19 @@
 
 package org.apache.hadoop.yarn.server.applicationhistoryservice.webapp;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.util.Log4jWarningErrorMetricsAppender;
 import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
+import static org.apache.hadoop.util.GenericsUtil.isLog4jLogger;
+
 public class NavBlock extends HtmlBlock {
 
   @Override
   public void render(Block html) {
     boolean addErrorsAndWarningsLink = false;
-    Log log = LogFactory.getLog(NavBlock.class);
-    if (log instanceof Log4JLogger) {
+    if (isLog4jLogger(NavBlock.class)) {
       Log4jWarningErrorMetricsAppender appender =
           Log4jWarningErrorMetricsAppender.findAppender();
       if (appender != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/839e077f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/KeyValueBasedTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/KeyValueBasedTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/KeyValueBasedTimelineStore.java
index 79e2bf2..82db770 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/KeyValueBasedTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/KeyValueBasedTimelineStore.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.yarn.server.timeline;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.service.AbstractService;
@@ -33,6 +31,8 @@ import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents.EventsOfOneEnt
 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.TimelineDataManager.CheckAcl;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -71,8 +71,8 @@ abstract class KeyValueBasedTimelineStore
 
   private boolean serviceStopped = false;
 
-  private static final Log LOG
-      = LogFactory.getLog(KeyValueBasedTimelineStore.class);
+  private static final Logger LOG
+      = LoggerFactory.getLogger(KeyValueBasedTimelineStore.class);
 
   public KeyValueBasedTimelineStore() {
     super(KeyValueBasedTimelineStore.class.getName());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/839e077f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java
index ffe0413..e3db1dc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java
@@ -22,8 +22,6 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.commons.collections.map.LRUMap;
 import org.apache.commons.io.FileUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -48,6 +46,7 @@ import org.apache.hadoop.yarn.server.timeline.util.LeveldbUtils.KeyParser;
 import org.apache.hadoop.yarn.server.utils.LeveldbIterator;
 import org.fusesource.leveldbjni.JniDBFactory;
 import org.iq80.leveldb.*;
+import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.IOException;
@@ -118,8 +117,8 @@ import static org.fusesource.leveldbjni.JniDBFactory.bytes;
 @InterfaceStability.Unstable
 public class LeveldbTimelineStore extends AbstractService
     implements TimelineStore {
-  private static final Log LOG = LogFactory
-      .getLog(LeveldbTimelineStore.class);
+  private static final org.slf4j.Logger LOG = LoggerFactory
+      .getLogger(LeveldbTimelineStore.class);
 
   @Private
   @VisibleForTesting
@@ -240,7 +239,7 @@ public class LeveldbTimelineStore extends AbstractService
         localFS.setPermission(dbPath, LEVELDB_DIR_UMASK);
       }
     } finally {
-      IOUtils.cleanup(LOG, localFS);
+      IOUtils.cleanupWithLogger(LOG, localFS);
     }
     LOG.info("Using leveldb path " + dbPath);
     try {
@@ -284,7 +283,7 @@ public class LeveldbTimelineStore extends AbstractService
             " closing db now", e);
       }
     }
-    IOUtils.cleanup(LOG, db);
+    IOUtils.cleanupWithLogger(LOG, db);
     super.serviceStop();
   }
 
@@ -320,7 +319,7 @@ public class LeveldbTimelineStore extends AbstractService
           discardOldEntities(timestamp);
           Thread.sleep(ttlInterval);
         } catch (IOException e) {
-          LOG.error(e);
+          LOG.error(e.toString());
         } catch (InterruptedException e) {
           LOG.info("Deletion thread received interrupt, exiting");
           break;
@@ -394,7 +393,7 @@ public class LeveldbTimelineStore extends AbstractService
     } catch(DBException e) {
       throw new IOException(e);            	
     } finally {
-      IOUtils.cleanup(LOG, iterator);
+      IOUtils.cleanupWithLogger(LOG, iterator);
     }
   }
 
@@ -570,7 +569,7 @@ public class LeveldbTimelineStore extends AbstractService
     } catch(DBException e) {
       throw new IOException(e);            	
     } finally {
-      IOUtils.cleanup(LOG, iterator);
+      IOUtils.cleanupWithLogger(LOG, iterator);
     }
     return events;
   }
@@ -753,7 +752,7 @@ public class LeveldbTimelineStore extends AbstractService
     } catch(DBException e) {
       throw new IOException(e);   	
     } finally {
-      IOUtils.cleanup(LOG, iterator);
+      IOUtils.cleanupWithLogger(LOG, iterator);
     }
   }
   
@@ -925,7 +924,7 @@ public class LeveldbTimelineStore extends AbstractService
     } finally {
       lock.unlock();
       writeLocks.returnLock(lock);
-      IOUtils.cleanup(LOG, writeBatch);
+      IOUtils.cleanupWithLogger(LOG, writeBatch);
     }
 
     for (EntityIdentifier relatedEntity : relatedEntitiesWithoutStartTimes) {
@@ -1376,7 +1375,7 @@ public class LeveldbTimelineStore extends AbstractService
     } catch(DBException e) {
       throw new IOException(e);            	
     } finally {
-      IOUtils.cleanup(LOG, iterator);
+      IOUtils.cleanupWithLogger(LOG, iterator);
     }
   }
 
@@ -1506,7 +1505,7 @@ public class LeveldbTimelineStore extends AbstractService
     } catch(DBException e) {
       throw new IOException(e);
     } finally {
-      IOUtils.cleanup(LOG, writeBatch);
+      IOUtils.cleanupWithLogger(LOG, writeBatch);
     }
   }
 
@@ -1548,7 +1547,7 @@ public class LeveldbTimelineStore extends AbstractService
           LOG.error("Got IOException while deleting entities for type " +
               entityType + ", continuing to next type", e);
         } finally {
-          IOUtils.cleanup(LOG, iterator, pfIterator);
+          IOUtils.cleanupWithLogger(LOG, iterator, pfIterator);
           deleteLock.writeLock().unlock();
           if (typeCount > 0) {
             LOG.info("Deleted " + typeCount + " entities of type " +
@@ -1629,7 +1628,7 @@ public class LeveldbTimelineStore extends AbstractService
       String incompatibleMessage = 
           "Incompatible version for timeline store: expecting version " 
               + getCurrentVersion() + ", but loading version " + loadedVersion;
-      LOG.fatal(incompatibleMessage);
+      LOG.error(incompatibleMessage);
       throw new IOException(incompatibleMessage);
     }
   }
@@ -1718,7 +1717,7 @@ public class LeveldbTimelineStore extends AbstractService
     } catch(DBException e) {
       throw new IOException(e);            	
     } finally {
-      IOUtils.cleanup(LOG, writeBatch);
+      IOUtils.cleanupWithLogger(LOG, writeBatch);
     }
   }
 
@@ -1755,7 +1754,7 @@ public class LeveldbTimelineStore extends AbstractService
     } catch(DBException e) {
       throw new IOException(e);            	
     } finally {
-      IOUtils.cleanup(LOG, iterator);
+      IOUtils.cleanupWithLogger(LOG, iterator);
     }
   }
 
@@ -1805,7 +1804,7 @@ public class LeveldbTimelineStore extends AbstractService
     } catch(DBException e) {
       throw new IOException(e);            	
     } finally {
-      IOUtils.cleanup(LOG, iterator);
+      IOUtils.cleanupWithLogger(LOG, iterator);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/839e077f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/RollingLevelDB.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/RollingLevelDB.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/RollingLevelDB.java
index 6d10671..5c511a3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/RollingLevelDB.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/RollingLevelDB.java
@@ -33,8 +33,6 @@ import java.util.Map.Entry;
 
 import org.apache.commons.io.FilenameUtils;
 import org.apache.commons.lang.time.FastDateFormat;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -45,6 +43,8 @@ import org.fusesource.leveldbjni.JniDBFactory;
 import org.iq80.leveldb.DB;
 import org.iq80.leveldb.Options;
 import org.iq80.leveldb.WriteBatch;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Contains the logic to lookup a leveldb by timestamp so that multiple smaller
@@ -54,7 +54,8 @@ import org.iq80.leveldb.WriteBatch;
 class RollingLevelDB {
 
   /** Logger for this class. */
-  private static final Log LOG = LogFactory.getLog(RollingLevelDB.class);
+  private static final Logger LOG = LoggerFactory.
+      getLogger(RollingLevelDB.class);
   /** Factory to open and create new leveldb instances. */
   private static JniDBFactory factory = new JniDBFactory();
   /** Thread safe date formatter. */
@@ -151,7 +152,7 @@ class RollingLevelDB {
     }
 
     public void close() {
-      IOUtils.cleanup(LOG, writeBatch);
+      IOUtils.cleanupWithLogger(LOG, writeBatch);
     }
   }
 
@@ -346,7 +347,7 @@ class RollingLevelDB {
         .iterator();
     while (iterator.hasNext()) {
       Entry<Long, DB> entry = iterator.next();
-      IOUtils.cleanup(LOG, entry.getValue());
+      IOUtils.cleanupWithLogger(LOG, entry.getValue());
       String dbName = fdf.format(entry.getKey());
       Path path = new Path(rollingDBPath, getName() + "." + dbName);
       try {
@@ -361,9 +362,9 @@ class RollingLevelDB {
 
   public void stop() throws Exception {
     for (DB db : rollingdbs.values()) {
-      IOUtils.cleanup(LOG, db);
+      IOUtils.cleanupWithLogger(LOG, db);
     }
-    IOUtils.cleanup(LOG, lfs);
+    IOUtils.cleanupWithLogger(LOG, lfs);
   }
 
   private long computeNextCheckMillis(long now) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/839e077f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/RollingLevelDBTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/RollingLevelDBTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/RollingLevelDBTimelineStore.java
index 00f6630..1ac170c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/RollingLevelDBTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/RollingLevelDBTimelineStore.java
@@ -38,8 +38,6 @@ import java.util.TreeMap;
 
 import org.apache.commons.collections.map.LRUMap;
 import org.apache.commons.lang.StringUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -76,6 +74,8 @@ import org.iq80.leveldb.ReadOptions;
 import org.iq80.leveldb.WriteBatch;
 import org.nustaq.serialization.FSTConfiguration;
 import org.nustaq.serialization.FSTClazzNameRegistry;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 
@@ -168,8 +168,8 @@ import static org.fusesource.leveldbjni.JniDBFactory.bytes;
 @InterfaceStability.Unstable
 public class RollingLevelDBTimelineStore extends AbstractService implements
     TimelineStore {
-  private static final Log LOG = LogFactory
-      .getLog(RollingLevelDBTimelineStore.class);
+  private static final Logger LOG = LoggerFactory
+      .getLogger(RollingLevelDBTimelineStore.class);
   private static FSTConfiguration fstConf =
       FSTConfiguration.createDefaultConfiguration();
   // Fall back to 2.24 parsing if 2.50 parsing fails
@@ -368,9 +368,9 @@ public class RollingLevelDBTimelineStore extends AbstractService implements
             + " closing db now", e);
       }
     }
-    IOUtils.cleanup(LOG, domaindb);
-    IOUtils.cleanup(LOG, starttimedb);
-    IOUtils.cleanup(LOG, ownerdb);
+    IOUtils.cleanupWithLogger(LOG, domaindb);
+    IOUtils.cleanupWithLogger(LOG, starttimedb);
+    IOUtils.cleanupWithLogger(LOG, ownerdb);
     entitydb.stop();
     indexdb.stop();
     super.serviceStop();
@@ -399,7 +399,7 @@ public class RollingLevelDBTimelineStore extends AbstractService implements
           discardOldEntities(timestamp);
           Thread.sleep(ttlInterval);
         } catch (IOException e) {
-          LOG.error(e);
+          LOG.error(e.toString());
         } catch (InterruptedException e) {
           LOG.info("Deletion thread received interrupt, exiting");
           break;
@@ -1525,7 +1525,7 @@ public class RollingLevelDBTimelineStore extends AbstractService implements
                   + ". Total start times deleted so far this cycle: "
                   + startTimesCount);
             }
-            IOUtils.cleanup(LOG, writeBatch);
+            IOUtils.cleanupWithLogger(LOG, writeBatch);
             writeBatch = starttimedb.createWriteBatch();
             batchSize = 0;
           }
@@ -1545,7 +1545,7 @@ public class RollingLevelDBTimelineStore extends AbstractService implements
       LOG.info("Deleted " + startTimesCount + "/" + totalCount
           + " start time entities earlier than " + minStartTime);
     } finally {
-      IOUtils.cleanup(LOG, writeBatch);
+      IOUtils.cleanupWithLogger(LOG, writeBatch);
     }
     return startTimesCount;
   }
@@ -1622,7 +1622,7 @@ public class RollingLevelDBTimelineStore extends AbstractService implements
       String incompatibleMessage = "Incompatible version for timeline store: "
           + "expecting version " + getCurrentVersion()
           + ", but loading version " + loadedVersion;
-      LOG.fatal(incompatibleMessage);
+      LOG.error(incompatibleMessage);
       throw new IOException(incompatibleMessage);
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/839e077f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
index 57a9346..56b71fa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
@@ -26,8 +26,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.SortedSet;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.AbstractService;
@@ -45,6 +43,8 @@ import org.apache.hadoop.yarn.server.timeline.security.TimelineACLsManager;
 import org.apache.hadoop.yarn.webapp.BadRequestException;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The class wrap over the timeline store and the ACLs manager. It does some non
@@ -54,7 +54,8 @@ import com.google.common.annotations.VisibleForTesting;
  */
 public class TimelineDataManager extends AbstractService {
 
-  private static final Log LOG = LogFactory.getLog(TimelineDataManager.class);
+  private static final Logger LOG =
+          LoggerFactory.getLogger(TimelineDataManager.class);
   @VisibleForTesting
   public static final String DEFAULT_DOMAIN_ID = "DEFAULT";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/839e077f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/recovery/LeveldbTimelineStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/recovery/LeveldbTimelineStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/recovery/LeveldbTimelineStateStore.java
index b62a541..bcd57ef 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/recovery/LeveldbTimelineStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/recovery/LeveldbTimelineStateStore.java
@@ -28,8 +28,6 @@ import java.io.File;
 import java.io.IOException;
 
 import com.google.common.annotations.VisibleForTesting;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -50,6 +48,8 @@ import org.iq80.leveldb.DB;
 import org.iq80.leveldb.DBException;
 import org.iq80.leveldb.Options;
 import org.iq80.leveldb.WriteBatch;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import static org.fusesource.leveldbjni.JniDBFactory.bytes;
 
@@ -60,8 +60,8 @@ import static org.fusesource.leveldbjni.JniDBFactory.bytes;
 public class LeveldbTimelineStateStore extends
     TimelineStateStore {
 
-  public static final Log LOG =
-      LogFactory.getLog(LeveldbTimelineStateStore.class);
+  public static final Logger LOG =
+      LoggerFactory.getLogger(LeveldbTimelineStateStore.class);
 
   private static final String DB_NAME = "timeline-state-store.ldb";
   private static final FsPermission LEVELDB_DIR_UMASK = FsPermission
@@ -103,7 +103,7 @@ public class LeveldbTimelineStateStore extends
         localFS.setPermission(dbPath, LEVELDB_DIR_UMASK);
       }
     } finally {
-      IOUtils.cleanup(LOG, localFS);
+      IOUtils.cleanupWithLogger(LOG, localFS);
     }
     JniDBFactory factory = new JniDBFactory();
     try {
@@ -131,7 +131,7 @@ public class LeveldbTimelineStateStore extends
 
   @Override
   protected void closeStorage() throws IOException {
-    IOUtils.cleanup(LOG, db);
+    IOUtils.cleanupWithLogger(LOG, db);
   }
 
   @Override
@@ -168,8 +168,8 @@ public class LeveldbTimelineStateStore extends
     } catch (DBException e) {
       throw new IOException(e);
     } finally {
-      IOUtils.cleanup(LOG, ds);
-      IOUtils.cleanup(LOG, batch);
+      IOUtils.cleanupWithLogger(LOG, ds);
+      IOUtils.cleanupWithLogger(LOG, batch);
     }
   }
 
@@ -239,7 +239,7 @@ public class LeveldbTimelineStateStore extends
       key.write(dataStream);
       dataStream.close();
     } finally {
-      IOUtils.cleanup(LOG, dataStream);
+      IOUtils.cleanupWithLogger(LOG, dataStream);
     }
     return memStream.toByteArray();
   }
@@ -253,7 +253,7 @@ public class LeveldbTimelineStateStore extends
     try {
       key.readFields(in);
     } finally {
-      IOUtils.cleanup(LOG, in);
+      IOUtils.cleanupWithLogger(LOG, in);
     }
     state.tokenMasterKeyState.add(key);
   }
@@ -267,7 +267,7 @@ public class LeveldbTimelineStateStore extends
     try {
       data.readFields(in);
     } finally {
-      IOUtils.cleanup(LOG, in);
+      IOUtils.cleanupWithLogger(LOG, in);
     }
     state.tokenState.put(data.getTokenIdentifier(), data.getRenewDate());
   }
@@ -290,7 +290,7 @@ public class LeveldbTimelineStateStore extends
         ++numKeys;
       }
     } finally {
-      IOUtils.cleanup(LOG, iterator);
+      IOUtils.cleanupWithLogger(LOG, iterator);
     }
     return numKeys;
   }
@@ -314,7 +314,7 @@ public class LeveldbTimelineStateStore extends
     } catch (DBException e) {
       throw new IOException(e);
     } finally {
-      IOUtils.cleanup(LOG, iterator);
+      IOUtils.cleanupWithLogger(LOG, iterator);
     }
     return numTokens;
   }
@@ -332,7 +332,7 @@ public class LeveldbTimelineStateStore extends
       try {
         state.latestSequenceNumber = in.readInt();
       } finally {
-        IOUtils.cleanup(LOG, in);
+        IOUtils.cleanupWithLogger(LOG, in);
       }
     }
   }
@@ -412,7 +412,7 @@ public class LeveldbTimelineStateStore extends
       String incompatibleMessage =
           "Incompatible version for timeline state store: expecting version "
               + getCurrentVersion() + ", but loading version " + loadedVersion;
-      LOG.fatal(incompatibleMessage);
+      LOG.error(incompatibleMessage);
       throw new IOException(incompatibleMessage);
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/839e077f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineACLsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineACLsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineACLsManager.java
index 25252fc..6c32eec 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineACLsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineACLsManager.java
@@ -24,8 +24,6 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.commons.collections.map.LRUMap;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -41,6 +39,8 @@ import org.apache.hadoop.yarn.server.timeline.TimelineStore;
 import org.apache.hadoop.yarn.util.StringHelper;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * <code>TimelineACLsManager</code> check the entity level timeline data access.
@@ -48,7 +48,8 @@ import com.google.common.annotations.VisibleForTesting;
 @Private
 public class TimelineACLsManager {
 
-  private static final Log LOG = LogFactory.getLog(TimelineACLsManager.class);
+  private static final Logger LOG = LoggerFactory.
+      getLogger(TimelineACLsManager.class);
   private static final int DOMAIN_ACCESS_ENTRY_CACHE_SIZE = 100;
 
   private AdminACLsManager adminAclsManager;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/839e077f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelegationTokenSecretManagerService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelegationTokenSecretManagerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelegationTokenSecretManagerService.java
index 60a0348..0c6892a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelegationTokenSecretManagerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelegationTokenSecretManagerService.java
@@ -21,8 +21,6 @@ package org.apache.hadoop.yarn.server.timeline.security;
 import java.io.IOException;
 import java.util.Map.Entry;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -35,6 +33,8 @@ import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.timeline.recovery.LeveldbTimelineStateStore;
 import org.apache.hadoop.yarn.server.timeline.recovery.TimelineStateStore;
 import org.apache.hadoop.yarn.server.timeline.recovery.TimelineStateStore.TimelineServiceState;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The service wrapper of {@link TimelineDelegationTokenSecretManager}
@@ -118,8 +118,8 @@ public class TimelineDelegationTokenSecretManagerService extends
   public static class TimelineDelegationTokenSecretManager extends
       AbstractDelegationTokenSecretManager<TimelineDelegationTokenIdentifier> {
 
-    public static final Log LOG =
-        LogFactory.getLog(TimelineDelegationTokenSecretManager.class);
+    public static final Logger LOG =
+        LoggerFactory.getLogger(TimelineDelegationTokenSecretManager.class);
 
     private TimelineStateStore stateStore;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/839e077f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java
index ad4e2bb..be8e3c5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java
@@ -43,8 +43,6 @@ import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
@@ -68,13 +66,16 @@ import org.apache.hadoop.yarn.webapp.NotFoundException;
 
 import com.google.inject.Inject;
 import com.google.inject.Singleton;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @Singleton
 @Path("/ws/v1/timeline")
 //TODO: support XML serialization/deserialization
 public class TimelineWebServices {
 
-  private static final Log LOG = LogFactory.getLog(TimelineWebServices.class);
+  private static final Logger LOG = LoggerFactory
+      .getLogger(TimelineWebServices.class);
 
   private TimelineDataManager timelineDataManager;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/839e077f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestFileSystemApplicationHistoryStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestFileSystemApplicationHistoryStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestFileSystemApplicationHistoryStore.java
index 15a00d2..df4adbe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestFileSystemApplicationHistoryStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestFileSystemApplicationHistoryStore.java
@@ -32,8 +32,6 @@ import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -51,12 +49,14 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TestFileSystemApplicationHistoryStore extends
     ApplicationHistoryStoreTestUtils {
 
-  private static Log LOG = LogFactory
-    .getLog(TestFileSystemApplicationHistoryStore.class.getName());
+  private static final Logger LOG = LoggerFactory
+      .getLogger(TestFileSystemApplicationHistoryStore.class.getName());
 
   private FileSystem fs;
   private Path fsWorkingPath;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/839e077f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLeveldbTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLeveldbTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLeveldbTimelineStore.java
index 0c292d8..f68a1c4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLeveldbTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLeveldbTimelineStore.java
@@ -160,7 +160,7 @@ public class TestLeveldbTimelineStore extends TimelineStoreTestUtils {
     } catch(DBException e) {
       throw new IOException(e);
     } finally {
-      IOUtils.cleanup(null, iterator, pfIterator);
+      IOUtils.cleanupWithLogger(null, iterator, pfIterator);
     }
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[39/51] [abbrv] hadoop git commit: HDFS-10326. Disable setting tcp socket send/receive buffers for write pipelines. Contributed by Daryn Sharp.

Posted by st...@apache.org.
HDFS-10326. Disable setting tcp socket send/receive buffers for write pipelines. Contributed by Daryn Sharp.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/71b8dda4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/71b8dda4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/71b8dda4

Branch: refs/heads/HADOOP-13345
Commit: 71b8dda4f6ff6006410f3a9fe7717aa096004b1b
Parents: e0c2414
Author: Haohui Mai <wh...@apache.org>
Authored: Tue Aug 8 14:58:11 2017 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Tue Aug 8 14:58:16 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/protocol/HdfsConstants.java     |  4 ++--
 .../src/main/resources/hdfs-default.xml         |  9 ++++++---
 .../hadoop/hdfs/TestDFSClientSocketSize.java    | 20 ++++++++++++--------
 3 files changed, 20 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/71b8dda4/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
index b636121..2681f12 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
@@ -48,8 +48,8 @@ public final class HdfsConstants {
   public static final byte COLD_STORAGE_POLICY_ID = 2;
   public static final String COLD_STORAGE_POLICY_NAME = "COLD";
 
-  // TODO should be conf injected?
-  public static final int DEFAULT_DATA_SOCKET_SIZE = 128 * 1024;
+  public static final int DEFAULT_DATA_SOCKET_SIZE = 0;
+
   /**
    * A special path component contained in the path for a snapshot file/dir
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/71b8dda4/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 8bf2b8c..bb62359 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -2545,13 +2545,14 @@
 
 <property>
   <name>dfs.client.socket.send.buffer.size</name>
-  <value>131072</value>
+  <value>0</value>
   <description>
     Socket send buffer size for a write pipeline in DFSClient side.
     This may affect TCP connection throughput.
     If it is set to zero or negative value,
     no buffer size will be set explicitly,
     thus enable tcp auto-tuning on some system.
+    The default value is 0.
   </description>
 </property>
 
@@ -3025,23 +3026,25 @@
 
 <property>
   <name>dfs.datanode.transfer.socket.send.buffer.size</name>
-  <value>131072</value>
+  <value>0</value>
   <description>
     Socket send buffer size for DataXceiver (mirroring packets to downstream
     in pipeline). This may affect TCP connection throughput.
     If it is set to zero or negative value, no buffer size will be set
     explicitly, thus enable tcp auto-tuning on some system.
+    The default value is 0.
   </description>
 </property>
 
 <property>
   <name>dfs.datanode.transfer.socket.recv.buffer.size</name>
-  <value>131072</value>
+  <value>0</value>
   <description>
     Socket receive buffer size for DataXceiver (receiving packets from client
     during block writing). This may affect TCP connection throughput.
     If it is set to zero or negative value, no buffer size will be set
     explicitly, thus enable tcp auto-tuning on some system.
+    The default value is 0.
   </description>
 </property>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/71b8dda4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientSocketSize.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientSocketSize.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientSocketSize.java
index fa12f34..40cd676 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientSocketSize.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientSocketSize.java
@@ -30,7 +30,6 @@ import org.slf4j.LoggerFactory;
 import java.io.IOException;
 import java.net.Socket;
 
-import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_SOCKET_SEND_BUFFER_SIZE_DEFAULT;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_SOCKET_SEND_BUFFER_SIZE_KEY;
 import static org.junit.Assert.assertTrue;
 
@@ -42,15 +41,16 @@ public class TestDFSClientSocketSize {
   }
 
   /**
-   * The setting of socket send buffer size in
-   * {@link java.net.Socket#setSendBufferSize(int)} is only a hint.  Actual
-   * value may differ.  We just sanity check that it is somewhere close.
+   * Test that the send buffer size default value is 0, in which case the socket
+   * will use a TCP auto-tuned value.
    */
   @Test
   public void testDefaultSendBufferSize() throws IOException {
-    assertTrue("Send buffer size should be somewhere near default.",
-        getSendBufferSize(new Configuration()) >=
-            DFS_CLIENT_SOCKET_SEND_BUFFER_SIZE_DEFAULT / 2);
+    final int sendBufferSize = getSendBufferSize(new Configuration());
+    LOG.info("If not specified, the auto tuned send buffer size is: {}",
+        sendBufferSize);
+    assertTrue("Send buffer size should be non-negative value which is " +
+        "determined by system (kernel).", sendBufferSize > 0);
   }
 
   /**
@@ -73,6 +73,10 @@ public class TestDFSClientSocketSize {
         sendBufferSize1 > sendBufferSize2);
   }
 
+  /**
+   * Test that if the send buffer size is 0, the socket will use a TCP
+   * auto-tuned value.
+   */
   @Test
   public void testAutoTuningSendBufferSize() throws IOException {
     final Configuration conf = new Configuration();
@@ -80,7 +84,7 @@ public class TestDFSClientSocketSize {
     final int sendBufferSize = getSendBufferSize(conf);
     LOG.info("The auto tuned send buffer size is: {}", sendBufferSize);
     assertTrue("Send buffer size should be non-negative value which is " +
-          "determined by system (kernel).", sendBufferSize > 0);
+        "determined by system (kernel).", sendBufferSize > 0);
   }
 
   private int getSendBufferSize(Configuration conf) throws IOException {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[35/51] [abbrv] hadoop git commit: YARN-6890. Not display killApp button on UI if UI is unsecured but cluster is secured. Contributed by Junping Du

Posted by st...@apache.org.
YARN-6890. Not display killApp button on UI if UI is unsecured but cluster is secured. Contributed by Junping Du


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/acf9bd8b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/acf9bd8b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/acf9bd8b

Branch: refs/heads/HADOOP-13345
Commit: acf9bd8b1d87b9c46821ecf0461d8dcd0a6ec6d6
Parents: 47b145b
Author: Jian He <ji...@apache.org>
Authored: Tue Aug 8 11:09:38 2017 -0700
Committer: Jian He <ji...@apache.org>
Committed: Tue Aug 8 11:09:38 2017 -0700

----------------------------------------------------------------------
 .../hadoop/fs/CommonConfigurationKeysPublic.java      |  2 ++
 .../apache/hadoop/yarn/server/webapp/AppBlock.java    | 14 +++++++++++++-
 2 files changed, 15 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/acf9bd8b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
index e8d4b4c..4fda2b8 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
@@ -608,6 +608,8 @@ public class CommonConfigurationKeysPublic {
    */
   public static final String HADOOP_TOKEN_FILES =
       "hadoop.token.files";
+  public static final String HADOOP_HTTP_AUTHENTICATION_TYPE =
+    "hadoop.http.authentication.type";
 
   /**
    * @see

http://git-wip-us.apache.org/repos/asf/hadoop/blob/acf9bd8b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
index d4090aa..693aa04 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
@@ -30,6 +30,7 @@ import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.http.RestCsrfPreventionFilter;
@@ -70,6 +71,8 @@ public class AppBlock extends HtmlBlock {
   protected ApplicationBaseProtocol appBaseProt;
   protected Configuration conf;
   protected ApplicationId appID = null;
+  private boolean unsecuredUI = true;
+
 
   @Inject
   protected AppBlock(ApplicationBaseProtocol appBaseProt, ViewContext ctx,
@@ -77,6 +80,9 @@ public class AppBlock extends HtmlBlock {
     super(ctx);
     this.appBaseProt = appBaseProt;
     this.conf = conf;
+    // check if UI is unsecured.
+    String httpAuth = conf.get(CommonConfigurationKeys.HADOOP_HTTP_AUTHENTICATION_TYPE);
+    this.unsecuredUI = (httpAuth != null) && httpAuth.equals("simple");
   }
 
   @Override
@@ -129,10 +135,16 @@ public class AppBlock extends HtmlBlock {
 
     setTitle(join("Application ", aid));
 
+    // YARN-6890. for secured cluster allow anonymous UI access, application kill
+    // shouldn't be there.
+    boolean unsecuredUIForSecuredCluster = UserGroupInformation.isSecurityEnabled()
+        && this.unsecuredUI;
+
     if (webUiType != null
         && webUiType.equals(YarnWebParams.RM_WEB_UI)
         && conf.getBoolean(YarnConfiguration.RM_WEBAPP_UI_ACTIONS_ENABLED,
-          YarnConfiguration.DEFAULT_RM_WEBAPP_UI_ACTIONS_ENABLED)) {
+          YarnConfiguration.DEFAULT_RM_WEBAPP_UI_ACTIONS_ENABLED)
+            && !unsecuredUIForSecuredCluster) {
       // Application Kill
       html.div()
         .button()


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[46/51] [abbrv] hadoop git commit: HDFS-12182. BlockManager.metaSave does not distinguish between "under replicated" and "missing" blocks. Contributed by Wellington Chevreuil.

Posted by st...@apache.org.
HDFS-12182. BlockManager.metaSave does not distinguish between "under replicated" and "missing" blocks. Contributed by Wellington Chevreuil.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/9a3c2379
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9a3c2379
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9a3c2379

Branch: refs/heads/HADOOP-13345
Commit: 9a3c2379ef24cdca5153abf4b63fde1131ff8989
Parents: 07694fc
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Tue Aug 8 23:43:24 2017 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Tue Aug 8 23:44:18 2017 -0700

----------------------------------------------------------------------
 .../server/blockmanagement/BlockManager.java    | 27 ++++++++--
 .../blockmanagement/TestBlockManager.java       | 54 ++++++++++++++++++++
 .../hdfs/server/namenode/TestMetaSave.java      |  2 +
 3 files changed, 79 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a3c2379/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index fc754a0..6129db8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -705,17 +705,36 @@ public class BlockManager implements BlockStatsMXBean {
     datanodeManager.fetchDatanodes(live, dead, false);
     out.println("Live Datanodes: " + live.size());
     out.println("Dead Datanodes: " + dead.size());
+
     //
-    // Dump contents of neededReconstruction
+    // Need to iterate over all queues from neededReplications
+    // except for the QUEUE_WITH_CORRUPT_BLOCKS)
     //
     synchronized (neededReconstruction) {
       out.println("Metasave: Blocks waiting for reconstruction: "
-          + neededReconstruction.size());
-      for (Block block : neededReconstruction) {
+          + neededReconstruction.getLowRedundancyBlockCount());
+      for (int i = 0; i < neededReconstruction.LEVEL; i++) {
+        if (i != neededReconstruction.QUEUE_WITH_CORRUPT_BLOCKS) {
+          for (Iterator<BlockInfo> it = neededReconstruction.iterator(i);
+               it.hasNext();) {
+            Block block = it.next();
+            dumpBlockMeta(block, out);
+          }
+        }
+      }
+      //
+      // Now prints corrupt blocks separately
+      //
+      out.println("Metasave: Blocks currently missing: " +
+          neededReconstruction.getCorruptBlockSize());
+      for (Iterator<BlockInfo> it = neededReconstruction.
+          iterator(neededReconstruction.QUEUE_WITH_CORRUPT_BLOCKS);
+           it.hasNext();) {
+        Block block = it.next();
         dumpBlockMeta(block, out);
       }
     }
-    
+
     // Dump any postponed over-replicated blocks
     out.println("Mis-replicated blocks that have been postponed:");
     for (Block block : postponedMisreplicatedBlocks) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a3c2379/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
index 6b1a979..42aeadf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
@@ -1459,4 +1459,58 @@ public class TestBlockManager {
     }
   }
 
+  @Test
+  public void testMetaSaveMissingReplicas() throws Exception {
+    List<DatanodeStorageInfo> origStorages = getStorages(0, 1);
+    List<DatanodeDescriptor> origNodes = getNodes(origStorages);
+    BlockInfo block = makeBlockReplicasMissing(0, origNodes);
+    File file = new File("test.log");
+    PrintWriter out = new PrintWriter(file);
+    bm.metaSave(out);
+    out.flush();
+    FileInputStream fstream = new FileInputStream(file);
+    DataInputStream in = new DataInputStream(fstream);
+    BufferedReader reader = new BufferedReader(new InputStreamReader(in));
+    StringBuffer buffer = new StringBuffer();
+    String line;
+    try {
+      while ((line = reader.readLine()) != null) {
+        buffer.append(line);
+      }
+      String output = buffer.toString();
+      assertTrue("Metasave output should have reported missing blocks.",
+          output.contains("Metasave: Blocks currently missing: 1"));
+      assertTrue("There should be 0 blocks waiting for reconstruction",
+          output.contains("Metasave: Blocks waiting for reconstruction: 0"));
+      String blockNameGS = block.getBlockName() + "_" +
+          block.getGenerationStamp();
+      assertTrue("Block " + blockNameGS + " should be MISSING.",
+          output.contains(blockNameGS + " MISSING"));
+    } finally {
+      reader.close();
+      file.delete();
+    }
+  }
+
+  private BlockInfo makeBlockReplicasMissing(long blockId,
+      List<DatanodeDescriptor> nodesList) throws IOException {
+    long inodeId = ++mockINodeId;
+    final INodeFile bc = TestINodeFile.createINodeFile(inodeId);
+
+    BlockInfo blockInfo = blockOnNodes(blockId, nodesList);
+    blockInfo.setReplication((short) 3);
+    blockInfo.setBlockCollectionId(inodeId);
+
+    Mockito.doReturn(bc).when(fsn).getBlockCollection(inodeId);
+    bm.blocksMap.addBlockCollection(blockInfo, bc);
+    bm.markBlockReplicasAsCorrupt(blockInfo, blockInfo,
+        blockInfo.getGenerationStamp() + 1,
+        blockInfo.getNumBytes(),
+        new DatanodeStorageInfo[]{});
+    BlockCollection mockedBc = Mockito.mock(BlockCollection.class);
+    Mockito.when(mockedBc.getBlocks()).thenReturn(new BlockInfo[]{blockInfo});
+    bm.checkRedundancy(mockedBc);
+    return blockInfo;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a3c2379/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java
index 0303a5d..8cc1433 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java
@@ -155,6 +155,8 @@ public class TestMetaSave {
       line = reader.readLine();
       assertTrue(line.equals("Metasave: Blocks waiting for reconstruction: 0"));
       line = reader.readLine();
+      assertTrue(line.equals("Metasave: Blocks currently missing: 0"));
+      line = reader.readLine();
       assertTrue(line.equals("Mis-replicated blocks that have been postponed:"));
       line = reader.readLine();
       assertTrue(line.equals("Metasave: Blocks being reconstructed: 0"));


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[20/51] [abbrv] hadoop git commit: HADOOP-14685. Exclude some test jars from hadoop-client-minicluster jar. Contributed by Bharat Viswanadham.

Posted by st...@apache.org.
HADOOP-14685. Exclude some test jars from hadoop-client-minicluster jar. Contributed by Bharat Viswanadham.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/024c3ec4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/024c3ec4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/024c3ec4

Branch: refs/heads/HADOOP-13345
Commit: 024c3ec4a3ad47cf30501497c7ae810a30634f82
Parents: f44b349
Author: Arpit Agarwal <ar...@apache.org>
Authored: Fri Aug 4 16:46:59 2017 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Fri Aug 4 16:46:59 2017 -0700

----------------------------------------------------------------------
 hadoop-client-modules/hadoop-client-minicluster/pom.xml | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/024c3ec4/hadoop-client-modules/hadoop-client-minicluster/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client-minicluster/pom.xml b/hadoop-client-modules/hadoop-client-minicluster/pom.xml
index f4b2329..5255640 100644
--- a/hadoop-client-modules/hadoop-client-minicluster/pom.xml
+++ b/hadoop-client-modules/hadoop-client-minicluster/pom.xml
@@ -634,6 +634,13 @@
                         <exclude>**/*</exclude>
                       </excludes>
                     </filter>
+                    <filter>
+                      <artifact>org.apache.hadoop:hadoop-mapreduce-client-jobclient:*</artifact>
+                      <excludes>
+                        <exclude>testjar/*</exclude>
+                        <exclude>testshell/*</exclude>
+                      </excludes>
+                    </filter>
                   </filters>
                   <relocations>
                     <relocation>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[37/51] [abbrv] hadoop git commit: YARN-6726. Fix issues with docker commands executed by container-executor. (Shane Kumpf via wangda)

Posted by st...@apache.org.
YARN-6726. Fix issues with docker commands executed by container-executor. (Shane Kumpf via wangda)

Change-Id: If1b1827345f98f0a49cc7e39d1ba41fbeed5e911


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/1794de3e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1794de3e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1794de3e

Branch: refs/heads/HADOOP-13345
Commit: 1794de3ea4bbd6863fb43dbae9f5a46b6e4230a0
Parents: 735fce5
Author: Wangda Tan <wa...@apache.org>
Authored: Tue Aug 8 12:56:29 2017 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Tue Aug 8 12:56:29 2017 -0700

----------------------------------------------------------------------
 .../src/CMakeLists.txt                          |   1 +
 .../impl/container-executor.c                   |  78 +++++++++++-
 .../impl/container-executor.h                   |  17 ++-
 .../impl/utils/string-utils.c                   |  86 ++++++++++++++
 .../impl/utils/string-utils.h                   |  32 +++++
 .../test/test-container-executor.c              | 119 ++++++++++++++++++-
 6 files changed, 327 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1794de3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
index f7fe83d..5b52536 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
@@ -89,6 +89,7 @@ add_library(container
     main/native/container-executor/impl/configuration.c
     main/native/container-executor/impl/container-executor.c
     main/native/container-executor/impl/get_executable.c
+    main/native/container-executor/impl/utils/string-utils.c
 )
 
 add_executable(container-executor

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1794de3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
index 99f7b56..def628e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
@@ -18,6 +18,7 @@
 
 #include "configuration.h"
 #include "container-executor.h"
+#include "utils/string-utils.h"
 
 #include <inttypes.h>
 #include <libgen.h>
@@ -40,6 +41,7 @@
 #include <sys/mount.h>
 #include <sys/wait.h>
 #include <getopt.h>
+#include <regex.h>
 
 #include "config.h"
 
@@ -79,6 +81,11 @@ static const char* TC_READ_STATS_OPTS [] = { "-s",  "-b", NULL};
 //struct to store the user details
 struct passwd *user_detail = NULL;
 
+//Docker container related constants.
+static const char* DOCKER_CONTAINER_NAME_PREFIX = "container_";
+static const char* DOCKER_CLIENT_CONFIG_ARG = "--config=";
+static const char* DOCKER_PULL_COMMAND = "pull";
+
 FILE* LOGFILE = NULL;
 FILE* ERRORFILE = NULL;
 
@@ -1208,6 +1215,27 @@ char** tokenize_docker_command(const char *input, int *split_counter) {
   return linesplit;
 }
 
+int execute_regex_match(const char *regex_str, const char *input) {
+  regex_t regex;
+  int regex_match;
+  if (0 != regcomp(&regex, regex_str, REG_EXTENDED|REG_NOSUB)) {
+    fprintf(LOGFILE, "Unable to compile regex.");
+    fflush(LOGFILE);
+    exit(ERROR_COMPILING_REGEX);
+  }
+  regex_match = regexec(&regex, input, (size_t) 0, NULL, 0);
+  regfree(&regex);
+  if(0 == regex_match) {
+    return 0;
+  }
+  return 1;
+}
+
+int validate_docker_image_name(const char *image_name) {
+  char *regex_str = "^(([a-zA-Z0-9.-]+)(:[0-9]+)?/)?([a-z0-9_./-]+)(:[a-zA-Z0-9_.-]+)?$";
+  return execute_regex_match(regex_str, image_name);
+}
+
 char* sanitize_docker_command(const char *line) {
   static struct option long_options[] = {
     {"name", required_argument, 0, 'n' },
@@ -1222,6 +1250,7 @@ char* sanitize_docker_command(const char *line) {
     {"cap-drop", required_argument, 0, 'o' },
     {"device", required_argument, 0, 'i' },
     {"detach", required_argument, 0, 't' },
+    {"format", required_argument, 0, 'f' },
     {0, 0, 0, 0}
   };
 
@@ -1240,6 +1269,35 @@ char* sanitize_docker_command(const char *line) {
   if(output == NULL) {
     exit(OUT_OF_MEMORY);
   }
+
+  // Handle docker client config option.
+  if(0 == strncmp(linesplit[0], DOCKER_CLIENT_CONFIG_ARG, strlen(DOCKER_CLIENT_CONFIG_ARG))) {
+    strcat(output, linesplit[0]);
+    strcat(output, " ");
+    long index = 0;
+    while(index < split_counter) {
+      linesplit[index] = linesplit[index + 1];
+      if (linesplit[index] == NULL) {
+        split_counter--;
+        break;
+      }
+      index++;
+    }
+  }
+
+  // Handle docker pull and image name validation.
+  if (0 == strncmp(linesplit[0], DOCKER_PULL_COMMAND, strlen(DOCKER_PULL_COMMAND))) {
+    if (0 != validate_docker_image_name(linesplit[1])) {
+      fprintf(ERRORFILE, "Invalid Docker image name, exiting.");
+      fflush(ERRORFILE);
+      exit(DOCKER_IMAGE_INVALID);
+    }
+    strcat(output, linesplit[0]);
+    strcat(output, " ");
+    strcat(output, linesplit[1]);
+    return output;
+  }
+
   strcat(output, linesplit[0]);
   strcat(output, " ");
   optind = 1;
@@ -1287,6 +1345,11 @@ char* sanitize_docker_command(const char *line) {
       case 't':
         quote_and_append_arg(&output, &output_size, "--detach=", optarg);
         break;
+      case 'f':
+        strcat(output, "--format=");
+        strcat(output, optarg);
+        strcat(output, " ");
+        break;
       default:
         fprintf(LOGFILE, "Unknown option in docker command, character %d %c, optionindex = %d\n", c, c, optind);
         fflush(LOGFILE);
@@ -1297,7 +1360,16 @@ char* sanitize_docker_command(const char *line) {
 
   if(optind < split_counter) {
     while(optind < split_counter) {
-      quote_and_append_arg(&output, &output_size, "", linesplit[optind++]);
+      if (0 == strncmp(linesplit[optind], DOCKER_CONTAINER_NAME_PREFIX, strlen(DOCKER_CONTAINER_NAME_PREFIX))) {
+        if (1 != validate_container_id(linesplit[optind])) {
+          fprintf(ERRORFILE, "Specified container_id=%s is invalid\n", linesplit[optind]);
+          fflush(ERRORFILE);
+          exit(DOCKER_CONTAINER_NAME_INVALID);
+        }
+        strcat(output, linesplit[optind++]);
+      } else {
+        quote_and_append_arg(&output, &output_size, "", linesplit[optind++]);
+      }
     }
   }
 
@@ -1328,8 +1400,8 @@ char* parse_docker_command_file(const char* command_file) {
   if(ret == NULL) {
     exit(ERROR_SANITIZING_DOCKER_COMMAND);
   }
-  fprintf(LOGFILE, "Using command %s\n", ret);
-  fflush(LOGFILE);
+  fprintf(ERRORFILE, "Using command %s\n", ret);
+  fflush(ERRORFILE);
 
   return ret;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1794de3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h
index e40bd90..1dc0491 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h
@@ -74,7 +74,10 @@ enum errorcodes {
   COULD_NOT_CREATE_APP_LOG_DIRECTORIES = 36,
   COULD_NOT_CREATE_TMP_DIRECTORIES = 37,
   ERROR_CREATE_CONTAINER_DIRECTORIES_ARGUMENTS = 38,
-  ERROR_SANITIZING_DOCKER_COMMAND = 39
+  ERROR_SANITIZING_DOCKER_COMMAND = 39,
+  DOCKER_IMAGE_INVALID = 40,
+  DOCKER_CONTAINER_NAME_INVALID = 41,
+  ERROR_COMPILING_REGEX = 42
 };
 
 enum operations {
@@ -309,3 +312,15 @@ int run_docker(const char *command_file);
  * Sanitize docker commands. Returns NULL if there was any failure.
 */
 char* sanitize_docker_command(const char *line);
+
+/*
+ * Compile the regex_str and determine if the input string matches.
+ * Return 0 on match, 1 of non-match.
+ */
+int execute_regex_match(const char *regex_str, const char *input);
+
+/**
+ * Validate the docker image name matches the expected input.
+ * Return 0 on success.
+ */
+int validate_docker_image_name(const char *image_name);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1794de3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.c
new file mode 100644
index 0000000..703d484
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.c
@@ -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.
+ */
+
+#include <strings.h>
+#include <string.h>
+#include <stdio.h>
+#include <stdlib.h>
+
+/*
+ * if all chars in the input str are numbers
+ * return true/false
+ */
+static int all_numbers(char* input) {
+  if (0 == strlen(input)) {
+    return 0;
+  }
+
+  for (int i = 0; i < strlen(input); i++) {
+    if (input[i] < '0' || input[i] > '9') {
+      return 0;
+    }
+  }
+  return 1;
+}
+
+int validate_container_id(const char* input) {
+  /*
+   * Two different forms of container_id
+   * container_e17_1410901177871_0001_01_000005
+   * container_1410901177871_0001_01_000005
+   */
+  char* input_cpy = malloc(strlen(input));
+  strcpy(input_cpy, input);
+  char* p = strtok(input_cpy, "_");
+  int idx = 0;
+  while (p != NULL) {
+    if (0 == idx) {
+      if (0 != strcmp("container", p)) {
+        return 0;
+      }
+    } else if (1 == idx) {
+      // this could be e[n][n], or [n][n]...
+      if (!all_numbers(p)) {
+        if (strlen(p) == 0) {
+          return 0;
+        }
+        if (p[0] != 'e') {
+          return 0;
+        }
+        if (!all_numbers(p + 1)) {
+          return 0;
+        }
+      }
+    } else {
+      // otherwise, should be all numbers
+      if (!all_numbers(p)) {
+        return 0;
+      }
+    }
+
+    p = strtok(NULL, "_");
+    idx++;
+  }
+  free(input_cpy);
+
+  // We should have [5,6] elements split by '_'
+  if (idx > 6 || idx < 5) {
+    return 0;
+  }
+  return 1;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1794de3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.h
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.h b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.h
new file mode 100644
index 0000000..0a41ad1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.h
@@ -0,0 +1,32 @@
+/**
+ * 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.
+ */
+
+#ifdef __FreeBSD__
+#define _WITH_GETLINE
+#endif
+
+#ifndef _UTILS_STRING_UTILS_H_
+#define _UTILS_STRING_UTILS_H_
+
+/*
+ * Get numbers split by comma from a input string
+ * return false/true
+ */
+int validate_container_id(const char* input);
+
+#endif
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1794de3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
index cf5f119..3202652 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
@@ -17,6 +17,7 @@
  */
 #include "configuration.h"
 #include "container-executor.h"
+#include "utils/string-utils.h"
 
 #include <inttypes.h>
 #include <errno.h>
@@ -1176,7 +1177,13 @@ void test_sanitize_docker_command() {
     "run --name=$CID --user=nobody -d --workdir=/yarn/local/cdir --privileged --rm --device=/sys/fs/cgroup/device:/sys/fs/cgroup/device --detach=true --cgroup-parent=/sys/fs/cgroup/cpu/yarn/cid --net=host --hostname=test.host.name --cap-drop=ALL --cap-add=SYS_CHROOT --cap-add=MKNOD --cap-add=SETFCAP --cap-add=SETPCAP --cap-add=FSETID --cap-add=CHOWN --cap-add=AUDIT_WRITE --cap-add=SETGID --cap-add=NET_RAW --cap-add=FOWNER --cap-add=SETUID --cap-add=DAC_OVERRIDE --cap-add=KILL --cap-add=NET_BIND_SERVICE -v /sys/fs/cgroup:/sys/fs/cgroup:ro -v /yarn/local/cdir:/yarn/local/cdir -v /yarn/local/usercache/test/:/yarn/local/usercache/test/ ubuntu bash /yarn/local/usercache/test/appcache/aid/cid/launch_container.sh",
     "run --name=cname --user=nobody -d --workdir=/yarn/local/cdir --privileged --rm --device=/sys/fs/cgroup/device:/sys/fs/cgroup/device --detach=true --cgroup-parent=/sys/fs/cgroup/cpu/yarn/cid --net=host --hostname=test.host.name --cap-drop=ALL --cap-add=SYS_CHROOT --cap-add=MKNOD --cap-add=SETFCAP --cap-add=SETPCAP --cap-add=FSETID --cap-add=CHOWN --cap-add=AUDIT_WRITE --cap-add=SETGID --cap-add=NET_RAW --cap-add=FOWNER --cap-add=SETUID --cap-add=DAC_OVERRIDE --cap-add=KILL --cap-add=NET_BIND_SERVICE -v /sys/fs/cgroup:/sys/fs/cgroup:ro -v /yarn/local/cdir:/yarn/local/cdir -v /yarn/local/usercache/test/:/yarn/local/usercache/test/ ubuntu || touch /tmp/file # bash /yarn/local/usercache/test/appcache/aid/cid/launch_container.sh",
     "run --name=cname --user=nobody -d --workdir=/yarn/local/cdir --privileged --rm --device=/sys/fs/cgroup/device:/sys/fs/cgroup/device --detach=true --cgroup-parent=/sys/fs/cgroup/cpu/yarn/cid --net=host --hostname=test.host.name --cap-drop=ALL --cap-add=SYS_CHROOT --cap-add=MKNOD --cap-add=SETFCAP --cap-add=SETPCAP --cap-add=FSETID --cap-add=CHOWN --cap-add=AUDIT_WRITE --cap-add=SETGID --cap-add=NET_RAW --cap-add=FOWNER --cap-add=SETUID --cap-add=DAC_OVERRIDE --cap-add=KILL --cap-add=NET_BIND_SERVICE -v /sys/fs/cgroup:/sys/fs/cgroup:ro -v /yarn/local/cdir:/yarn/local/cdir -v /yarn/local/usercache/test/:/yarn/local/usercache/test/ ubuntu' || touch /tmp/file # bash /yarn/local/usercache/test/appcache/aid/cid/launch_container.sh",
-    "run ''''''''"
+    "run ''''''''",
+    "inspect --format='{{range(.NetworkSettings.Networks)}}{{.IPAddress}},{{end}}{{.Config.Hostname}}' container_e111_1111111111111_1111_01_111111",
+    "rm container_e111_1111111111111_1111_01_111111",
+    "stop container_e111_1111111111111_1111_01_111111",
+    "pull ubuntu",
+    "pull registry.com/user/ubuntu",
+    "--config=/yarn/local/cdir/ pull registry.com/user/ubuntu"
   };
   char *expected_output[] = {
       "run --name='cname' --user='nobody' -d --workdir='/yarn/local/cdir' --privileged --rm --device='/sys/fs/cgroup/device:/sys/fs/cgroup/device' --detach='true' --cgroup-parent='/sys/fs/cgroup/cpu/yarn/cid' --net='host' --hostname='test.host.name' --cap-drop='ALL' --cap-add='SYS_CHROOT' --cap-add='MKNOD' --cap-add='SETFCAP' --cap-add='SETPCAP' --cap-add='FSETID' --cap-add='CHOWN' --cap-add='AUDIT_WRITE' --cap-add='SETGID' --cap-add='NET_RAW' --cap-add='FOWNER' --cap-add='SETUID' --cap-add='DAC_OVERRIDE' --cap-add='KILL' --cap-add='NET_BIND_SERVICE' -v '/sys/fs/cgroup:/sys/fs/cgroup:ro' -v '/yarn/local/cdir:/yarn/local/cdir' -v '/yarn/local/usercache/test/:/yarn/local/usercache/test/' 'ubuntu' 'bash' '/yarn/local/usercache/test/appcache/aid/cid/launch_container.sh' ",
@@ -1184,12 +1191,18 @@ void test_sanitize_docker_command() {
       "run --name='cname' --user='nobody' -d --workdir='/yarn/local/cdir' --privileged --rm --device='/sys/fs/cgroup/device:/sys/fs/cgroup/device' --detach='true' --cgroup-parent='/sys/fs/cgroup/cpu/yarn/cid' --net='host' --hostname='test.host.name' --cap-drop='ALL' --cap-add='SYS_CHROOT' --cap-add='MKNOD' --cap-add='SETFCAP' --cap-add='SETPCAP' --cap-add='FSETID' --cap-add='CHOWN' --cap-add='AUDIT_WRITE' --cap-add='SETGID' --cap-add='NET_RAW' --cap-add='FOWNER' --cap-add='SETUID' --cap-add='DAC_OVERRIDE' --cap-add='KILL' --cap-add='NET_BIND_SERVICE' -v '/sys/fs/cgroup:/sys/fs/cgroup:ro' -v '/yarn/local/cdir:/yarn/local/cdir' -v '/yarn/local/usercache/test/:/yarn/local/usercache/test/' 'ubuntu' '||' 'touch' '/tmp/file' '#' 'bash' '/yarn/local/usercache/test/appcache/aid/cid/launch_container.sh' ",
       "run --name='cname' --user='nobody' -d --workdir='/yarn/local/cdir' --privileged --rm --device='/sys/fs/cgroup/device:/sys/fs/cgroup/device' --detach='true' --cgroup-parent='/sys/fs/cgroup/cpu/yarn/cid' --net='host' --hostname='test.host.name' --cap-drop='ALL' --cap-add='SYS_CHROOT' --cap-add='MKNOD' --cap-add='SETFCAP' --cap-add='SETPCAP' --cap-add='FSETID' --cap-add='CHOWN' --cap-add='AUDIT_WRITE' --cap-add='SETGID' --cap-add='NET_RAW' --cap-add='FOWNER' --cap-add='SETUID' --cap-add='DAC_OVERRIDE' --cap-add='KILL' --cap-add='NET_BIND_SERVICE' -v '/sys/fs/cgroup:/sys/fs/cgroup:ro' -v '/yarn/local/cdir:/yarn/local/cdir' -v '/yarn/local/usercache/test/:/yarn/local/usercache/test/' 'ubuntu'\"'\"'' '||' 'touch' '/tmp/file' '#' 'bash' '/yarn/local/usercache/test/appcache/aid/cid/launch_container.sh' ",
       "run ''\"'\"''\"'\"''\"'\"''\"'\"''\"'\"''\"'\"''\"'\"''\"'\"'' ",
+      "inspect --format='{{range(.NetworkSettings.Networks)}}{{.IPAddress}},{{end}}{{.Config.Hostname}}' container_e111_1111111111111_1111_01_111111",
+      "rm container_e111_1111111111111_1111_01_111111",
+      "stop container_e111_1111111111111_1111_01_111111",
+      "pull ubuntu",
+      "pull registry.com/user/ubuntu",
+      "--config=/yarn/local/cdir/ pull registry.com/user/ubuntu"
   };
 
   int input_size = sizeof(input) / sizeof(char *);
   int i = 0;
   for(i = 0;  i < input_size; i++) {
-    char *command = (char *) calloc(strlen(input[i]), sizeof(char));
+    char *command = (char *) calloc(strlen(input[i]) + 1 , sizeof(char));
     strncpy(command, input[i], strlen(input[i]));
     char *op = sanitize_docker_command(command);
     if(strncmp(expected_output[i], op, strlen(expected_output[i])) != 0) {
@@ -1200,6 +1213,102 @@ void test_sanitize_docker_command() {
   }
 }
 
+void test_validate_docker_image_name() {
+
+  char *good_input[] = {
+    "ubuntu",
+    "ubuntu:latest",
+    "ubuntu:14.04",
+    "ubuntu:LATEST",
+    "registry.com:5000/user/ubuntu",
+    "registry.com:5000/user/ubuntu:latest",
+    "registry.com:5000/user/ubuntu:0.1.2.3",
+    "registry.com/user/ubuntu",
+    "registry.com/user/ubuntu:latest",
+    "registry.com/user/ubuntu:0.1.2.3",
+    "registry.com/user/ubuntu:test-image",
+    "registry.com/user/ubuntu:test_image",
+    "registry.com/ubuntu",
+    "user/ubuntu",
+    "user/ubuntu:0.1.2.3",
+    "user/ubuntu:latest",
+    "user/ubuntu:test_image",
+    "user/ubuntu.test:test_image",
+    "user/ubuntu-test:test-image",
+    "registry.com/ubuntu/ubuntu/ubuntu"
+  };
+
+  char *bad_input[] = {
+    "UBUNTU",
+    "registry.com|5000/user/ubuntu",
+    "registry.com | 5000/user/ubuntu",
+    "ubuntu' || touch /tmp/file #",
+    "ubuntu || touch /tmp/file #",
+    "''''''''",
+    "bad_host_name:5000/user/ubuntu",
+    "registry.com:foo/ubuntu/ubuntu/ubuntu",
+    "registry.com/ubuntu:foo/ubuntu/ubuntu"
+  };
+
+  int good_input_size = sizeof(good_input) / sizeof(char *);
+  int i = 0;
+  for(i = 0; i < good_input_size; i++) {
+    int op = validate_docker_image_name(good_input[i]);
+    if(0 != op) {
+      printf("\nFAIL: docker image name %s is invalid", good_input[i]);
+      exit(1);
+    }
+  }
+
+  int bad_input_size = sizeof(bad_input) / sizeof(char *);
+  int j = 0;
+  for(j = 0; j < bad_input_size; j++) {
+    int op = validate_docker_image_name(bad_input[j]);
+    if(1 != op) {
+      printf("\nFAIL: docker image name %s is valid, expected invalid", bad_input[j]);
+      exit(1);
+    }
+  }
+}
+
+void test_validate_container_id() {
+  char *good_input[] = {
+    "container_e134_1499953498516_50875_01_000007",
+    "container_1499953498516_50875_01_000007",
+    "container_e1_12312_11111_02_000001"
+  };
+
+  char *bad_input[] = {
+    "CONTAINER",
+    "container_e1_12312_11111_02_000001 | /tmp/file"
+    "container_e1_12312_11111_02_000001 || # /tmp/file",
+    "container_e1_12312_11111_02_000001 # /tmp/file",
+    "container_e1_12312_11111_02_000001' || touch /tmp/file #",
+    "ubuntu || touch /tmp/file #",
+    "''''''''"
+  };
+
+  int good_input_size = sizeof(good_input) / sizeof(char *);
+  int i = 0;
+  for(i = 0; i < good_input_size; i++) {
+    int op = validate_container_id(good_input[i]);
+    if(1 != op) {
+      printf("FAIL: docker container name %s is invalid\n", good_input[i]);
+      exit(1);
+    }
+  }
+
+  int bad_input_size = sizeof(bad_input) / sizeof(char *);
+  int j = 0;
+  for(j = 0; j < bad_input_size; j++) {
+    int op = validate_container_id(bad_input[j]);
+    if(0 != op) {
+      printf("FAIL: docker container name %s is valid, expected invalid\n", bad_input[j]);
+      exit(1);
+    }
+  }
+}
+
 // This test is expected to be executed either by a regular
 // user or by root. If executed by a regular user it doesn't
 // test all the functions that would depend on changing the
@@ -1297,6 +1406,12 @@ int main(int argc, char **argv) {
   printf("\nTesting sanitize docker commands()\n");
   test_sanitize_docker_command();
 
+  printf("\nTesting validate_docker_image_name()\n");
+  test_validate_docker_image_name();
+
+  printf("\nTesting validate_container_id()\n");
+  test_validate_container_id();
+
   test_check_user(0);
 
 #ifdef __APPLE__


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[12/51] [abbrv] hadoop git commit: YARN-5977. ContainerManagementProtocol changes to support change of container ExecutionType. (Kartheek Muthyala via asuresh)

Posted by st...@apache.org.
YARN-5977. ContainerManagementProtocol changes to support change of container ExecutionType. (Kartheek Muthyala via asuresh)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/35dc7829
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/35dc7829
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/35dc7829

Branch: refs/heads/HADOOP-13345
Commit: 35dc7829236f92054d5ce6ea78d3a44ca6c8f3d3
Parents: f4c6b00
Author: Arun Suresh <as...@apache.org>
Authored: Thu Aug 3 21:15:40 2017 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Thu Aug 3 21:15:40 2017 -0700

----------------------------------------------------------------------
 .../v2/app/launcher/TestContainerLauncher.java  |   9 +
 .../app/launcher/TestContainerLauncherImpl.java |   9 +
 .../yarn/api/ContainerManagementProtocol.java   |  23 ++
 .../protocolrecords/ContainerUpdateRequest.java |  77 ++++++
 .../ContainerUpdateResponse.java                |  95 ++++++++
 .../proto/containermanagement_protocol.proto    |   1 +
 .../src/main/proto/yarn_service_protos.proto    |   9 +
 .../yarn/client/api/impl/NMClientImpl.java      |  18 +-
 ...ContainerManagementProtocolPBClientImpl.java |  40 ++-
 ...ontainerManagementProtocolPBServiceImpl.java |  28 ++-
 .../impl/pb/ContainerUpdateRequestPBImpl.java   | 171 +++++++++++++
 .../impl/pb/ContainerUpdateResponsePBImpl.java  | 241 +++++++++++++++++++
 .../hadoop/yarn/TestContainerLaunchRPC.java     |   9 +
 .../yarn/TestContainerResourceIncreaseRPC.java  |  20 +-
 .../hadoop/yarn/api/TestPBImplRecords.java      |   4 +
 .../java/org/apache/hadoop/yarn/TestRPC.java    |   9 +
 .../containermanager/ContainerManagerImpl.java  |  25 +-
 .../nodemanager/TestNodeManagerResync.java      |  30 +--
 .../containermanager/TestContainerManager.java  |  46 ++--
 .../TestContainerManagerRecovery.java           |  22 +-
 .../server/resourcemanager/NodeManager.java     |   9 +
 .../resourcemanager/TestAMAuthorization.java    |   9 +
 .../TestApplicationMasterLauncher.java          |   9 +
 23 files changed, 834 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/35dc7829/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
index 1520929..a93bf88 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
@@ -31,6 +31,8 @@ import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hadoop.yarn.api.protocolrecords.CommitResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.ReInitializeContainerRequest;
@@ -462,6 +464,7 @@ public class TestContainerLauncher {
     }
 
     @Override
+    @Deprecated
     public IncreaseContainersResourceResponse increaseContainersResource(
         IncreaseContainersResourceRequest request) throws IOException,
         IOException {
@@ -506,5 +509,11 @@ public class TestContainerLauncher {
         throws YarnException, IOException {
       return null;
     }
+
+    @Override
+    public ContainerUpdateResponse updateContainer(ContainerUpdateRequest
+        request) throws YarnException, IOException {
+      return null;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35dc7829/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java
index 225570c..53af631 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java
@@ -47,6 +47,8 @@ import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncher.EventType;
 import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.CommitResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
@@ -465,6 +467,7 @@ public class TestContainerLauncherImpl {
     }
 
     @Override
+    @Deprecated
     public IncreaseContainersResourceResponse increaseContainersResource(
         IncreaseContainersResourceRequest request) throws YarnException,
         IOException {
@@ -511,6 +514,12 @@ public class TestContainerLauncherImpl {
         throws YarnException, IOException {
       return null;
     }
+
+    @Override
+    public ContainerUpdateResponse updateContainer(ContainerUpdateRequest
+        request) throws YarnException, IOException {
+      return null;
+    }
   }
   
   @SuppressWarnings("serial")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35dc7829/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java
index 10708a0..8fceb46 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java
@@ -24,6 +24,8 @@ import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.protocolrecords.CommitResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
@@ -196,10 +198,31 @@ public interface ContainerManagementProtocol {
    */
   @Public
   @Unstable
+  @Deprecated
   IncreaseContainersResourceResponse increaseContainersResource(
       IncreaseContainersResourceRequest request) throws YarnException,
       IOException;
 
+  /**
+   * <p>
+   * The API used by the <code>ApplicationMaster</code> to request for
+   * resource update of running containers on the <code>NodeManager</code>.
+   * </p>
+   *
+   * @param request
+   *         request to update resource of a list of containers
+   * @return response which includes a list of containerIds of containers
+   *         whose resource has been successfully updated and a
+   *         containerId-to-exception map for failed requests.
+   *
+   * @throws YarnException Exception specific to YARN
+   * @throws IOException IOException thrown from NodeManager
+   */
+  @Public
+  @Unstable
+  ContainerUpdateResponse updateContainer(ContainerUpdateRequest request)
+      throws YarnException, IOException;
+
   SignalContainerResponse signalToContainer(SignalContainerRequest request)
       throws YarnException, IOException;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35dc7829/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ContainerUpdateRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ContainerUpdateRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ContainerUpdateRequest.java
new file mode 100644
index 0000000..0242c74
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ContainerUpdateRequest.java
@@ -0,0 +1,77 @@
+/**
+ * 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.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
+import org.apache.hadoop.yarn.api.records.NMToken;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.util.Records;
+
+import java.util.List;
+
+/**
+ * <p>The request sent by <code>Application Master</code> to the
+ * <code>Node Manager</code> to change the resource quota of a container.</p>
+ *
+ * @see ContainerManagementProtocol#updateContainer(ContainerUpdateRequest)
+ */
+@Public
+@Unstable
+public abstract class ContainerUpdateRequest {
+
+  @Public
+  @Unstable
+  public static ContainerUpdateRequest newInstance(
+      List<Token> containersToIncrease) {
+    ContainerUpdateRequest request =
+        Records.newRecord(ContainerUpdateRequest.class);
+    request.setContainersToUpdate(containersToIncrease);
+    return request;
+  }
+
+  /**
+   * Get a list of container tokens to be used for authorization during
+   * container resource update.
+   * <p>
+   * Note: {@link NMToken} will be used for authenticating communication with
+   * {@code NodeManager}.
+   * @return the list of container tokens to be used for authorization during
+   * container resource update.
+   * @see NMToken
+   */
+  @Public
+  @Unstable
+  public abstract List<Token> getContainersToUpdate();
+
+  /**
+   * Set container tokens to be used during container resource increase.
+   * The token is acquired from
+   * <code>AllocateResponse.getUpdatedContainers</code>.
+   * The token contains the container id and resource capability required for
+   * container resource update.
+   * @param containersToUpdate the list of container tokens to be used
+   *                             for container resource increase.
+   */
+  @Public
+  @Unstable
+  public abstract void setContainersToUpdate(
+      List<Token> containersToUpdate);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35dc7829/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ContainerUpdateResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ContainerUpdateResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ContainerUpdateResponse.java
new file mode 100644
index 0000000..aa132f4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ContainerUpdateResponse.java
@@ -0,0 +1,95 @@
+/**
+ * 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.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.SerializedException;
+import org.apache.hadoop.yarn.util.Records;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * <p>
+ * The response sent by the <code>NodeManager</code> to the
+ * <code>ApplicationMaster</code> when asked to update container resource.
+ * </p>
+ *
+ * @see ContainerManagementProtocol#updateContainer(ContainerUpdateRequest)
+ */
+@Public
+@Unstable
+public abstract class ContainerUpdateResponse {
+
+  public static ContainerUpdateResponse newInstance(
+      List<ContainerId> successfullyUpdatedContainers,
+      Map<ContainerId, SerializedException> failedRequests) {
+    ContainerUpdateResponse response =
+        Records.newRecord(ContainerUpdateResponse.class);
+    response.setSuccessfullyUpdatedContainers(
+        successfullyUpdatedContainers);
+    response.setFailedRequests(failedRequests);
+    return response;
+  }
+
+  /**
+   * Get the list of containerIds of containers whose resource
+   * have been successfully update.
+   *
+   * @return the list of containerIds of containers whose resource have
+   * been successfully updated.
+   */
+  @Public
+  @Unstable
+  public abstract List<ContainerId> getSuccessfullyUpdatedContainers();
+
+  /**
+   * Set the list of containerIds of containers whose resource have
+   * been successfully updated.
+   * @param succeedUpdatedContainers Containers whose update request were
+   *                                 successfully completed.
+   */
+  @Private
+  @Unstable
+  public abstract void setSuccessfullyUpdatedContainers(
+      List<ContainerId> succeedUpdatedContainers);
+
+  /**
+   * Get the containerId-to-exception map in which the exception indicates
+   * error from each container for failed requests.
+   * @return map of containerId-to-exception
+   */
+  @Public
+  @Unstable
+  public abstract Map<ContainerId, SerializedException> getFailedRequests();
+
+  /**
+   * Set the containerId-to-exception map in which the exception indicates
+   * error from each container for failed requests.
+   * @param failedRequests Containers whose update request were failed
+   */
+  @Private
+  @Unstable
+  public abstract void setFailedRequests(
+      Map<ContainerId, SerializedException> failedRequests);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35dc7829/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto
index 7c53d2e..22b4406 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto
@@ -36,6 +36,7 @@ service ContainerManagementProtocolService {
   rpc stopContainers(StopContainersRequestProto) returns (StopContainersResponseProto);
   rpc getContainerStatuses(GetContainerStatusesRequestProto) returns (GetContainerStatusesResponseProto);
   rpc increaseContainersResource(IncreaseContainersResourceRequestProto) returns (IncreaseContainersResourceResponseProto);
+  rpc updateContainer(ContainerUpdateRequestProto) returns (ContainerUpdateResponseProto);
   rpc signalToContainer(SignalContainerRequestProto) returns (SignalContainerResponseProto);
   rpc localize(ResourceLocalizationRequestProto) returns (ResourceLocalizationResponseProto);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35dc7829/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
index 0e14896..b92c46e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
@@ -368,6 +368,15 @@ message IncreaseContainersResourceResponseProto {
   repeated ContainerExceptionMapProto failed_requests = 2;
 }
 
+message ContainerUpdateRequestProto {
+  repeated hadoop.common.TokenProto update_container_token = 1;
+}
+
+message ContainerUpdateResponseProto {
+  repeated ContainerIdProto succeeded_requests = 1;
+  repeated ContainerExceptionMapProto failed_requests = 2;
+}
+
 //////////////////////////////////////////////////////
 /////// Application_History_Protocol /////////////////
 //////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35dc7829/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/NMClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/NMClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/NMClientImpl.java
index c81d448..8171de2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/NMClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/NMClientImpl.java
@@ -34,17 +34,17 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
-
 import org.apache.hadoop.yarn.api.protocolrecords.ReInitializeContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
+
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
@@ -239,12 +239,12 @@ public class NMClientImpl extends NMClient {
           container.getNodeId().toString(), container.getId());
       List<Token> increaseTokens = new ArrayList<>();
       increaseTokens.add(container.getContainerToken());
-      IncreaseContainersResourceRequest increaseRequest =
-          IncreaseContainersResourceRequest
-              .newInstance(increaseTokens);
-      IncreaseContainersResourceResponse response =
-          proxy.getContainerManagementProtocol()
-              .increaseContainersResource(increaseRequest);
+
+      ContainerUpdateRequest request =
+          ContainerUpdateRequest.newInstance(increaseTokens);
+      ContainerUpdateResponse response =
+          proxy.getContainerManagementProtocol().updateContainer(request);
+
       if (response.getFailedRequests() != null
           && response.getFailedRequests().containsKey(container.getId())) {
         Throwable t = response.getFailedRequests().get(container.getId())

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35dc7829/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java
index 873dcb7..7e471f3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java
@@ -28,6 +28,8 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB;
 import org.apache.hadoop.yarn.api.protocolrecords.CommitResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
@@ -45,10 +47,10 @@ import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.CommitResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ContainerUpdateRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ContainerUpdateResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesResponsePBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersResourceRequestPBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersResourceResponsePBImpl;
 
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReInitializeContainerRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReInitializeContainerResponsePBImpl;
@@ -56,8 +58,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ResourceLocalizationRe
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ResourceLocalizationResponsePBImpl;
 
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RestartContainerResponsePBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
-    .RollbackResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RollbackResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SignalContainerRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SignalContainerResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersRequestPBImpl;
@@ -71,8 +72,8 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.proto.YarnProtos;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ContainerUpdateRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesRequestProto;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.ResourceLocalizationRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SignalContainerRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersRequestProto;
@@ -161,14 +162,35 @@ public class ContainerManagementProtocolPBClientImpl implements ContainerManagem
   }
 
   @Override
+  @Deprecated
   public IncreaseContainersResourceResponse increaseContainersResource(
       IncreaseContainersResourceRequest request) throws YarnException,
       IOException {
-    IncreaseContainersResourceRequestProto requestProto =
-        ((IncreaseContainersResourceRequestPBImpl)request).getProto();
     try {
-      return new IncreaseContainersResourceResponsePBImpl(
-          proxy.increaseContainersResource(null, requestProto));
+      ContainerUpdateRequest req =
+          ContainerUpdateRequest.newInstance(request.getContainersToIncrease());
+      ContainerUpdateRequestProto reqProto =
+          ((ContainerUpdateRequestPBImpl) req).getProto();
+      ContainerUpdateResponse resp = new ContainerUpdateResponsePBImpl(
+          proxy.updateContainer(null, reqProto));
+      return IncreaseContainersResourceResponse
+          .newInstance(resp.getSuccessfullyUpdatedContainers(),
+              resp.getFailedRequests());
+
+    } catch (ServiceException e) {
+      RPCUtil.unwrapAndThrowException(e);
+      return null;
+    }
+  }
+
+  @Override
+  public ContainerUpdateResponse updateContainer(ContainerUpdateRequest
+      request) throws YarnException, IOException {
+    ContainerUpdateRequestProto requestProto =
+        ((ContainerUpdateRequestPBImpl)request).getProto();
+    try {
+      return new ContainerUpdateResponsePBImpl(
+          proxy.updateContainer(null, requestProto));
     } catch (ServiceException e) {
       RPCUtil.unwrapAndThrowException(e);
       return null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35dc7829/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java
index fb8eead..68e1645 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java
@@ -24,6 +24,8 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB;
 import org.apache.hadoop.yarn.api.protocolrecords.CommitResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.ReInitializeContainerResponse;
@@ -34,6 +36,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.CommitResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ContainerUpdateRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ContainerUpdateResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersResourceRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersResourceResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesRequestPBImpl;
@@ -74,6 +78,8 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersResponsePro
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainersRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainersResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.CommitResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ContainerUpdateRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ContainerUpdateResponseProto;
 
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
@@ -137,8 +143,12 @@ public class ContainerManagementProtocolPBServiceImpl implements ContainerManage
     IncreaseContainersResourceRequestPBImpl request =
         new IncreaseContainersResourceRequestPBImpl(proto);
     try {
+      ContainerUpdateResponse resp = real.updateContainer(ContainerUpdateRequest
+          .newInstance(request.getContainersToIncrease()));
       IncreaseContainersResourceResponse response =
-          real.increaseContainersResource(request);
+          IncreaseContainersResourceResponse
+              .newInstance(resp.getSuccessfullyUpdatedContainers(),
+                  resp.getFailedRequests());
       return ((IncreaseContainersResourceResponsePBImpl)response).getProto();
     } catch (YarnException e) {
       throw new ServiceException(e);
@@ -148,6 +158,22 @@ public class ContainerManagementProtocolPBServiceImpl implements ContainerManage
   }
 
   @Override
+  public ContainerUpdateResponseProto updateContainer(
+      RpcController controller, ContainerUpdateRequestProto proto)
+      throws ServiceException {
+    ContainerUpdateRequestPBImpl request =
+        new ContainerUpdateRequestPBImpl(proto);
+    try {
+      ContainerUpdateResponse response = real.updateContainer(request);
+      return ((ContainerUpdateResponsePBImpl)response).getProto();
+    } catch (YarnException e) {
+      throw new ServiceException(e);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
   public SignalContainerResponseProto signalToContainer(RpcController arg0,
       SignalContainerRequestProto proto) throws ServiceException {
     final SignalContainerRequestPBImpl request =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35dc7829/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ContainerUpdateRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ContainerUpdateRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ContainerUpdateRequestPBImpl.java
new file mode 100644
index 0000000..2736c8a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ContainerUpdateRequestPBImpl.java
@@ -0,0 +1,171 @@
+/**
+ * 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.protocolrecords.impl.pb;
+
+import com.google.protobuf.TextFormat;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateRequest;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ContainerUpdateRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ContainerUpdateRequestProtoOrBuilder;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * <p>An implementation of <code>ContainerUpdateRequest</code>.</p>
+ *
+ * @see ContainerUpdateRequest
+ */
+@Private
+@Unstable
+public class ContainerUpdateRequestPBImpl extends ContainerUpdateRequest {
+  private ContainerUpdateRequestProto proto =
+      ContainerUpdateRequestProto.getDefaultInstance();
+  private ContainerUpdateRequestProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  private List<Token> containersToUpdate = null;
+
+  public ContainerUpdateRequestPBImpl() {
+    builder = ContainerUpdateRequestProto.newBuilder();
+  }
+
+  public ContainerUpdateRequestPBImpl(ContainerUpdateRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  @Override
+  public List<Token> getContainersToUpdate() {
+    if (containersToUpdate != null) {
+      return containersToUpdate;
+    }
+    ContainerUpdateRequestProtoOrBuilder p = viaProto ? proto : builder;
+    List<TokenProto> list = p.getUpdateContainerTokenList();
+    containersToUpdate = new ArrayList<>();
+    for (TokenProto c : list) {
+      containersToUpdate.add(convertFromProtoFormat(c));
+    }
+    return containersToUpdate;
+  }
+
+  @Override
+  public void setContainersToUpdate(List<Token> containersToUpdate) {
+    maybeInitBuilder();
+    if (containersToUpdate == null) {
+      builder.clearUpdateContainerToken();
+    }
+    this.containersToUpdate = containersToUpdate;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  public ContainerUpdateRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private Token convertFromProtoFormat(TokenProto p) {
+    return new TokenPBImpl(p);
+  }
+
+  private TokenProto convertToProtoFormat(Token t) {
+    return ((TokenPBImpl) t).getProto();
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ContainerUpdateRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.containersToUpdate != null) {
+      addUpdateContainersToProto();
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void addUpdateContainersToProto() {
+    maybeInitBuilder();
+    builder.clearUpdateContainerToken();
+    if (this.containersToUpdate == null) {
+      return;
+    }
+    Iterable<TokenProto> iterable = new Iterable<TokenProto>() {
+      @Override
+      public Iterator<TokenProto> iterator() {
+        return new Iterator<TokenProto>() {
+          private Iterator<Token> iter = containersToUpdate.iterator();
+
+          @Override
+          public boolean hasNext() {
+            return iter.hasNext();
+          }
+
+          @Override
+          public TokenProto next() {
+            return convertToProtoFormat(iter.next());
+          }
+
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+          }
+        };
+      }
+    };
+    builder.addAllUpdateContainerToken(iterable);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35dc7829/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ContainerUpdateResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ContainerUpdateResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ContainerUpdateResponsePBImpl.java
new file mode 100644
index 0000000..78e74ec
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/ContainerUpdateResponsePBImpl.java
@@ -0,0 +1,241 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
+
+import com.google.protobuf.TextFormat;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateResponse;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.SerializedException;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.SerializedExceptionProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ContainerExceptionMapProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ContainerUpdateResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ContainerUpdateResponseProtoOrBuilder;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * <p>An implementation of <code>ContainerUpdateResponse</code>.</p>
+ *
+ * @see ContainerUpdateResponse
+ */
+@Private
+@Unstable
+public class ContainerUpdateResponsePBImpl extends ContainerUpdateResponse {
+  private ContainerUpdateResponseProto proto =
+      ContainerUpdateResponseProto.getDefaultInstance();
+  private ContainerUpdateResponseProto.Builder builder = null;
+  private boolean viaProto = false;
+  private List<ContainerId> succeededRequests = null;
+  private Map<ContainerId, SerializedException> failedRequests = null;
+
+  public ContainerUpdateResponsePBImpl() {
+    builder = ContainerUpdateResponseProto.newBuilder();
+  }
+
+  public ContainerUpdateResponsePBImpl(ContainerUpdateResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  @Override
+  public List<ContainerId> getSuccessfullyUpdatedContainers() {
+    initSucceededRequests();
+    return this.succeededRequests;
+  }
+
+  @Override
+  public void setSuccessfullyUpdatedContainers(List<ContainerId> succeeded) {
+    maybeInitBuilder();
+    if (succeeded == null) {
+      builder.clearSucceededRequests();
+    }
+    this.succeededRequests = succeeded;
+  }
+
+  @Override
+  public Map<ContainerId, SerializedException> getFailedRequests() {
+    initFailedRequests();
+    return this.failedRequests;
+  }
+
+  @Override
+  public void setFailedRequests(
+      Map<ContainerId, SerializedException> failedRequests) {
+    maybeInitBuilder();
+    if (failedRequests == null) {
+      builder.clearFailedRequests();
+    }
+    this.failedRequests = failedRequests;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  public ContainerUpdateResponseProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void initSucceededRequests() {
+    if (this.succeededRequests != null) {
+      return;
+    }
+    ContainerUpdateResponseProtoOrBuilder p = viaProto ? proto : builder;
+    List<ContainerIdProto> list = p.getSucceededRequestsList();
+    this.succeededRequests = new ArrayList<ContainerId>();
+    for (ContainerIdProto c : list) {
+      this.succeededRequests.add(convertFromProtoFormat(c));
+    }
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ContainerUpdateResponseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void initFailedRequests() {
+    if (this.failedRequests != null) {
+      return;
+    }
+    ContainerUpdateResponseProtoOrBuilder p = viaProto ? proto : builder;
+    List<ContainerExceptionMapProto> protoList = p.getFailedRequestsList();
+    this.failedRequests = new HashMap<ContainerId, SerializedException>();
+    for (ContainerExceptionMapProto ce : protoList) {
+      this.failedRequests.put(convertFromProtoFormat(ce.getContainerId()),
+          convertFromProtoFormat(ce.getException()));
+    }
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.succeededRequests != null) {
+      addSucceededRequestsToProto();
+    }
+    if (this.failedRequests != null) {
+      addFailedRequestsToProto();
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void addSucceededRequestsToProto() {
+    maybeInitBuilder();
+    builder.clearSucceededRequests();
+    if (this.succeededRequests == null) {
+      return;
+    }
+    Iterable<ContainerIdProto> iterable = new Iterable<ContainerIdProto>() {
+      @Override
+      public Iterator<ContainerIdProto> iterator() {
+        return new Iterator<ContainerIdProto>() {
+          private Iterator<ContainerId> iter = succeededRequests.iterator();
+
+          @Override
+          public boolean hasNext() {
+            return iter.hasNext();
+          }
+
+          @Override
+          public ContainerIdProto next() {
+            return convertToProtoFormat(iter.next());
+          }
+
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+          }
+        };
+      }
+    };
+    builder.addAllSucceededRequests(iterable);
+  }
+
+  private void addFailedRequestsToProto() {
+    maybeInitBuilder();
+    builder.clearFailedRequests();
+    if (this.failedRequests == null) {
+      return;
+    }
+    List<ContainerExceptionMapProto> protoList =
+        new ArrayList<ContainerExceptionMapProto>();
+
+    for (Map.Entry<ContainerId, SerializedException> entry : this.failedRequests
+        .entrySet()) {
+      protoList.add(ContainerExceptionMapProto.newBuilder()
+          .setContainerId(convertToProtoFormat(entry.getKey()))
+          .setException(convertToProtoFormat(entry.getValue())).build());
+    }
+    builder.addAllFailedRequests(protoList);
+  }
+
+  private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
+    return new ContainerIdPBImpl(p);
+  }
+
+  private ContainerIdProto convertToProtoFormat(ContainerId t) {
+    return ((ContainerIdPBImpl) t).getProto();
+  }
+
+  private SerializedExceptionPBImpl convertFromProtoFormat(
+      SerializedExceptionProto p) {
+    return new SerializedExceptionPBImpl(p);
+  }
+
+  private SerializedExceptionProto convertToProtoFormat(SerializedException t) {
+    return ((SerializedExceptionPBImpl) t).getProto();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35dc7829/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java
index 90c7573..dfe7534 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java
@@ -33,6 +33,8 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.CommitResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
@@ -193,6 +195,7 @@ public class TestContainerLaunchRPC {
     }
 
     @Override
+    @Deprecated
     public IncreaseContainersResourceResponse increaseContainersResource(
         IncreaseContainersResourceRequest request) throws YarnException, IOException {
       return null;
@@ -236,5 +239,11 @@ public class TestContainerLaunchRPC {
         throws YarnException, IOException {
       return null;
     }
+
+    @Override
+    public ContainerUpdateResponse updateContainer(ContainerUpdateRequest
+        request) throws YarnException, IOException {
+      return null;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35dc7829/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerResourceIncreaseRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerResourceIncreaseRPC.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerResourceIncreaseRPC.java
index f97f7c7..6e97284 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerResourceIncreaseRPC.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerResourceIncreaseRPC.java
@@ -27,6 +27,8 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.CommitResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
@@ -111,11 +113,11 @@ public class TestContainerResourceIncreaseRPC {
       // Construct container resource increase request,
       List<Token> increaseTokens = new ArrayList<>();
       increaseTokens.add(containerToken);
-      IncreaseContainersResourceRequest increaseRequest =
-          IncreaseContainersResourceRequest
-              .newInstance(increaseTokens);
+      ContainerUpdateRequest request = ContainerUpdateRequest
+          .newInstance(increaseTokens);
+
       try {
-        proxy.increaseContainersResource(increaseRequest);
+        proxy.updateContainer(request);
       } catch (Exception e) {
         LOG.info(StringUtils.stringifyException(e));
         Assert.assertEquals("Error, exception is not: "
@@ -170,8 +172,16 @@ public class TestContainerResourceIncreaseRPC {
     }
 
     @Override
+    @Deprecated
     public IncreaseContainersResourceResponse increaseContainersResource(
-        IncreaseContainersResourceRequest request) throws YarnException, IOException {
+        IncreaseContainersResourceRequest request)
+        throws YarnException, IOException {
+      return null;
+    }
+
+    @Override
+    public ContainerUpdateResponse updateContainer(ContainerUpdateRequest
+        request) throws YarnException, IOException {
       try {
         // make the thread sleep to look like its not going to respond
         Thread.sleep(10000);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35dc7829/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
index b62b4ee..bb688c9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
@@ -27,6 +27,8 @@ import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenReque
 import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenResponseProto;
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.yarn.api.protocolrecords.CommitResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.ReInitializeContainerRequest;
@@ -379,6 +381,8 @@ public class TestPBImplRecords extends BasePBImplRecordsTest {
     generateByNewInstance(StartContainerRequest.class);
     generateByNewInstance(NodeLabel.class);
     generateByNewInstance(UpdatedContainer.class);
+    generateByNewInstance(ContainerUpdateRequest.class);
+    generateByNewInstance(ContainerUpdateResponse.class);
     // genByNewInstance does not apply to QueueInfo, cause
     // it is recursive(has sub queues)
     typeValueCache.put(QueueInfo.class, QueueInfo.newInstance("root", 1.0f,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35dc7829/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
index e5d159b..9775f5c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
@@ -34,6 +34,8 @@ import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB;
 import org.apache.hadoop.yarn.api.protocolrecords.CommitResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
@@ -341,6 +343,7 @@ public class TestRPC {
     }
 
     @Override
+    @Deprecated
     public IncreaseContainersResourceResponse increaseContainersResource(
         IncreaseContainersResourceRequest request)
             throws YarnException, IOException {
@@ -385,6 +388,12 @@ public class TestRPC {
         throws YarnException, IOException {
       return null;
     }
+
+    @Override
+    public ContainerUpdateResponse updateContainer(ContainerUpdateRequest
+        request) throws YarnException, IOException {
+      return null;
+    }
   }
 
   public static ContainerTokenIdentifier newContainerTokenIdentifier(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35dc7829/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
index 167d15d..84ed3c1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
@@ -39,6 +39,8 @@ import org.apache.hadoop.service.Service;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.CommitResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
@@ -1133,13 +1135,26 @@ public class ContainerManagerImpl extends CompositeService implements
    * Increase resource of a list of containers on this NodeManager.
    */
   @Override
+  @Deprecated
   public IncreaseContainersResourceResponse increaseContainersResource(
       IncreaseContainersResourceRequest requests)
           throws YarnException, IOException {
+    ContainerUpdateResponse resp = updateContainer(
+        ContainerUpdateRequest.newInstance(requests.getContainersToIncrease()));
+    return IncreaseContainersResourceResponse.newInstance(
+        resp.getSuccessfullyUpdatedContainers(), resp.getFailedRequests());
+  }
+
+  /**
+   * Update resource of a list of containers on this NodeManager.
+   */
+  @Override
+  public ContainerUpdateResponse updateContainer(ContainerUpdateRequest
+      request) throws YarnException, IOException {
     UserGroupInformation remoteUgi = getRemoteUgi();
     NMTokenIdentifier nmTokenIdentifier = selectNMTokenIdentifier(remoteUgi);
     authorizeUser(remoteUgi, nmTokenIdentifier);
-    List<ContainerId> successfullyIncreasedContainers
+    List<ContainerId> successfullyUpdatedContainers
         = new ArrayList<ContainerId>();
     Map<ContainerId, SerializedException> failedContainers =
         new HashMap<ContainerId, SerializedException>();
@@ -1151,7 +1166,7 @@ public class ContainerManagerImpl extends CompositeService implements
     synchronized (this.context) {
       // Process container resource increase requests
       for (org.apache.hadoop.yarn.api.records.Token token :
-          requests.getContainersToIncrease()) {
+          request.getContainersToUpdate()) {
         ContainerId containerId = null;
         try {
           if (token.getIdentifier() == null) {
@@ -1171,7 +1186,7 @@ public class ContainerManagerImpl extends CompositeService implements
           Resource resource = containerTokenIdentifier.getResource();
           changeContainerResourceInternal(containerId,
               containerTokenIdentifier.getVersion(), resource, true);
-          successfullyIncreasedContainers.add(containerId);
+          successfullyUpdatedContainers.add(containerId);
         } catch (YarnException | InvalidToken e) {
           failedContainers.put(containerId, SerializedException.newInstance(e));
         } catch (IOException e) {
@@ -1179,8 +1194,8 @@ public class ContainerManagerImpl extends CompositeService implements
         }
       }
     }
-    return IncreaseContainersResourceResponse.newInstance(
-        successfullyIncreasedContainers, failedContainers);
+    return ContainerUpdateResponse.newInstance(
+        successfullyUpdatedContainers, failedContainers);
   }
 
   @SuppressWarnings("unchecked")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35dc7829/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
index c5c74d1..0c025ac 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
@@ -45,9 +45,9 @@ import org.apache.hadoop.net.ServerSocketUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -224,7 +224,7 @@ public class TestNodeManagerResync {
     // Start a container and make sure it is in RUNNING state
     ((TestNodeManager4)nm).startContainer();
     // Simulate a container resource increase in a separate thread
-    ((TestNodeManager4)nm).increaseContainersResource();
+    ((TestNodeManager4)nm).updateContainerResource();
     // Simulate RM restart by sending a RESYNC event
     LOG.info("Sending out RESYNC event");
     nm.getNMDispatcher().getEventHandler().handle(
@@ -505,7 +505,7 @@ public class TestNodeManagerResync {
 
   class TestNodeManager4 extends NodeManager {
 
-    private Thread increaseContainerResourceThread = null;
+    private Thread containerUpdateResourceThread = null;
 
     @Override
     protected NodeStatusUpdater createNodeStatusUpdater(Context context,
@@ -621,11 +621,11 @@ public class TestNodeManagerResync {
     }
 
     // Increase container resource in a thread
-    public void increaseContainersResource()
+    public void updateContainerResource()
         throws InterruptedException {
       LOG.info("Increase a container resource in a separate thread");
-      increaseContainerResourceThread = new IncreaseContainersResourceThread();
-      increaseContainerResourceThread.start();
+      containerUpdateResourceThread = new ContainerUpdateResourceThread();
+      containerUpdateResourceThread.start();
     }
 
     class TestNodeStatusUpdaterImpl4 extends MockNodeStatusUpdater {
@@ -652,7 +652,7 @@ public class TestNodeManagerResync {
             updateBarrier.await();
             // Call the actual rebootNodeStatusUpdaterAndRegisterWithRM().
             // This function should be synchronized with
-            // increaseContainersResource().
+            // updateContainer().
             updateBarrier.await();
             super.rebootNodeStatusUpdaterAndRegisterWithRM();
             // Check status after registerWithRM
@@ -672,7 +672,7 @@ public class TestNodeManagerResync {
       }
     }
 
-    class IncreaseContainersResourceThread extends Thread {
+    class ContainerUpdateResourceThread extends Thread {
       @Override
       public void run() {
         // Construct container resource increase request
@@ -683,15 +683,15 @@ public class TestNodeManagerResync {
           try {
             updateBarrier.await();
             increaseTokens.add(getContainerToken(targetResource));
-            IncreaseContainersResourceRequest increaseRequest =
-                IncreaseContainersResourceRequest.newInstance(increaseTokens);
-            IncreaseContainersResourceResponse increaseResponse =
+            ContainerUpdateRequest updateRequest =
+                ContainerUpdateRequest.newInstance(increaseTokens);
+            ContainerUpdateResponse updateResponse =
                 getContainerManager()
-                    .increaseContainersResource(increaseRequest);
+                    .updateContainer(updateRequest);
             Assert.assertEquals(
-                1, increaseResponse.getSuccessfullyIncreasedContainers()
+                1, updateResponse.getSuccessfullyUpdatedContainers()
                     .size());
-            Assert.assertTrue(increaseResponse.getFailedRequests().isEmpty());
+            Assert.assertTrue(updateResponse.getFailedRequests().isEmpty());
           } catch (Exception e) {
             e.printStackTrace();
           } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35dc7829/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
index ba0ecce..f2d2037 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
@@ -47,10 +47,10 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.Service;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.ResourceLocalizationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
@@ -1549,16 +1549,15 @@ public class TestContainerManager extends BaseContainerManagerTest {
             context.getContainerTokenSecretManager(), null);
     increaseTokens.add(containerToken);
 
-    IncreaseContainersResourceRequest increaseRequest =
-        IncreaseContainersResourceRequest
-          .newInstance(increaseTokens);
-    IncreaseContainersResourceResponse increaseResponse =
-        containerManager.increaseContainersResource(increaseRequest);
+    ContainerUpdateRequest updateRequest =
+        ContainerUpdateRequest.newInstance(increaseTokens);
+    ContainerUpdateResponse updateResponse =
+        containerManager.updateContainer(updateRequest);
     // Check response
     Assert.assertEquals(
-        0, increaseResponse.getSuccessfullyIncreasedContainers().size());
-    Assert.assertEquals(2, increaseResponse.getFailedRequests().size());
-    for (Map.Entry<ContainerId, SerializedException> entry : increaseResponse
+        0, updateResponse.getSuccessfullyUpdatedContainers().size());
+    Assert.assertEquals(2, updateResponse.getFailedRequests().size());
+    for (Map.Entry<ContainerId, SerializedException> entry : updateResponse
         .getFailedRequests().entrySet()) {
       Assert.assertNotNull("Failed message", entry.getValue().getMessage());
       if (cId0.equals(entry.getKey())) {
@@ -1635,16 +1634,15 @@ public class TestContainerManager extends BaseContainerManagerTest {
             Resource.newInstance(512, 1),
             context.getContainerTokenSecretManager(), null);
     increaseTokens.add(containerToken);
-    IncreaseContainersResourceRequest increaseRequest =
-        IncreaseContainersResourceRequest
-            .newInstance(increaseTokens);
-    IncreaseContainersResourceResponse increaseResponse =
-        containerManager.increaseContainersResource(increaseRequest);
+    ContainerUpdateRequest updateRequest =
+        ContainerUpdateRequest.newInstance(increaseTokens);
+    ContainerUpdateResponse updateResponse =
+        containerManager.updateContainer(updateRequest);
     // Check response
     Assert.assertEquals(
-        0, increaseResponse.getSuccessfullyIncreasedContainers().size());
-    Assert.assertEquals(1, increaseResponse.getFailedRequests().size());
-    for (Map.Entry<ContainerId, SerializedException> entry : increaseResponse
+        0, updateResponse.getSuccessfullyUpdatedContainers().size());
+    Assert.assertEquals(1, updateResponse.getFailedRequests().size());
+    for (Map.Entry<ContainerId, SerializedException> entry : updateResponse
         .getFailedRequests().entrySet()) {
       if (cId.equals(entry.getKey())) {
         Assert.assertNotNull("Failed message", entry.getValue().getMessage());
@@ -1717,13 +1715,13 @@ public class TestContainerManager extends BaseContainerManagerTest {
         context.getNodeId(), user, targetResource,
             context.getContainerTokenSecretManager(), null);
     increaseTokens.add(containerToken);
-    IncreaseContainersResourceRequest increaseRequest =
-        IncreaseContainersResourceRequest.newInstance(increaseTokens);
-    IncreaseContainersResourceResponse increaseResponse =
-        containerManager.increaseContainersResource(increaseRequest);
+    ContainerUpdateRequest updateRequest =
+        ContainerUpdateRequest.newInstance(increaseTokens);
+    ContainerUpdateResponse updateResponse =
+        containerManager.updateContainer(updateRequest);
     Assert.assertEquals(
-        1, increaseResponse.getSuccessfullyIncreasedContainers().size());
-    Assert.assertTrue(increaseResponse.getFailedRequests().isEmpty());
+        1, updateResponse.getSuccessfullyUpdatedContainers().size());
+    Assert.assertTrue(updateResponse.getFailedRequests().isEmpty());
     // Check status
     List<ContainerId> containerIds = new ArrayList<>();
     containerIds.add(cId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35dc7829/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
index b1a7b4b..d2bd79c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
@@ -49,9 +49,9 @@ import org.apache.hadoop.net.ServerSocketUtil;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
@@ -460,9 +460,9 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
         org.apache.hadoop.yarn.server.nodemanager
             .containermanager.container.ContainerState.RUNNING);
     Resource targetResource = Resource.newInstance(2048, 2);
-    IncreaseContainersResourceResponse increaseResponse =
-        increaseContainersResource(context, cm, cid, targetResource);
-    assertTrue(increaseResponse.getFailedRequests().isEmpty());
+    ContainerUpdateResponse updateResponse =
+        updateContainers(context, cm, cid, targetResource);
+    assertTrue(updateResponse.getFailedRequests().isEmpty());
     // check status
     ContainerStatus containerStatus = getContainerStatus(context, cm, cid);
     assertEquals(targetResource, containerStatus.getCapability());
@@ -643,7 +643,7 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
     });
   }
 
-  private IncreaseContainersResourceResponse increaseContainersResource(
+  private ContainerUpdateResponse updateContainers(
       Context context, final ContainerManagerImpl cm, ContainerId cid,
       Resource capability) throws Exception {
     UserGroupInformation user = UserGroupInformation.createRemoteUser(
@@ -655,18 +655,18 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
         cid, 0, context.getNodeId(), user.getShortUserName(),
         capability, context.getContainerTokenSecretManager(), null);
     increaseTokens.add(containerToken);
-    final IncreaseContainersResourceRequest increaseRequest =
-        IncreaseContainersResourceRequest.newInstance(increaseTokens);
+    final ContainerUpdateRequest updateRequest =
+        ContainerUpdateRequest.newInstance(increaseTokens);
     NMTokenIdentifier nmToken = new NMTokenIdentifier(
         cid.getApplicationAttemptId(), context.getNodeId(),
         user.getShortUserName(),
         context.getNMTokenSecretManager().getCurrentKey().getKeyId());
     user.addTokenIdentifier(nmToken);
     return user.doAs(
-        new PrivilegedExceptionAction<IncreaseContainersResourceResponse>() {
+        new PrivilegedExceptionAction<ContainerUpdateResponse>() {
           @Override
-          public IncreaseContainersResourceResponse run() throws Exception {
-            return cm.increaseContainersResource(increaseRequest);
+          public ContainerUpdateResponse run() throws Exception {
+            return cm.updateContainer(updateRequest);
           }
         });
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35dc7829/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
index 2fc4b3f..ee974e3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
@@ -26,6 +26,8 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.yarn.api.protocolrecords.CommitResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.ReInitializeContainerRequest;
@@ -306,12 +308,19 @@ public class NodeManager implements ContainerManagementProtocol {
   }
 
   @Override
+  @Deprecated
   public IncreaseContainersResourceResponse increaseContainersResource(
       IncreaseContainersResourceRequest request)
           throws YarnException, IOException {
     return null;
   }
 
+  @Override
+  public ContainerUpdateResponse updateContainer(ContainerUpdateRequest
+      request) throws YarnException, IOException {
+    return null;
+  }
+
   public static org.apache.hadoop.yarn.server.api.records.NodeStatus
   createNodeStatus(NodeId nodeId, List<ContainerStatus> containers) {
     RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35dc7829/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java
index b3d4d34..1acf658 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java
@@ -42,6 +42,8 @@ import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.CommitResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
@@ -156,12 +158,19 @@ public class TestAMAuthorization {
       return GetContainerStatusesResponse.newInstance(null, null);
     }
 
+    @Deprecated
     @Override
     public IncreaseContainersResourceResponse increaseContainersResource(IncreaseContainersResourceRequest request)
         throws YarnException {
       return IncreaseContainersResourceResponse.newInstance(null, null);
     }
 
+    @Override
+    public ContainerUpdateResponse updateContainer(ContainerUpdateRequest
+        request) throws YarnException, IOException {
+      return ContainerUpdateResponse.newInstance(null, null);
+    }
+
     public Credentials getContainerCredentials() throws IOException {
       Credentials credentials = new Credentials();
       DataInputByteBuffer buf = new DataInputByteBuffer();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35dc7829/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
index 8e14176..172993b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
@@ -34,6 +34,8 @@ import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.CommitResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
@@ -148,6 +150,7 @@ public class TestApplicationMasterLauncher {
     }
 
     @Override
+    @Deprecated
     public IncreaseContainersResourceResponse increaseContainersResource(
         IncreaseContainersResourceRequest request)
             throws YarnException {
@@ -190,6 +193,12 @@ public class TestApplicationMasterLauncher {
         throws YarnException, IOException {
       return null;
     }
+
+    @Override
+    public ContainerUpdateResponse updateContainer(ContainerUpdateRequest
+        request) throws YarnException, IOException {
+      return null;
+    }
   }
 
   @Test


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[50/51] [abbrv] hadoop git commit: YARN-6958. Moving logging APIs over to slf4j in hadoop-yarn-server-timelineservice. Contributed by Yeliang Cang.

Posted by st...@apache.org.
YARN-6958. Moving logging APIs over to slf4j in hadoop-yarn-server-timelineservice. Contributed by Yeliang Cang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/63cfcb90
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/63cfcb90
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/63cfcb90

Branch: refs/heads/HADOOP-13345
Commit: 63cfcb90ac6fbb79ba9ed6b3044cd999fc74e58c
Parents: 69afa26
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Aug 9 23:58:22 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed Aug 9 23:58:22 2017 +0900

----------------------------------------------------------------------
 .../server/timeline/LevelDBCacheTimelineStore.java    | 14 +++++++-------
 .../reader/filter/TimelineFilterUtils.java            |  7 ++++---
 .../storage/HBaseTimelineReaderImpl.java              |  8 ++++----
 .../storage/HBaseTimelineWriterImpl.java              |  8 ++++----
 .../storage/TimelineSchemaCreator.java                |  7 ++++---
 .../storage/application/ApplicationTable.java         |  7 ++++---
 .../storage/apptoflow/AppToFlowTable.java             |  7 ++++---
 .../timelineservice/storage/common/ColumnHelper.java  |  8 +++++---
 .../storage/common/HBaseTimelineStorageUtils.java     |  8 ++++----
 .../timelineservice/storage/entity/EntityTable.java   |  7 ++++---
 .../storage/flow/FlowActivityTable.java               |  7 ++++---
 .../storage/flow/FlowRunCoprocessor.java              |  7 ++++---
 .../timelineservice/storage/flow/FlowRunTable.java    |  7 ++++---
 .../timelineservice/storage/flow/FlowScanner.java     |  7 ++++---
 .../storage/reader/TimelineEntityReader.java          |  7 ++++---
 .../collector/AppLevelTimelineCollector.java          |  7 ++++---
 .../collector/NodeTimelineCollectorManager.java       |  8 ++++----
 .../PerNodeTimelineCollectorsAuxService.java          | 10 +++++-----
 .../timelineservice/collector/TimelineCollector.java  |  7 ++++---
 .../collector/TimelineCollectorManager.java           |  8 ++++----
 .../collector/TimelineCollectorWebService.java        |  8 ++++----
 .../timelineservice/reader/TimelineReaderServer.java  |  9 +++++----
 .../reader/TimelineReaderWebServices.java             |  8 ++++----
 .../storage/FileSystemTimelineReaderImpl.java         |  8 ++++----
 .../storage/common/TimelineStorageUtils.java          |  4 ----
 25 files changed, 102 insertions(+), 91 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/63cfcb90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/LevelDBCacheTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/LevelDBCacheTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/LevelDBCacheTimelineStore.java
index 7379dd6..f7a3d01 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/LevelDBCacheTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/LevelDBCacheTimelineStore.java
@@ -19,8 +19,6 @@
 package org.apache.hadoop.yarn.server.timeline;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -34,6 +32,8 @@ import org.fusesource.leveldbjni.JniDBFactory;
 import org.iq80.leveldb.DB;
 import org.iq80.leveldb.DBIterator;
 import org.iq80.leveldb.Options;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.IOException;
@@ -58,8 +58,8 @@ import java.util.Map;
 @Private
 @Unstable
 public class LevelDBCacheTimelineStore extends KeyValueBasedTimelineStore {
-  private static final Log LOG
-      = LogFactory.getLog(LevelDBCacheTimelineStore.class);
+  private static final Logger LOG
+      = LoggerFactory.getLogger(LevelDBCacheTimelineStore.class);
   private static final String CACHED_LDB_FILE_PREFIX = "-timeline-cache.ldb";
   private String dbId;
   private DB entityDb;
@@ -102,7 +102,7 @@ public class LevelDBCacheTimelineStore extends KeyValueBasedTimelineStore {
         localFS.setPermission(dbPath, LeveldbUtils.LEVELDB_DIR_UMASK);
       }
     } finally {
-      IOUtils.cleanup(LOG, localFS);
+      IOUtils.cleanupWithLogger(LOG, localFS);
     }
     LOG.info("Using leveldb path " + dbPath);
     entityDb = factory.open(new File(dbPath.toString()), options);
@@ -113,7 +113,7 @@ public class LevelDBCacheTimelineStore extends KeyValueBasedTimelineStore {
 
   @Override
   protected synchronized void serviceStop() throws Exception {
-    IOUtils.cleanup(LOG, entityDb);
+    IOUtils.cleanupWithLogger(LOG, entityDb);
     Path dbPath = new Path(
         configuration.get(YarnConfiguration.TIMELINE_SERVICE_LEVELDB_PATH),
         dbId + CACHED_LDB_FILE_PREFIX);
@@ -125,7 +125,7 @@ public class LevelDBCacheTimelineStore extends KeyValueBasedTimelineStore {
               "timeline store " + dbPath);
       }
     } finally {
-      IOUtils.cleanup(LOG, localFS);
+      IOUtils.cleanupWithLogger(LOG, localFS);
     }
     super.serviceStop();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63cfcb90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilterUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilterUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilterUtils.java
index cccae26..a934a3d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilterUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilterUtils.java
@@ -22,8 +22,6 @@ import java.io.IOException;
 import java.util.HashSet;
 import java.util.Set;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.filter.BinaryPrefixComparator;
 import org.apache.hadoop.hbase.filter.FamilyFilter;
@@ -36,13 +34,16 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
 import org.apache.hadoop.hbase.filter.QualifierFilter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Set of utility methods used by timeline filter classes.
  */
 public final class TimelineFilterUtils {
 
-  private static final Log LOG = LogFactory.getLog(TimelineFilterUtils.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TimelineFilterUtils.class);
 
   private TimelineFilterUtils() {
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63cfcb90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java
index a384a84..dc50f42 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java
@@ -21,8 +21,6 @@ package org.apache.hadoop.yarn.server.timelineservice.storage;
 import java.io.IOException;
 import java.util.Set;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.client.Connection;
@@ -34,6 +32,8 @@ import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilter
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
 import org.apache.hadoop.yarn.server.timelineservice.storage.reader.TimelineEntityReader;
 import org.apache.hadoop.yarn.server.timelineservice.storage.reader.TimelineEntityReaderFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * HBase based implementation for {@link TimelineReader}.
@@ -41,8 +41,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.reader.TimelineEnti
 public class HBaseTimelineReaderImpl
     extends AbstractService implements TimelineReader {
 
-  private static final Log LOG = LogFactory
-      .getLog(HBaseTimelineReaderImpl.class);
+  private static final Logger LOG = LoggerFactory
+      .getLogger(HBaseTimelineReaderImpl.class);
 
   private Configuration hbaseConf = null;
   private Connection conn;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63cfcb90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java
index b94b85f..afa58cb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java
@@ -21,8 +21,6 @@ import java.io.IOException;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -65,6 +63,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumn;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumnPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKey;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This implements a hbase based backend for storing the timeline entity
@@ -76,8 +76,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable;
 public class HBaseTimelineWriterImpl extends AbstractService implements
     TimelineWriter {
 
-  private static final Log LOG = LogFactory
-      .getLog(HBaseTimelineWriterImpl.class);
+  private static final Logger LOG = LoggerFactory
+      .getLogger(HBaseTimelineWriterImpl.class);
 
   private Connection conn;
   private TypedBufferedMutator<EntityTable> entityTable;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63cfcb90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
index b3b749e..dbed05d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
@@ -29,8 +29,6 @@ import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
 import org.apache.commons.cli.PosixParser;
 import org.apache.commons.lang.StringUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -46,6 +44,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTa
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This creates the schema for a hbase based backend for storing application
@@ -58,7 +58,8 @@ public final class TimelineSchemaCreator {
   }
 
   final static String NAME = TimelineSchemaCreator.class.getSimpleName();
-  private static final Log LOG = LogFactory.getLog(TimelineSchemaCreator.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TimelineSchemaCreator.class);
   private static final String SKIP_EXISTING_TABLE_OPTION_SHORT = "s";
   private static final String APP_METRICS_TTL_OPTION_SHORT = "ma";
   private static final String APP_TABLE_NAME_SHORT = "a";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63cfcb90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java
index cb4fc92..d3bdd39 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java
@@ -19,8 +19,6 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.application;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -30,6 +28,8 @@ import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The application table as column families info, config and metrics. Info
@@ -99,7 +99,8 @@ public class ApplicationTable extends BaseTable<ApplicationTable> {
   /** default max number of versions. */
   private static final int DEFAULT_METRICS_MAX_VERSIONS = 10000;
 
-  private static final Log LOG = LogFactory.getLog(ApplicationTable.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ApplicationTable.class);
 
   public ApplicationTable() {
     super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63cfcb90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.java
index 301cf99..40d95a4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.java
@@ -18,8 +18,6 @@
 package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow;
 
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -29,6 +27,8 @@ import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 
@@ -68,7 +68,8 @@ public class AppToFlowTable extends BaseTable<AppToFlowTable> {
   /** default value for app_flow table name. */
   private static final String DEFAULT_TABLE_NAME = "timelineservice.app_flow";
 
-  private static final Log LOG = LogFactory.getLog(AppToFlowTable.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AppToFlowTable.class);
 
   public AppToFlowTable() {
     super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63cfcb90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.java
index be55db5..a9c2148 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.java
@@ -24,13 +24,14 @@ import java.util.Map.Entry;
 import java.util.NavigableMap;
 import java.util.TreeMap;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationCompactionDimension;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 /**
  * This class is meant to be used only by explicit Columns, and not directly to
  * write by clients.
@@ -38,7 +39,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
  * @param <T> refers to the table.
  */
 public class ColumnHelper<T> {
-  private static final Log LOG = LogFactory.getLog(ColumnHelper.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ColumnHelper.class);
 
   private final ColumnFamily<T> columnFamily;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63cfcb90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java
index e93b470..b6f1157 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java
@@ -17,8 +17,6 @@
 
 package org.apache.hadoop.yarn.server.timelineservice.storage.common;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
@@ -30,6 +28,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationCom
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationOperation;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.List;
@@ -41,8 +41,8 @@ import java.util.Map;
 public final class HBaseTimelineStorageUtils {
   /** milliseconds in one day. */
   public static final long MILLIS_ONE_DAY = 86400000L;
-  private static final Log LOG =
-      LogFactory.getLog(HBaseTimelineStorageUtils.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(HBaseTimelineStorageUtils.class);
 
   private HBaseTimelineStorageUtils() {
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63cfcb90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java
index ddf0406..df5ce69 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java
@@ -19,8 +19,6 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.entity;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -30,6 +28,8 @@ import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The entity table as column families info, config and metrics. Info stores
@@ -99,7 +99,8 @@ public class EntityTable extends BaseTable<EntityTable> {
   /** default max number of versions. */
   private static final int DEFAULT_METRICS_MAX_VERSIONS = 10000;
 
-  private static final Log LOG = LogFactory.getLog(EntityTable.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(EntityTable.class);
 
   public EntityTable() {
     super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63cfcb90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java
index 8a0430c..e646eb2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java
@@ -19,8 +19,6 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.flow;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -29,6 +27,8 @@ import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The flow activity table has column family info
@@ -63,7 +63,8 @@ public class FlowActivityTable extends BaseTable<FlowActivityTable> {
   public static final String DEFAULT_TABLE_NAME =
       "timelineservice.flowactivity";
 
-  private static final Log LOG = LogFactory.getLog(FlowActivityTable.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FlowActivityTable.class);
 
   /** default max number of versions. */
   public static final int DEFAULT_METRICS_MAX_VERSIONS = Integer.MAX_VALUE;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63cfcb90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java
index 2be6ef8..221420e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java
@@ -24,8 +24,6 @@ import java.util.Map;
 import java.util.NavigableMap;
 import java.util.TreeMap;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
@@ -50,13 +48,16 @@ import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimestampGenerator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Coprocessor for flow run table.
  */
 public class FlowRunCoprocessor extends BaseRegionObserver {
 
-  private static final Log LOG = LogFactory.getLog(FlowRunCoprocessor.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FlowRunCoprocessor.class);
   private boolean isFlowRunRegion = false;
 
   private Region region;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63cfcb90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java
index 547bef0..9c6549f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java
@@ -19,8 +19,6 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.flow;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -29,6 +27,8 @@ import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The flow run table has column family info
@@ -94,7 +94,8 @@ public class FlowRunTable extends BaseTable<FlowRunTable> {
   /** default value for flowrun table name. */
   public static final String DEFAULT_TABLE_NAME = "timelineservice.flowrun";
 
-  private static final Log LOG = LogFactory.getLog(FlowRunTable.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FlowRunTable.class);
 
   /** default max number of versions. */
   public static final int DEFAULT_METRICS_MAX_VERSIONS = Integer.MAX_VALUE;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63cfcb90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java
index 0e3c8ee..dbd0484 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java
@@ -27,8 +27,6 @@ import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
@@ -52,6 +50,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimestampGen
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Invoked via the coprocessor when a Get or a Scan is issued for flow run
@@ -62,7 +62,8 @@ import com.google.common.annotations.VisibleForTesting;
  */
 class FlowScanner implements RegionScanner, Closeable {
 
-  private static final Log LOG = LogFactory.getLog(FlowScanner.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FlowScanner.class);
 
   /**
    * use a special application id to represent the flow id this is needed since

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63cfcb90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.java
index 7b294a8..424d141 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.java
@@ -27,8 +27,6 @@ import java.util.NavigableSet;
 import java.util.Set;
 import java.util.TreeSet;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Result;
@@ -54,6 +52,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The base class for reading and deserializing timeline entities from the
@@ -61,7 +61,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn
  * entities that are being requested.
  */
 public abstract class TimelineEntityReader {
-  private static final Log LOG = LogFactory.getLog(TimelineEntityReader.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TimelineEntityReader.class);
 
   private final boolean singleEntityRead;
   private TimelineReaderContext context;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63cfcb90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollector.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollector.java
index 0b05309..56f7b2b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollector.java
@@ -19,8 +19,6 @@
 package org.apache.hadoop.yarn.server.timelineservice.collector;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -32,6 +30,8 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 
 import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.HashSet;
 import java.util.Map;
@@ -48,7 +48,8 @@ import java.util.concurrent.TimeUnit;
 @Private
 @Unstable
 public class AppLevelTimelineCollector extends TimelineCollector {
-  private static final Log LOG = LogFactory.getLog(TimelineCollector.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TimelineCollector.class);
 
   private final static int AGGREGATION_EXECUTOR_NUM_THREADS = 1;
   private final static int AGGREGATION_EXECUTOR_EXEC_INTERVAL_SECS = 15;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63cfcb90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/NodeTimelineCollectorManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/NodeTimelineCollectorManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/NodeTimelineCollectorManager.java
index 0323d7b..1719782 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/NodeTimelineCollectorManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/NodeTimelineCollectorManager.java
@@ -26,8 +26,6 @@ import java.net.InetSocketAddress;
 import java.net.URI;
 import java.util.HashMap;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -47,6 +45,8 @@ import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Class on the NodeManager side that manages adding and removing collectors and
@@ -55,8 +55,8 @@ import com.google.common.annotations.VisibleForTesting;
 @Private
 @Unstable
 public class NodeTimelineCollectorManager extends TimelineCollectorManager {
-  private static final Log LOG =
-      LogFactory.getLog(NodeTimelineCollectorManager.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NodeTimelineCollectorManager.class);
 
   // REST server for this collector manager.
   private HttpServer2 timelineRestServer;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63cfcb90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.java
index 266bd04..e4e6421 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.java
@@ -23,8 +23,6 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -43,6 +41,8 @@ import org.apache.hadoop.yarn.server.api.ContainerTerminationContext;
 import org.apache.hadoop.yarn.server.api.ContainerType;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The top-level server for the per-node timeline collector manager. Currently
@@ -52,8 +52,8 @@ import com.google.common.annotations.VisibleForTesting;
 @Private
 @Unstable
 public class PerNodeTimelineCollectorsAuxService extends AuxiliaryService {
-  private static final Log LOG =
-      LogFactory.getLog(PerNodeTimelineCollectorsAuxService.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(PerNodeTimelineCollectorsAuxService.class);
   private static final int SHUTDOWN_HOOK_PRIORITY = 30;
 
   private final NodeTimelineCollectorManager collectorManager;
@@ -209,7 +209,7 @@ public class PerNodeTimelineCollectorsAuxService extends AuxiliaryService {
       auxService.init(conf);
       auxService.start();
     } catch (Throwable t) {
-      LOG.fatal("Error starting PerNodeTimelineCollectorServer", t);
+      LOG.error("Error starting PerNodeTimelineCollectorServer", t);
       ExitUtil.terminate(-1, "Error starting PerNodeTimelineCollectorServer");
     }
     return auxService;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63cfcb90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
index 5416b26..37387f1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
@@ -26,8 +26,6 @@ import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -39,6 +37,8 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineWriteResponse;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Service that handles writes to the timeline service and writes them to the
@@ -51,7 +51,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
 @Unstable
 public abstract class TimelineCollector extends CompositeService {
 
-  private static final Log LOG = LogFactory.getLog(TimelineCollector.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TimelineCollector.class);
   public static final String SEPARATOR = "_";
 
   private TimelineWriter writer;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63cfcb90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.java
index 07cbb2b..94b95ad 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.java
@@ -26,8 +26,6 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -39,6 +37,8 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Class that manages adding and removing collectors and their lifecycle. It
@@ -48,8 +48,8 @@ import com.google.common.annotations.VisibleForTesting;
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class TimelineCollectorManager extends AbstractService {
-  private static final Log LOG =
-      LogFactory.getLog(TimelineCollectorManager.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TimelineCollectorManager.class);
 
   private TimelineWriter writer;
   private ScheduledExecutorService writerFlusher;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63cfcb90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorWebService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorWebService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorWebService.java
index fe04b7a..efb5d6b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorWebService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorWebService.java
@@ -36,8 +36,6 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
@@ -58,6 +56,8 @@ import org.apache.hadoop.yarn.webapp.ForbiddenException;
 import org.apache.hadoop.yarn.webapp.NotFoundException;
 
 import com.google.inject.Singleton;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The main per-node REST end point for timeline service writes. It is
@@ -69,8 +69,8 @@ import com.google.inject.Singleton;
 @Singleton
 @Path("/ws/v2/timeline")
 public class TimelineCollectorWebService {
-  private static final Log LOG =
-      LogFactory.getLog(TimelineCollectorWebService.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TimelineCollectorWebService.class);
 
   private @Context ServletContext context;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63cfcb90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
index 2faf4b6..d7eff32 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
@@ -25,8 +25,6 @@ import java.net.URI;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -48,12 +46,15 @@ import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /** Main class for Timeline Reader. */
 @Private
 @Unstable
 public class TimelineReaderServer extends CompositeService {
-  private static final Log LOG = LogFactory.getLog(TimelineReaderServer.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TimelineReaderServer.class);
   private static final int SHUTDOWN_HOOK_PRIORITY = 30;
   static final String TIMELINE_READER_MANAGER_ATTR =
       "timeline.reader.manager";
@@ -203,7 +204,7 @@ public class TimelineReaderServer extends CompositeService {
       timelineReaderServer.init(conf);
       timelineReaderServer.start();
     } catch (Throwable t) {
-      LOG.fatal("Error starting TimelineReaderWebServer", t);
+      LOG.error("Error starting TimelineReaderWebServer", t);
       ExitUtil.terminate(-1, "Error starting TimelineReaderWebServer");
     }
     return timelineReaderServer;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63cfcb90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java
index 139a1be..b3e3cdc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java
@@ -40,8 +40,6 @@ import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.http.JettyUtils;
@@ -57,6 +55,8 @@ import org.apache.hadoop.yarn.webapp.NotFoundException;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.inject.Singleton;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /** REST end point for Timeline Reader. */
 @Private
@@ -64,8 +64,8 @@ import com.google.inject.Singleton;
 @Singleton
 @Path("/ws/v2/timeline")
 public class TimelineReaderWebServices {
-  private static final Log LOG =
-      LogFactory.getLog(TimelineReaderWebServices.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TimelineReaderWebServices.class);
 
   @Context private ServletContext ctxt;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63cfcb90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineReaderImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineReaderImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineReaderImpl.java
index 967702b..b4e792b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineReaderImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineReaderImpl.java
@@ -39,8 +39,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.commons.csv.CSVFormat;
 import org.apache.commons.csv.CSVParser;
 import org.apache.commons.csv.CSVRecord;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
@@ -54,6 +52,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStor
 import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  *  File System based implementation for TimelineReader. This implementation may
@@ -64,8 +64,8 @@ import com.google.common.annotations.VisibleForTesting;
 public class FileSystemTimelineReaderImpl extends AbstractService
     implements TimelineReader {
 
-  private static final Log LOG =
-      LogFactory.getLog(FileSystemTimelineReaderImpl.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FileSystemTimelineReaderImpl.class);
 
   private String rootPath;
   private static final String ENTITIES_DIR = "entities";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63cfcb90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineStorageUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineStorageUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineStorageUtils.java
index 9b83659..7f7d640 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineStorageUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineStorageUtils.java
@@ -23,8 +23,6 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
@@ -48,8 +46,6 @@ public final class TimelineStorageUtils {
   private TimelineStorageUtils() {
   }
 
-  private static final Log LOG = LogFactory.getLog(TimelineStorageUtils.class);
-
   /**
    * Matches key-values filter. Used for relatesTo/isRelatedTo filters.
    *


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[45/51] [abbrv] hadoop git commit: HADOOP-14355. Update maven-war-plugin to 3.1.0.

Posted by st...@apache.org.
HADOOP-14355. Update maven-war-plugin to 3.1.0.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/07694fc6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/07694fc6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/07694fc6

Branch: refs/heads/HADOOP-13345
Commit: 07694fc65ae6d97a430a7dd67a6277e5795c321f
Parents: ebabc70
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Aug 9 13:20:03 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed Aug 9 13:20:03 2017 +0900

----------------------------------------------------------------------
 hadoop-project/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/07694fc6/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 5aabdc7..8151016 100755
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -126,7 +126,7 @@
     <maven-resources-plugin.version>2.6</maven-resources-plugin.version>
     <maven-shade-plugin.version>2.4.3</maven-shade-plugin.version>
     <maven-jar-plugin.version>2.5</maven-jar-plugin.version>
-    <maven-war-plugin.version>2.4</maven-war-plugin.version>
+    <maven-war-plugin.version>3.1.0</maven-war-plugin.version>
     <maven-source-plugin.version>2.3</maven-source-plugin.version>
     <maven-pdf-plugin.version>1.2</maven-pdf-plugin.version>
     <maven-remote-resources-plugin.version>1.5</maven-remote-resources-plugin.version>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[11/51] [abbrv] hadoop git commit: HDFS-12131. Add some of the FSNamesystem JMX values as metrics. Contributed by Erik Krogen.

Posted by st...@apache.org.
HDFS-12131. Add some of the FSNamesystem JMX values as metrics. Contributed by Erik Krogen.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f4c6b00a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f4c6b00a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f4c6b00a

Branch: refs/heads/HADOOP-13345
Commit: f4c6b00a9f48ae7667db4035b641769efc3bb7cf
Parents: 0542e6f
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Aug 3 15:44:51 2017 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Thu Aug 3 15:44:51 2017 -0700

----------------------------------------------------------------------
 .../hadoop-common/src/site/markdown/Metrics.md  |  11 ++
 .../hdfs/server/namenode/FSNamesystem.java      |  20 ++++
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  |   8 +-
 .../namenode/metrics/TestNameNodeMetrics.java   | 112 +++++++++++++++++++
 4 files changed, 150 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4c6b00a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
index 4b89bc2..852a1e9 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
@@ -213,7 +213,15 @@ Each metrics record contains tags such as HAState and Hostname as additional inf
 | `PendingDataNodeMessageCount` | (HA-only) Current number of pending block-related messages for later processing in the standby NameNode |
 | `MillisSinceLastLoadedEdits` | (HA-only) Time in milliseconds since the last time standby NameNode load edit log. In active NameNode, set to 0 |
 | `BlockCapacity` | Current number of block capacity |
+| `NumLiveDataNodes` | Number of datanodes which are currently live |
+| `NumDeadDataNodes` | Number of datanodes which are currently dead |
+| `NumDecomLiveDataNodes` | Number of datanodes which have been decommissioned and are now live |
+| `NumDecomDeadDataNodes` | Number of datanodes which have been decommissioned and are now dead |
+| `NumDecommissioningDataNodes` | Number of datanodes in decommissioning state |
+| `VolumeFailuresTotal` | Total number of volume failures across all Datanodes |
+| `EstimatedCapacityLostTotal` | An estimate of the total capacity lost due to volume failures |
 | `StaleDataNodes` | Current number of DataNodes marked stale due to delayed heartbeat |
+| `NumStaleStorages` | Number of storages marked as content stale (after NameNode restart/failover before first block report is received) |
 | `MissingReplOneBlocks` | Current number of missing blocks with replication factor 1 |
 | `NumFilesUnderConstruction` | Current number of files under construction |
 | `NumActiveClients` | Current number of active clients holding lease |
@@ -224,6 +232,9 @@ Each metrics record contains tags such as HAState and Hostname as additional inf
 | `TotalSyncTimes` | Total number of milliseconds spent by various edit logs in sync operation|
 | `NameDirSize` | NameNode name directories size in bytes |
 | `NumTimedOutPendingReconstructions` | The number of timed out reconstructions. Not the number of unique blocks that timed out. |
+| `NumInMaintenanceLiveDataNodes` | Number of live Datanodes which are in maintenance state |
+| `NumInMaintenanceDeadDataNodes` | Number of dead Datanodes which are in maintenance state |
+| `NumEnteringMaintenanceDataNodes` | Number of Datanodes that are entering the maintenance state |
 | `FSN(Read|Write)Lock`*OperationName*`NumOps` | Total number of acquiring lock by operations |
 | `FSN(Read|Write)Lock`*OperationName*`AvgTime` | Average time of holding the lock by operations in milliseconds |
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4c6b00a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 5849712..229de05 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -4843,16 +4843,20 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   @Override // FSNamesystemMBean
+  @Metric({"NumLiveDataNodes", "Number of datanodes which are currently live"})
   public int getNumLiveDataNodes() {
     return getBlockManager().getDatanodeManager().getNumLiveDataNodes();
   }
 
   @Override // FSNamesystemMBean
+  @Metric({"NumDeadDataNodes", "Number of datanodes which are currently dead"})
   public int getNumDeadDataNodes() {
     return getBlockManager().getDatanodeManager().getNumDeadDataNodes();
   }
   
   @Override // FSNamesystemMBean
+  @Metric({"NumDecomLiveDataNodes",
+      "Number of datanodes which have been decommissioned and are now live"})
   public int getNumDecomLiveDataNodes() {
     final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
     getBlockManager().getDatanodeManager().fetchDatanodes(live, null, false);
@@ -4864,6 +4868,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   @Override // FSNamesystemMBean
+  @Metric({"NumDecomDeadDataNodes",
+      "Number of datanodes which have been decommissioned and are now dead"})
   public int getNumDecomDeadDataNodes() {
     final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
     getBlockManager().getDatanodeManager().fetchDatanodes(null, dead, false);
@@ -4875,6 +4881,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   @Override // FSNamesystemMBean
+  @Metric({"VolumeFailuresTotal",
+      "Total number of volume failures across all Datanodes"})
   public int getVolumeFailuresTotal() {
     List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
     getBlockManager().getDatanodeManager().fetchDatanodes(live, null, false);
@@ -4886,6 +4894,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   @Override // FSNamesystemMBean
+  @Metric({"EstimatedCapacityLostTotal",
+      "An estimate of the total capacity lost due to volume failures"})
   public long getEstimatedCapacityLostTotal() {
     List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
     getBlockManager().getDatanodeManager().fetchDatanodes(live, null, false);
@@ -4901,6 +4911,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   @Override // FSNamesystemMBean
+  @Metric({"NumDecommissioningDataNodes",
+      "Number of datanodes in decommissioning state"})
   public int getNumDecommissioningDataNodes() {
     return getBlockManager().getDatanodeManager().getDecommissioningNodes()
         .size();
@@ -4918,6 +4930,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * before NN receives the first Heartbeat followed by the first Blockreport.
    */
   @Override // FSNamesystemMBean
+  @Metric({"NumStaleStorages",
+      "Number of storages marked as content stale"})
   public int getNumStaleStorages() {
     return getBlockManager().getDatanodeManager().getNumStaleStorages();
   }
@@ -7542,6 +7556,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
 
   @Override // FSNamesystemMBean
+  @Metric({"NumInMaintenanceLiveDataNodes",
+      "Number of live Datanodes which are in maintenance state"})
   public int getNumInMaintenanceLiveDataNodes() {
     final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
     getBlockManager().getDatanodeManager().fetchDatanodes(live, null, true);
@@ -7553,6 +7569,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   @Override // FSNamesystemMBean
+  @Metric({"NumInMaintenanceDeadDataNodes",
+      "Number of dead Datanodes which are in maintenance state"})
   public int getNumInMaintenanceDeadDataNodes() {
     final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
     getBlockManager().getDatanodeManager().fetchDatanodes(null, dead, true);
@@ -7564,6 +7582,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   @Override // FSNamesystemMBean
+  @Metric({"NumEnteringMaintenanceDataNodes",
+      "Number of Datanodes that are entering the maintenance state"})
   public int getNumEnteringMaintenanceDataNodes() {
     return getBlockManager().getDatanodeManager().getEnteringMaintenanceNodes()
         .size();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4c6b00a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
index f9908fe..0345cf5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
@@ -61,6 +61,7 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
 import com.google.common.base.Supplier;
@@ -148,6 +149,8 @@ public class MiniDFSCluster implements AutoCloseable {
   public static final String HDFS_MINIDFS_BASEDIR = "hdfs.minidfs.basedir";
   public static final String  DFS_NAMENODE_SAFEMODE_EXTENSION_TESTING_KEY
       = DFS_NAMENODE_SAFEMODE_EXTENSION_KEY + ".testing";
+  public static final String  DFS_NAMENODE_DECOMMISSION_INTERVAL_TESTING_KEY
+      = DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY + ".testing";
 
   // Changing this default may break some tests that assume it is 2.
   private static final int DEFAULT_STORAGES_PER_DATANODE = 2;
@@ -818,7 +821,10 @@ public class MiniDFSCluster implements AutoCloseable {
       int safemodeExtension = conf.getInt(
           DFS_NAMENODE_SAFEMODE_EXTENSION_TESTING_KEY, 0);
       conf.setInt(DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, safemodeExtension);
-      conf.setInt(DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY, 3); // 3 second
+      long decommissionInterval = conf.getTimeDuration(
+          DFS_NAMENODE_DECOMMISSION_INTERVAL_TESTING_KEY, 3, TimeUnit.SECONDS);
+      conf.setTimeDuration(DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY,
+          decommissionInterval, TimeUnit.SECONDS);
       if (!useConfiguredTopologyMappingClass) {
         conf.setClass(NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
             StaticMapping.class, DNSToSwitchMapping.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4c6b00a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
index c3bb255..b983fd1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.metrics;
 
+import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.crypto.key.JavaKeyStoreProvider;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystemTestHelper;
@@ -41,7 +42,9 @@ import java.io.DataInputStream;
 import java.io.File;
 import java.io.IOException;
 import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
 import java.util.EnumSet;
+import java.util.List;
 import java.util.Random;
 import com.google.common.collect.ImmutableList;
 
@@ -69,12 +72,15 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.MockNameNodeResourceChecker;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
 import org.apache.hadoop.hdfs.tools.NNHAServiceTarget;
+import org.apache.hadoop.hdfs.util.HostsFileWriter;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.metrics2.MetricsSource;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
@@ -115,6 +121,15 @@ public class TestNameNodeMetrics {
     CONF.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 1);
     CONF.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
         DFS_REDUNDANCY_INTERVAL);
+    // Set it long enough to essentially disable unless we manually call it
+    // Used for decommissioning DataNode metrics
+    CONF.setTimeDuration(
+        MiniDFSCluster.DFS_NAMENODE_DECOMMISSION_INTERVAL_TESTING_KEY, 999,
+        TimeUnit.DAYS);
+    // Next two configs used for checking failed volume metrics
+    CONF.setTimeDuration(DFSConfigKeys.DFS_DATANODE_DISK_CHECK_MIN_GAP_KEY,
+        10, TimeUnit.MILLISECONDS);
+    CONF.setInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, 1);
     CONF.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
         DFS_REDUNDANCY_INTERVAL);
     CONF.set(DFSConfigKeys.DFS_METRICS_PERCENTILES_INTERVALS_KEY, 
@@ -133,6 +148,7 @@ public class TestNameNodeMetrics {
   private DistributedFileSystem fs;
   private final Random rand = new Random();
   private FSNamesystem namesystem;
+  private HostsFileWriter hostsFileWriter;
   private BlockManager bm;
   private Path ecDir;
 
@@ -142,6 +158,8 @@ public class TestNameNodeMetrics {
 
   @Before
   public void setUp() throws Exception {
+    hostsFileWriter = new HostsFileWriter();
+    hostsFileWriter.initialize(CONF, "temp/decommission");
     cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(DATANODE_COUNT)
         .build();
     cluster.waitActive();
@@ -161,6 +179,10 @@ public class TestNameNodeMetrics {
       MetricsRecordBuilder rb = getMetrics(source);
       assertQuantileGauges("GetGroups1s", rb);
     }
+    if (hostsFileWriter != null) {
+      hostsFileWriter.cleanup();
+      hostsFileWriter = null;
+    }
     if (cluster != null) {
       cluster.shutdown();
       cluster = null;
@@ -235,6 +257,96 @@ public class TestNameNodeMetrics {
         .getBlockManager());
     assertGauge("StaleDataNodes", 0, getMetrics(NS_METRICS));
   }
+
+  /**
+   * Test metrics associated with volume failures.
+   */
+  @Test
+  public void testVolumeFailures() throws Exception {
+    assertGauge("VolumeFailuresTotal", 0, getMetrics(NS_METRICS));
+    assertGauge("EstimatedCapacityLostTotal", 0L, getMetrics(NS_METRICS));
+    DataNode dn = cluster.getDataNodes().get(0);
+    FsDatasetSpi.FsVolumeReferences volumeReferences =
+        DataNodeTestUtils.getFSDataset(dn).getFsVolumeReferences();
+    FsVolumeImpl fsVolume = (FsVolumeImpl) volumeReferences.get(0);
+    File dataDir = new File(fsVolume.getBaseURI());
+    long capacity = fsVolume.getCapacity();
+    volumeReferences.close();
+    DataNodeTestUtils.injectDataDirFailure(dataDir);
+    DataNodeTestUtils.waitForDiskError(dn, fsVolume);
+    DataNodeTestUtils.triggerHeartbeat(dn);
+    BlockManagerTestUtil.checkHeartbeat(bm);
+    assertGauge("VolumeFailuresTotal", 1, getMetrics(NS_METRICS));
+    assertGauge("EstimatedCapacityLostTotal", capacity, getMetrics(NS_METRICS));
+  }
+
+  /**
+   * Test metrics associated with liveness and decommission status of DataNodes.
+   */
+  @Test
+  public void testDataNodeLivenessAndDecom() throws Exception {
+    List<DataNode> dataNodes = cluster.getDataNodes();
+    DatanodeDescriptor[] dnDescriptors = new DatanodeDescriptor[DATANODE_COUNT];
+    String[] dnAddresses = new String[DATANODE_COUNT];
+    for (int i = 0; i < DATANODE_COUNT; i++) {
+      dnDescriptors[i] = bm.getDatanodeManager()
+          .getDatanode(dataNodes.get(i).getDatanodeId());
+      dnAddresses[i] = dnDescriptors[i].getXferAddr();
+    }
+    // First put all DNs into include
+    hostsFileWriter.initIncludeHosts(dnAddresses);
+    bm.getDatanodeManager().refreshNodes(CONF);
+    assertGauge("NumDecomLiveDataNodes", 0, getMetrics(NS_METRICS));
+    assertGauge("NumLiveDataNodes", DATANODE_COUNT, getMetrics(NS_METRICS));
+
+    // Now decommission one DN
+    hostsFileWriter.initExcludeHost(dnAddresses[0]);
+    bm.getDatanodeManager().refreshNodes(CONF);
+    assertGauge("NumDecommissioningDataNodes", 1, getMetrics(NS_METRICS));
+    BlockManagerTestUtil.recheckDecommissionState(bm.getDatanodeManager());
+    assertGauge("NumDecommissioningDataNodes", 0, getMetrics(NS_METRICS));
+    assertGauge("NumDecomLiveDataNodes", 1, getMetrics(NS_METRICS));
+    assertGauge("NumLiveDataNodes", DATANODE_COUNT, getMetrics(NS_METRICS));
+
+    // Now kill all DNs by expiring their heartbeats
+    for (int i = 0; i < DATANODE_COUNT; i++) {
+      DataNodeTestUtils.setHeartbeatsDisabledForTests(dataNodes.get(i), true);
+      long expireInterval = CONF.getLong(
+          DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY,
+          DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_DEFAULT) * 2L
+          + CONF.getLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
+          DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT) * 10 * 1000L;
+      DFSTestUtil.resetLastUpdatesWithOffset(dnDescriptors[i],
+          -(expireInterval + 1));
+    }
+    BlockManagerTestUtil.checkHeartbeat(bm);
+    assertGauge("NumDecomLiveDataNodes", 0, getMetrics(NS_METRICS));
+    assertGauge("NumDecomDeadDataNodes", 1, getMetrics(NS_METRICS));
+    assertGauge("NumLiveDataNodes", 0, getMetrics(NS_METRICS));
+    assertGauge("NumDeadDataNodes", DATANODE_COUNT, getMetrics(NS_METRICS));
+
+    // Now remove the decommissioned DN altogether
+    String[] includeHosts = new String[dnAddresses.length - 1];
+    for (int i = 0; i < includeHosts.length; i++) {
+      includeHosts[i] = dnAddresses[i + 1];
+    }
+    hostsFileWriter.initIncludeHosts(includeHosts);
+    hostsFileWriter.initExcludeHosts(new ArrayList<>());
+    bm.getDatanodeManager().refreshNodes(CONF);
+    assertGauge("NumDecomLiveDataNodes", 0, getMetrics(NS_METRICS));
+    assertGauge("NumDecomDeadDataNodes", 0, getMetrics(NS_METRICS));
+    assertGauge("NumLiveDataNodes", 0, getMetrics(NS_METRICS));
+    assertGauge("NumDeadDataNodes", DATANODE_COUNT - 1, getMetrics(NS_METRICS));
+
+    // Finally mark the remaining DNs as live again
+    for (int i = 1; i < dataNodes.size(); i++) {
+      DataNodeTestUtils.setHeartbeatsDisabledForTests(dataNodes.get(i), false);
+      DFSTestUtil.resetLastUpdatesWithOffset(dnDescriptors[i], 0);
+    }
+    BlockManagerTestUtil.checkHeartbeat(bm);
+    assertGauge("NumLiveDataNodes", DATANODE_COUNT - 1, getMetrics(NS_METRICS));
+    assertGauge("NumDeadDataNodes", 0, getMetrics(NS_METRICS));
+  }
   
   /** Test metrics associated with addition of a file */
   @Test


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[18/51] [abbrv] hadoop git commit: HDFS-12224. Add tests to TestJournalNodeSync for sync after JN downtime. Contributed by Hanisha Koneru.

Posted by st...@apache.org.
HDFS-12224. Add tests to TestJournalNodeSync for sync after JN downtime. Contributed by Hanisha Koneru.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/bbc6d254
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/bbc6d254
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/bbc6d254

Branch: refs/heads/HADOOP-13345
Commit: bbc6d254c8a953abba69415d80edeede3ee6269d
Parents: fe33417
Author: Arpit Agarwal <ar...@apache.org>
Authored: Fri Aug 4 12:51:33 2017 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Fri Aug 4 12:51:33 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/qjournal/server/Journal.java    |   3 +-
 .../hdfs/qjournal/server/JournalMetrics.java    |  11 +
 .../hdfs/qjournal/server/JournalNodeSyncer.java |   4 +
 .../hdfs/qjournal/TestJournalNodeSync.java      | 265 -----------
 .../hdfs/qjournal/server/TestJournalNode.java   |   6 +-
 .../qjournal/server/TestJournalNodeSync.java    | 439 +++++++++++++++++++
 6 files changed, 458 insertions(+), 270 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bbc6d254/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
index 0041d5e..0f4091d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
@@ -286,8 +286,7 @@ public class Journal implements Closeable {
     fjm.setLastReadableTxId(val);
   }
 
-  @VisibleForTesting
-  JournalMetrics getMetricsForTests() {
+  JournalMetrics getMetrics() {
     return metrics;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bbc6d254/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalMetrics.java
index cffe2c1..fcfd901 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalMetrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalMetrics.java
@@ -45,6 +45,9 @@ class JournalMetrics {
   
   @Metric("Number of batches written where this node was lagging")
   MutableCounterLong batchesWrittenWhileLagging;
+
+  @Metric("Number of edit logs downloaded by JournalNodeSyncer")
+  private MutableCounterLong numEditLogsSynced;
   
   private final int[] QUANTILE_INTERVALS = new int[] {
       1*60, // 1m
@@ -120,4 +123,12 @@ class JournalMetrics {
       q.add(us);
     }
   }
+
+  public MutableCounterLong getNumEditLogsSynced() {
+    return numEditLogsSynced;
+  }
+
+  public void incrNumEditLogsSynced() {
+    numEditLogsSynced.incr();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bbc6d254/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeSyncer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeSyncer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeSyncer.java
index 479f6a0..537ba0a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeSyncer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeSyncer.java
@@ -77,6 +77,7 @@ public class JournalNodeSyncer {
   private final long journalSyncInterval;
   private final int logSegmentTransferTimeout;
   private final DataTransferThrottler throttler;
+  private final JournalMetrics metrics;
 
   JournalNodeSyncer(JournalNode jouranlNode, Journal journal, String jid,
       Configuration conf) {
@@ -93,6 +94,7 @@ public class JournalNodeSyncer {
         DFSConfigKeys.DFS_EDIT_LOG_TRANSFER_TIMEOUT_KEY,
         DFSConfigKeys.DFS_EDIT_LOG_TRANSFER_TIMEOUT_DEFAULT);
     throttler = getThrottler(conf);
+    metrics = journal.getMetrics();
   }
 
   void stopSync() {
@@ -411,6 +413,8 @@ public class JournalNodeSyncer {
         LOG.warn("Deleting " + tmpEditsFile + " has failed");
       }
       return false;
+    } else {
+      metrics.incrNumEditLogsSynced();
     }
     return true;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bbc6d254/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestJournalNodeSync.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestJournalNodeSync.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestJournalNodeSync.java
deleted file mode 100644
index 8415a6f..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestJournalNodeSync.java
+++ /dev/null
@@ -1,265 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.qjournal;
-
-import com.google.common.base.Supplier;
-import com.google.common.collect.Lists;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
-import static org.apache.hadoop.hdfs.server.namenode.FileJournalManager
-    .getLogFile;
-
-import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.List;
-import java.util.Random;
-
-/**
- * Unit test for Journal Node formatting upon re-installation and syncing.
- */
-public class TestJournalNodeSync {
-  private MiniQJMHACluster qjmhaCluster;
-  private MiniDFSCluster dfsCluster;
-  private MiniJournalCluster jCluster;
-  private FileSystem fs;
-  private FSNamesystem namesystem;
-  private int editsPerformed = 0;
-  private final String jid = "ns1";
-
-  @Before
-  public void setUpMiniCluster() throws IOException {
-    final Configuration conf = new HdfsConfiguration();
-    conf.setBoolean(DFSConfigKeys.DFS_JOURNALNODE_ENABLE_SYNC_KEY, true);
-    conf.setLong(DFSConfigKeys.DFS_JOURNALNODE_SYNC_INTERVAL_KEY, 1000L);
-    qjmhaCluster = new MiniQJMHACluster.Builder(conf).setNumNameNodes(2)
-      .build();
-    dfsCluster = qjmhaCluster.getDfsCluster();
-    jCluster = qjmhaCluster.getJournalCluster();
-
-    dfsCluster.transitionToActive(0);
-    fs = dfsCluster.getFileSystem(0);
-    namesystem = dfsCluster.getNamesystem(0);
-  }
-
-  @After
-  public void shutDownMiniCluster() throws IOException {
-    if (qjmhaCluster != null) {
-      qjmhaCluster.shutdown();
-    }
-  }
-
-  @Test(timeout=30000)
-  public void testJournalNodeSync() throws Exception {
-    File firstJournalDir = jCluster.getJournalDir(0, jid);
-    File firstJournalCurrentDir = new StorageDirectory(firstJournalDir)
-        .getCurrentDir();
-
-    // Generate some edit logs and delete one.
-    long firstTxId = generateEditLog();
-    generateEditLog();
-
-    File missingLog = deleteEditLog(firstJournalCurrentDir, firstTxId);
-
-    GenericTestUtils.waitFor(editLogExists(Lists.newArrayList(missingLog)),
-        500, 10000);
-  }
-
-  @Test(timeout=30000)
-  public void testSyncForMultipleMissingLogs() throws Exception {
-    File firstJournalDir = jCluster.getJournalDir(0, jid);
-    File firstJournalCurrentDir = new StorageDirectory(firstJournalDir)
-        .getCurrentDir();
-
-    // Generate some edit logs and delete two.
-    long firstTxId = generateEditLog();
-    long nextTxId = generateEditLog();
-
-    List<File> missingLogs = Lists.newArrayList();
-    missingLogs.add(deleteEditLog(firstJournalCurrentDir, firstTxId));
-    missingLogs.add(deleteEditLog(firstJournalCurrentDir, nextTxId));
-
-    GenericTestUtils.waitFor(editLogExists(missingLogs), 500, 10000);
-  }
-
-  @Test(timeout=30000)
-  public void testSyncForDiscontinuousMissingLogs() throws Exception {
-    File firstJournalDir = jCluster.getJournalDir(0, jid);
-    File firstJournalCurrentDir = new StorageDirectory(firstJournalDir)
-        .getCurrentDir();
-
-    // Generate some edit logs and delete two discontinuous logs.
-    long firstTxId = generateEditLog();
-    generateEditLog();
-    long nextTxId = generateEditLog();
-
-    List<File> missingLogs = Lists.newArrayList();
-    missingLogs.add(deleteEditLog(firstJournalCurrentDir, firstTxId));
-    missingLogs.add(deleteEditLog(firstJournalCurrentDir, nextTxId));
-
-    GenericTestUtils.waitFor(editLogExists(missingLogs), 500, 10000);
-  }
-
-  @Test(timeout=30000)
-  public void testMultipleJournalsMissingLogs() throws Exception {
-    File firstJournalDir = jCluster.getJournalDir(0, jid);
-    File firstJournalCurrentDir = new StorageDirectory(firstJournalDir)
-        .getCurrentDir();
-
-    File secondJournalDir = jCluster.getJournalDir(1, jid);
-    StorageDirectory sd = new StorageDirectory(secondJournalDir);
-    File secondJournalCurrentDir = sd.getCurrentDir();
-
-    // Generate some edit logs and delete one log from two journals.
-    long firstTxId = generateEditLog();
-    generateEditLog();
-
-    List<File> missingLogs = Lists.newArrayList();
-    missingLogs.add(deleteEditLog(firstJournalCurrentDir, firstTxId));
-    missingLogs.add(deleteEditLog(secondJournalCurrentDir, firstTxId));
-
-    GenericTestUtils.waitFor(editLogExists(missingLogs), 500, 10000);
-  }
-
-  @Test(timeout=60000)
-  public void testMultipleJournalsMultipleMissingLogs() throws Exception {
-    File firstJournalDir = jCluster.getJournalDir(0, jid);
-    File firstJournalCurrentDir = new StorageDirectory(firstJournalDir)
-        .getCurrentDir();
-
-    File secondJournalDir = jCluster.getJournalDir(1, jid);
-    File secondJournalCurrentDir = new StorageDirectory(secondJournalDir)
-        .getCurrentDir();
-
-    File thirdJournalDir = jCluster.getJournalDir(2, jid);
-    File thirdJournalCurrentDir = new StorageDirectory(thirdJournalDir)
-        .getCurrentDir();
-
-    // Generate some edit logs and delete multiple logs in multiple journals.
-    long firstTxId = generateEditLog();
-    long secondTxId = generateEditLog();
-    long thirdTxId = generateEditLog();
-
-    List<File> missingLogs = Lists.newArrayList();
-    missingLogs.add(deleteEditLog(firstJournalCurrentDir, firstTxId));
-    missingLogs.add(deleteEditLog(secondJournalCurrentDir, firstTxId));
-    missingLogs.add(deleteEditLog(secondJournalCurrentDir, secondTxId));
-    missingLogs.add(deleteEditLog(thirdJournalCurrentDir, thirdTxId));
-
-    GenericTestUtils.waitFor(editLogExists(missingLogs), 500, 30000);
-  }
-
-  // Test JournalNode Sync by randomly deleting edit logs from one or two of
-  // the journals.
-  @Test(timeout=60000)
-  public void testRandomJournalMissingLogs() throws Exception {
-    Random randomJournal = new Random();
-
-    List<File> journalCurrentDirs = Lists.newArrayList();
-
-    for (int i = 0; i < 3; i++) {
-      journalCurrentDirs.add(new StorageDirectory(jCluster.getJournalDir(i,
-          jid)).getCurrentDir());
-    }
-
-    int count = 0;
-    long lastStartTxId;
-    int journalIndex;
-    List<File> missingLogs = Lists.newArrayList();
-    while (count < 5) {
-      lastStartTxId = generateEditLog();
-
-      // Delete the last edit log segment from randomly selected journal node
-      journalIndex = randomJournal.nextInt(3);
-      missingLogs.add(deleteEditLog(journalCurrentDirs.get(journalIndex),
-          lastStartTxId));
-
-      // Delete the last edit log segment from two journals for some logs
-      if (count % 2 == 0) {
-        journalIndex = (journalIndex + 1) % 3;
-        missingLogs.add(deleteEditLog(journalCurrentDirs.get(journalIndex),
-            lastStartTxId));
-      }
-
-      count++;
-    }
-
-    GenericTestUtils.waitFor(editLogExists(missingLogs), 500, 30000);
-  }
-
-  private File deleteEditLog(File currentDir, long startTxId)
-      throws IOException {
-    EditLogFile logFile = getLogFile(currentDir, startTxId);
-    while (logFile.isInProgress()) {
-      dfsCluster.getNameNode(0).getRpcServer().rollEditLog();
-      logFile = getLogFile(currentDir, startTxId);
-    }
-    File deleteFile = logFile.getFile();
-    Assert.assertTrue("Couldn't delete edit log file", deleteFile.delete());
-
-    return deleteFile;
-  }
-
-  /**
-   * Do a mutative metadata operation on the file system.
-   *
-   * @return true if the operation was successful, false otherwise.
-   */
-  private boolean doAnEdit() throws IOException {
-    return fs.mkdirs(new Path("/tmp", Integer.toString(editsPerformed++)));
-  }
-
-  /**
-   * Does an edit and rolls the Edit Log.
-   *
-   * @return the startTxId of next segment after rolling edits.
-   */
-  private long generateEditLog() throws IOException {
-    long startTxId = namesystem.getFSImage().getEditLog().getLastWrittenTxId();
-    Assert.assertTrue("Failed to do an edit", doAnEdit());
-    dfsCluster.getNameNode(0).getRpcServer().rollEditLog();
-    return startTxId;
-  }
-
-  private Supplier<Boolean> editLogExists(List<File> editLogs) {
-    Supplier<Boolean> supplier = new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        for (File editLog : editLogs) {
-          if (!editLog.exists()) {
-            return false;
-          }
-        }
-        return true;
-      }
-    };
-    return supplier;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bbc6d254/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
index 9dd6846..28ec708 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
@@ -102,7 +102,7 @@ public class TestJournalNode {
   @Test(timeout=100000)
   public void testJournal() throws Exception {
     MetricsRecordBuilder metrics = MetricsAsserts.getMetrics(
-        journal.getMetricsForTests().getName());
+        journal.getMetrics().getName());
     MetricsAsserts.assertCounter("BatchesWritten", 0L, metrics);
     MetricsAsserts.assertCounter("BatchesWrittenWhileLagging", 0L, metrics);
     MetricsAsserts.assertGauge("CurrentLagTxns", 0L, metrics);
@@ -117,7 +117,7 @@ public class TestJournalNode {
     ch.sendEdits(1L, 1, 1, "hello".getBytes(Charsets.UTF_8)).get();
     
     metrics = MetricsAsserts.getMetrics(
-        journal.getMetricsForTests().getName());
+        journal.getMetrics().getName());
     MetricsAsserts.assertCounter("BatchesWritten", 1L, metrics);
     MetricsAsserts.assertCounter("BatchesWrittenWhileLagging", 0L, metrics);
     MetricsAsserts.assertGauge("CurrentLagTxns", 0L, metrics);
@@ -130,7 +130,7 @@ public class TestJournalNode {
     ch.sendEdits(1L, 2, 1, "goodbye".getBytes(Charsets.UTF_8)).get();
 
     metrics = MetricsAsserts.getMetrics(
-        journal.getMetricsForTests().getName());
+        journal.getMetrics().getName());
     MetricsAsserts.assertCounter("BatchesWritten", 2L, metrics);
     MetricsAsserts.assertCounter("BatchesWrittenWhileLagging", 1L, metrics);
     MetricsAsserts.assertGauge("CurrentLagTxns", 98L, metrics);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bbc6d254/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeSync.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeSync.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeSync.java
new file mode 100644
index 0000000..2964f05
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeSync.java
@@ -0,0 +1,439 @@
+/**
+ * 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.qjournal.server;
+
+import com.google.common.base.Supplier;
+import com.google.common.collect.Lists;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.qjournal.MiniJournalCluster;
+import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
+import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
+import static org.apache.hadoop.hdfs.server.namenode.FileJournalManager
+    .getLogFile;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+
+/**
+ * Unit test for Journal Node formatting upon re-installation and syncing.
+ */
+public class TestJournalNodeSync {
+  private Configuration conf;
+  private MiniQJMHACluster qjmhaCluster;
+  private MiniDFSCluster dfsCluster;
+  private MiniJournalCluster jCluster;
+  private FileSystem fs;
+  private FSNamesystem namesystem;
+  private int editsPerformed = 0;
+  private final String jid = "ns1";
+
+  @Rule
+  public TestName testName = new TestName();
+
+  @Before
+  public void setUpMiniCluster() throws IOException {
+    conf = new HdfsConfiguration();
+    conf.setBoolean(DFSConfigKeys.DFS_JOURNALNODE_ENABLE_SYNC_KEY, true);
+    conf.setLong(DFSConfigKeys.DFS_JOURNALNODE_SYNC_INTERVAL_KEY, 1000L);
+    if (testName.getMethodName().equals(
+        "testSyncAfterJNdowntimeWithoutQJournalQueue")) {
+      conf.setInt(DFSConfigKeys.DFS_QJOURNAL_QUEUE_SIZE_LIMIT_KEY, 0);
+    }
+    qjmhaCluster = new MiniQJMHACluster.Builder(conf).setNumNameNodes(2)
+      .build();
+    dfsCluster = qjmhaCluster.getDfsCluster();
+    jCluster = qjmhaCluster.getJournalCluster();
+
+    dfsCluster.transitionToActive(0);
+    fs = dfsCluster.getFileSystem(0);
+    namesystem = dfsCluster.getNamesystem(0);
+  }
+
+  @After
+  public void shutDownMiniCluster() throws IOException {
+    if (qjmhaCluster != null) {
+      qjmhaCluster.shutdown();
+    }
+  }
+
+  @Test(timeout=30000)
+  public void testJournalNodeSync() throws Exception {
+    File firstJournalDir = jCluster.getJournalDir(0, jid);
+    File firstJournalCurrentDir = new StorageDirectory(firstJournalDir)
+        .getCurrentDir();
+
+    // Generate some edit logs and delete one.
+    long firstTxId = generateEditLog();
+    generateEditLog();
+
+    File missingLog = deleteEditLog(firstJournalCurrentDir, firstTxId);
+
+    GenericTestUtils.waitFor(editLogExists(Lists.newArrayList(missingLog)),
+        500, 10000);
+  }
+
+  @Test(timeout=30000)
+  public void testSyncForMultipleMissingLogs() throws Exception {
+    File firstJournalDir = jCluster.getJournalDir(0, jid);
+    File firstJournalCurrentDir = new StorageDirectory(firstJournalDir)
+        .getCurrentDir();
+
+    // Generate some edit logs and delete two.
+    long firstTxId = generateEditLog();
+    long nextTxId = generateEditLog();
+
+    List<File> missingLogs = Lists.newArrayList();
+    missingLogs.add(deleteEditLog(firstJournalCurrentDir, firstTxId));
+    missingLogs.add(deleteEditLog(firstJournalCurrentDir, nextTxId));
+
+    GenericTestUtils.waitFor(editLogExists(missingLogs), 500, 10000);
+  }
+
+  @Test(timeout=30000)
+  public void testSyncForDiscontinuousMissingLogs() throws Exception {
+    File firstJournalDir = jCluster.getJournalDir(0, jid);
+    File firstJournalCurrentDir = new StorageDirectory(firstJournalDir)
+        .getCurrentDir();
+
+    // Generate some edit logs and delete two discontinuous logs.
+    long firstTxId = generateEditLog();
+    generateEditLog();
+    long nextTxId = generateEditLog();
+
+    List<File> missingLogs = Lists.newArrayList();
+    missingLogs.add(deleteEditLog(firstJournalCurrentDir, firstTxId));
+    missingLogs.add(deleteEditLog(firstJournalCurrentDir, nextTxId));
+
+    GenericTestUtils.waitFor(editLogExists(missingLogs), 500, 10000);
+  }
+
+  @Test(timeout=30000)
+  public void testMultipleJournalsMissingLogs() throws Exception {
+    File firstJournalDir = jCluster.getJournalDir(0, jid);
+    File firstJournalCurrentDir = new StorageDirectory(firstJournalDir)
+        .getCurrentDir();
+
+    File secondJournalDir = jCluster.getJournalDir(1, jid);
+    StorageDirectory sd = new StorageDirectory(secondJournalDir);
+    File secondJournalCurrentDir = sd.getCurrentDir();
+
+    // Generate some edit logs and delete one log from two journals.
+    long firstTxId = generateEditLog();
+    generateEditLog();
+
+    List<File> missingLogs = Lists.newArrayList();
+    missingLogs.add(deleteEditLog(firstJournalCurrentDir, firstTxId));
+    missingLogs.add(deleteEditLog(secondJournalCurrentDir, firstTxId));
+
+    GenericTestUtils.waitFor(editLogExists(missingLogs), 500, 10000);
+  }
+
+  @Test(timeout=60000)
+  public void testMultipleJournalsMultipleMissingLogs() throws Exception {
+    File firstJournalDir = jCluster.getJournalDir(0, jid);
+    File firstJournalCurrentDir = new StorageDirectory(firstJournalDir)
+        .getCurrentDir();
+
+    File secondJournalDir = jCluster.getJournalDir(1, jid);
+    File secondJournalCurrentDir = new StorageDirectory(secondJournalDir)
+        .getCurrentDir();
+
+    File thirdJournalDir = jCluster.getJournalDir(2, jid);
+    File thirdJournalCurrentDir = new StorageDirectory(thirdJournalDir)
+        .getCurrentDir();
+
+    // Generate some edit logs and delete multiple logs in multiple journals.
+    long firstTxId = generateEditLog();
+    long secondTxId = generateEditLog();
+    long thirdTxId = generateEditLog();
+
+    List<File> missingLogs = Lists.newArrayList();
+    missingLogs.add(deleteEditLog(firstJournalCurrentDir, firstTxId));
+    missingLogs.add(deleteEditLog(secondJournalCurrentDir, firstTxId));
+    missingLogs.add(deleteEditLog(secondJournalCurrentDir, secondTxId));
+    missingLogs.add(deleteEditLog(thirdJournalCurrentDir, thirdTxId));
+
+    GenericTestUtils.waitFor(editLogExists(missingLogs), 500, 30000);
+  }
+
+  // Test JournalNode Sync by randomly deleting edit logs from one or two of
+  // the journals.
+  @Test(timeout=60000)
+  public void testRandomJournalMissingLogs() throws Exception {
+    Random randomJournal = new Random();
+
+    List<File> journalCurrentDirs = Lists.newArrayList();
+
+    for (int i = 0; i < 3; i++) {
+      journalCurrentDirs.add(new StorageDirectory(jCluster.getJournalDir(i,
+          jid)).getCurrentDir());
+    }
+
+    int count = 0;
+    long lastStartTxId;
+    int journalIndex;
+    List<File> missingLogs = Lists.newArrayList();
+    while (count < 5) {
+      lastStartTxId = generateEditLog();
+
+      // Delete the last edit log segment from randomly selected journal node
+      journalIndex = randomJournal.nextInt(3);
+      missingLogs.add(deleteEditLog(journalCurrentDirs.get(journalIndex),
+          lastStartTxId));
+
+      // Delete the last edit log segment from two journals for some logs
+      if (count % 2 == 0) {
+        journalIndex = (journalIndex + 1) % 3;
+        missingLogs.add(deleteEditLog(journalCurrentDirs.get(journalIndex),
+            lastStartTxId));
+      }
+
+      count++;
+    }
+
+    GenericTestUtils.waitFor(editLogExists(missingLogs), 500, 30000);
+  }
+
+  // Test JournalNode Sync when a JN id down while NN is actively writing
+  // logs and comes back up after some time.
+  @Test (timeout=300_000)
+  public void testSyncAfterJNdowntime() throws Exception {
+    File firstJournalDir = jCluster.getJournalDir(0, jid);
+    File firstJournalCurrentDir = new StorageDirectory(firstJournalDir)
+        .getCurrentDir();
+    File secondJournalDir = jCluster.getJournalDir(1, jid);
+    File secondJournalCurrentDir = new StorageDirectory(secondJournalDir)
+        .getCurrentDir();
+
+    long[] startTxIds = new long[10];
+
+    startTxIds[0] = generateEditLog();
+    startTxIds[1] = generateEditLog();
+
+    // Stop the first JN
+    jCluster.getJournalNode(0).stop(0);
+
+    // Roll some more edits while the first JN is down
+    for (int i = 2; i < 10; i++) {
+      startTxIds[i] = generateEditLog(5);
+    }
+
+    // Re-start the first JN
+    jCluster.restartJournalNode(0);
+
+    // Roll an edit to update the committed tx id of the first JN
+    generateEditLog();
+
+    // List the edit logs rolled during JN down time.
+    List<File> missingLogs = Lists.newArrayList();
+    for (int i = 2; i < 10; i++) {
+      EditLogFile logFile = getLogFile(secondJournalCurrentDir, startTxIds[i],
+          false);
+      missingLogs.add(new File(firstJournalCurrentDir,
+          logFile.getFile().getName()));
+    }
+
+    // Check that JNSync downloaded the edit logs rolled during JN down time.
+    GenericTestUtils.waitFor(editLogExists(missingLogs), 500, 30000);
+  }
+
+  /**
+   * Test JournalNode Sync when a JN id down while NN is actively writing
+   * logs and comes back up after some time with no edit log queueing.
+   * Queuing disabled during the cluster setup {@link #setUpMiniCluster()}
+   * @throws Exception
+   */
+  @Test (timeout=300_000)
+  public void testSyncAfterJNdowntimeWithoutQJournalQueue() throws Exception{
+    // Queuing is disabled during the cluster setup {@link #setUpMiniCluster()}
+    File firstJournalDir = jCluster.getJournalDir(0, jid);
+    File firstJournalCurrentDir = new StorageDirectory(firstJournalDir)
+        .getCurrentDir();
+    File secondJournalDir = jCluster.getJournalDir(1, jid);
+    File secondJournalCurrentDir = new StorageDirectory(secondJournalDir)
+        .getCurrentDir();
+
+    long[] startTxIds = new long[10];
+
+    startTxIds[0] = generateEditLog();
+    startTxIds[1] = generateEditLog(2);
+
+    // Stop the first JN
+    jCluster.getJournalNode(0).stop(0);
+
+    // Roll some more edits while the first JN is down
+    for (int i = 2; i < 10; i++) {
+      startTxIds[i] = generateEditLog(5);
+    }
+
+    // Re-start the first JN
+    jCluster.restartJournalNode(0);
+
+    // After JN restart and before rolling another edit, the missing edit
+    // logs will not by synced as the committed tx id of the JN will be
+    // less than the start tx id's of the missing edit logs and edit log queuing
+    // has been disabled.
+    // Roll an edit to update the committed tx id of the first JN
+    generateEditLog(2);
+
+    // List the edit logs rolled during JN down time.
+    List<File> missingLogs = Lists.newArrayList();
+    for (int i = 2; i < 10; i++) {
+      EditLogFile logFile = getLogFile(secondJournalCurrentDir, startTxIds[i],
+          false);
+      missingLogs.add(new File(firstJournalCurrentDir,
+          logFile.getFile().getName()));
+    }
+
+    // Check that JNSync downloaded the edit logs rolled during JN down time.
+    GenericTestUtils.waitFor(editLogExists(missingLogs), 500, 30000);
+
+    // Check that all the missing edit logs have been downloaded via
+    // JournalNodeSyncer alone (as the edit log queueing has been disabled)
+    long numEditLogsSynced = jCluster.getJournalNode(0).getOrCreateJournal(jid)
+        .getMetrics().getNumEditLogsSynced().value();
+    Assert.assertTrue("Edit logs downloaded outside syncer. Expected 8 or " +
+            "more downloads, got " + numEditLogsSynced + " downloads instead",
+        numEditLogsSynced >= 8);
+  }
+
+  // Test JournalNode Sync when a JN is formatted while NN is actively writing
+  // logs.
+  @Test (timeout=300_000)
+  public void testSyncAfterJNformat() throws Exception{
+    File firstJournalDir = jCluster.getJournalDir(0, jid);
+    File firstJournalCurrentDir = new StorageDirectory(firstJournalDir)
+        .getCurrentDir();
+    File secondJournalDir = jCluster.getJournalDir(1, jid);
+    File secondJournalCurrentDir = new StorageDirectory(secondJournalDir)
+        .getCurrentDir();
+
+    long[] startTxIds = new long[10];
+
+    startTxIds[0] = generateEditLog(1);
+    startTxIds[1] = generateEditLog(2);
+    startTxIds[2] = generateEditLog(4);
+    startTxIds[3] = generateEditLog(6);
+
+    Journal journal1 = jCluster.getJournalNode(0).getOrCreateJournal(jid);
+    NamespaceInfo nsInfo = journal1.getStorage().getNamespaceInfo();
+
+    // Delete contents of current directory of one JN
+    for (File file : firstJournalCurrentDir.listFiles()) {
+      file.delete();
+    }
+
+    // Format the JN
+    journal1.format(nsInfo);
+
+    // Roll some more edits
+    for (int i = 4; i < 10; i++) {
+      startTxIds[i] = generateEditLog(5);
+    }
+
+    // List the edit logs rolled during JN down time.
+    List<File> missingLogs = Lists.newArrayList();
+    for (int i = 0; i < 10; i++) {
+      EditLogFile logFile = getLogFile(secondJournalCurrentDir, startTxIds[i],
+          false);
+      missingLogs.add(new File(firstJournalCurrentDir,
+          logFile.getFile().getName()));
+    }
+
+    // Check that the formatted JN has all the edit logs.
+    GenericTestUtils.waitFor(editLogExists(missingLogs), 500, 30000);
+  }
+
+  private File deleteEditLog(File currentDir, long startTxId)
+      throws IOException {
+    EditLogFile logFile = getLogFile(currentDir, startTxId);
+    while (logFile.isInProgress()) {
+      dfsCluster.getNameNode(0).getRpcServer().rollEditLog();
+      logFile = getLogFile(currentDir, startTxId);
+    }
+    File deleteFile = logFile.getFile();
+    Assert.assertTrue("Couldn't delete edit log file", deleteFile.delete());
+
+    return deleteFile;
+  }
+
+  /**
+   * Do a mutative metadata operation on the file system.
+   *
+   * @return true if the operation was successful, false otherwise.
+   */
+  private boolean doAnEdit() throws IOException {
+    return fs.mkdirs(new Path("/tmp", Integer.toString(editsPerformed++)));
+  }
+
+  /**
+   * Does an edit and rolls the Edit Log.
+   *
+   * @return the startTxId of next segment after rolling edits.
+   */
+  private long generateEditLog() throws IOException {
+    return generateEditLog(1);
+  }
+
+  /**
+   * Does specified number of edits and rolls the Edit Log.
+   *
+   * @param numEdits number of Edits to perform
+   * @return the startTxId of next segment after rolling edits.
+   */
+  private long generateEditLog(int numEdits) throws IOException {
+    long startTxId = namesystem.getFSImage().getEditLog().getLastWrittenTxId();
+    for (int i = 1; i <= numEdits; i++) {
+      Assert.assertTrue("Failed to do an edit", doAnEdit());
+    }
+    dfsCluster.getNameNode(0).getRpcServer().rollEditLog();
+    return startTxId;
+  }
+
+  private Supplier<Boolean> editLogExists(List<File> editLogs) {
+    Supplier<Boolean> supplier = new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        for (File editLog : editLogs) {
+          if (!editLog.exists()) {
+            return false;
+          }
+        }
+        return true;
+      }
+    };
+    return supplier;
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[09/51] [abbrv] hadoop git commit: MAPREDUCE-6914. Tests use assertTrue(....equals(...)) instead of assertEquals()). (Daniel Templeton via Yufei Gu)

Posted by st...@apache.org.
MAPREDUCE-6914. Tests use assertTrue(....equals(...)) instead of assertEquals()). (Daniel Templeton via Yufei Gu)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b8e82418
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b8e82418
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b8e82418

Branch: refs/heads/HADOOP-13345
Commit: b8e8241854904e3a92b636ffaa6f4c651e47cea2
Parents: c617fe0
Author: Yufei Gu <yu...@apache.org>
Authored: Thu Aug 3 11:44:34 2017 -0700
Committer: Yufei Gu <yu...@apache.org>
Committed: Thu Aug 3 11:44:34 2017 -0700

----------------------------------------------------------------------
 .../hadoop/mapreduce/v2/app/TestJobEndNotifier.java | 16 ++++++++--------
 .../v2/app/rm/TestRMContainerAllocator.java         |  3 +--
 .../java/org/apache/hadoop/mapred/TestQueue.java    |  8 ++++----
 .../org/apache/hadoop/mapred/TestYARNRunner.java    |  8 ++++----
 .../mapreduce/lib/input/TestMultipleInputs.java     | 10 +++++-----
 5 files changed, 22 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8e82418/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestJobEndNotifier.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestJobEndNotifier.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestJobEndNotifier.java
index d122a9b..5af79d6 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestJobEndNotifier.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestJobEndNotifier.java
@@ -124,20 +124,20 @@ public class TestJobEndNotifier extends JobEndNotifier {
       proxyToUse.type() == Proxy.Type.DIRECT);
     conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_PROXY, "somehost:1000");
     setConf(conf);
-    Assert.assertTrue("Proxy should have been set but wasn't ",
-      proxyToUse.toString().equals("HTTP @ somehost:1000"));
+    Assert.assertEquals("Proxy should have been set but wasn't ",
+      "HTTP @ somehost:1000", proxyToUse.toString());
     conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_PROXY, "socks@somehost:1000");
     setConf(conf);
-    Assert.assertTrue("Proxy should have been socks but wasn't ",
-      proxyToUse.toString().equals("SOCKS @ somehost:1000"));
+    Assert.assertEquals("Proxy should have been socks but wasn't ",
+      "SOCKS @ somehost:1000", proxyToUse.toString());
     conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_PROXY, "SOCKS@somehost:1000");
     setConf(conf);
-    Assert.assertTrue("Proxy should have been socks but wasn't ",
-      proxyToUse.toString().equals("SOCKS @ somehost:1000"));
+    Assert.assertEquals("Proxy should have been socks but wasn't ",
+      "SOCKS @ somehost:1000", proxyToUse.toString());
     conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_PROXY, "sfafn@somehost:1000");
     setConf(conf);
-    Assert.assertTrue("Proxy should have been http but wasn't ",
-      proxyToUse.toString().equals("HTTP @ somehost:1000"));
+    Assert.assertEquals("Proxy should have been http but wasn't ",
+      "HTTP @ somehost:1000", proxyToUse.toString());
     
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8e82418/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
index bc05c62..6c51626 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
@@ -329,8 +329,7 @@ public class TestRMContainerAllocator {
     for(TaskAttemptContainerAssignedEvent event : assigned) {
       if(event.getTaskAttemptID().equals(event3.getAttemptID())) {
         assigned.remove(event);
-        Assert.assertTrue(
-                    event.getContainer().getNodeId().getHost().equals("h3"));
+        Assert.assertEquals("h3", event.getContainer().getNodeId().getHost());
         break;
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8e82418/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestQueue.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestQueue.java
index fb2d5e0..796bbee 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestQueue.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestQueue.java
@@ -74,13 +74,13 @@ public class TestQueue {
       assertTrue(root.getChildren().size() == 2);
       Iterator<Queue> iterator = root.getChildren().iterator();
       Queue firstSubQueue = iterator.next();
-      assertTrue(firstSubQueue.getName().equals("first"));
+      assertEquals("first", firstSubQueue.getName());
       assertEquals(
           firstSubQueue.getAcls().get("mapred.queue.first.acl-submit-job")
               .toString(),
           "Users [user1, user2] and members of the groups [group1, group2] are allowed");
       Queue secondSubQueue = iterator.next();
-      assertTrue(secondSubQueue.getName().equals("second"));
+      assertEquals("second", secondSubQueue.getName());
       assertEquals(secondSubQueue.getProperties().getProperty("key"), "value");
       assertEquals(secondSubQueue.getProperties().getProperty("key1"), "value1");
       // test status
@@ -207,13 +207,13 @@ public class TestQueue {
     assertTrue(root.getChildren().size() == 2);
     Iterator<Queue> iterator = root.getChildren().iterator();
     Queue firstSubQueue = iterator.next();
-    assertTrue(firstSubQueue.getName().equals("first"));
+    assertEquals("first", firstSubQueue.getName());
     assertEquals(
         firstSubQueue.getAcls().get("mapred.queue.first.acl-submit-job")
             .toString(),
         "Users [user1, user2] and members of the groups [group1, group2] are allowed");
     Queue secondSubQueue = iterator.next();
-    assertTrue(secondSubQueue.getName().equals("second"));
+    assertEquals("second", secondSubQueue.getName());
 
     assertEquals(firstSubQueue.getState().getStateName(), "running");
     assertEquals(secondSubQueue.getState().getStateName(), "stopped");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8e82418/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java
index bd3e524..55ddea6 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java
@@ -871,10 +871,10 @@ public class TestYARNRunner {
     Configuration confSent = BuilderUtils.parseTokensConf(submissionContext);
 
     // configs that match regex should be included
-    Assert.assertTrue(confSent.get("dfs.namenode.rpc-address.mycluster2.nn1")
-        .equals("123.0.0.1"));
-    Assert.assertTrue(confSent.get("dfs.namenode.rpc-address.mycluster2.nn2")
-        .equals("123.0.0.2"));
+    Assert.assertEquals("123.0.0.1",
+        confSent.get("dfs.namenode.rpc-address.mycluster2.nn1"));
+    Assert.assertEquals("123.0.0.2",
+        confSent.get("dfs.namenode.rpc-address.mycluster2.nn2"));
 
     // configs that aren't matching regex should not be included
     Assert.assertTrue(confSent.get("hadoop.tmp.dir") == null || !confSent

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8e82418/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
index 632c40e..a6f8a72 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
@@ -134,11 +134,11 @@ public class TestMultipleInputs extends HadoopTestCase {
     BufferedReader output = new BufferedReader(new InputStreamReader(fs
         .open(new Path(outDir, "part-r-00000"))));
     // reducer should have counted one key from each file
-    assertTrue(output.readLine().equals("a 2"));
-    assertTrue(output.readLine().equals("b 2"));
-    assertTrue(output.readLine().equals("c 2"));
-    assertTrue(output.readLine().equals("d 2"));
-    assertTrue(output.readLine().equals("e 2"));
+    assertEquals("a 2", output.readLine());
+    assertEquals("b 2", output.readLine());
+    assertEquals("c 2", output.readLine());
+    assertEquals("d 2", output.readLine());
+    assertEquals("e 2", output.readLine());
   }
 
   @Test


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[34/51] [abbrv] hadoop git commit: YARN-6757. Refactor the usage of yarn.nodemanager.linux-container-executor.cgroups.mount-path (Contributed by Miklos Szegedi via Daniel Templeton)

Posted by st...@apache.org.
YARN-6757. Refactor the usage of yarn.nodemanager.linux-container-executor.cgroups.mount-path
(Contributed by Miklos Szegedi via Daniel Templeton)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/47b145b9
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/47b145b9
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/47b145b9

Branch: refs/heads/HADOOP-13345
Commit: 47b145b9b4e81d781891abce8a6638f0b436acc4
Parents: 9891295
Author: Daniel Templeton <te...@apache.org>
Authored: Tue Aug 8 10:33:26 2017 -0700
Committer: Daniel Templeton <te...@apache.org>
Committed: Tue Aug 8 10:33:26 2017 -0700

----------------------------------------------------------------------
 .../src/main/resources/yarn-default.xml         | 43 ++++++++++-----
 .../linux/resources/CGroupsHandler.java         | 15 +++++
 .../linux/resources/CGroupsHandlerImpl.java     | 26 +++++----
 .../linux/resources/ResourceHandlerModule.java  | 58 ++++++++++++++++++--
 .../util/CgroupsLCEResourcesHandler.java        | 53 ++++++++++++------
 .../linux/resources/TestCGroupsHandlerImpl.java | 27 ++++++++-
 .../util/TestCgroupsLCEResourcesHandler.java    | 31 +++++++++++
 .../src/site/markdown/GracefulDecommission.md   | 12 ++--
 .../src/site/markdown/NodeManagerCgroups.md     | 17 +++++-
 .../site/markdown/WritingYarnApplications.md    |  4 +-
 .../src/site/markdown/registry/yarn-registry.md | 14 ++---
 11 files changed, 237 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/47b145b9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index 95b8a88..000e892 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -134,7 +134,7 @@
 
   <property>
       <description>
-        This configures the HTTP endpoint for Yarn Daemons.The following
+        This configures the HTTP endpoint for YARN Daemons.The following
         values are supported:
         - HTTP_ONLY : Service is provided only on http
         - HTTPS_ONLY : Service is provided only on https
@@ -1063,14 +1063,14 @@
       DeletionService will delete the application's localized file directory
       and log directory.
       
-      To diagnose Yarn application problems, set this property's value large
+      To diagnose YARN application problems, set this property's value large
       enough (for example, to 600 = 10 minutes) to permit examination of these
       directories. After changing the property's value, you must restart the 
       nodemanager in order for it to have an effect.
 
-      The roots of Yarn applications' work directories is configurable with
+      The roots of YARN applications' work directories is configurable with
       the yarn.nodemanager.local-dirs property (see below), and the roots
-      of the Yarn applications' log directories is configurable with the 
+      of the YARN applications' log directories is configurable with the
       yarn.nodemanager.log-dirs property (see also below).
     </description>
     <name>yarn.nodemanager.delete.debug-delay-sec</name>
@@ -1510,28 +1510,45 @@
   <property>
     <description>The cgroups hierarchy under which to place YARN proccesses (cannot contain commas).
     If yarn.nodemanager.linux-container-executor.cgroups.mount is false
-    (that is, if cgroups have been pre-configured) and the Yarn user has write
+    (that is, if cgroups have been pre-configured) and the YARN user has write
     access to the parent directory, then the directory will be created.
-    If the directory already exists, the administrator has to give Yarn
+    If the directory already exists, the administrator has to give YARN
     write permissions to it recursively.
-    Only used when the LCE resources handler is set to the CgroupsLCEResourcesHandler.</description>
+    This property only applies when the LCE resources handler is set to
+    CgroupsLCEResourcesHandler.</description>
     <name>yarn.nodemanager.linux-container-executor.cgroups.hierarchy</name>
     <value>/hadoop-yarn</value>
   </property>
 
   <property>
     <description>Whether the LCE should attempt to mount cgroups if not found.
-    Only used when the LCE resources handler is set to the CgroupsLCEResourcesHandler.</description>
+    This property only applies when the LCE resources handler is set to
+    CgroupsLCEResourcesHandler.
+    </description>
     <name>yarn.nodemanager.linux-container-executor.cgroups.mount</name>
     <value>false</value>
   </property>
 
   <property>
-    <description>Where the LCE should attempt to mount cgroups if not found. Common locations
-    include /sys/fs/cgroup and /cgroup; the default location can vary depending on the Linux
-    distribution in use. This path must exist before the NodeManager is launched.
-    Only used when the LCE resources handler is set to the CgroupsLCEResourcesHandler, and
-    yarn.nodemanager.linux-container-executor.cgroups.mount is true.</description>
+    <description>This property sets the path from which YARN will read the
+    CGroups configuration. YARN has built-in functionality to discover the
+    system CGroup mount paths, so use this property only if YARN's automatic
+    mount path discovery does not work.
+
+    The path specified by this property must exist before the NodeManager is
+    launched.
+    If yarn.nodemanager.linux-container-executor.cgroups.mount is set to true,
+    YARN will first try to mount the CGroups at the specified path before
+    reading them.
+    If yarn.nodemanager.linux-container-executor.cgroups.mount is set to
+    false, YARN will read the CGroups at the specified path.
+    If this property is empty, YARN tries to detect the CGroups location.
+
+    Please refer to NodeManagerCgroups.html in the documentation for further
+    details.
+    This property only applies when the LCE resources handler is set to
+    CgroupsLCEResourcesHandler.
+    </description>
     <name>yarn.nodemanager.linux-container-executor.cgroups.mount-path</name>
   </property>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47b145b9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandler.java
index 8fc35a8..82bd366 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandler.java
@@ -23,6 +23,9 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resourc
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
+import java.util.HashSet;
+import java.util.Set;
+
 /**
  * Provides CGroups functionality. Implementations are expected to be
  * thread-safe
@@ -54,6 +57,18 @@ public interface CGroupsHandler {
     String getName() {
       return name;
     }
+
+    /**
+     * Get the list of valid cgroup names.
+     * @return The set of cgroup name strings
+     */
+    public static Set<String> getValidCGroups() {
+      HashSet<String> validCgroups = new HashSet<>();
+      for (CGroupController controller : CGroupController.values()) {
+        validCgroups.add(controller.getName());
+      }
+      return validCgroups;
+    }
   }
 
   String CGROUP_FILE_TASKS = "tasks";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47b145b9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandlerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandlerImpl.java
index 85b01cd..9fd20eb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandlerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsHandlerImpl.java
@@ -83,7 +83,7 @@ class CGroupsHandlerImpl implements CGroupsHandler {
    * @param mtab mount file location
    * @throws ResourceHandlerException if initialization failed
    */
-  public CGroupsHandlerImpl(Configuration conf, PrivilegedOperationExecutor
+  CGroupsHandlerImpl(Configuration conf, PrivilegedOperationExecutor
       privilegedOperationExecutor, String mtab)
       throws ResourceHandlerException {
     this.cGroupPrefix = conf.get(YarnConfiguration.
@@ -115,7 +115,7 @@ class CGroupsHandlerImpl implements CGroupsHandler {
    *                                    PrivilegedContainerOperations
    * @throws ResourceHandlerException if initialization failed
    */
-  public CGroupsHandlerImpl(Configuration conf, PrivilegedOperationExecutor
+  CGroupsHandlerImpl(Configuration conf, PrivilegedOperationExecutor
       privilegedOperationExecutor) throws ResourceHandlerException {
     this(conf, privilegedOperationExecutor, MTAB_FILE);
   }
@@ -142,11 +142,18 @@ class CGroupsHandlerImpl implements CGroupsHandler {
     // the same hierarchy will be mounted at each mount point with the same
     // subsystem set.
 
-    Map<String, Set<String>> newMtab;
+    Map<String, Set<String>> newMtab = null;
     Map<CGroupController, String> cPaths;
     try {
-      // parse mtab
-      newMtab = parseMtab(mtabFile);
+      if (this.cGroupMountPath != null && !this.enableCGroupMount) {
+        newMtab = ResourceHandlerModule.
+            parseConfiguredCGroupPath(this.cGroupMountPath);
+      }
+
+      if (newMtab == null) {
+        // parse mtab
+        newMtab = parseMtab(mtabFile);
+      }
 
       // find cgroup controller paths
       cPaths = initializeControllerPathsFromMtab(newMtab);
@@ -203,10 +210,8 @@ class CGroupsHandlerImpl implements CGroupsHandler {
       throws IOException {
     Map<String, Set<String>> ret = new HashMap<>();
     BufferedReader in = null;
-    HashSet<String> validCgroups = new HashSet<>();
-    for (CGroupController controller : CGroupController.values()) {
-      validCgroups.add(controller.getName());
-    }
+    Set<String> validCgroups =
+        CGroupsHandler.CGroupController.getValidCGroups();
 
     try {
       FileInputStream fis = new FileInputStream(new File(mtab));
@@ -487,7 +492,8 @@ class CGroupsHandlerImpl implements CGroupsHandler {
       try (BufferedReader inl =
           new BufferedReader(new InputStreamReader(new FileInputStream(cgf
               + "/tasks"), "UTF-8"))) {
-        if ((str = inl.readLine()) != null) {
+        str = inl.readLine();
+        if (str != null) {
           LOG.debug("First line in cgroup tasks file: " + cgf + " " + str);
         }
       } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47b145b9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerModule.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerModule.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerModule.java
index 7fc04bd..4d137f0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerModule.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerModule.java
@@ -31,6 +31,13 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileg
 import org.apache.hadoop.yarn.server.nodemanager.util.CgroupsLCEResourcesHandler;
 import org.apache.hadoop.yarn.server.nodemanager.util.DefaultLCEResourcesHandler;
 
+import java.io.File;
+import java.io.IOException;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.Arrays;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -113,8 +120,8 @@ public class ResourceHandlerModule {
   }
 
   private static TrafficControlBandwidthHandlerImpl
-    getTrafficControlBandwidthHandler(Configuration conf)
-      throws ResourceHandlerException {
+      getTrafficControlBandwidthHandler(Configuration conf)
+        throws ResourceHandlerException {
     if (conf.getBoolean(YarnConfiguration.NM_NETWORK_RESOURCE_ENABLED,
         YarnConfiguration.DEFAULT_NM_NETWORK_RESOURCE_ENABLED)) {
       if (trafficControlBandwidthHandler == null) {
@@ -137,8 +144,8 @@ public class ResourceHandlerModule {
   }
 
   public static OutboundBandwidthResourceHandler
-    getOutboundBandwidthResourceHandler(Configuration conf)
-      throws ResourceHandlerException {
+      getOutboundBandwidthResourceHandler(Configuration conf)
+        throws ResourceHandlerException {
     return getTrafficControlBandwidthHandler(conf);
   }
 
@@ -176,7 +183,7 @@ public class ResourceHandlerModule {
   }
 
   private static CGroupsMemoryResourceHandlerImpl
-    getCgroupsMemoryResourceHandler(
+      getCgroupsMemoryResourceHandler(
       Configuration conf) throws ResourceHandlerException {
     if (cGroupsMemoryResourceHandler == null) {
       synchronized (MemoryResourceHandler.class) {
@@ -229,4 +236,45 @@ public class ResourceHandlerModule {
   static void nullifyResourceHandlerChain() throws ResourceHandlerException {
     resourceHandlerChain = null;
   }
+
+  /**
+   * If a cgroup mount directory is specified, it returns cgroup directories
+   * with valid names.
+   * The requirement is that each hierarchy has to be named with the comma
+   * separated names of subsystems supported.
+   * For example: /sys/fs/cgroup/cpu,cpuacct
+   * @param cgroupMountPath Root cgroup mount path (/sys/fs/cgroup in the
+   *                        example above)
+   * @return A path to cgroup subsystem set mapping in the same format as
+   *         {@link CGroupsHandlerImpl#parseMtab(String)}
+   * @throws IOException if the specified directory cannot be listed
+   */
+  public static Map<String, Set<String>> parseConfiguredCGroupPath(
+      String cgroupMountPath) throws IOException {
+    File cgroupDir = new File(cgroupMountPath);
+    File[] list = cgroupDir.listFiles();
+    if (list == null) {
+      throw new IOException("Empty cgroup mount directory specified: " +
+          cgroupMountPath);
+    }
+
+    Map<String, Set<String>> pathSubsystemMappings = new HashMap<>();
+    Set<String> validCGroups =
+        CGroupsHandler.CGroupController.getValidCGroups();
+    for (File candidate: list) {
+      Set<String> cgroupList =
+          new HashSet<>(Arrays.asList(candidate.getName().split(",")));
+      // Collect the valid subsystem names
+      cgroupList.retainAll(validCGroups);
+      if (!cgroupList.isEmpty()) {
+        if (candidate.isDirectory() && candidate.canWrite()) {
+          pathSubsystemMappings.put(candidate.getAbsolutePath(), cgroupList);
+        } else {
+          LOG.warn("The following cgroup is not a directory or it is not"
+              + " writable" + candidate.getAbsolutePath());
+        }
+      }
+    }
+    return pathSubsystemMappings;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47b145b9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/CgroupsLCEResourcesHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/CgroupsLCEResourcesHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/CgroupsLCEResourcesHandler.java
index bca4fdc..7a89285 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/CgroupsLCEResourcesHandler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/CgroupsLCEResourcesHandler.java
@@ -27,6 +27,7 @@ import java.io.InputStreamReader;
 import java.io.OutputStreamWriter;
 import java.io.PrintWriter;
 import java.io.Writer;
+import java.util.Arrays;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -39,7 +40,6 @@ import java.util.regex.Pattern;
 
 import com.google.common.annotations.VisibleForTesting;
 
-import com.google.common.collect.Sets;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -51,6 +51,8 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.nodemanager.LinuxContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.CGroupsCpuResourceHandlerImpl;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.CGroupsHandler;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.ResourceHandlerModule;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
 import org.apache.hadoop.yarn.util.SystemClock;
@@ -87,11 +89,11 @@ public class CgroupsLCEResourcesHandler implements LCEResourcesHandler {
 
   private long deleteCgroupTimeout;
   private long deleteCgroupDelay;
-  // package private for testing purposes
+  @VisibleForTesting
   Clock clock;
 
   private float yarnProcessors;
-  int nodeVCores;
+  private int nodeVCores;
 
   public CgroupsLCEResourcesHandler() {
     this.controllerPaths = new HashMap<String, String>();
@@ -132,8 +134,10 @@ public class CgroupsLCEResourcesHandler implements LCEResourcesHandler {
     this.strictResourceUsageMode =
         conf
           .getBoolean(
-            YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_STRICT_RESOURCE_USAGE,
-            YarnConfiguration.DEFAULT_NM_LINUX_CONTAINER_CGROUPS_STRICT_RESOURCE_USAGE);
+            YarnConfiguration
+                .NM_LINUX_CONTAINER_CGROUPS_STRICT_RESOURCE_USAGE,
+            YarnConfiguration
+                .DEFAULT_NM_LINUX_CONTAINER_CGROUPS_STRICT_RESOURCE_USAGE);
 
     int len = cgroupPrefix.length();
     if (cgroupPrefix.charAt(len - 1) == '/') {
@@ -169,8 +173,10 @@ public class CgroupsLCEResourcesHandler implements LCEResourcesHandler {
     if (systemProcessors != (int) yarnProcessors) {
       LOG.info("YARN containers restricted to " + yarnProcessors + " cores");
       int[] limits = getOverallLimits(yarnProcessors);
-      updateCgroup(CONTROLLER_CPU, "", CPU_PERIOD_US, String.valueOf(limits[0]));
-      updateCgroup(CONTROLLER_CPU, "", CPU_QUOTA_US, String.valueOf(limits[1]));
+      updateCgroup(CONTROLLER_CPU, "", CPU_PERIOD_US,
+          String.valueOf(limits[0]));
+      updateCgroup(CONTROLLER_CPU, "", CPU_QUOTA_US,
+          String.valueOf(limits[1]));
     } else if (CGroupsCpuResourceHandlerImpl.cpuLimitsExist(
         pathForCgroup(CONTROLLER_CPU, ""))) {
       LOG.info("Removing CPU constraints for YARN containers.");
@@ -178,8 +184,8 @@ public class CgroupsLCEResourcesHandler implements LCEResourcesHandler {
     }
   }
 
-  int[] getOverallLimits(float yarnProcessors) {
-    return CGroupsCpuResourceHandlerImpl.getOverallLimits(yarnProcessors);
+  int[] getOverallLimits(float yarnProcessorsArg) {
+    return CGroupsCpuResourceHandlerImpl.getOverallLimits(yarnProcessorsArg);
   }
 
 
@@ -204,7 +210,7 @@ public class CgroupsLCEResourcesHandler implements LCEResourcesHandler {
       LOG.debug("createCgroup: " + path);
     }
 
-    if (! new File(path).mkdir()) {
+    if (!new File(path).mkdir()) {
       throw new IOException("Failed to create cgroup at " + path);
     }
   }
@@ -251,7 +257,8 @@ public class CgroupsLCEResourcesHandler implements LCEResourcesHandler {
       try (BufferedReader inl =
             new BufferedReader(new InputStreamReader(new FileInputStream(cgf
               + "/tasks"), "UTF-8"))) {
-        if ((str = inl.readLine()) != null) {
+        str = inl.readLine();
+        if (str != null) {
           LOG.debug("First line in cgroup tasks file: " + cgf + " " + str);
         }
       } catch (IOException e) {
@@ -337,9 +344,9 @@ public class CgroupsLCEResourcesHandler implements LCEResourcesHandler {
               (containerVCores * yarnProcessors) / (float) nodeVCores;
           int[] limits = getOverallLimits(containerCPU);
           updateCgroup(CONTROLLER_CPU, containerName, CPU_PERIOD_US,
-            String.valueOf(limits[0]));
+              String.valueOf(limits[0]));
           updateCgroup(CONTROLLER_CPU, containerName, CPU_QUOTA_US,
-            String.valueOf(limits[1]));
+              String.valueOf(limits[1]));
         }
       }
     }
@@ -400,6 +407,8 @@ public class CgroupsLCEResourcesHandler implements LCEResourcesHandler {
   private Map<String, Set<String>> parseMtab() throws IOException {
     Map<String, Set<String>> ret = new HashMap<String, Set<String>>();
     BufferedReader in = null;
+    Set<String> validCgroups =
+        CGroupsHandler.CGroupController.getValidCGroups();
 
     try {
       FileInputStream fis = new FileInputStream(new File(getMtabFileName()));
@@ -415,8 +424,11 @@ public class CgroupsLCEResourcesHandler implements LCEResourcesHandler {
           String options = m.group(3);
 
           if (type.equals(CGROUPS_FSTYPE)) {
-            HashSet<String> value = Sets.newHashSet(options.split(","));
-            ret.put(path, value);
+            Set<String> cgroupList =
+                new HashSet<>(Arrays.asList(options.split(",")));
+            // Collect the valid subsystem names
+            cgroupList.retainAll(validCgroups);
+            ret.put(path, cgroupList);
           }
         }
       }
@@ -448,7 +460,16 @@ public class CgroupsLCEResourcesHandler implements LCEResourcesHandler {
 
   private void initializeControllerPaths() throws IOException {
     String controllerPath;
-    Map<String, Set<String>> parsedMtab = parseMtab();
+    Map<String, Set<String>> parsedMtab = null;
+
+    if (this.cgroupMountPath != null && !this.cgroupMount) {
+      parsedMtab = ResourceHandlerModule.
+          parseConfiguredCGroupPath(this.cgroupMountPath);
+    }
+
+    if (parsedMtab == null) {
+      parsedMtab = parseMtab();
+    }
 
     // CPU
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47b145b9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsHandlerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsHandlerImpl.java
index 7a4d39f..ab989cf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsHandlerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsHandlerImpl.java
@@ -573,4 +573,29 @@ public class TestCGroupsHandlerImpl {
         new File(new File(newMountPoint, "cpu"), this.hierarchy);
     assertTrue("Yarn cgroup should exist", hierarchyFile.exists());
   }
-}
+
+
+  @Test
+  public void testManualCgroupSetting() throws ResourceHandlerException {
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.set(YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_MOUNT_PATH, tmpPath);
+    conf.set(YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_HIERARCHY,
+        "/hadoop-yarn");
+    File cpu = new File(new File(tmpPath, "cpuacct,cpu"), "/hadoop-yarn");
+
+    try {
+      Assert.assertTrue("temp dir should be created", cpu.mkdirs());
+
+      CGroupsHandlerImpl cGroupsHandler = new CGroupsHandlerImpl(conf, null);
+      cGroupsHandler.initializeCGroupController(
+              CGroupsHandler.CGroupController.CPU);
+
+      Assert.assertEquals("CPU CGRoup path was not set", cpu.getAbsolutePath(),
+              new File(cGroupsHandler.getPathForCGroup(
+                  CGroupsHandler.CGroupController.CPU, "")).getAbsolutePath());
+
+    } finally {
+      FileUtils.deleteQuietly(cpu);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47b145b9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/util/TestCgroupsLCEResourcesHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/util/TestCgroupsLCEResourcesHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/util/TestCgroupsLCEResourcesHandler.java
index 1ed8fd8..7d8704f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/util/TestCgroupsLCEResourcesHandler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/util/TestCgroupsLCEResourcesHandler.java
@@ -41,6 +41,8 @@ import java.util.Scanner;
 import java.util.Set;
 import java.util.concurrent.CountDownLatch;
 
+import static org.mockito.Mockito.when;
+
 @Deprecated
 public class TestCgroupsLCEResourcesHandler {
   private static File cgroupDir = null;
@@ -388,4 +390,33 @@ public class TestCgroupsLCEResourcesHandler {
       FileUtils.deleteQuietly(memory);
     }
   }
+
+  @Test
+  public void testManualCgroupSetting() throws IOException {
+    CgroupsLCEResourcesHandler handler = new CgroupsLCEResourcesHandler();
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.set(YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_MOUNT_PATH,
+        cgroupDir.getAbsolutePath());
+    handler.setConf(conf);
+    File cpu = new File(new File(cgroupDir, "cpuacct,cpu"), "/hadoop-yarn");
+
+    try {
+      Assert.assertTrue("temp dir should be created", cpu.mkdirs());
+
+      final int numProcessors = 4;
+      ResourceCalculatorPlugin plugin =
+              Mockito.mock(ResourceCalculatorPlugin.class);
+      Mockito.doReturn(numProcessors).when(plugin).getNumProcessors();
+      Mockito.doReturn(numProcessors).when(plugin).getNumCores();
+      when(plugin.getNumProcessors()).thenReturn(8);
+      handler.init(null, plugin);
+
+      Assert.assertEquals("CPU CGRoup path was not set", cpu.getParent(),
+          handler.getControllerPaths().get("cpu"));
+
+    } finally {
+      FileUtils.deleteQuietly(cpu);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47b145b9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/GracefulDecommission.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/GracefulDecommission.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/GracefulDecommission.md
index 2acb3d2..2e83ca2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/GracefulDecommission.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/GracefulDecommission.md
@@ -13,7 +13,7 @@
 -->
 
 
-Graceful Decommission of Yarn Nodes
+Graceful Decommission of YARN Nodes
 ===============
 
 * [Overview](#overview)
@@ -29,19 +29,19 @@ Graceful Decommission of Yarn Nodes
 Overview
 --------
 
-Yarn is scalable very easily: any new NodeManager could join to the configured ResourceManager and start to execute jobs. But to achieve full elasticity we need a decommissioning process which helps to remove existing nodes and down-scale the cluster.
+YARN is scalable very easily: any new NodeManager could join to the configured ResourceManager and start to execute jobs. But to achieve full elasticity we need a decommissioning process which helps to remove existing nodes and down-scale the cluster.
 
-Yarn Nodes could be decommissioned NORMAL or GRACEFUL.
+YARN Nodes could be decommissioned NORMAL or GRACEFUL.
 
-Normal Decommission of Yarn Nodes means an immediate shutdown.
+Normal Decommission of YARN Nodes means an immediate shutdown.
 
-Graceful Decommission of Yarn Nodes is the mechanism to decommission NMs while minimize the impact to running applications. Once a node is in DECOMMISSIONING state, RM won't schedule new containers on it and will wait for running containers and applications to complete (or until decommissioning timeout exceeded) before transition the node into DECOMMISSIONED.
+Graceful Decommission of YARN Nodes is the mechanism to decommission NMs while minimize the impact to running applications. Once a node is in DECOMMISSIONING state, RM won't schedule new containers on it and will wait for running containers and applications to complete (or until decommissioning timeout exceeded) before transition the node into DECOMMISSIONED.
 
 ## Quick start
 
 To do a normal decommissioning:
 
-1. Start a Yarn cluster (with NodeManageres and ResourceManager)
+1. Start a YARN cluster (with NodeManageres and ResourceManager)
 2. Start a yarn job (for example with `yarn jar...` )
 3. Add `yarn.resourcemanager.nodes.exclude-path` property to your `yarn-site.xml` (Note: you don't need to restart the ResourceManager)
 4. Create a text file (the location is defined in the previous step) with one line which contains the name of a selected NodeManager 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47b145b9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManagerCgroups.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManagerCgroups.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManagerCgroups.md
index 2704f10..d362801 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManagerCgroups.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManagerCgroups.md
@@ -17,7 +17,7 @@ Using CGroups with YARN
 
 <!-- MACRO{toc|fromDepth=0|toDepth=3} -->
 
-CGroups is a mechanism for aggregating/partitioning sets of tasks, and all their future children, into hierarchical groups with specialized behaviour. CGroups is a Linux kernel feature and was merged into kernel version 2.6.24. From a YARN perspective, this allows containers to be limited in their resource usage. A good example of this is CPU usage. Without CGroups, it becomes hard to limit container CPU usage. Currently, CGroups is only used for limiting CPU usage.
+CGroups is a mechanism for aggregating/partitioning sets of tasks, and all their future children, into hierarchical groups with specialized behaviour. CGroups is a Linux kernel feature and was merged into kernel version 2.6.24. From a YARN perspective, this allows containers to be limited in their resource usage. A good example of this is CPU usage. Without CGroups, it becomes hard to limit container CPU usage.
 
 CGroups Configuration
 ---------------------
@@ -30,9 +30,9 @@ The following settings are related to setting up CGroups. These need to be set i
 |:---- |:---- |
 | `yarn.nodemanager.container-executor.class` | This should be set to "org.apache.hadoop.yarn.server.nodemanager.LinuxContainerExecutor". CGroups is a Linux kernel feature and is exposed via the LinuxContainerExecutor. |
 | `yarn.nodemanager.linux-container-executor.resources-handler.class` | This should be set to "org.apache.hadoop.yarn.server.nodemanager.util.CgroupsLCEResourcesHandler". Using the LinuxContainerExecutor doesn't force you to use CGroups. If you wish to use CGroups, the resource-handler-class must be set to CGroupsLCEResourceHandler. |
-| `yarn.nodemanager.linux-container-executor.cgroups.hierarchy` | The cgroups hierarchy under which to place YARN proccesses(cannot contain commas). If yarn.nodemanager.linux-container-executor.cgroups.mount is false (that is, if cgroups have been pre-configured) and the Yarn user has write access to the parent directory, then the directory will be created. If the directory already exists, the administrator has to give Yarn write permissions to it recursively. |
+| `yarn.nodemanager.linux-container-executor.cgroups.hierarchy` | The cgroups hierarchy under which to place YARN proccesses(cannot contain commas). If yarn.nodemanager.linux-container-executor.cgroups.mount is false (that is, if cgroups have been pre-configured) and the YARN user has write access to the parent directory, then the directory will be created. If the directory already exists, the administrator has to give YARN write permissions to it recursively. |
 | `yarn.nodemanager.linux-container-executor.cgroups.mount` | Whether the LCE should attempt to mount cgroups if not found - can be true or false. |
-| `yarn.nodemanager.linux-container-executor.cgroups.mount-path` | Where the LCE should attempt to mount cgroups if not found. Common locations include /sys/fs/cgroup and /cgroup; the default location can vary depending on the Linux distribution in use. This path must exist before the NodeManager is launched. Only used when the LCE resources handler is set to the CgroupsLCEResourcesHandler, and yarn.nodemanager.linux-container-executor.cgroups.mount is true. A point to note here is that the container-executor binary will try to mount the path specified + "/" + the subsystem. In our case, since we are trying to limit CPU the binary tries to mount the path specified + "/cpu" and that's the path it expects to exist. |
+| `yarn.nodemanager.linux-container-executor.cgroups.mount-path` | Optional. Where CGroups are located. LCE will try to mount them here, if `yarn.nodemanager.linux-container-executor.cgroups.mount` is true. LCE will try to use CGroups from this location, if `yarn.nodemanager.linux-container-executor.cgroups.mount` is false. If specified, this path and its subdirectories (CGroup hierarchies) must exist and they should be readable and writable by YARN before the NodeManager is launched. See CGroups mount options below for details. |
 | `yarn.nodemanager.linux-container-executor.group` | The Unix group of the NodeManager. It should match the setting in "container-executor.cfg". This configuration is required for validating the secure access of the container-executor binary. |
 
 The following settings are related to limiting resource usage of YARN containers:
@@ -42,6 +42,17 @@ The following settings are related to limiting resource usage of YARN containers
 | `yarn.nodemanager.resource.percentage-physical-cpu-limit` | This setting lets you limit the cpu usage of all YARN containers. It sets a hard upper limit on the cumulative CPU usage of the containers. For example, if set to 60, the combined CPU usage of all YARN containers will not exceed 60%. |
 | `yarn.nodemanager.linux-container-executor.cgroups.strict-resource-usage` | CGroups allows cpu usage limits to be hard or soft. When this setting is true, containers cannot use more CPU usage than allocated even if spare CPU is available. This ensures that containers can only use CPU that they were allocated. When set to false, containers can use spare CPU if available. It should be noted that irrespective of whether set to true or false, at no time can the combined CPU usage of all containers exceed the value specified in "yarn.nodemanager.resource.percentage-physical-cpu-limit". |
 
+CGroups mount options
+---------------------
+
+YARN uses CGroups through a directory structure mounted into the file system by the kernel. There are three options to attach to CGroups.
+
+| Option | Description |
+|:---- |:---- |
+| Discover CGroups mounted already | This should be used on newer systems like RHEL7 or Ubuntu16 or if the administrator mounts CGroups before YARN starts. Set `yarn.nodemanager.linux-container-executor.cgroups.mount` to false and leave other settings set to their defaults. YARN will locate the mount points in `/proc/mounts`. Common locations include `/sys/fs/cgroup` and `/cgroup`. The default location can vary depending on the Linux distribution in use.|
+| CGroups mounted by YARN | If the system does not have CGroups mounted or it is mounted to an inaccessible location then point `yarn.nodemanager.linux-container-executor.cgroups.mount-path` to an empty directory. Set `yarn.nodemanager.linux-container-executor.cgroups.mount` to true. A point to note here is that the container-executor binary will try to create and mount each subsystem as a subdirectory under this path. If `cpu` is already mounted somewhere with `cpuacct`, then the directory `cpu,cpuacct` will be created for the hierarchy.|
+| CGroups mounted already or linked but not in `/proc/mounts` | If cgroups is accessible through lxcfs or simulated by another filesystem, then point `yarn.nodemanager.linux-container-executor.cgroups.mount-path` to your CGroups root directory. Set `yarn.nodemanager.linux-container-executor.cgroups.mount` to false. YARN tries to use this path first, before any CGroup mount point discovery. The path should have a subdirectory for each CGroup hierarchy named by the comma separated CGroup subsystems supported like `<path>/cpu,cpuacct`. Valid subsystem names are `cpu, cpuacct, cpuset, memory, net_cls, blkio, freezer, devices`.|
+
 CGroups and security
 --------------------
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47b145b9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WritingYarnApplications.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WritingYarnApplications.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WritingYarnApplications.md
index 07c3765..f1308d5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WritingYarnApplications.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WritingYarnApplications.md
@@ -56,7 +56,7 @@ Following are the important interfaces:
 
 * Under very rare circumstances, programmer may want to directly use the 3 protocols to implement an application. However, note that *such behaviors are no longer encouraged for general use cases*.
 
-Writing a Simple Yarn Application
+Writing a Simple YARN Application
 ---------------------------------
 
 ### Writing a simple Client
@@ -574,4 +574,4 @@ Useful Links
 Sample Code
 -----------
 
-Yarn distributed shell: in `hadoop-yarn-applications-distributedshell` project after you set up your development environment.
+YARN distributed shell: in `hadoop-yarn-applications-distributedshell` project after you set up your development environment.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47b145b9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/registry/yarn-registry.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/registry/yarn-registry.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/registry/yarn-registry.md
index f5055d9..4973862 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/registry/yarn-registry.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/registry/yarn-registry.md
@@ -84,7 +84,7 @@ container ID.
 
 ## The binding problem
 Hadoop YARN allows applications to run on the Hadoop cluster. Some of these are
-batch jobs or queries that can managed via Yarn’s existing API using its
+batch jobs or queries that can managed via YARN’s existing API using its
 application ID. In addition YARN can deploy ong-lived services instances such a
 pool of Apache Tomcat web servers or an Apache HBase cluster. YARN will deploy
 them across the cluster depending on the individual each component requirements
@@ -121,7 +121,7 @@ services accessible from within the Hadoop cluster
         /services/yarn
         /services/oozie
 
-Yarn-deployed services belonging to individual users.
+YARN-deployed services belonging to individual users.
 
         /users/joe/org-apache-hbase/demo1
         /users/joe/org-apache-hbase/demo1/components/regionserver1
@@ -148,7 +148,7 @@ their application master, to which they heartbeat regularly.
 
 ## Unsupported Registration use cases:
 
-1. A short-lived Yarn application is registered automatically in the registry,
+1. A short-lived YARN application is registered automatically in the registry,
 including all its containers. and unregistered when the job terminates.
 Short-lived applications with many containers will place excessive load on a
 registry. All YARN applications will be given the option of registering, but it
@@ -259,7 +259,7 @@ service since it supports many of the properties, We pick a part of the ZK
 namespace to be the root of the service registry ( default: `yarnRegistry`).
 
 On top this base implementation we build our registry service API and the
-naming conventions that Yarn will use for its services. The registry will be
+naming conventions that YARN will use for its services. The registry will be
 accessed by the registry API, not directly via ZK - ZK is just an
 implementation choice (although unlikely to change in the future).
 
@@ -297,7 +297,7 @@ them.
 6. Core services will be registered using the following convention:
 `/services/{servicename}` e.g. `/services/hdfs`.
 
-7. Yarn services SHOULD be registered using the following convention:
+7. YARN services SHOULD be registered using the following convention:
 
         /users/{username}/{serviceclass}/{instancename}
 
@@ -823,8 +823,8 @@ The `RegistryPathStatus` class summarizes the contents of a node in the registry
 ## Security
 
 The registry will allow a service instance can only be registered under the
-path where it has permissions. Yarn will create directories with appropriate
-permissions for users where Yarn deployed services can be registered by a user.
+path where it has permissions. YARN will create directories with appropriate
+permissions for users where YARN deployed services can be registered by a user.
 of the user account of the service instance. The admin will also create
 directories (such as `/services`) with appropriate permissions (where core Hadoop
 services can register themselves.


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[08/51] [abbrv] hadoop git commit: YARN-6832. Tests use assertTrue(....equals(...)) instead of assertEquals()). (Daniel Templeton via Yufei Gu)

Posted by st...@apache.org.
YARN-6832. Tests use assertTrue(....equals(...)) instead of assertEquals()). (Daniel Templeton via Yufei Gu)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c617fe02
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c617fe02
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c617fe02

Branch: refs/heads/HADOOP-13345
Commit: c617fe02b3686f0d91c3f81acf9bccd8290c59b7
Parents: 293c74a
Author: Yufei Gu <yu...@apache.org>
Authored: Thu Aug 3 11:33:37 2017 -0700
Committer: Yufei Gu <yu...@apache.org>
Committed: Thu Aug 3 11:33:37 2017 -0700

----------------------------------------------------------------------
 .../yarn/client/api/impl/TestYarnClient.java    |  9 +++---
 .../yarn/util/TestProcfsBasedProcessTree.java   |  2 +-
 .../hadoop/yarn/util/TestYarnVersionInfo.java   | 11 ++++---
 .../containermanager/TestAuxServices.java       |  2 +-
 .../linux/resources/TestCGroupsHandlerImpl.java |  2 +-
 .../resourcemanager/TestClientRMService.java    |  4 +--
 .../TestReservationInputValidator.java          | 10 +++----
 .../scheduler/TestSchedulerUtils.java           |  4 +--
 .../capacity/TestCapacityScheduler.java         | 14 ++++-----
 .../TestCapacitySchedulerDynamicBehavior.java   |  2 +-
 .../hadoop/yarn/webapp/TestRMWithXFSFilter.java | 14 ++++-----
 ...stTimelineReaderWebServicesHBaseStorage.java | 30 ++++++++++----------
 12 files changed, 52 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c617fe02/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
index 7d7272a..41ef404 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
@@ -489,9 +489,8 @@ public class TestYarnClient {
     }
 
     reports = client.getApplications(appTypes, appStates);
-    Assert.assertEquals(reports.size(), 1);
-    Assert
-    .assertTrue((reports.get(0).getApplicationType().equals("NON-YARN")));
+    Assert.assertEquals(1, reports.size());
+    Assert.assertEquals("NON-YARN", reports.get(0).getApplicationType());
     for (ApplicationReport report : reports) {
       Assert.assertTrue(expectedReports.contains(report));
     }
@@ -1501,8 +1500,8 @@ public class TestYarnClient {
       ReservationRequests reservationRequests =
           response.getReservationAllocationState().get(0)
               .getReservationDefinition().getReservationRequests();
-      Assert.assertTrue(
-          reservationRequests.getInterpreter().toString().equals("R_ALL"));
+      Assert.assertEquals("R_ALL",
+          reservationRequests.getInterpreter().toString());
       Assert.assertTrue(reservationRequests.getReservationResources().get(0)
           .getDuration() == duration);
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c617fe02/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestProcfsBasedProcessTree.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestProcfsBasedProcessTree.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestProcfsBasedProcessTree.java
index aad513a..43a5182 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestProcfsBasedProcessTree.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestProcfsBasedProcessTree.java
@@ -236,7 +236,7 @@ public class TestProcfsBasedProcessTree {
       "vmem for the gone-process is " + p.getVirtualMemorySize()
           + " . It should be UNAVAILABLE(-1).",
           p.getVirtualMemorySize() == UNAVAILABLE);
-    Assert.assertTrue(p.toString().equals("[ ]"));
+    Assert.assertEquals("[ ]", p.toString());
   }
 
   protected ProcfsBasedProcessTree createProcessTree(String pid) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c617fe02/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestYarnVersionInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestYarnVersionInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestYarnVersionInfo.java
index 47ee822..7e41501 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestYarnVersionInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestYarnVersionInfo.java
@@ -20,9 +20,9 @@ package org.apache.hadoop.yarn.util;
 
 import java.io.IOException;
 
-import org.apache.hadoop.yarn.util.YarnVersionInfo;
 import org.junit.Test;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotNull;
 /**
  * A JUnit test to test {@link YarnVersionInfo}
@@ -38,9 +38,12 @@ public class TestYarnVersionInfo {
 
     // can't easily know what the correct values are going to be so just
     // make sure they aren't Unknown
-    assertTrue("getVersion returned Unknown", !YarnVersionInfo.getVersion().equals("Unknown"));
-    assertTrue("getUser returned Unknown", !YarnVersionInfo.getUser().equals("Unknown"));
-    assertTrue("getSrcChecksum returned Unknown", !YarnVersionInfo.getSrcChecksum().equals("Unknown"));
+    assertNotEquals("getVersion returned Unknown",
+        "Unknown", YarnVersionInfo.getVersion());
+    assertNotEquals("getUser returned Unknown",
+        "Unknown", YarnVersionInfo.getUser());
+    assertNotEquals("getSrcChecksum returned Unknown",
+        "Unknown", YarnVersionInfo.getSrcChecksum());
 
     // these could be Unknown if the VersionInfo generated from code not in svn or git
     // so just check that they return something

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c617fe02/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestAuxServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestAuxServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestAuxServices.java
index 26a1003..f075713 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestAuxServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestAuxServices.java
@@ -210,7 +210,7 @@ public class TestAuxServices {
       defaultAuxClassPath = new HashSet<String>(Arrays.asList(StringUtils
           .getTrimmedStrings(auxClassPath)));
     }
-    Assert.assertTrue(auxName.equals("ServiceC"));
+    Assert.assertEquals("ServiceC", auxName);
     aux.serviceStop();
 
     // create a new jar file, and configure it as customized class path

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c617fe02/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsHandlerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsHandlerImpl.java
index dd8e338..7a4d39f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsHandlerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsHandlerImpl.java
@@ -573,4 +573,4 @@ public class TestCGroupsHandlerImpl {
         new File(new File(newMountPoint, "cpu"), this.hierarchy);
     assertTrue("Yarn cgroup should exist", hierarchyFile.exists());
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c617fe02/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
index 678daa3..9ce02bc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
@@ -1612,8 +1612,8 @@ public class TestClientRMService {
     ReservationRequests reservationRequests =
         response.getReservationAllocationState().get(0)
             .getReservationDefinition().getReservationRequests();
-    Assert.assertTrue(
-        reservationRequests.getInterpreter().toString().equals("R_ALL"));
+    Assert.assertEquals("R_ALL",
+        reservationRequests.getInterpreter().toString());
     Assert.assertTrue(reservationRequests.getReservationResources().get(0)
         .getDuration() == duration);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c617fe02/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationInputValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationInputValidator.java
index 76dbc36..2917cd9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationInputValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationInputValidator.java
@@ -119,9 +119,8 @@ public class TestReservationInputValidator {
     } catch (YarnException e) {
       Assert.assertNull(plan);
       String message = e.getMessage();
-      Assert
-          .assertTrue(message
-              .equals("The queue is not specified. Please try again with a valid reservable queue."));
+      Assert.assertEquals("The queue is not specified. Please try again with a "
+          + "valid reservable queue.", message);
       LOG.info(message);
     }
   }
@@ -161,9 +160,8 @@ public class TestReservationInputValidator {
     } catch (YarnException e) {
       Assert.assertNull(plan);
       String message = e.getMessage();
-      Assert
-          .assertTrue(message
-              .equals("Missing reservation definition. Please try again by specifying a reservation definition."));
+      Assert.assertEquals("Missing reservation definition. Please try again by "
+          + "specifying a reservation definition.", message);
       LOG.info(message);
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c617fe02/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java
index 854a4f3..cdc67ed 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java
@@ -760,12 +760,12 @@ public class TestSchedulerUtils {
           mock(Priority.class), ResourceRequest.ANY, resource, 1);
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
           scheduler, rmContext);
-      Assert.assertTrue(resReq.getNodeLabelExpression().equals("x"));
+      Assert.assertEquals("x", resReq.getNodeLabelExpression());
       
       resReq.setNodeLabelExpression(" y ");
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
           scheduler, rmContext);
-      Assert.assertTrue(resReq.getNodeLabelExpression().equals("y"));
+      Assert.assertEquals("y", resReq.getNodeLabelExpression());
     } catch (InvalidResourceRequestException e) {
       e.printStackTrace();
       fail("Should be valid when request labels is a subset of queue labels");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c617fe02/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
index 0642cd9..f51f771 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
@@ -1529,7 +1529,7 @@ public class TestCapacityScheduler {
     String queue =
         scheduler.getApplicationAttempt(appsInA1.get(0)).getQueue()
             .getQueueName();
-    Assert.assertTrue(queue.equals("a1"));
+    Assert.assertEquals("a1", queue);
 
     List<ApplicationAttemptId> appsInA = scheduler.getAppsInQueue("a");
     assertTrue(appsInA.contains(appAttemptId));
@@ -1554,7 +1554,7 @@ public class TestCapacityScheduler {
     queue =
         scheduler.getApplicationAttempt(appsInB1.get(0)).getQueue()
             .getQueueName();
-    Assert.assertTrue(queue.equals("b1"));
+    Assert.assertEquals("b1", queue);
 
     appsInB = scheduler.getAppsInQueue("b");
     assertTrue(appsInB.contains(appAttemptId));
@@ -1591,7 +1591,7 @@ public class TestCapacityScheduler {
     String queue =
         scheduler.getApplicationAttempt(appsInA1.get(0)).getQueue()
             .getQueueName();
-    Assert.assertTrue(queue.equals("a1"));
+    Assert.assertEquals("a1", queue);
 
     List<ApplicationAttemptId> appsInA = scheduler.getAppsInQueue("a");
     assertTrue(appsInA.contains(appAttemptId));
@@ -1613,7 +1613,7 @@ public class TestCapacityScheduler {
     queue =
         scheduler.getApplicationAttempt(appsInA2.get(0)).getQueue()
             .getQueueName();
-    Assert.assertTrue(queue.equals("a2"));
+    Assert.assertEquals("a2", queue);
 
     appsInA1 = scheduler.getAppsInQueue("a1");
     assertTrue(appsInA1.isEmpty());
@@ -2111,7 +2111,7 @@ public class TestCapacityScheduler {
     String queue =
         scheduler.getApplicationAttempt(appsInA1.get(0)).getQueue()
             .getQueueName();
-    Assert.assertTrue(queue.equals("a1"));
+    Assert.assertEquals("a1", queue);
 
     List<ApplicationAttemptId> appsInRoot = scheduler.getAppsInQueue("root");
     assertTrue(appsInRoot.contains(appAttemptId));
@@ -2133,7 +2133,7 @@ public class TestCapacityScheduler {
     queue =
         scheduler.getApplicationAttempt(appsInB1.get(0)).getQueue()
             .getQueueName();
-    Assert.assertTrue(queue.equals("b1"));
+    Assert.assertEquals("b1", queue);
 
     appsInB = scheduler.getAppsInQueue("b");
     assertTrue(appsInB.contains(appAttemptId));
@@ -2489,7 +2489,7 @@ public class TestCapacityScheduler {
     String queue =
         scheduler.getApplicationAttempt(appsInA1.get(0)).getQueue()
             .getQueueName();
-    Assert.assertTrue(queue.equals("a1"));
+    Assert.assertEquals("a1", queue);
 
     List<ApplicationAttemptId> appsInRoot = scheduler.getAppsInQueue("root");
     assertTrue(appsInRoot.contains(appAttemptId));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c617fe02/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerDynamicBehavior.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerDynamicBehavior.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerDynamicBehavior.java
index 483ba1b..9aba30c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerDynamicBehavior.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerDynamicBehavior.java
@@ -222,7 +222,7 @@ public class TestCapacitySchedulerDynamicBehavior {
     String queue =
         scheduler.getApplicationAttempt(appsInB1.get(0)).getQueue()
             .getQueueName();
-    Assert.assertTrue(queue.equals("b1"));
+    Assert.assertEquals("b1", queue);
 
     List<ApplicationAttemptId> appsInRoot = scheduler.getAppsInQueue("root");
     assertTrue(appsInRoot.contains(appAttemptId));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c617fe02/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithXFSFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithXFSFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithXFSFilter.java
index bcf2b21..2f7ecde 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithXFSFilter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/webapp/TestRMWithXFSFilter.java
@@ -39,8 +39,8 @@ import org.junit.Test;
 import java.util.HashMap;
 import java.util.Map;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
 
 /**
  * Used TestRMWebServices as an example of web invocations of RM and added
@@ -72,9 +72,9 @@ public class TestRMWithXFSFilter extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("info").accept("application/xml")
         .get(ClientResponse.class);
-    assertTrue("Should have received DENY x-frame options header",
-        response.getHeaders().get(XFrameOptionsFilter.X_FRAME_OPTIONS).get(0)
-            .equals("DENY"));
+    assertEquals("Should have received DENY x-frame options header",
+        "DENY",
+        response.getHeaders().get(XFrameOptionsFilter.X_FRAME_OPTIONS).get(0));
   }
 
   protected void createInjector(String headerValue) {
@@ -123,9 +123,9 @@ public class TestRMWithXFSFilter extends JerseyTestBase {
     ClientResponse response = r.path("ws").path("v1").path("cluster")
         .path("info").accept("application/xml")
         .get(ClientResponse.class);
-    assertTrue("Should have received SAMEORIGIN x-frame options header",
-        response.getHeaders().get(XFrameOptionsFilter.X_FRAME_OPTIONS).get(0)
-            .equals("SAMEORIGIN"));
+    assertEquals("Should have received SAMEORIGIN x-frame options header",
+        "SAMEORIGIN",
+        response.getHeaders().get(XFrameOptionsFilter.X_FRAME_OPTIONS).get(0));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c617fe02/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
index 63a75d3..3f8978c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
@@ -1325,7 +1325,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       int cfgCnt = 0;
       for (TimelineEntity entity : entities) {
         cfgCnt += entity.getConfigs().size();
-        assertTrue(entity.getId().equals("entity2"));
+        assertEquals("entity2", entity.getId());
       }
       assertEquals(0, cfgCnt);
 
@@ -1343,7 +1343,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       cfgCnt = 0;
       for (TimelineEntity entity : entities) {
         cfgCnt += entity.getConfigs().size();
-        assertTrue(entity.getId().equals("entity2"));
+        assertEquals("entity2", entity.getId());
       }
       assertEquals(3, cfgCnt);
 
@@ -1360,7 +1360,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       cfgCnt = 0;
       for (TimelineEntity entity : entities) {
         cfgCnt += entity.getConfigs().size();
-        assertTrue(entity.getId().equals("entity2"));
+        assertEquals("entity2", entity.getId());
         for (String configKey : entity.getConfigs().keySet()) {
           assertTrue(configKey.startsWith("cfg_") ||
               configKey.startsWith("configuration_"));
@@ -1393,7 +1393,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       assertNotNull(entities);
       assertEquals(1, entities.size());
       for (TimelineEntity entity : entities) {
-        assertTrue(entity.getId().equals("entity2"));
+        assertEquals("entity2", entity.getId());
       }
     } finally {
       client.destroy();
@@ -1457,7 +1457,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       int infoCnt = 0;
       for (TimelineEntity entity : entities) {
         infoCnt += entity.getInfo().size();
-        assertTrue(entity.getId().equals("entity2"));
+        assertEquals("entity2", entity.getId());
       }
       // Includes UID in info field even if fields not specified as INFO.
       assertEquals(1, infoCnt);
@@ -1476,7 +1476,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       infoCnt = 0;
       for (TimelineEntity entity : entities) {
         infoCnt += entity.getInfo().size();
-        assertTrue(entity.getId().equals("entity2"));
+        assertEquals("entity2", entity.getId());
       }
       // Includes UID in info field.
       assertEquals(4, infoCnt);
@@ -1506,7 +1506,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       assertNotNull(entities);
       assertEquals(1, entities.size());
       for (TimelineEntity entity : entities) {
-        assertTrue(entity.getId().equals("entity1"));
+        assertEquals("entity1", entity.getId());
       }
     } finally {
       client.destroy();
@@ -1556,7 +1556,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       int metricCnt = 0;
       for (TimelineEntity entity : entities) {
         metricCnt += entity.getMetrics().size();
-        assertTrue(entity.getId().equals("entity2"));
+        assertEquals("entity2", entity.getId());
       }
       assertEquals(0, metricCnt);
 
@@ -1574,7 +1574,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       metricCnt = 0;
       for (TimelineEntity entity : entities) {
         metricCnt += entity.getMetrics().size();
-        assertTrue(entity.getId().equals("entity2"));
+        assertEquals("entity2", entity.getId());
       }
       assertEquals(3, metricCnt);
 
@@ -1593,7 +1593,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       metricCnt = 0;
       for (TimelineEntity entity : entities) {
         metricCnt += entity.getMetrics().size();
-        assertTrue(entity.getId().equals("entity2"));
+        assertEquals("entity2", entity.getId());
         for (TimelineMetric metric : entity.getMetrics()) {
           assertTrue(metric.getId().startsWith("MAP1"));
           assertEquals(TimelineMetric.Type.SINGLE_VALUE, metric.getType());
@@ -1614,7 +1614,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       metricCnt = 0;
       for (TimelineEntity entity : entities) {
         metricCnt += entity.getMetrics().size();
-        assertTrue(entity.getId().equals("entity2"));
+        assertEquals("entity2", entity.getId());
         for (TimelineMetric metric : entity.getMetrics()) {
           assertTrue(metric.getId().startsWith("MAP1"));
           if (metric.getId().equals("MAP1_SLOT_MILLIS")) {
@@ -1654,7 +1654,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       assertNotNull(entities);
       assertEquals(1, entities.size());
       for (TimelineEntity entity : entities) {
-        assertTrue(entity.getId().equals("entity2"));
+        assertEquals("entity2", entity.getId());
       }
     } finally {
       client.destroy();
@@ -1695,7 +1695,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       assertNotNull(entities);
       assertEquals(1, entities.size());
       for (TimelineEntity entity : entities) {
-        assertTrue(entity.getId().equals("entity2"));
+        assertEquals("entity2", entity.getId());
       }
 
       //  eventfilters=(!(event1,event3) OR event5,event6) OR
@@ -1753,7 +1753,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       assertNotNull(entities);
       assertEquals(1, entities.size());
       for (TimelineEntity entity : entities) {
-        assertTrue(entity.getId().equals("entity2"));
+        assertEquals("entity2", entity.getId());
       }
 
       // isrelatedto=(!(type3:entity31,type2:entity21:entity22)OR type5:
@@ -1785,7 +1785,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       assertNotNull(entities);
       assertEquals(1, entities.size());
       for (TimelineEntity entity : entities) {
-        assertTrue(entity.getId().equals("entity2"));
+        assertEquals("entity2", entity.getId());
       }
 
       // relatesto=(!(type3:entity31,type2:entity21:entity22)OR type5:entity51,


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[05/51] [abbrv] hadoop git commit: YARN-6678. Handle IllegalStateException in Async Scheduling mode of CapacityScheduler. Contributed by Tao Yang.

Posted by st...@apache.org.
YARN-6678. Handle IllegalStateException in Async Scheduling mode of CapacityScheduler. Contributed by Tao Yang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f64cfeaf
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f64cfeaf
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f64cfeaf

Branch: refs/heads/HADOOP-13345
Commit: f64cfeaf61ec65a465decdd8215f567d4e6677a9
Parents: 79df1e7
Author: Sunil G <su...@apache.org>
Authored: Thu Aug 3 19:27:10 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Thu Aug 3 19:27:10 2017 +0530

----------------------------------------------------------------------
 .../scheduler/common/fica/FiCaSchedulerApp.java |  13 ++
 .../TestCapacitySchedulerAsyncScheduling.java   | 147 +++++++++++++++++++
 2 files changed, 160 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f64cfeaf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
index ad4c8ce..17bb104 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
@@ -426,6 +426,19 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
           // accepted & confirmed, it will become RESERVED state
           if (schedulerContainer.getRmContainer().getState()
               == RMContainerState.RESERVED) {
+            // Check if node currently reserved by other application, there may
+            // be some outdated proposals in async-scheduling environment
+            if (schedulerContainer.getRmContainer() != schedulerContainer
+                .getSchedulerNode().getReservedContainer()) {
+              if (LOG.isDebugEnabled()) {
+                LOG.debug("Try to re-reserve a container, but node "
+                    + schedulerContainer.getSchedulerNode()
+                    + " is already reserved by another container"
+                    + schedulerContainer.getSchedulerNode()
+                    .getReservedContainer().getContainerId());
+              }
+              return false;
+            }
             // Set reReservation == true
             reReservation = true;
           } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f64cfeaf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAsyncScheduling.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAsyncScheduling.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAsyncScheduling.java
index 0eb89d7..0c3130d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAsyncScheduling.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAsyncScheduling.java
@@ -20,7 +20,10 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
@@ -41,20 +44,26 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnSched
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerAllocationProposal;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceCommitRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.SchedulerContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 public class TestCapacitySchedulerAsyncScheduling {
   private final int GB = 1024;
@@ -257,6 +266,144 @@ public class TestCapacitySchedulerAsyncScheduling {
     rm.stop();
   }
 
+  // Testcase for YARN-6678
+  @Test(timeout = 30000)
+  public void testCommitOutdatedReservedProposal() throws Exception {
+    // disable async-scheduling for simulating complex since scene
+    Configuration disableAsyncConf = new Configuration(conf);
+    disableAsyncConf.setBoolean(
+        CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_ENABLE, false);
+
+    // init RM & NMs & Nodes
+    final MockRM rm = new MockRM(disableAsyncConf);
+    rm.start();
+    final MockNM nm1 = rm.registerNode("h1:1234", 9 * GB);
+    final MockNM nm2 = rm.registerNode("h2:2234", 9 * GB);
+
+    // init scheduler nodes
+    int waitTime = 1000;
+    while (waitTime > 0 &&
+        ((AbstractYarnScheduler) rm.getRMContext().getScheduler())
+            .getNodeTracker().nodeCount() < 2) {
+      waitTime -= 10;
+      Thread.sleep(10);
+    }
+    Assert.assertEquals(2,
+        ((AbstractYarnScheduler) rm.getRMContext().getScheduler())
+            .getNodeTracker().nodeCount());
+
+    YarnScheduler scheduler = rm.getRMContext().getScheduler();
+    final SchedulerNode sn1 =
+        ((CapacityScheduler) scheduler).getSchedulerNode(nm1.getNodeId());
+    final SchedulerNode sn2 =
+        ((CapacityScheduler) scheduler).getSchedulerNode(nm2.getNodeId());
+
+    // submit app1, am1 is running on nm1
+    RMApp app = rm.submitApp(200, "app", "user", null, "default");
+    final MockAM am = MockRM.launchAndRegisterAM(app, rm, nm1);
+    // submit app2, am2 is running on nm1
+    RMApp app2 = rm.submitApp(200, "app", "user", null, "default");
+    final MockAM am2 = MockRM.launchAndRegisterAM(app2, rm, nm1);
+
+    // allocate and launch 2 containers for app1
+    allocateAndLaunchContainers(am, nm1, rm, 1,
+        Resources.createResource(5 * GB), 0, 2);
+    allocateAndLaunchContainers(am, nm2, rm, 1,
+        Resources.createResource(5 * GB), 0, 3);
+
+    // nm1 runs 3 containers(app1-container_01/AM, app1-container_02,
+    //                       app2-container_01/AM)
+    // nm2 runs 1 container(app1-container_03)
+    Assert.assertEquals(3, sn1.getNumContainers());
+    Assert.assertEquals(1, sn2.getNumContainers());
+
+    // reserve 1 container(app1-container_04) for app1 on nm1
+    ResourceRequest rr2 = ResourceRequest
+        .newInstance(Priority.newInstance(0), "*",
+            Resources.createResource(5 * GB), 1);
+    am.allocate(Arrays.asList(rr2), null);
+    nm1.nodeHeartbeat(true);
+    // wait app1-container_04 reserved on nm1
+    waitTime = 1000;
+    while (waitTime > 0 && sn1.getReservedContainer() == null) {
+      waitTime -= 10;
+      Thread.sleep(10);
+    }
+    Assert.assertNotNull(sn1.getReservedContainer());
+
+    final CapacityScheduler cs = (CapacityScheduler) scheduler;
+    final CapacityScheduler spyCs = Mockito.spy(cs);
+    final AtomicBoolean isFirstReserve = new AtomicBoolean(true);
+    final AtomicBoolean isChecked = new AtomicBoolean(false);
+    // handle CapacityScheduler#tryCommit,
+    // reproduce the process that can raise IllegalStateException before
+    Mockito.doAnswer(new Answer<Object>() {
+      public Object answer(InvocationOnMock invocation) throws Exception {
+        ResourceCommitRequest request =
+            (ResourceCommitRequest) invocation.getArguments()[1];
+        if (request.getContainersToReserve().size() > 0 && isFirstReserve
+            .compareAndSet(true, false)) {
+          // release app1-container_03 on nm2
+          RMContainer killableContainer =
+              sn2.getCopiedListOfRunningContainers().get(0);
+          cs.completedContainer(killableContainer, ContainerStatus
+                  .newInstance(killableContainer.getContainerId(),
+                      ContainerState.COMPLETE, "",
+                      ContainerExitStatus.KILLED_BY_RESOURCEMANAGER),
+              RMContainerEventType.KILL);
+          Assert.assertEquals(0, sn2.getCopiedListOfRunningContainers().size());
+          // unreserve app1-container_04 on nm1
+          // and allocate app1-container_05 on nm2
+          cs.handle(new NodeUpdateSchedulerEvent(sn2.getRMNode()));
+          int waitTime = 1000;
+          while (waitTime > 0
+              && sn2.getCopiedListOfRunningContainers().size() == 0) {
+            waitTime -= 10;
+            Thread.sleep(10);
+          }
+          Assert.assertEquals(1, sn2.getCopiedListOfRunningContainers().size());
+          Assert.assertNull(sn1.getReservedContainer());
+
+          // reserve app2-container_02 on nm1
+          ResourceRequest rr3 = ResourceRequest
+              .newInstance(Priority.newInstance(0), "*",
+                  Resources.createResource(5 * GB), 1);
+          am2.allocate(Arrays.asList(rr3), null);
+          cs.handle(new NodeUpdateSchedulerEvent(sn1.getRMNode()));
+          waitTime = 1000;
+          while (waitTime > 0 && sn1.getReservedContainer() == null) {
+            waitTime -= 10;
+            Thread.sleep(10);
+          }
+          Assert.assertNotNull(sn1.getReservedContainer());
+
+          // call real apply
+          try {
+            cs.tryCommit((Resource) invocation.getArguments()[0],
+                (ResourceCommitRequest) invocation.getArguments()[1]);
+          } catch (Exception e) {
+            e.printStackTrace();
+            Assert.fail();
+          }
+          isChecked.set(true);
+        } else {
+          cs.tryCommit((Resource) invocation.getArguments()[0],
+              (ResourceCommitRequest) invocation.getArguments()[1]);
+        }
+        return null;
+      }
+    }).when(spyCs).tryCommit(Mockito.any(Resource.class),
+        Mockito.any(ResourceCommitRequest.class));
+
+    spyCs.handle(new NodeUpdateSchedulerEvent(sn1.getRMNode()));
+
+    waitTime = 1000;
+    while (waitTime > 0 && !isChecked.get()) {
+      waitTime -= 10;
+      Thread.sleep(10);
+    }
+    rm.stop();
+  }
 
   private void allocateAndLaunchContainers(MockAM am, MockNM nm, MockRM rm,
       int nContainer, Resource resource, int priority, int startContainerId)


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[02/51] [abbrv] hadoop git commit: HDFS-6984. Serialize FileStatus via protobuf.

Posted by st...@apache.org.
HDFS-6984. Serialize FileStatus via protobuf.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/12e44e7b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/12e44e7b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/12e44e7b

Branch: refs/heads/HADOOP-13345
Commit: 12e44e7bdaf53d3720a89d32f0cc2717241bd6b2
Parents: 1a1bf6b
Author: Chris Douglas <cd...@apache.org>
Authored: Wed Aug 2 12:12:48 2017 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Wed Aug 2 12:12:48 2017 -0700

----------------------------------------------------------------------
 .../dev-support/findbugsExcludeFile.xml         |   4 +
 hadoop-common-project/hadoop-common/pom.xml     |   1 +
 .../apache/hadoop/fs/FileEncryptionInfo.java    |   6 +-
 .../java/org/apache/hadoop/fs/FileStatus.java   | 152 ++++++++-----
 .../org/apache/hadoop/fs/LocatedFileStatus.java |  64 ++++--
 .../hadoop/fs/permission/FsPermission.java      |  12 +
 .../apache/hadoop/fs/protocolPB/PBHelper.java   | 131 +++++++++++
 .../hadoop/fs/protocolPB/package-info.java      |  18 ++
 .../apache/hadoop/io/erasurecode/ECSchema.java  |   6 +-
 .../hadoop-common/src/main/proto/FSProtos.proto |  69 ++++++
 .../org/apache/hadoop/fs/TestFileStatus.java    |   1 +
 .../fs/protocolPB/TestFSSerialization.java      |  85 +++++++
 .../hdfs/protocol/ErasureCodingPolicy.java      |   8 +-
 .../hdfs/protocol/FsPermissionExtension.java    |   5 +
 .../hadoop/hdfs/protocol/HdfsFileStatus.java    | 228 +++++++++----------
 .../hdfs/protocol/HdfsLocatedFileStatus.java    |  42 ++--
 .../protocol/SnapshottableDirectoryStatus.java  |  10 +-
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  |  65 +++++-
 .../apache/hadoop/hdfs/web/JsonUtilClient.java  |  50 ++--
 .../hadoop/hdfs/web/WebHdfsConstants.java       |   8 +-
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |  16 +-
 .../hadoop-hdfs-client/src/main/proto/acl.proto |   7 +-
 .../src/main/proto/hdfs.proto                   |  16 +-
 .../hadoop/fs/http/client/HttpFSFileSystem.java |  45 ++--
 .../fs/http/client/BaseTestHttpFSWith.java      |  19 +-
 .../dev-support/findbugsExcludeFile.xml         |  12 +
 .../hadoop/hdfs/protocol/SnapshotInfo.java      |   2 +-
 .../hadoop/hdfs/protocolPB/package-info.java    |  18 ++
 .../server/namenode/FSDirStatAndListingOp.java  |  58 ++---
 .../hdfs/server/namenode/FSDirectory.java       |  20 +-
 .../hdfs/server/namenode/FSNamesystem.java      |   7 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |   3 +-
 .../namenode/snapshot/SnapshotManager.java      |   7 +-
 .../org/apache/hadoop/hdfs/web/JsonUtil.java    |  28 ++-
 .../hadoop/hdfs/TestDFSClientRetries.java       |   6 +-
 .../apache/hadoop/hdfs/TestEncryptionZones.java |   3 +-
 .../hdfs/TestFileStatusSerialization.java       | 153 +++++++++++++
 .../java/org/apache/hadoop/hdfs/TestLease.java  |   7 +-
 .../hdfs/server/namenode/AclTestHelpers.java    |  13 +-
 .../hadoop/hdfs/server/namenode/TestFsck.java   |   4 +-
 .../apache/hadoop/hdfs/web/TestJsonUtil.java    |  11 +-
 41 files changed, 1054 insertions(+), 366 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml b/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
index de76afb..4bafd8e 100644
--- a/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
@@ -323,6 +323,10 @@
       <!-- protobuf generated code -->
       <Class name="~org\.apache\.hadoop\.tracing\.TraceAdminPB.*"/>
     </Match>
+    <Match>
+      <!-- protobuf generated code -->
+      <Class name="~org\.apache\.hadoop\.fs\.FSProto.*"/>
+    </Match>
 
     <!--
        Manually checked, misses child thread manually syncing on parent's intrinsic lock.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-common-project/hadoop-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml
index 3e73bce..ad6f734 100644
--- a/hadoop-common-project/hadoop-common/pom.xml
+++ b/hadoop-common-project/hadoop-common/pom.xml
@@ -393,6 +393,7 @@
                   <include>RefreshUserMappingsProtocol.proto</include>
                   <include>RefreshCallQueueProtocol.proto</include>
                   <include>GenericRefreshProtocol.proto</include>
+                  <include>FSProtos.proto</include>
                 </includes>
               </source>
             </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileEncryptionInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileEncryptionInfo.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileEncryptionInfo.java
index 1129e07..ce5ed56 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileEncryptionInfo.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileEncryptionInfo.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.fs;
 
+import java.io.Serializable;
+
 import org.apache.commons.codec.binary.Hex;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.crypto.CipherSuite;
@@ -30,7 +32,9 @@ import static com.google.common.base.Preconditions.checkNotNull;
  * an encrypted file.
  */
 @InterfaceAudience.Private
-public class FileEncryptionInfo {
+public class FileEncryptionInfo implements Serializable {
+
+  private static final long serialVersionUID = 0x156abe03;
 
   private final CipherSuite cipherSuite;
   private final CryptoProtocolVersion version;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
index 7ce6363..2f22ea0 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
@@ -23,11 +23,15 @@ import java.io.IOException;
 import java.io.InvalidObjectException;
 import java.io.ObjectInputValidation;
 import java.io.Serializable;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.Set;
 
+import org.apache.hadoop.fs.FSProtos.FileStatusProto;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.io.Text;
+import org.apache.hadoop.fs.protocolPB.PBHelper;
 import org.apache.hadoop.io.Writable;
 
 /** Interface that represents the client side information for a file.
@@ -50,7 +54,31 @@ public class FileStatus implements Writable, Comparable<Object>,
   private String owner;
   private String group;
   private Path symlink;
-  
+  private Set<AttrFlags> attr;
+
+  private enum AttrFlags {
+    HAS_ACL,
+    HAS_CRYPT,
+    HAS_EC,
+  };
+  private static final Set<AttrFlags> NONE = Collections.<AttrFlags>emptySet();
+  private static Set<AttrFlags> flags(boolean acl, boolean crypt, boolean ec) {
+    if (!(acl || crypt || ec)) {
+      return NONE;
+    }
+    EnumSet<AttrFlags> ret = EnumSet.noneOf(AttrFlags.class);
+    if (acl) {
+      ret.add(AttrFlags.HAS_ACL);
+    }
+    if (crypt) {
+      ret.add(AttrFlags.HAS_CRYPT);
+    }
+    if (ec) {
+      ret.add(AttrFlags.HAS_EC);
+    }
+    return ret;
+  }
+
   public FileStatus() { this(0, false, 0, 0, 0, 0, null, null, null, null); }
   
   //We should deprecate this soon?
@@ -79,6 +107,15 @@ public class FileStatus implements Writable, Comparable<Object>,
                     FsPermission permission, String owner, String group, 
                     Path symlink,
                     Path path) {
+    this(length, isdir, block_replication, blocksize, modification_time,
+        access_time, permission, owner, group, symlink, path,
+        false, false, false);
+  }
+
+  public FileStatus(long length, boolean isdir, int block_replication,
+      long blocksize, long modification_time, long access_time,
+      FsPermission permission, String owner, String group, Path symlink,
+      Path path, boolean hasAcl, boolean isEncrypted, boolean isErasureCoded) {
     this.length = length;
     this.isdir = isdir;
     this.block_replication = (short)block_replication;
@@ -89,7 +126,7 @@ public class FileStatus implements Writable, Comparable<Object>,
       this.permission = permission;
     } else if (isdir) {
       this.permission = FsPermission.getDirDefault();
-    } else if (symlink!=null) {
+    } else if (symlink != null) {
       this.permission = FsPermission.getDefault();
     } else {
       this.permission = FsPermission.getFileDefault();
@@ -98,6 +135,8 @@ public class FileStatus implements Writable, Comparable<Object>,
     this.group = (group == null) ? "" : group;
     this.symlink = symlink;
     this.path = path;
+    attr = flags(hasAcl, isEncrypted, isErasureCoded);
+
     // The variables isdir and symlink indicate the type:
     // 1. isdir implies directory, in which case symlink must be null.
     // 2. !isdir implies a file or symlink, symlink != null implies a
@@ -213,7 +252,7 @@ public class FileStatus implements Writable, Comparable<Object>,
    * @return true if the underlying file or directory has ACLs set.
    */
   public boolean hasAcl() {
-    return permission.getAclBit();
+    return attr.contains(AttrFlags.HAS_ACL);
   }
 
   /**
@@ -222,7 +261,7 @@ public class FileStatus implements Writable, Comparable<Object>,
    * @return true if the underlying file is encrypted.
    */
   public boolean isEncrypted() {
-    return permission.getEncryptedBit();
+    return attr.contains(AttrFlags.HAS_CRYPT);
   }
 
   /**
@@ -231,7 +270,7 @@ public class FileStatus implements Writable, Comparable<Object>,
    * @return true if the underlying file or directory is erasure coded.
    */
   public boolean isErasureCoded() {
-    return permission.getErasureCodedBit();
+    return attr.contains(AttrFlags.HAS_EC);
   }
 
   /**
@@ -304,47 +343,6 @@ public class FileStatus implements Writable, Comparable<Object>,
   public void setSymlink(final Path p) {
     symlink = p;
   }
-  
-  //////////////////////////////////////////////////
-  // Writable
-  //////////////////////////////////////////////////
-  @Override
-  public void write(DataOutput out) throws IOException {
-    Text.writeString(out, getPath().toString(), Text.DEFAULT_MAX_LEN);
-    out.writeLong(getLen());
-    out.writeBoolean(isDirectory());
-    out.writeShort(getReplication());
-    out.writeLong(getBlockSize());
-    out.writeLong(getModificationTime());
-    out.writeLong(getAccessTime());
-    getPermission().write(out);
-    Text.writeString(out, getOwner(), Text.DEFAULT_MAX_LEN);
-    Text.writeString(out, getGroup(), Text.DEFAULT_MAX_LEN);
-    out.writeBoolean(isSymlink());
-    if (isSymlink()) {
-      Text.writeString(out, getSymlink().toString(), Text.DEFAULT_MAX_LEN);
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    String strPath = Text.readString(in, Text.DEFAULT_MAX_LEN);
-    this.path = new Path(strPath);
-    this.length = in.readLong();
-    this.isdir = in.readBoolean();
-    this.block_replication = in.readShort();
-    blocksize = in.readLong();
-    modification_time = in.readLong();
-    access_time = in.readLong();
-    permission.readFields(in);
-    owner = Text.readString(in, Text.DEFAULT_MAX_LEN);
-    group = Text.readString(in, Text.DEFAULT_MAX_LEN);
-    if (in.readBoolean()) {
-      this.symlink = new Path(Text.readString(in, Text.DEFAULT_MAX_LEN));
-    } else {
-      this.symlink = null;
-    }
-  }
 
   /**
    * Compare this FileStatus to another FileStatus
@@ -377,15 +375,12 @@ public class FileStatus implements Writable, Comparable<Object>,
    */
   @Override
   public boolean equals(Object o) {
-    if (o == null) {
+    if (!(o instanceof FileStatus)) {
       return false;
     }
     if (this == o) {
       return true;
     }
-    if (!(o instanceof FileStatus)) {
-      return false;
-    }
     FileStatus other = (FileStatus)o;
     return this.getPath().equals(other.getPath());
   }
@@ -420,7 +415,11 @@ public class FileStatus implements Writable, Comparable<Object>,
     sb.append("; permission=" + permission);
     sb.append("; isSymlink=" + isSymlink());
     if(isSymlink()) {
-      sb.append("; symlink=" + symlink);
+      try {
+        sb.append("; symlink=" + getSymlink());
+      } catch (IOException e) {
+        throw new RuntimeException("Unexpected exception", e);
+      }
     }
     sb.append("; hasAcl=" + hasAcl());
     sb.append("; isEncrypted=" + isEncrypted());
@@ -429,6 +428,55 @@ public class FileStatus implements Writable, Comparable<Object>,
     return sb.toString();
   }
 
+  /**
+   * Read instance encoded as protobuf from stream.
+   * @param in Input stream
+   * @see PBHelper#convert(FileStatus)
+   * @deprecated Use the {@link PBHelper} and protobuf serialization directly.
+   */
+  @Override
+  @Deprecated
+  public void readFields(DataInput in) throws IOException {
+    int size = in.readInt();
+    if (size < 0) {
+      throw new IOException("Can't read FileStatusProto with negative " +
+          "size of " + size);
+    }
+    byte[] buf = new byte[size];
+    in.readFully(buf);
+    FileStatusProto proto = FileStatusProto.parseFrom(buf);
+    FileStatus other = PBHelper.convert(proto);
+    isdir = other.isDirectory();
+    length = other.getLen();
+    isdir = other.isDirectory();
+    block_replication = other.getReplication();
+    blocksize = other.getBlockSize();
+    modification_time = other.getModificationTime();
+    access_time = other.getAccessTime();
+    setPermission(other.getPermission());
+    setOwner(other.getOwner());
+    setGroup(other.getGroup());
+    setSymlink((other.isSymlink() ? other.getSymlink() : null));
+    setPath(other.getPath());
+    attr = flags(other.hasAcl(), other.isEncrypted(), other.isErasureCoded());
+    assert (isDirectory() && getSymlink() == null) || !isDirectory();
+  }
+
+  /**
+   * Write instance encoded as protobuf to stream.
+   * @param out Output stream
+   * @see PBHelper#convert(FileStatus)
+   * @deprecated Use the {@link PBHelper} and protobuf serialization directly.
+   */
+  @Override
+  @Deprecated
+  public void write(DataOutput out) throws IOException {
+    FileStatusProto proto = PBHelper.convert(this);
+    int size = proto.getSerializedSize();
+    out.writeInt(size);
+    out.write(proto.toByteArray());
+  }
+
   @Override
   public void validateObject() throws InvalidObjectException {
     if (null == path) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocatedFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocatedFileStatus.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocatedFileStatus.java
index 588fd6a..885da07 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocatedFileStatus.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocatedFileStatus.java
@@ -30,6 +30,9 @@ import org.apache.hadoop.fs.permission.FsPermission;
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class LocatedFileStatus extends FileStatus {
+
+  private static final long serialVersionUID = 0x17339920;
+
   private BlockLocation[] locations;
 
 
@@ -42,14 +45,18 @@ public class LocatedFileStatus extends FileStatus {
    * @param stat a file status
    * @param locations a file's block locations
    */
-  public LocatedFileStatus(FileStatus stat, BlockLocation[] locations)
-  throws IOException {
+  public LocatedFileStatus(FileStatus stat, BlockLocation[] locations) {
     this(stat.getLen(), stat.isDirectory(), stat.getReplication(),
         stat.getBlockSize(), stat.getModificationTime(),
-        stat.getAccessTime(), stat.getPermission(), stat.getOwner(),
-        stat.getGroup(), null, stat.getPath(), locations);
+        stat.getAccessTime(), stat.getPermission(),
+        stat.getOwner(), stat.getGroup(), null, stat.getPath(),
+        stat.hasAcl(), stat.isEncrypted(), stat.isErasureCoded(), locations);
     if (stat.isSymlink()) {
-      setSymlink(stat.getSymlink());
+      try {
+        setSymlink(stat.getSymlink());
+      } catch (IOException e) {
+        throw new RuntimeException("Unexpected exception", e);
+      }
     }
   }
 
@@ -69,24 +76,55 @@ public class LocatedFileStatus extends FileStatus {
    * @param path the path's qualified name
    * @param locations a file's block locations
    */
+  @Deprecated
   public LocatedFileStatus(long length, boolean isdir,
           int block_replication,
           long blocksize, long modification_time, long access_time,
           FsPermission permission, String owner, String group, 
-          Path symlink,
-          Path path,
-          BlockLocation[] locations) {
-	  super(length, isdir, block_replication, blocksize, modification_time,
-			  access_time, permission, owner, group, symlink, path);
-	  this.locations = locations;
+          Path symlink, Path path, BlockLocation[] locations) {
+    this(length, isdir, block_replication, blocksize, modification_time,
+        access_time, permission, owner, group, symlink, path,
+        permission.getAclBit(), permission.getEncryptedBit(),
+        permission.getErasureCodedBit(), locations);
   }
-  
+
+  /**
+   * Constructor.
+   *
+   * @param length a file's length
+   * @param isdir if the path is a directory
+   * @param block_replication the file's replication factor
+   * @param blocksize a file's block size
+   * @param modification_time a file's modification time
+   * @param access_time a file's access time
+   * @param permission a file's permission
+   * @param owner a file's owner
+   * @param group a file's group
+   * @param symlink symlink if the path is a symbolic link
+   * @param path the path's qualified name
+   * @param hasAcl entity has associated ACLs
+   * @param isEncrypted entity is encrypted
+   * @param isErasureCoded entity is erasure coded
+   * @param locations a file's block locations
+   */
+  public LocatedFileStatus(long length, boolean isdir,
+      int block_replication, long blocksize, long modification_time,
+      long access_time, FsPermission permission, String owner, String group,
+      Path symlink, Path path,
+      boolean hasAcl, boolean isEncrypted, boolean isErasureCoded,
+      BlockLocation[] locations) {
+    super(length, isdir, block_replication, blocksize, modification_time,
+        access_time, permission, owner, group, symlink, path,
+        hasAcl, isEncrypted, isErasureCoded);
+    this.locations = locations;
+  }
+
   /**
    * Get the file's block locations
    * @return the file's block locations
    */
   public BlockLocation[] getBlockLocations() {
-	  return locations;
+    return locations;
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
index 73ab5f6..23692de 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
@@ -133,11 +133,13 @@ public class FsPermission implements Writable, Serializable,
   }
 
   @Override
+  @Deprecated
   public void write(DataOutput out) throws IOException {
     out.writeShort(toShort());
   }
 
   @Override
+  @Deprecated
   public void readFields(DataInput in) throws IOException {
     fromShort(in.readShort());
   }
@@ -184,6 +186,7 @@ public class FsPermission implements Writable, Serializable,
    *
    * @return short extended short representation of this permission
    */
+  @Deprecated
   public short toExtendedShort() {
     return toShort();
   }
@@ -299,7 +302,10 @@ public class FsPermission implements Writable, Serializable,
    * Returns true if there is also an ACL (access control list).
    *
    * @return boolean true if there is also an ACL (access control list).
+   * @deprecated Get acl bit from the {@link org.apache.hadoop.fs.FileStatus}
+   * object.
    */
+  @Deprecated
   public boolean getAclBit() {
     // File system subclasses that support the ACL bit would override this.
     return false;
@@ -307,14 +313,20 @@ public class FsPermission implements Writable, Serializable,
 
   /**
    * Returns true if the file is encrypted or directory is in an encryption zone
+   * @deprecated Get encryption bit from the
+   * {@link org.apache.hadoop.fs.FileStatus} object.
    */
+  @Deprecated
   public boolean getEncryptedBit() {
     return false;
   }
 
   /**
    * Returns true if the file or directory is erasure coded.
+   * @deprecated Get ec bit from the {@link org.apache.hadoop.fs.FileStatus}
+   * object.
    */
+  @Deprecated
   public boolean getErasureCodedBit() {
     return false;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/protocolPB/PBHelper.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/protocolPB/PBHelper.java
new file mode 100644
index 0000000..aa7d8f5
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/protocolPB/PBHelper.java
@@ -0,0 +1,131 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.protocolPB;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.FSProtos.*;
+
+/**
+ * Utility methods aiding conversion of fs data structures.
+ */
+public final class PBHelper {
+
+  private PBHelper() {
+    // prevent construction
+  }
+
+  public static FsPermission convert(FsPermissionProto proto)
+      throws IOException {
+    return new FsPermission((short)proto.getPerm());
+  }
+
+  public static FsPermissionProto convert(FsPermission p) throws IOException {
+    FsPermissionProto.Builder bld = FsPermissionProto.newBuilder();
+    bld.setPerm(p.toShort());
+    return bld.build();
+  }
+
+  public static FileStatus convert(FileStatusProto proto) throws IOException {
+    final Path path;
+    final long length;
+    final boolean isdir;
+    final short blockReplication;
+    final long blocksize;
+    final long mtime;
+    final long atime;
+    final String owner;
+    final String group;
+    final FsPermission permission;
+    final Path symlink;
+    switch (proto.getFileType()) {
+    case FT_DIR:
+      isdir = true;
+      symlink = null;
+      blocksize = 0;
+      length = 0;
+      blockReplication = 0;
+      break;
+    case FT_SYMLINK:
+      isdir = false;
+      symlink = new Path(proto.getSymlink());
+      blocksize = 0;
+      length = 0;
+      blockReplication = 0;
+      break;
+    case FT_FILE:
+      isdir = false;
+      symlink = null;
+      blocksize = proto.getBlockSize();
+      length = proto.getLength();
+      int brep = proto.getBlockReplication();
+      if ((brep & 0xffff0000) != 0) {
+        throw new IOException(String.format("Block replication 0x%08x " +
+            "doesn't fit in 16 bits.", brep));
+      }
+      blockReplication = (short)brep;
+      break;
+    default:
+      throw new IllegalStateException("Unknown type: " + proto.getFileType());
+    }
+    path = new Path(proto.getPath());
+    mtime = proto.getModificationTime();
+    atime = proto.getAccessTime();
+    permission = convert(proto.getPermission());
+    owner = proto.getOwner();
+    group = proto.getGroup();
+    int flags = proto.getFlags();
+    return new FileStatus(length, isdir, blockReplication, blocksize,
+        mtime, atime, permission, owner, group, symlink, path,
+        (flags & FileStatusProto.Flags.HAS_ACL_VALUE)   != 0,
+        (flags & FileStatusProto.Flags.HAS_CRYPT_VALUE) != 0,
+        (flags & FileStatusProto.Flags.HAS_EC_VALUE)    != 0);
+  }
+
+  public static FileStatusProto convert(FileStatus stat) throws IOException {
+    FileStatusProto.Builder bld = FileStatusProto.newBuilder();
+    bld.setPath(stat.getPath().toString());
+    if (stat.isDirectory()) {
+      bld.setFileType(FileStatusProto.FileType.FT_DIR);
+    } else if (stat.isSymlink()) {
+      bld.setFileType(FileStatusProto.FileType.FT_SYMLINK)
+         .setSymlink(stat.getSymlink().toString());
+    } else {
+      bld.setFileType(FileStatusProto.FileType.FT_FILE)
+         .setLength(stat.getLen())
+         .setBlockReplication(stat.getReplication())
+         .setBlockSize(stat.getBlockSize());
+    }
+    bld.setAccessTime(stat.getAccessTime())
+       .setModificationTime(stat.getModificationTime())
+       .setOwner(stat.getOwner())
+       .setGroup(stat.getGroup())
+       .setPermission(convert(stat.getPermission()));
+    int flags = 0;
+    flags |= stat.hasAcl()         ? FileStatusProto.Flags.HAS_ACL_VALUE   : 0;
+    flags |= stat.isEncrypted()    ? FileStatusProto.Flags.HAS_CRYPT_VALUE : 0;
+    flags |= stat.isErasureCoded() ? FileStatusProto.Flags.HAS_EC_VALUE    : 0;
+    bld.setFlags(flags);
+    return bld.build();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/protocolPB/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/protocolPB/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/protocolPB/package-info.java
new file mode 100644
index 0000000..dbc9bce
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/protocolPB/package-info.java
@@ -0,0 +1,18 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.protocolPB;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java
index 4d66019..f008e24 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.io.erasurecode;
 
+import java.io.Serializable;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
@@ -31,7 +32,10 @@ import org.apache.hadoop.classification.InterfaceStability;
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
-public final class ECSchema {
+public final class ECSchema implements Serializable {
+
+  private static final long serialVersionUID = 0x10953aa0;
+
   public static final String NUM_DATA_UNITS_KEY = "numDataUnits";
   public static final String NUM_PARITY_UNITS_KEY = "numParityUnits";
   public static final String CODEC_NAME_KEY = "codec";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-common-project/hadoop-common/src/main/proto/FSProtos.proto
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/proto/FSProtos.proto b/hadoop-common-project/hadoop-common/src/main/proto/FSProtos.proto
new file mode 100644
index 0000000..6fe7980
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/proto/FSProtos.proto
@@ -0,0 +1,69 @@
+/**
+ * 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.
+ */
+
+/**
+ * These .proto interfaces are private and stable.
+ * Please see http://wiki.apache.org/hadoop/Compatibility
+ * for what changes are allowed for a *stable* .proto interface.
+ */
+
+option java_package = "org.apache.hadoop.fs";
+option java_outer_classname = "FSProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+package hadoop.fs;
+
+message FsPermissionProto {
+  required uint32 perm = 1; // UNIX-style mode bits
+}
+
+/*
+ * FileStatus encoding. Field IDs match those from HdfsFileStatusProto, but
+ * cross-serialization is not an explicitly supported use case. Unlike HDFS,
+ * most fields are optional and do not define defaults.
+ */
+message FileStatusProto {
+  enum FileType {
+    FT_DIR     = 1;
+    FT_FILE    = 2;
+    FT_SYMLINK = 3;
+  }
+  enum Flags {
+    HAS_ACL    = 0x01; // has ACLs
+    HAS_CRYPT  = 0x02; // encrypted
+    HAS_EC     = 0x04; // erasure coded
+  }
+  required FileType fileType            = 1;
+  required string path                  = 2;
+  optional uint64 length                = 3;
+  optional FsPermissionProto permission = 4;
+  optional string owner                 = 5;
+  optional string group                 = 6;
+  optional uint64 modification_time     = 7;
+  optional uint64 access_time           = 8;
+  optional string symlink               = 9;
+  optional uint32 block_replication     = 10;
+  optional uint64 block_size            = 11;
+  // locations                          = 12
+  // alias                              = 13
+  // childrenNum                        = 14
+  optional bytes encryption_data        = 15;
+  // storagePolicy                      = 16
+  optional bytes ec_data                = 17;
+  optional uint32 flags                 = 18 [default = 0];
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileStatus.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileStatus.java
index 1962f49..61a688e 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileStatus.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileStatus.java
@@ -36,6 +36,7 @@ import org.junit.Test;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/protocolPB/TestFSSerialization.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/protocolPB/TestFSSerialization.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/protocolPB/TestFSSerialization.java
new file mode 100644
index 0000000..31cacf7
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/protocolPB/TestFSSerialization.java
@@ -0,0 +1,85 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.protocolPB;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import static org.apache.hadoop.fs.FSProtos.*;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+/**
+ * Verify PB serialization of FS data structures.
+ */
+public class TestFSSerialization {
+
+  @Test
+  @SuppressWarnings("deprecation")
+  public void testWritableFlagSerialization() throws Exception {
+    final Path p = new Path("hdfs://yaks:4344/dingos/f");
+    for (int i = 0; i < 0x8; ++i) {
+      final boolean acl   = 0 != (i & 0x1);
+      final boolean crypt = 0 != (i & 0x2);
+      final boolean ec    = 0 != (i & 0x4);
+      FileStatus stat = new FileStatus(1024L, false, 3, 1L << 31,
+          12345678L, 87654321L, FsPermission.getFileDefault(),
+          "hadoop", "unqbbc", null, p, acl, crypt, ec);
+      DataOutputBuffer dob = new DataOutputBuffer();
+      stat.write(dob);
+      DataInputBuffer dib = new DataInputBuffer();
+      dib.reset(dob.getData(), 0, dob.getLength());
+      FileStatus fstat = new FileStatus();
+      fstat.readFields(dib);
+      assertEquals(stat, fstat);
+      checkFields(stat, fstat);
+    }
+  }
+
+  @Test
+  public void testUtilitySerialization() throws Exception {
+    final Path p = new Path("hdfs://yaks:4344/dingos/f");
+    FileStatus stat = new FileStatus(1024L, false, 3, 1L << 31,
+        12345678L, 87654321L, FsPermission.createImmutable((short)0111),
+        "hadoop", "unqbbc", null, p);
+    FileStatusProto fsp = PBHelper.convert(stat);
+    FileStatus stat2 = PBHelper.convert(fsp);
+    assertEquals(stat, stat2);
+    checkFields(stat, stat2);
+  }
+
+  private static void checkFields(FileStatus expected, FileStatus actual) {
+    assertEquals(expected.getPath(), actual.getPath());
+    assertEquals(expected.isDirectory(), actual.isDirectory());
+    assertEquals(expected.getLen(), actual.getLen());
+    assertEquals(expected.getPermission(), actual.getPermission());
+    assertEquals(expected.getOwner(), actual.getOwner());
+    assertEquals(expected.getGroup(), actual.getGroup());
+    assertEquals(expected.getModificationTime(), actual.getModificationTime());
+    assertEquals(expected.getAccessTime(), actual.getAccessTime());
+    assertEquals(expected.getReplication(), actual.getReplication());
+    assertEquals(expected.getBlockSize(), actual.getBlockSize());
+    assertEquals(expected.hasAcl(), actual.hasAcl());
+    assertEquals(expected.isEncrypted(), actual.isEncrypted());
+    assertEquals(expected.isErasureCoded(), actual.isErasureCoded());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.java
index 368a2f2..7afc377 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.protocol;
 
+import java.io.Serializable;
+
 import com.google.common.base.Preconditions;
 import org.apache.commons.lang.builder.EqualsBuilder;
 import org.apache.commons.lang.builder.HashCodeBuilder;
@@ -29,11 +31,13 @@ import org.apache.hadoop.io.erasurecode.ECSchema;
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
-public final class ErasureCodingPolicy {
+public final class ErasureCodingPolicy implements Serializable {
+
+  private static final long serialVersionUID = 0x0079fe4e;
 
+  private String name;
   private final ECSchema schema;
   private final int cellSize;
-  private String name;
   private byte id;
 
   public ErasureCodingPolicy(String name, ECSchema schema,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/FsPermissionExtension.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/FsPermissionExtension.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/FsPermissionExtension.java
index e0dd0d7..37d04e3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/FsPermissionExtension.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/FsPermissionExtension.java
@@ -27,6 +27,11 @@ import org.apache.hadoop.fs.permission.FsPermission;
  * done for backwards compatibility in case any existing clients assume the
  * value of FsPermission is in a particular range.
  */
+
+/**
+ * @deprecated ACLs, encryption, and erasure coding are managed on FileStatus.
+ */
+@Deprecated
 @InterfaceAudience.Private
 public class FsPermissionExtension extends FsPermission {
   private static final long serialVersionUID = 0x13c298a4;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
index c386602..8438b01 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
@@ -17,7 +17,9 @@
  */
 package org.apache.hadoop.hdfs.protocol;
 
+import java.io.IOException;
 import java.net.URI;
+import java.util.EnumSet;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -31,24 +33,15 @@ import org.apache.hadoop.hdfs.DFSUtilClient;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class HdfsFileStatus {
+public class HdfsFileStatus extends FileStatus {
+
+  private static final long serialVersionUID = 0x126eb82a;
 
   // local name of the inode that's encoded in java UTF8
-  private final byte[] path;
-  private final byte[] symlink; // symlink target encoded in java UTF8 or null
-  private final long length;
-  private final boolean isdir;
-  private final short block_replication;
-  private final long blocksize;
-  private final long modification_time;
-  private final long access_time;
-  private final FsPermission permission;
-  private final String owner;
-  private final String group;
+  private byte[] uPath;
+  private byte[] uSymlink; // symlink target encoded in java UTF8/null
   private final long fileId;
-
   private final FileEncryptionInfo feInfo;
-
   private final ErasureCodingPolicy ecPolicy;
 
   // Used by dir, not including dot and dotdot. Always zero for a regular file.
@@ -58,11 +51,21 @@ public class HdfsFileStatus {
   public static final byte[] EMPTY_NAME = new byte[0];
 
   /**
+   * Set of features potentially active on an instance.
+   */
+  public enum Flags {
+    HAS_ACL,
+    HAS_CRYPT,
+    HAS_EC;
+  }
+  private final EnumSet<Flags> flags;
+
+  /**
    * Constructor.
-   * @param length the number of bytes the file has
-   * @param isdir if the path is a directory
+   * @param length            the number of bytes the file has
+   * @param isdir             if the path is a directory
    * @param block_replication the replication factor
-   * @param blocksize the block size
+   * @param blocksize         the block size
    * @param modification_time modification time
    * @param access_time access time
    * @param permission permission
@@ -77,25 +80,18 @@ public class HdfsFileStatus {
    * @param ecPolicy the erasure coding policy
    */
   public HdfsFileStatus(long length, boolean isdir, int block_replication,
-      long blocksize, long modification_time, long access_time,
-      FsPermission permission, String owner, String group, byte[] symlink,
-      byte[] path, long fileId, int childrenNum, FileEncryptionInfo feInfo,
-      byte storagePolicy, ErasureCodingPolicy ecPolicy) {
-    this.length = length;
-    this.isdir = isdir;
-    this.block_replication = (short) block_replication;
-    this.blocksize = blocksize;
-    this.modification_time = modification_time;
-    this.access_time = access_time;
-    this.permission = (permission == null) ?
-        ((isdir || symlink!=null) ?
-            FsPermission.getDefault() :
-            FsPermission.getFileDefault()) :
-        permission;
-    this.owner = (owner == null) ? "" : owner;
-    this.group = (group == null) ? "" : group;
-    this.symlink = symlink;
-    this.path = path;
+                        long blocksize, long modification_time,
+                        long access_time, FsPermission permission,
+                        EnumSet<Flags> flags, String owner, String group,
+                        byte[] symlink, byte[] path, long fileId,
+                        int childrenNum, FileEncryptionInfo feInfo,
+                        byte storagePolicy, ErasureCodingPolicy ecPolicy) {
+    super(length, isdir, block_replication, blocksize, modification_time,
+        access_time, convert(isdir, symlink != null, permission, flags),
+        owner, group, null, null);
+    this.flags = flags;
+    this.uSymlink = symlink;
+    this.uPath = path;
     this.fileId = fileId;
     this.childrenNum = childrenNum;
     this.feInfo = feInfo;
@@ -104,83 +100,48 @@ public class HdfsFileStatus {
   }
 
   /**
-   * Get the length of this file, in bytes.
-   * @return the length of this file, in bytes.
+   * Set redundant flags for compatibility with existing applications.
    */
-  public final long getLen() {
-    return length;
-  }
-
-  /**
-   * Is this a directory?
-   * @return true if this is a directory
-   */
-  public final boolean isDir() {
-    return isdir;
+  protected static FsPermission convert(boolean isdir, boolean symlink,
+      FsPermission p, EnumSet<Flags> f) {
+    if (p instanceof FsPermissionExtension) {
+      // verify flags are set consistently
+      assert p.getAclBit() == f.contains(HdfsFileStatus.Flags.HAS_ACL);
+      assert p.getEncryptedBit() == f.contains(HdfsFileStatus.Flags.HAS_CRYPT);
+      assert p.getErasureCodedBit() == f.contains(HdfsFileStatus.Flags.HAS_EC);
+      return p;
+    }
+    if (null == p) {
+      if (isdir) {
+        p = FsPermission.getDirDefault();
+      } else if (symlink) {
+        p = FsPermission.getDefault();
+      } else {
+        p = FsPermission.getFileDefault();
+      }
+    }
+    return new FsPermissionExtension(p, f.contains(Flags.HAS_ACL),
+        f.contains(Flags.HAS_CRYPT), f.contains(Flags.HAS_EC));
   }
 
-  /**
-   * Is this a symbolic link?
-   * @return true if this is a symbolic link
-   */
+  @Override
   public boolean isSymlink() {
-    return symlink != null;
+    return uSymlink != null;
   }
 
-  /**
-   * Get the block size of the file.
-   * @return the number of bytes
-   */
-  public final long getBlockSize() {
-    return blocksize;
+  @Override
+  public boolean hasAcl() {
+    return flags.contains(Flags.HAS_ACL);
   }
 
-  /**
-   * Get the replication factor of a file.
-   * @return the replication factor of a file.
-   */
-  public final short getReplication() {
-    return block_replication;
+  @Override
+  public boolean isEncrypted() {
+    return flags.contains(Flags.HAS_CRYPT);
   }
 
-  /**
-   * Get the modification time of the file.
-   * @return the modification time of file in milliseconds since January 1, 1970 UTC.
-   */
-  public final long getModificationTime() {
-    return modification_time;
-  }
-
-  /**
-   * Get the access time of the file.
-   * @return the access time of file in milliseconds since January 1, 1970 UTC.
-   */
-  public final long getAccessTime() {
-    return access_time;
-  }
-
-  /**
-   * Get FsPermission associated with the file.
-   * @return permission
-   */
-  public final FsPermission getPermission() {
-    return permission;
-  }
-
-  /**
-   * Get the owner of the file.
-   * @return owner of the file
-   */
-  public final String getOwner() {
-    return owner;
-  }
-
-  /**
-   * Get the group associated with the file.
-   * @return group for the file.
-   */
-  public final String getGroup() {
-    return group;
+  @Override
+  public boolean isErasureCoded() {
+    return flags.contains(Flags.HAS_EC);
   }
 
   /**
@@ -188,7 +149,7 @@ public class HdfsFileStatus {
    * @return true if the name is empty
    */
   public final boolean isEmptyLocalName() {
-    return path.length == 0;
+    return uPath.length == 0;
   }
 
   /**
@@ -196,7 +157,7 @@ public class HdfsFileStatus {
    * @return the local name in string
    */
   public final String getLocalName() {
-    return DFSUtilClient.bytes2String(path);
+    return DFSUtilClient.bytes2String(uPath);
   }
 
   /**
@@ -204,7 +165,7 @@ public class HdfsFileStatus {
    * @return the local name in java UTF8
    */
   public final byte[] getLocalNameInBytes() {
-    return path;
+    return uPath;
   }
 
   /**
@@ -238,16 +199,24 @@ public class HdfsFileStatus {
     return new Path(parent, getLocalName());
   }
 
-  /**
-   * Get the string representation of the symlink.
-   * @return the symlink as a string.
-   */
-  public final String getSymlink() {
-    return DFSUtilClient.bytes2String(symlink);
+  @Override
+  public Path getSymlink() throws IOException {
+    if (isSymlink()) {
+      return new Path(DFSUtilClient.bytes2String(uSymlink));
+    }
+    throw new IOException("Path " + getPath() + " is not a symbolic link");
+  }
+
+  @Override
+  public void setSymlink(Path sym) {
+    uSymlink = DFSUtilClient.string2Bytes(sym.toString());
   }
 
+  /**
+   * Opaque referant for the symlink, to be resolved at the client.
+   */
   public final byte[] getSymlinkInBytes() {
-    return symlink;
+    return uSymlink;
   }
 
   public final long getFileId() {
@@ -275,13 +244,30 @@ public class HdfsFileStatus {
     return storagePolicy;
   }
 
-  public final FileStatus makeQualified(URI defaultUri, Path path) {
-    return new FileStatus(getLen(), isDir(), getReplication(),
-        getBlockSize(), getModificationTime(),
-        getAccessTime(),
-        getPermission(), getOwner(), getGroup(),
-        isSymlink() ? new Path(getSymlink()) : null,
-        (getFullPath(path)).makeQualified(
-            defaultUri, null)); // fully-qualify path
+  @Override
+  public boolean equals(Object o) {
+    // satisfy findbugs
+    return super.equals(o);
+  }
+
+  @Override
+  public int hashCode() {
+    // satisfy findbugs
+    return super.hashCode();
+  }
+
+  /**
+   * Resolve the short name of the Path given the URI, parent provided. This
+   * FileStatus reference will not contain a valid Path until it is resolved
+   * by this method.
+   * @param defaultUri FileSystem to fully qualify HDFS path.
+   * @param parent Parent path of this element.
+   * @return Reference to this instance.
+   */
+  public final FileStatus makeQualified(URI defaultUri, Path parent) {
+    // fully-qualify path
+    setPath(getFullPath(parent).makeQualified(defaultUri, null));
+    return this; // API compatibility
   }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
index 0fd2039..b82a860 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.protocol;
 
 import java.net.URI;
+import java.util.EnumSet;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -34,7 +35,14 @@ import org.apache.hadoop.hdfs.DFSUtilClient;
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class HdfsLocatedFileStatus extends HdfsFileStatus {
-  private final LocatedBlocks locations;
+
+  private static final long serialVersionUID = 0x23c73328;
+
+  /**
+   * Left transient, because {@link #makeQualifiedLocated(URI,Path)}
+   * is the user-facing type.
+   */
+  private transient LocatedBlocks locations;
 
   /**
    * Constructor
@@ -56,12 +64,12 @@ public class HdfsLocatedFileStatus extends HdfsFileStatus {
    */
   public HdfsLocatedFileStatus(long length, boolean isdir,
       int block_replication, long blocksize, long modification_time,
-      long access_time, FsPermission permission, String owner, String group,
-      byte[] symlink, byte[] path, long fileId, LocatedBlocks locations,
-      int childrenNum, FileEncryptionInfo feInfo, byte storagePolicy,
-      ErasureCodingPolicy ecPolicy) {
+      long access_time, FsPermission permission, EnumSet<Flags> flags,
+      String owner, String group, byte[] symlink, byte[] path, long fileId,
+      LocatedBlocks locations, int childrenNum, FileEncryptionInfo feInfo,
+      byte storagePolicy, ErasureCodingPolicy ecPolicy) {
     super(length, isdir, block_replication, blocksize, modification_time,
-        access_time, permission, owner, group, symlink, path, fileId,
+        access_time, permission, flags, owner, group, symlink, path, fileId,
         childrenNum, feInfo, storagePolicy, ecPolicy);
     this.locations = locations;
   }
@@ -72,13 +80,21 @@ public class HdfsLocatedFileStatus extends HdfsFileStatus {
 
   public final LocatedFileStatus makeQualifiedLocated(URI defaultUri,
       Path path) {
-    return new LocatedFileStatus(getLen(), isDir(), getReplication(),
-        getBlockSize(), getModificationTime(),
-        getAccessTime(),
-        getPermission(), getOwner(), getGroup(),
-        isSymlink() ? new Path(getSymlink()) : null,
-        (getFullPath(path)).makeQualified(
-            defaultUri, null), // fully-qualify path
+    makeQualified(defaultUri, path);
+    return new LocatedFileStatus(this,
         DFSUtilClient.locatedBlocks2Locations(getBlockLocations()));
   }
+
+  @Override
+  public boolean equals(Object o) {
+    // satisfy findbugs
+    return super.equals(o);
+  }
+
+  @Override
+  public int hashCode() {
+    // satisfy findbugs
+    return super.hashCode();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
index 583d027..61e5316 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
@@ -21,6 +21,7 @@ import java.io.PrintStream;
 import java.text.SimpleDateFormat;
 import java.util.Comparator;
 import java.util.Date;
+import java.util.EnumSet;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -57,11 +58,12 @@ public class SnapshottableDirectoryStatus {
   private final byte[] parentFullPath;
 
   public SnapshottableDirectoryStatus(long modification_time, long access_time,
-      FsPermission permission, String owner, String group, byte[] localName,
-      long inodeId, int childrenNum,
-      int snapshotNumber, int snapshotQuota, byte[] parentFullPath) {
+      FsPermission permission, EnumSet<HdfsFileStatus.Flags> flags,
+      String owner, String group, byte[] localName, long inodeId,
+      int childrenNum, int snapshotNumber, int snapshotQuota,
+      byte[] parentFullPath) {
     this.dirStatus = new HdfsFileStatus(0, true, 0, 0, modification_time,
-        access_time, permission, owner, group, null, localName, inodeId,
+        access_time, permission, flags, owner, group, null, localName, inodeId,
         childrenNum, null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
         null);
     this.snapshotNumber = snapshotNumber;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
index feb3061..a0d2c5a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
@@ -104,6 +104,7 @@ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.AclEntrySco
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.AclEntryTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.FsActionProto;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclStatusProto;
+import org.apache.hadoop.hdfs.protocol.proto.AclProtos.FsPermissionProto;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockFlagProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveEntryProto;
@@ -149,7 +150,6 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto.Sto
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DirectoryListingProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ErasureCodingPolicyProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsPermissionProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsServerDefaultsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto.FileType;
@@ -1142,7 +1142,7 @@ public class PBHelperClient {
   }
 
   public static FsPermission convert(FsPermissionProto p) {
-    return new FsPermissionExtension((short)p.getPerm());
+    return new FsPermission((short)p.getPerm());
   }
 
   private static Event.CreateEvent.INodeType createTypeConvert(
@@ -1501,10 +1501,14 @@ public class PBHelperClient {
       return null;
     }
     final HdfsFileStatusProto status = sdirStatusProto.getDirStatus();
+    EnumSet<HdfsFileStatus.Flags> flags = status.hasFlags()
+        ? convertFlags(status.getFlags())
+        : convertFlags(status.getPermission());
     return new SnapshottableDirectoryStatus(
         status.getModificationTime(),
         status.getAccessTime(),
         convert(status.getPermission()),
+        flags,
         status.getOwner(),
         status.getGroup(),
         status.getPath().toByteArray(),
@@ -1546,17 +1550,23 @@ public class PBHelperClient {
   }
 
   public static FsPermissionProto convert(FsPermission p) {
-    return FsPermissionProto.newBuilder().setPerm(p.toExtendedShort()).build();
+    return FsPermissionProto.newBuilder().setPerm(p.toShort()).build();
   }
 
   public static HdfsFileStatus convert(HdfsFileStatusProto fs) {
-    if (fs == null)
+    if (fs == null) {
       return null;
+    }
+    EnumSet<HdfsFileStatus.Flags> flags = fs.hasFlags()
+        ? convertFlags(fs.getFlags())
+        : convertFlags(fs.getPermission());
     return new HdfsLocatedFileStatus(
         fs.getLength(), fs.getFileType().equals(FileType.IS_DIR),
         fs.getBlockReplication(), fs.getBlocksize(),
         fs.getModificationTime(), fs.getAccessTime(),
-        convert(fs.getPermission()), fs.getOwner(), fs.getGroup(),
+        convert(fs.getPermission()),
+        flags,
+        fs.getOwner(), fs.getGroup(),
         fs.getFileType().equals(FileType.IS_SYMLINK) ?
             fs.getSymlink().toByteArray() : null,
         fs.getPath().toByteArray(),
@@ -1569,6 +1579,47 @@ public class PBHelperClient {
         fs.hasEcPolicy() ? convertErasureCodingPolicy(fs.getEcPolicy()) : null);
   }
 
+  private static EnumSet<HdfsFileStatus.Flags> convertFlags(int flags) {
+    EnumSet<HdfsFileStatus.Flags> f =
+        EnumSet.noneOf(HdfsFileStatus.Flags.class);
+    for (HdfsFileStatusProto.Flags pbf : HdfsFileStatusProto.Flags.values()) {
+      if ((pbf.getNumber() & flags) != 0) {
+        switch (pbf) {
+        case HAS_ACL:
+          f.add(HdfsFileStatus.Flags.HAS_ACL);
+          break;
+        case HAS_CRYPT:
+          f.add(HdfsFileStatus.Flags.HAS_CRYPT);
+          break;
+        case HAS_EC:
+          f.add(HdfsFileStatus.Flags.HAS_EC);
+          break;
+        default:
+          // ignore unknown
+          break;
+        }
+      }
+    }
+    return f;
+  }
+
+  private static EnumSet<HdfsFileStatus.Flags> convertFlags(
+      FsPermissionProto pbp) {
+    EnumSet<HdfsFileStatus.Flags> f =
+        EnumSet.noneOf(HdfsFileStatus.Flags.class);
+    FsPermission p = new FsPermissionExtension((short)pbp.getPerm());
+    if (p.getAclBit()) {
+      f.add(HdfsFileStatus.Flags.HAS_ACL);
+    }
+    if (p.getEncryptedBit()) {
+      f.add(HdfsFileStatus.Flags.HAS_CRYPT);
+    }
+    if (p.getErasureCodedBit()) {
+      f.add(HdfsFileStatus.Flags.HAS_EC);
+    }
+    return f;
+  }
+
   public static CorruptFileBlocks convert(CorruptFileBlocksProto c) {
     if (c == null)
       return null;
@@ -2082,6 +2133,10 @@ public class PBHelperClient {
       builder.setEcPolicy(convertErasureCodingPolicy(
           fs.getErasureCodingPolicy()));
     }
+    int flags = fs.hasAcl()   ? HdfsFileStatusProto.Flags.HAS_ACL_VALUE   : 0;
+    flags |= fs.isEncrypted() ? HdfsFileStatusProto.Flags.HAS_CRYPT_VALUE : 0;
+    flags |= fs.isErasureCoded() ? HdfsFileStatusProto.Flags.HAS_EC_VALUE : 0;
+    builder.setFlags(flags);
     return builder.build();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
index 5e9396e..7ec5fe5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
@@ -41,7 +41,6 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.DatanodeInfoBuilder;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -61,6 +60,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
 
@@ -97,17 +97,8 @@ class JsonUtilClient {
   }
 
   /** Convert a string to a FsPermission object. */
-  static FsPermission toFsPermission(
-      final String s, Boolean aclBit, Boolean encBit, Boolean erasureBit) {
-    FsPermission perm = new FsPermission(Short.parseShort(s, 8));
-    final boolean aBit = (aclBit != null) ? aclBit : false;
-    final boolean eBit = (encBit != null) ? encBit : false;
-    final boolean ecBit = (erasureBit != null) ? erasureBit : false;
-    if (aBit || eBit || ecBit) {
-      return new FsPermissionExtension(perm, aBit, eBit, ecBit);
-    } else {
-      return perm;
-    }
+  static FsPermission toFsPermission(final String s) {
+    return null == s ? null : new FsPermission(Short.parseShort(s, 8));
   }
 
   /** Convert a Json map to a HdfsFileStatus object. */
@@ -128,10 +119,23 @@ class JsonUtilClient {
     final long len = ((Number) m.get("length")).longValue();
     final String owner = (String) m.get("owner");
     final String group = (String) m.get("group");
-    final FsPermission permission = toFsPermission((String) m.get("permission"),
-        (Boolean) m.get("aclBit"),
-        (Boolean) m.get("encBit"),
-        (Boolean) m.get("ecBit"));
+    final FsPermission permission = toFsPermission((String)m.get("permission"));
+
+    Boolean aclBit = (Boolean) m.get("aclBit");
+    Boolean encBit = (Boolean) m.get("encBit");
+    Boolean erasureBit  = (Boolean) m.get("ecBit");
+    EnumSet<HdfsFileStatus.Flags> f =
+        EnumSet.noneOf(HdfsFileStatus.Flags.class);
+    if (aclBit != null && aclBit) {
+      f.add(HdfsFileStatus.Flags.HAS_ACL);
+    }
+    if (encBit != null && encBit) {
+      f.add(HdfsFileStatus.Flags.HAS_CRYPT);
+    }
+    if (erasureBit != null && erasureBit) {
+      f.add(HdfsFileStatus.Flags.HAS_EC);
+    }
+
     final long aTime = ((Number) m.get("accessTime")).longValue();
     final long mTime = ((Number) m.get("modificationTime")).longValue();
     final long blockSize = ((Number) m.get("blockSize")).longValue();
@@ -143,11 +147,11 @@ class JsonUtilClient {
     final byte storagePolicy = m.containsKey("storagePolicy") ?
         (byte) ((Number) m.get("storagePolicy")).longValue() :
         HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
-    return new HdfsFileStatus(len, type == WebHdfsConstants.PathType.DIRECTORY,
-        replication, blockSize, mTime, aTime, permission, owner, group,
-        symlink, DFSUtilClient.string2Bytes(localName),
-        fileId, childrenNum, null,
-        storagePolicy, null);
+    return new HdfsFileStatus(len,
+        type == WebHdfsConstants.PathType.DIRECTORY, replication, blockSize,
+        mTime, aTime, permission, f, owner, group, symlink,
+        DFSUtilClient.string2Bytes(localName), fileId, childrenNum,
+        null, storagePolicy, null);
   }
 
   static HdfsFileStatus[] toHdfsFileStatusArray(final Map<?, ?> json) {
@@ -465,9 +469,7 @@ class JsonUtilClient {
     aclStatusBuilder.stickyBit((Boolean) m.get("stickyBit"));
     String permString = (String) m.get("permission");
     if (permString != null) {
-      final FsPermission permission = toFsPermission(permString,
-          (Boolean) m.get("aclBit"), (Boolean) m.get("encBit"),
-          (Boolean) m.get("ecBit"));
+      final FsPermission permission = toFsPermission(permString);
       aclStatusBuilder.setPermission(permission);
     }
     final List<?> entries = (List<?>) m.get("entries");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsConstants.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsConstants.java
index 50da899..f690dd0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsConstants.java
@@ -32,7 +32,13 @@ public class WebHdfsConstants {
     FILE, DIRECTORY, SYMLINK;
 
     static PathType valueOf(HdfsFileStatus status) {
-      return status.isDir()? DIRECTORY: status.isSymlink()? SYMLINK: FILE;
+      if (status.isDirectory()) {
+        return DIRECTORY;
+      }
+      if (status.isSymlink()) {
+        return SYMLINK;
+      }
+      return FILE;
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
index 3861cba..1159e50 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
@@ -1016,15 +1016,7 @@ public class WebHdfsFileSystem extends FileSystem
   public FileStatus getFileStatus(Path f) throws IOException {
     statistics.incrementReadOps(1);
     storageStatistics.incrementOpCounter(OpType.GET_FILE_STATUS);
-    return makeQualified(getHdfsFileStatus(f), f);
-  }
-
-  private FileStatus makeQualified(HdfsFileStatus f, Path parent) {
-    return new FileStatus(f.getLen(), f.isDir(), f.getReplication(),
-        f.getBlockSize(), f.getModificationTime(), f.getAccessTime(),
-        f.getPermission(), f.getOwner(), f.getGroup(),
-        f.isSymlink() ? new Path(f.getSymlink()) : null,
-        f.getFullPath(parent).makeQualified(getUri(), getWorkingDirectory()));
+    return getHdfsFileStatus(f).makeQualified(getUri(), f);
   }
 
   @Override
@@ -1507,6 +1499,7 @@ public class WebHdfsFileSystem extends FileSystem
     statistics.incrementReadOps(1);
     storageStatistics.incrementOpCounter(OpType.LIST_STATUS);
 
+    final URI fsUri = getUri();
     final HttpOpParam.Op op = GetOpParam.Op.LISTSTATUS;
     return new FsPathResponseRunner<FileStatus[]>(op, f) {
       @Override
@@ -1515,7 +1508,7 @@ public class WebHdfsFileSystem extends FileSystem
             JsonUtilClient.toHdfsFileStatusArray(json);
         final FileStatus[] statuses = new FileStatus[hdfsStatuses.length];
         for (int i = 0; i < hdfsStatuses.length; i++) {
-          statuses[i] = makeQualified(hdfsStatuses[i], f);
+          statuses[i] = hdfsStatuses[i].makeQualified(fsUri, f);
         }
 
         return statuses;
@@ -1541,10 +1534,11 @@ public class WebHdfsFileSystem extends FileSystem
       }
     }.run();
     // Qualify the returned FileStatus array
+    final URI fsUri = getUri();
     final HdfsFileStatus[] statuses = listing.getPartialListing();
     FileStatus[] qualified = new FileStatus[statuses.length];
     for (int i = 0; i < statuses.length; i++) {
-      qualified[i] = makeQualified(statuses[i], f);
+      qualified[i] = statuses[i].makeQualified(fsUri, f);
     }
     return new DirectoryEntries(qualified, listing.getLastName(),
         listing.hasMore());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/acl.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/acl.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/acl.proto
index bb7fdb0..c2529c9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/acl.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/acl.proto
@@ -21,7 +21,12 @@ option java_outer_classname = "AclProtos";
 option java_generate_equals_and_hash = true;
 package hadoop.hdfs;
 
-import "hdfs.proto";
+/**
+ * File or Directory permision - same spec as posix
+ */
+message FsPermissionProto {
+  required uint32 perm = 1;       // Actually a short - only 16bits used
+}
 
 message AclEntryProto {
   enum AclEntryScopeProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
index b306fcf..7109980 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
@@ -32,6 +32,7 @@ option java_generate_equals_and_hash = true;
 package hadoop.hdfs;
 
 import "Security.proto";
+import "acl.proto";
 
 /**
  * Extended block idenfies a block
@@ -197,13 +198,6 @@ message CorruptFileBlocksProto {
 }
 
 /**
- * File or Directory permision - same spec as posix
- */
-message FsPermissionProto {
-  required uint32 perm = 1;       // Actually a short - only 16bits used
-}
-
-/**
  * Types of recognized storage media.
  */
 enum StorageTypeProto {
@@ -388,6 +382,11 @@ message HdfsFileStatusProto {
     IS_FILE = 2;
     IS_SYMLINK = 3;
   }
+  enum Flags {
+    HAS_ACL   = 0x01; // has ACLs
+    HAS_CRYPT = 0x02; // encrypted
+    HAS_EC    = 0x04; // erasure coded
+  }
   required FileType fileType = 1;
   required bytes path = 2;          // local name of inode encoded java UTF8
   required uint64 length = 3;
@@ -415,6 +414,9 @@ message HdfsFileStatusProto {
 
   // Optional field for erasure coding
   optional ErasureCodingPolicyProto ecPolicy = 17;
+
+  // Set of flags
+  optional uint32 flags = 18 [default = 0];
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
index 1ab890f..d139100 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
@@ -1047,18 +1047,7 @@ public class HttpFSFileSystem extends FileSystem
   /** Convert a string to a FsPermission object. */
   static FsPermission toFsPermission(JSONObject json) {
     final String s = (String) json.get(PERMISSION_JSON);
-    final Boolean aclBit = (Boolean) json.get(ACL_BIT_JSON);
-    final Boolean encBit = (Boolean) json.get(ENC_BIT_JSON);
-    final Boolean erasureBit = (Boolean) json.get(EC_BIT_JSON);
-    FsPermission perm = new FsPermission(Short.parseShort(s, 8));
-    final boolean aBit = (aclBit != null) ? aclBit : false;
-    final boolean eBit = (encBit != null) ? encBit : false;
-    final boolean ecBit = (erasureBit != null) ? erasureBit : false;
-    if (aBit || eBit || ecBit) {
-      return new FsPermissionExtension(perm, aBit, eBit, ecBit);
-    } else {
-      return perm;
-    }
+    return new FsPermission(Short.parseShort(s, 8));
   }
 
   private FileStatus createFileStatus(Path parent, JSONObject json) {
@@ -1073,23 +1062,23 @@ public class HttpFSFileSystem extends FileSystem
     long mTime = (Long) json.get(MODIFICATION_TIME_JSON);
     long blockSize = (Long) json.get(BLOCK_SIZE_JSON);
     short replication = ((Long) json.get(REPLICATION_JSON)).shortValue();
-    FileStatus fileStatus = null;
-
-    switch (type) {
-      case FILE:
-      case DIRECTORY:
-        fileStatus = new FileStatus(len, (type == FILE_TYPE.DIRECTORY),
-                                    replication, blockSize, mTime, aTime,
-                                    permission, owner, group, path);
-        break;
-      case SYMLINK:
-        Path symLink = null;
-        fileStatus = new FileStatus(len, false,
-                                    replication, blockSize, mTime, aTime,
-                                    permission, owner, group, symLink,
-                                    path);
+
+    final Boolean aclBit = (Boolean) json.get(ACL_BIT_JSON);
+    final Boolean encBit = (Boolean) json.get(ENC_BIT_JSON);
+    final Boolean erasureBit = (Boolean) json.get(EC_BIT_JSON);
+    final boolean aBit = (aclBit != null) ? aclBit : false;
+    final boolean eBit = (encBit != null) ? encBit : false;
+    final boolean ecBit = (erasureBit != null) ? erasureBit : false;
+    if (aBit || eBit || ecBit) {
+      // include this for compatibility with 2.x
+      FsPermissionExtension deprecatedPerm =
+          new FsPermissionExtension(permission, aBit, eBit, ecBit);
+      return new FileStatus(len, FILE_TYPE.DIRECTORY == type,
+          replication, blockSize, mTime, aTime, deprecatedPerm, owner, group,
+          null, path, aBit, eBit, ecBit);
     }
-    return fileStatus;
+    return new FileStatus(len, FILE_TYPE.DIRECTORY == type,
+        replication, blockSize, mTime, aTime, permission, owner, group, path);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
index e23093e..ca11c66 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
@@ -852,10 +852,11 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     }
   }
 
-  private static void assertSameAclBit(FileSystem expected, FileSystem actual,
+  private static void assertSameAcls(FileSystem expected, FileSystem actual,
       Path path) throws IOException {
     FileStatus expectedFileStatus = expected.getFileStatus(path);
     FileStatus actualFileStatus = actual.getFileStatus(path);
+    assertEquals(actualFileStatus.hasAcl(), expectedFileStatus.hasAcl());
     assertEquals(actualFileStatus.getPermission().getAclBit(),
         expectedFileStatus.getPermission().getAclBit());
   }
@@ -888,31 +889,31 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     AclStatus proxyAclStat = proxyFs.getAclStatus(path);
     AclStatus httpfsAclStat = httpfs.getAclStatus(path);
     assertSameAcls(httpfsAclStat, proxyAclStat);
-    assertSameAclBit(httpfs, proxyFs, path);
+    assertSameAcls(httpfs, proxyFs, path);
 
     httpfs.setAcl(path, AclEntry.parseAclSpec(aclSet,true));
     proxyAclStat = proxyFs.getAclStatus(path);
     httpfsAclStat = httpfs.getAclStatus(path);
     assertSameAcls(httpfsAclStat, proxyAclStat);
-    assertSameAclBit(httpfs, proxyFs, path);
+    assertSameAcls(httpfs, proxyFs, path);
 
     httpfs.modifyAclEntries(path, AclEntry.parseAclSpec(aclUser2, true));
     proxyAclStat = proxyFs.getAclStatus(path);
     httpfsAclStat = httpfs.getAclStatus(path);
     assertSameAcls(httpfsAclStat, proxyAclStat);
-    assertSameAclBit(httpfs, proxyFs, path);
+    assertSameAcls(httpfs, proxyFs, path);
 
     httpfs.removeAclEntries(path, AclEntry.parseAclSpec(rmAclUser1, false));
     proxyAclStat = proxyFs.getAclStatus(path);
     httpfsAclStat = httpfs.getAclStatus(path);
     assertSameAcls(httpfsAclStat, proxyAclStat);
-    assertSameAclBit(httpfs, proxyFs, path);
+    assertSameAcls(httpfs, proxyFs, path);
 
     httpfs.removeAcl(path);
     proxyAclStat = proxyFs.getAclStatus(path);
     httpfsAclStat = httpfs.getAclStatus(path);
     assertSameAcls(httpfsAclStat, proxyAclStat);
-    assertSameAclBit(httpfs, proxyFs, path);
+    assertSameAcls(httpfs, proxyFs, path);
   }
 
   /**
@@ -935,21 +936,21 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     AclStatus proxyAclStat = proxyFs.getAclStatus(dir);
     AclStatus httpfsAclStat = httpfs.getAclStatus(dir);
     assertSameAcls(httpfsAclStat, proxyAclStat);
-    assertSameAclBit(httpfs, proxyFs, dir);
+    assertSameAcls(httpfs, proxyFs, dir);
 
     /* Set a default ACL on the directory */
     httpfs.setAcl(dir, (AclEntry.parseAclSpec(defUser1,true)));
     proxyAclStat = proxyFs.getAclStatus(dir);
     httpfsAclStat = httpfs.getAclStatus(dir);
     assertSameAcls(httpfsAclStat, proxyAclStat);
-    assertSameAclBit(httpfs, proxyFs, dir);
+    assertSameAcls(httpfs, proxyFs, dir);
 
     /* Remove the default ACL */
     httpfs.removeDefaultAcl(dir);
     proxyAclStat = proxyFs.getAclStatus(dir);
     httpfsAclStat = httpfs.getAclStatus(dir);
     assertSameAcls(httpfsAclStat, proxyAclStat);
-    assertSameAclBit(httpfs, proxyFs, dir);
+    assertSameAcls(httpfs, proxyFs, dir);
   }
 
   private void testEncryption() throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
index be54efb..2a7824a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
@@ -252,4 +252,16 @@
         <Class name="org.apache.hadoop.hdfs.server.datanode.checker.AbstractFuture" />
         <Bug pattern="NS_DANGEROUS_NON_SHORT_CIRCUIT" />
     </Match>
+    <Match>
+        <Class name="org.apache.hadoop.hdfs.server.namenode.NNUpgradeUtil$1" />
+        <Method name="visitFile" />
+        <Bug pattern="NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE" />
+    </Match>
+    <!-- HdfsFileStatus is user-facing, but HdfsLocatedFileStatus is not.
+         Defensible compatibility choices over time create odd corners. -->
+    <Match>
+        <Class name="org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus" />
+        <Field name="locations" />
+        <Bug pattern="SE_TRANSIENT_FIELD_NOT_RESTORED" />
+    </Match>
  </FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotInfo.java
index 3ddfc85..676e827 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotInfo.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.hdfs.protocol;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsPermissionProto;
+import org.apache.hadoop.hdfs.protocol.proto.AclProtos.FsPermissionProto;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e44e7b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/package-info.java
new file mode 100644
index 0000000..6233024
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/package-info.java
@@ -0,0 +1,18 @@
+/**
+ * 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.protocolPB;
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[48/51] [abbrv] hadoop git commit: YARN-6515. Fix warnings from Spotbugs in hadoop-yarn-server-nodemanager. Contributed by Naganarasimha G R.

Posted by st...@apache.org.
YARN-6515. Fix warnings from Spotbugs in hadoop-yarn-server-nodemanager. Contributed by Naganarasimha G R.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/1a18d5e5
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1a18d5e5
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1a18d5e5

Branch: refs/heads/HADOOP-13345
Commit: 1a18d5e514d13aa3a88e9b6089394a27296d6bc3
Parents: 8a4bff0
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Aug 9 21:56:34 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed Aug 9 21:56:43 2017 +0900

----------------------------------------------------------------------
 .../server/nodemanager/NodeStatusUpdaterImpl.java    | 11 +++++------
 .../localizer/ContainerLocalizer.java                | 15 ++++++++-------
 .../containermanager/monitor/ContainerMetrics.java   |  2 +-
 3 files changed, 14 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a18d5e5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
index 00073d8..b5ec383 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
@@ -639,7 +639,6 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
   public void removeOrTrackCompletedContainersFromContext(
       List<ContainerId> containerIds) throws IOException {
     Set<ContainerId> removedContainers = new HashSet<ContainerId>();
-    Set<ContainerId> removedNullContainers = new HashSet<ContainerId>();
 
     pendingContainersToRemove.addAll(containerIds);
     Iterator<ContainerId> iter = pendingContainersToRemove.iterator();
@@ -649,7 +648,6 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
       Container nmContainer = context.getContainers().get(containerId);
       if (nmContainer == null) {
         iter.remove();
-        removedNullContainers.add(containerId);
       } else if (nmContainer.getContainerState().equals(
         org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState.DONE)) {
         context.getContainers().remove(containerId);
@@ -712,11 +710,12 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
   public void removeVeryOldStoppedContainersFromCache() {
     synchronized (recentlyStoppedContainers) {
       long currentTime = System.currentTimeMillis();
-      Iterator<ContainerId> i =
-          recentlyStoppedContainers.keySet().iterator();
+      Iterator<Entry<ContainerId, Long>> i =
+          recentlyStoppedContainers.entrySet().iterator();
       while (i.hasNext()) {
-        ContainerId cid = i.next();
-        if (recentlyStoppedContainers.get(cid) < currentTime) {
+        Entry<ContainerId, Long> mapEntry = i.next();
+        ContainerId cid = mapEntry.getKey();
+        if (mapEntry.getValue() < currentTime) {
           if (!context.getContainers().containsKey(cid)) {
             i.remove();
             try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a18d5e5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java
index 8a46491..bb4b7f3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java
@@ -17,6 +17,8 @@
 */
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer;
 
+import static org.apache.hadoop.util.Shell.getAllShells;
+
 import java.io.DataInputStream;
 import java.io.File;
 import java.io.IOException;
@@ -30,6 +32,7 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 import java.util.Stack;
 import java.util.concurrent.Callable;
@@ -81,8 +84,6 @@ import org.apache.hadoop.yarn.util.FSDownload;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
-import static org.apache.hadoop.util.Shell.getAllShells;
-
 public class ContainerLocalizer {
 
   static final Log LOG = LogFactory.getLog(ContainerLocalizer.class);
@@ -348,13 +349,13 @@ public class ContainerLocalizer {
     final List<LocalResourceStatus> currentResources =
       new ArrayList<LocalResourceStatus>();
     // TODO: Synchronization??
-    for (Iterator<LocalResource> i = pendingResources.keySet().iterator();
-         i.hasNext();) {
-      LocalResource rsrc = i.next();
+    for (Iterator<Entry<LocalResource, Future<Path>>> i =
+        pendingResources.entrySet().iterator(); i.hasNext();) {
+      Entry<LocalResource, Future<Path>> mapEntry = i.next();
       LocalResourceStatus stat =
         recordFactory.newRecordInstance(LocalResourceStatus.class);
-      stat.setResource(rsrc);
-      Future<Path> fPath = pendingResources.get(rsrc);
+      stat.setResource(mapEntry.getKey());
+      Future<Path> fPath = mapEntry.getValue();
       if (fPath.isDone()) {
         try {
           Path localPath = fPath.get();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a18d5e5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainerMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainerMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainerMetrics.java
index 07b3dea..a6aa337 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainerMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainerMetrics.java
@@ -130,7 +130,7 @@ public class ContainerMetrics implements MetricsSource {
   /**
    * Simple metrics cache to help prevent re-registrations.
    */
-  protected final static Map<ContainerId, ContainerMetrics>
+  private final static Map<ContainerId, ContainerMetrics>
       usageMetrics = new HashMap<>();
   // Create a timer to unregister container metrics,
   // whose associated thread run as a daemon.


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[33/51] [abbrv] hadoop git commit: YARN-6961. Remove commons-logging dependency from hadoop-yarn-server-applicationhistoryservice module. Contributed by Yeliang Cang.

Posted by st...@apache.org.
YARN-6961. Remove commons-logging dependency from hadoop-yarn-server-applicationhistoryservice module. Contributed by Yeliang Cang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/98912950
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/98912950
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/98912950

Branch: refs/heads/HADOOP-13345
Commit: 98912950b6167523f6238a90ce69da817db91308
Parents: 55a181f
Author: Akira Ajisaka <aa...@apache.org>
Authored: Tue Aug 8 19:38:58 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Tue Aug 8 19:38:58 2017 +0900

----------------------------------------------------------------------
 .../hadoop-yarn-server-applicationhistoryservice/pom.xml         | 4 ----
 1 file changed, 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/98912950/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml
index d732af4..cace493 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml
@@ -131,10 +131,6 @@
       <groupId>com.google.guava</groupId>
       <artifactId>guava</artifactId>
     </dependency>
-    <dependency>
-      <groupId>commons-logging</groupId>
-      <artifactId>commons-logging</artifactId>
-    </dependency>
 
     <!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
     <dependency>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[30/51] [abbrv] hadoop git commit: YARN-6955. Handle concurrent register AM requests in FederationInterceptor. (Botong Huang via Subru).

Posted by st...@apache.org.
YARN-6955. Handle concurrent register AM requests in FederationInterceptor. (Botong Huang via Subru).


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c61f2c41
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c61f2c41
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c61f2c41

Branch: refs/heads/HADOOP-13345
Commit: c61f2c419830e40bb47fb2b1fe1f7d6109ed29a9
Parents: bc20680
Author: Subru Krishnan <su...@apache.org>
Authored: Mon Aug 7 16:58:29 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Aug 7 16:58:29 2017 -0700

----------------------------------------------------------------------
 .../dev-support/findbugs-exclude.xml            |  4 +-
 .../yarn/server/MockResourceManagerFacade.java  | 18 ++--
 .../amrmproxy/FederationInterceptor.java        | 43 ++++------
 .../amrmproxy/TestFederationInterceptor.java    | 88 ++++++++++++++++++--
 4 files changed, 110 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61f2c41/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
index 034f03c..6825a36 100644
--- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
@@ -594,11 +594,9 @@
     <Bug pattern="UL_UNRELEASED_LOCK_EXCEPTION_PATH" />
   </Match>
 
-  <!-- Ignore false alert for RCN_REDUNDANT_NULLCHECK_OF_NULL_VALUE -->
   <Match>
     <Class name="org.apache.hadoop.yarn.server.nodemanager.amrmproxy.FederationInterceptor" />
-    <Method name="registerApplicationMaster" />
-    <Bug pattern="RCN_REDUNDANT_NULLCHECK_OF_NULL_VALUE" />
+    <Bug pattern="IS2_INCONSISTENT_SYNC" />
   </Match>
 
 </FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61f2c41/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
index 68c55ac..e33d7e1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
@@ -246,6 +246,16 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
 
     shouldReRegisterNext = false;
 
+    synchronized (applicationContainerIdMap) {
+      if (applicationContainerIdMap.containsKey(amrmToken)) {
+        throw new InvalidApplicationMasterRequestException(
+            AMRMClientUtils.APP_ALREADY_REGISTERED_MESSAGE);
+      }
+      // Keep track of the containers that are returned to this application
+      applicationContainerIdMap.put(amrmToken, new ArrayList<ContainerId>());
+    }
+
+    // Make sure we wait for certain test cases last in the method
     synchronized (syncObj) {
       syncObj.notifyAll();
       // We reuse the port number to indicate whether the unit test want us to
@@ -261,14 +271,6 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
       }
     }
 
-    synchronized (applicationContainerIdMap) {
-      if (applicationContainerIdMap.containsKey(amrmToken)) {
-        throw new InvalidApplicationMasterRequestException(
-            AMRMClientUtils.APP_ALREADY_REGISTERED_MESSAGE);
-      }
-      // Keep track of the containers that are returned to this application
-      applicationContainerIdMap.put(amrmToken, new ArrayList<ContainerId>());
-    }
     return RegisterApplicationMasterResponse.newInstance(null, null, null, null,
         null, request.getHost(), null);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61f2c41/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
index ffe47f4..28724aa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
@@ -208,22 +208,25 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
    * requests from AM because of timeout between AM and AMRMProxy, which is
    * shorter than the timeout + failOver between FederationInterceptor
    * (AMRMProxy) and RM.
+   *
+   * For the same reason, this method needs to be synchronized.
    */
   @Override
-  public RegisterApplicationMasterResponse registerApplicationMaster(
-      RegisterApplicationMasterRequest request)
-      throws YarnException, IOException {
+  public synchronized RegisterApplicationMasterResponse
+      registerApplicationMaster(RegisterApplicationMasterRequest request)
+          throws YarnException, IOException {
     // If AM is calling with a different request, complain
-    if (this.amRegistrationRequest != null
-        && !this.amRegistrationRequest.equals(request)) {
-      throw new YarnException("A different request body recieved. AM should"
-          + " not call registerApplicationMaster with different request body");
+    if (this.amRegistrationRequest != null) {
+      if (!this.amRegistrationRequest.equals(request)) {
+        throw new YarnException("AM should not call "
+            + "registerApplicationMaster with a different request body");
+      }
+    } else {
+      // Save the registration request. This will be used for registering with
+      // secondary sub-clusters using UAMs, as well as re-register later
+      this.amRegistrationRequest = request;
     }
 
-    // Save the registration request. This will be used for registering with
-    // secondary sub-clusters using UAMs, as well as re-register later
-    this.amRegistrationRequest = request;
-
     /*
      * Present to AM as if we are the RM that never fails over. When actual RM
      * fails over, we always re-register automatically.
@@ -245,22 +248,8 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
      * is running and will breaks the elasticity feature. The registration with
      * the other sub-cluster RM will be done lazily as needed later.
      */
-    try {
-      this.amRegistrationResponse =
-          this.homeRM.registerApplicationMaster(request);
-    } catch (InvalidApplicationMasterRequestException e) {
-      if (e.getMessage()
-          .contains(AMRMClientUtils.APP_ALREADY_REGISTERED_MESSAGE)) {
-        // Some other register thread might have succeeded in the meantime
-        if (this.amRegistrationResponse != null) {
-          LOG.info("Other concurrent thread registered successfully, "
-              + "simply return the same success register response");
-          return this.amRegistrationResponse;
-        }
-      }
-      // This is a real issue, throw back to AM
-      throw e;
-    }
+    this.amRegistrationResponse =
+        this.homeRM.registerApplicationMaster(request);
 
     // the queue this application belongs will be used for getting
     // AMRMProxy policy from state store.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61f2c41/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptor.java
index 4e15323..34b0741 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptor.java
@@ -21,6 +21,11 @@ package org.apache.hadoop.yarn.server.nodemanager.amrmproxy;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
 
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
@@ -36,6 +41,7 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.InvalidApplicationMasterRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.MockResourceManagerFacade;
 import org.apache.hadoop.yarn.server.federation.policies.manager.UniformBroadcastPolicyManager;
 import org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterRequest;
@@ -234,7 +240,7 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
     RegisterApplicationMasterRequest registerReq =
         Records.newRecord(RegisterApplicationMasterRequest.class);
     registerReq.setHost(Integer.toString(testAppId));
-    registerReq.setRpcPort(testAppId);
+    registerReq.setRpcPort(0);
     registerReq.setTrackingUrl("");
 
     RegisterApplicationMasterResponse registerResponse =
@@ -298,7 +304,7 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
     RegisterApplicationMasterRequest registerReq =
         Records.newRecord(RegisterApplicationMasterRequest.class);
     registerReq.setHost(Integer.toString(testAppId));
-    registerReq.setRpcPort(testAppId);
+    registerReq.setRpcPort(0);
     registerReq.setTrackingUrl("");
 
     RegisterApplicationMasterResponse registerResponse =
@@ -338,6 +344,78 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
     Assert.assertEquals(true, finshResponse.getIsUnregistered());
   }
 
+  /*
+   * Test concurrent register threads. This is possible because the timeout
+   * between AM and AMRMProxy is shorter than the timeout + failOver between
+   * FederationInterceptor (AMRMProxy) and RM. When first call is blocked due to
+   * RM failover and AM timeout, it will call us resulting in a second register
+   * thread.
+   */
+  @Test(timeout = 5000)
+  public void testConcurrentRegister()
+      throws InterruptedException, ExecutionException {
+    ExecutorService threadpool = Executors.newCachedThreadPool();
+    ExecutorCompletionService<RegisterApplicationMasterResponse> compSvc =
+        new ExecutorCompletionService<>(threadpool);
+
+    Object syncObj = MockResourceManagerFacade.getSyncObj();
+
+    // Two register threads
+    synchronized (syncObj) {
+      // Make sure first thread will block within RM, before the second thread
+      // starts
+      LOG.info("Starting first register thread");
+      compSvc.submit(new ConcurrentRegisterAMCallable());
+
+      try {
+        LOG.info("Test main starts waiting for the first thread to block");
+        syncObj.wait();
+        LOG.info("Test main wait finished");
+      } catch (Exception e) {
+        LOG.info("Test main wait interrupted", e);
+      }
+    }
+
+    // The second thread will get already registered exception from RM.
+    LOG.info("Starting second register thread");
+    compSvc.submit(new ConcurrentRegisterAMCallable());
+
+    // Notify the first register thread to return
+    LOG.info("Let first blocked register thread move on");
+    synchronized (syncObj) {
+      syncObj.notifyAll();
+    }
+
+    // Both thread should return without exception
+    RegisterApplicationMasterResponse response = compSvc.take().get();
+    Assert.assertNotNull(response);
+
+    response = compSvc.take().get();
+    Assert.assertNotNull(response);
+
+    threadpool.shutdown();
+  }
+
+  /**
+   * A callable that calls registerAM to RM with blocking.
+   */
+  public class ConcurrentRegisterAMCallable
+      implements Callable<RegisterApplicationMasterResponse> {
+    @Override
+    public RegisterApplicationMasterResponse call() throws Exception {
+      RegisterApplicationMasterResponse response = null;
+      try {
+        // Use port number 1001 to let mock RM block in the register call
+        response = interceptor.registerApplicationMaster(
+            RegisterApplicationMasterRequest.newInstance(null, 1001, null));
+      } catch (Exception e) {
+        LOG.info("Register thread exception", e);
+        response = null;
+      }
+      return response;
+    }
+  }
+
   @Test
   public void testRequestInterceptorChainCreation() throws Exception {
     RequestInterceptor root =
@@ -381,7 +459,7 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
     RegisterApplicationMasterRequest registerReq =
         Records.newRecord(RegisterApplicationMasterRequest.class);
     registerReq.setHost(Integer.toString(testAppId));
-    registerReq.setRpcPort(testAppId);
+    registerReq.setRpcPort(0);
     registerReq.setTrackingUrl("");
 
     for (int i = 0; i < 2; i++) {
@@ -397,7 +475,7 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
     RegisterApplicationMasterRequest registerReq =
         Records.newRecord(RegisterApplicationMasterRequest.class);
     registerReq.setHost(Integer.toString(testAppId));
-    registerReq.setRpcPort(testAppId);
+    registerReq.setRpcPort(0);
     registerReq.setTrackingUrl("");
 
     RegisterApplicationMasterResponse registerResponse =
@@ -407,7 +485,7 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
     // Register the application second time with a different request obj
     registerReq = Records.newRecord(RegisterApplicationMasterRequest.class);
     registerReq.setHost(Integer.toString(testAppId));
-    registerReq.setRpcPort(testAppId);
+    registerReq.setRpcPort(0);
     registerReq.setTrackingUrl("different");
     try {
       registerResponse = interceptor.registerApplicationMaster(registerReq);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[26/51] [abbrv] hadoop git commit: HDFS-12306. Add audit log for some erasure coding operations. Contributed by Huafeng Wang

Posted by st...@apache.org.
HDFS-12306. Add audit log for some erasure coding operations. Contributed by Huafeng Wang


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/0b674360
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/0b674360
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/0b674360

Branch: refs/heads/HADOOP-13345
Commit: 0b67436068899497e99c86f37fd4887ca188fae2
Parents: b0fbf17
Author: Kai Zheng <ka...@intel.com>
Authored: Mon Aug 7 19:30:10 2017 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Mon Aug 7 19:30:10 2017 +0800

----------------------------------------------------------------------
 .../hdfs/server/namenode/FSNamesystem.java      | 48 ++++++++++++--------
 .../hdfs/server/namenode/NameNodeRpcServer.java |  2 +-
 2 files changed, 29 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b674360/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 229de05..b1639b2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -7055,18 +7055,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       resultingStat = FSDirErasureCodingOp.setErasureCodingPolicy(this,
           srcArg, ecPolicyName, pc, logRetryCache);
       success = true;
-    } catch (AccessControlException ace) {
-      logAuditEvent(success, operationName, srcArg, null,
-          resultingStat);
-      throw ace;
     } finally {
       writeUnlock(operationName);
       if (success) {
         getEditLog().logSync();
       }
+      logAuditEvent(success, operationName, srcArg, null, resultingStat);
     }
-    logAuditEvent(success, operationName, srcArg, null,
-        resultingStat);
   }
 
   /**
@@ -7074,9 +7069,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @param policies The policies to add.
    * @return The according result of add operation.
    */
-  AddECPolicyResponse[] addECPolicies(ErasureCodingPolicy[] policies)
+  AddECPolicyResponse[] addErasureCodingPolicies(ErasureCodingPolicy[] policies)
       throws IOException {
-    final String operationName = "addECPolicies";
+    final String operationName = "addErasureCodingPolicies";
     String addECPolicyName = "";
     checkOperation(OperationCategory.WRITE);
     List<AddECPolicyResponse> responses = new ArrayList<>();
@@ -7201,18 +7196,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       resultingStat = FSDirErasureCodingOp.unsetErasureCodingPolicy(this,
           srcArg, pc, logRetryCache);
       success = true;
-    } catch (AccessControlException ace) {
-      logAuditEvent(success, operationName, srcArg, null,
-          resultingStat);
-      throw ace;
     } finally {
       writeUnlock(operationName);
       if (success) {
         getEditLog().logSync();
       }
+      logAuditEvent(success, operationName, srcArg, null, resultingStat);
     }
-    logAuditEvent(success, operationName, srcArg, null,
-        resultingStat);
   }
 
   /**
@@ -7220,14 +7210,20 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   ErasureCodingPolicy getErasureCodingPolicy(String src)
       throws AccessControlException, UnresolvedLinkException, IOException {
+    final String operationName = "getErasureCodingPolicy";
+    boolean success = false;
     checkOperation(OperationCategory.READ);
     FSPermissionChecker pc = getPermissionChecker();
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      return FSDirErasureCodingOp.getErasureCodingPolicy(this, src, pc);
+      final ErasureCodingPolicy ret =
+          FSDirErasureCodingOp.getErasureCodingPolicy(this, src, pc);
+      success = true;
+      return ret;
     } finally {
-      readUnlock("getErasureCodingPolicy");
+      readUnlock(operationName);
+      logAuditEvent(success, operationName, null);
     }
   }
 
@@ -7235,13 +7231,19 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * Get available erasure coding polices
    */
   ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException {
+    final String operationName = "getErasureCodingPolicies";
+    boolean success = false;
     checkOperation(OperationCategory.READ);
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      return FSDirErasureCodingOp.getErasureCodingPolicies(this);
+      final ErasureCodingPolicy[] ret =
+          FSDirErasureCodingOp.getErasureCodingPolicies(this);
+      success = true;
+      return ret;
     } finally {
-      readUnlock("getErasureCodingPolicies");
+      readUnlock(operationName);
+      logAuditEvent(success, operationName, null);
     }
   }
 
@@ -7249,13 +7251,19 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * Get available erasure coding codecs and corresponding coders.
    */
   HashMap<String, String> getErasureCodingCodecs() throws IOException {
+    final String operationName = "getErasureCodingCodecs";
+    boolean success = false;
     checkOperation(OperationCategory.READ);
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      return FSDirErasureCodingOp.getErasureCodingCodecs(this);
+      final HashMap<String, String> ret =
+          FSDirErasureCodingOp.getErasureCodingCodecs(this);
+      success = true;
+      return ret;
     } finally {
-      readUnlock("getErasureCodingCodecs");
+      readUnlock(operationName);
+      logAuditEvent(success, operationName, null);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b674360/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 52b422c..9265381 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -2298,7 +2298,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
       ErasureCodingPolicy[] policies) throws IOException {
     checkNNStartup();
     namesystem.checkSuperuserPrivilege();
-    return namesystem.addECPolicies(policies);
+    return namesystem.addErasureCodingPolicies(policies);
   }
 
   @Override


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[27/51] [abbrv] hadoop git commit: HADOOP-14727. Socket not closed properly when reading Configurations with BlockReaderRemote. Contributed by Jonathan Eagles.

Posted by st...@apache.org.
HADOOP-14727. Socket not closed properly when reading Configurations with BlockReaderRemote. Contributed by Jonathan Eagles.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a3a9c976
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a3a9c976
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a3a9c976

Branch: refs/heads/HADOOP-13345
Commit: a3a9c976c3cfa3ab6b0936eb8cf0889891bd0678
Parents: 0b67436
Author: Xiao Chen <xi...@apache.org>
Authored: Fri Aug 4 20:53:45 2017 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Mon Aug 7 10:25:52 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/conf/Configuration.java   | 15 ++++++++++-----
 .../org/apache/hadoop/conf/TestConfiguration.java    |  6 ++++--
 2 files changed, 14 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3a9c976/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
index e26d3a8..65e8569 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.conf;
 
+import com.ctc.wstx.io.StreamBootstrapper;
+import com.ctc.wstx.io.SystemId;
 import com.ctc.wstx.stax.WstxInputFactory;
 import com.fasterxml.jackson.core.JsonFactory;
 import com.fasterxml.jackson.core.JsonGenerator;
@@ -94,7 +96,6 @@ import org.apache.hadoop.security.alias.CredentialProviderFactory;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringInterner;
 import org.apache.hadoop.util.StringUtils;
-import org.codehaus.stax2.XMLInputFactory2;
 import org.codehaus.stax2.XMLStreamReader2;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -285,7 +286,8 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
    * Specify exact input factory to avoid time finding correct one.
    * Factory is reusable across un-synchronized threads once initialized
    */
-  private static final XMLInputFactory2 XML_INPUT_FACTORY = new WstxInputFactory();
+  private static final WstxInputFactory XML_INPUT_FACTORY =
+      new WstxInputFactory();
 
   /**
    * Class to keep the information about the keys which replace the deprecated
@@ -2647,15 +2649,18 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     return parse(connection.getInputStream(), url.toString());
   }
 
-  private XMLStreamReader parse(InputStream is,
-      String systemId) throws IOException, XMLStreamException {
+  private XMLStreamReader parse(InputStream is, String systemIdStr)
+      throws IOException, XMLStreamException {
     if (!quietmode) {
       LOG.debug("parsing input stream " + is);
     }
     if (is == null) {
       return null;
     }
-    return XML_INPUT_FACTORY.createXMLStreamReader(systemId, is);
+    SystemId systemId = SystemId.construct(systemIdStr);
+    return XML_INPUT_FACTORY.createSR(XML_INPUT_FACTORY.createPrivateConfig(),
+        systemId, StreamBootstrapper.getInstance(null, systemId, is), false,
+        true);
   }
 
   private void loadResources(Properties properties,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3a9c976/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
index 2af61c0..92d3290 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
@@ -155,11 +155,13 @@ public class TestConfiguration extends TestCase {
     startConfig();
     declareProperty("prop", "A", "A");
     endConfig();
-    
-    InputStream in1 = new ByteArrayInputStream(writer.toString().getBytes());
+
+    InputStream in1 = Mockito.spy(new ByteArrayInputStream(
+          writer.toString().getBytes()));
     Configuration conf = new Configuration(false);
     conf.addResource(in1);
     assertEquals("A", conf.get("prop"));
+    Mockito.verify(in1, Mockito.times(1)).close();
     InputStream in2 = new ByteArrayInputStream(writer.toString().getBytes());
     conf.addResource(in2);
     assertEquals("A", conf.get("prop"));


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[13/51] [abbrv] hadoop git commit: HADOOP-14706. Adding a helper method to determine whether a log is Log4j implement.

Posted by st...@apache.org.
HADOOP-14706. Adding a helper method to determine whether a log is Log4j implement.

This closes #257

Signed-off-by: Akira Ajisaka <aa...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/691bf5ec
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/691bf5ec
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/691bf5ec

Branch: refs/heads/HADOOP-13345
Commit: 691bf5ec5d641c0552f4b243c4efa34ec2e529dc
Parents: 35dc782
Author: wenxinhe <we...@gmail.com>
Authored: Tue Aug 1 16:19:00 2017 +0800
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Fri Aug 4 14:35:57 2017 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/pom.xml          |  2 +-
 .../java/org/apache/hadoop/util/GenericsUtil.java    | 15 +++++++++++++++
 .../org/apache/hadoop/util/TestGenericsUtil.java     |  5 +++++
 3 files changed, 21 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/691bf5ec/hadoop-common-project/hadoop-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml
index ad6f734..0cd5479 100644
--- a/hadoop-common-project/hadoop-common/pom.xml
+++ b/hadoop-common-project/hadoop-common/pom.xml
@@ -179,7 +179,7 @@
     <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-log4j12</artifactId>
-      <scope>runtime</scope>
+      <scope>compile</scope>
     </dependency>
     <dependency>
       <groupId>org.mockito</groupId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/691bf5ec/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericsUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericsUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericsUtil.java
index 611fff9..a53096a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericsUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericsUtil.java
@@ -23,6 +23,9 @@ import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.impl.Log4jLoggerAdapter;
 
 /**
  * Contains utility methods for dealing with Java Generics. 
@@ -72,4 +75,16 @@ public class GenericsUtil {
     return toArray(getClass(list.get(0)), list);
   }
 
+  /**
+   * Determine whether the log of <code>clazz</code> is Log4j implementation.
+   * @param clazz a class to be determined
+   * @return true if the log of <code>clazz</code> is Log4j implementation.
+   */
+  public static boolean isLog4jLogger(Class<?> clazz) {
+    if (clazz == null) {
+      return false;
+    }
+    Logger log = LoggerFactory.getLogger(clazz);
+    return log instanceof Log4jLoggerAdapter;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/691bf5ec/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericsUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericsUtil.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericsUtil.java
index 25e2ce9..58537ad 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericsUtil.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericsUtil.java
@@ -131,4 +131,9 @@ public class TestGenericsUtil extends TestCase {
             GenericClass.class, c2);
   }
 
+  public void testIsLog4jLogger() throws Exception {
+    assertFalse("False if clazz is null", GenericsUtil.isLog4jLogger(null));
+    assertTrue("The implementation is Log4j",
+        GenericsUtil.isLog4jLogger(TestGenericsUtil.class));
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[31/51] [abbrv] hadoop git commit: YARN-6920. Fix resource leak that happens during container re-initialization. (asuresh)

Posted by st...@apache.org.
YARN-6920. Fix resource leak that happens during container re-initialization. (asuresh)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8d3fd819
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8d3fd819
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8d3fd819

Branch: refs/heads/HADOOP-13345
Commit: 8d3fd81980275fa81e7a5539b1751f38a63b6911
Parents: c61f2c4
Author: Arun Suresh <as...@apache.org>
Authored: Mon Aug 7 18:59:25 2017 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Mon Aug 7 18:59:25 2017 -0700

----------------------------------------------------------------------
 .../yarn/client/api/impl/TestNMClient.java      | 37 +++++++++-----------
 .../container/ContainerImpl.java                |  4 +++
 .../scheduler/ContainerScheduler.java           |  4 +++
 .../containermanager/TestContainerManager.java  |  9 +++++
 4 files changed, 34 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3fd819/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java
index 1034f7e..6bd0816 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java
@@ -398,6 +398,8 @@ public class TestNMClient {
               "will be Rolled-back", Arrays.asList(new Integer[] {-1000}));
           testCommitContainer(container.getId(), true);
           testReInitializeContainer(container.getId(), clc, false);
+          testGetContainerStatus(container, i, ContainerState.RUNNING,
+              "will be Re-initialized", Arrays.asList(new Integer[] {-1000}));
           testCommitContainer(container.getId(), false);
         } else {
           testReInitializeContainer(container.getId(), clc, true);
@@ -449,24 +451,21 @@ public class TestNMClient {
       ContainerState state, String diagnostics, List<Integer> exitStatuses)
           throws YarnException, IOException {
     while (true) {
-      try {
-        ContainerStatus status = nmClient.getContainerStatus(
-            container.getId(), container.getNodeId());
-        // NodeManager may still need some time to get the stable
-        // container status
-        if (status.getState() == state) {
-          assertEquals(container.getId(), status.getContainerId());
-          assertTrue("" + index + ": " + status.getDiagnostics(),
-              status.getDiagnostics().contains(diagnostics));
-          
-          assertTrue("Exit Statuses are supposed to be in: " + exitStatuses +
-              ", but the actual exit status code is: " + status.getExitStatus(),
-              exitStatuses.contains(status.getExitStatus()));
-          break;
-        }
-        Thread.sleep(100);
-      } catch (InterruptedException e) {
-        e.printStackTrace();
+      sleep(250);
+      ContainerStatus status = nmClient.getContainerStatus(
+          container.getId(), container.getNodeId());
+      // NodeManager may still need some time to get the stable
+      // container status
+      if (status.getState() == state) {
+        assertEquals(container.getId(), status.getContainerId());
+        assertTrue("" + index + ": " + status.getDiagnostics(),
+            status.getDiagnostics().contains(diagnostics));
+
+        assertTrue("Exit Statuses are supposed to be in: " + exitStatuses +
+                ", but the actual exit status code is: " +
+                status.getExitStatus(),
+            exitStatuses.contains(status.getExitStatus()));
+        break;
       }
     }
   }
@@ -559,9 +558,7 @@ public class TestNMClient {
       ContainerLaunchContext clc, boolean autoCommit)
       throws YarnException, IOException {
     try {
-      sleep(250);
       nmClient.reInitializeContainer(containerId, clc, autoCommit);
-      sleep(250);
     } catch (YarnException e) {
       // NM container will only be in SCHEDULED state, so expect the increase
       // action to fail.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3fd819/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
index 46f8fa0..c0aa6b0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
@@ -1397,6 +1397,10 @@ public class ContainerImpl implements Container {
       container.resourceSet =
           container.reInitContext.mergedResourceSet(container.resourceSet);
       container.isMarkeForKilling = false;
+      // Ensure Resources are decremented.
+      container.dispatcher.getEventHandler().handle(
+          new ContainerSchedulerEvent(container,
+          ContainerSchedulerEventType.CONTAINER_COMPLETED));
       container.sendScheduleEvent();
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3fd819/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java
index c119bf2..60d6213 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java
@@ -466,4 +466,8 @@ public class ContainerScheduler extends AbstractService implements
     return this.context.getContainerManager().getContainersMonitor();
   }
 
+  @VisibleForTesting
+  public ResourceUtilization getCurrentUtilization() {
+    return this.utilizationTracker.getCurrentUtilization();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3fd819/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
index f2d2037..24d46b6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
@@ -74,6 +74,7 @@ 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.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceUtilization;
 import org.apache.hadoop.yarn.api.records.SerializedException;
 import org.apache.hadoop.yarn.api.records.SignalContainerCommand;
 import org.apache.hadoop.yarn.api.records.Token;
@@ -437,7 +438,15 @@ public class TestContainerManager extends BaseContainerManagerTest {
 
     File newStartFile = new File(tmpDir, "start_file_n.txt").getAbsoluteFile();
 
+    ResourceUtilization beforeUpgrade =
+        ResourceUtilization.newInstance(
+            containerManager.getContainerScheduler().getCurrentUtilization());
     prepareContainerUpgrade(autoCommit, false, false, cId, newStartFile);
+    ResourceUtilization afterUpgrade =
+        ResourceUtilization.newInstance(
+            containerManager.getContainerScheduler().getCurrentUtilization());
+    Assert.assertEquals("Possible resource leak detected !!",
+        beforeUpgrade, afterUpgrade);
 
     // Assert that the First process is not alive anymore
     Assert.assertFalse("Process is still alive!",


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[06/51] [abbrv] hadoop git commit: YARN-6674 Add memory cgroup settings for opportunistic containers. (Miklos Szegedi via Haibo Chen)

Posted by st...@apache.org.
YARN-6674 Add memory cgroup settings for opportunistic containers.  (Miklos Szegedi via Haibo Chen)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c5d256c7
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c5d256c7
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c5d256c7

Branch: refs/heads/HADOOP-13345
Commit: c5d256c7607d3fe80db881c07ad445de467ab24b
Parents: f64cfea
Author: Haibo Chen <ha...@apache.org>
Authored: Thu Aug 3 09:52:35 2017 -0700
Committer: Haibo Chen <ha...@apache.org>
Committed: Thu Aug 3 09:52:35 2017 -0700

----------------------------------------------------------------------
 .../CGroupsMemoryResourceHandlerImpl.java       | 35 +++++++++++----
 .../TestCGroupsMemoryResourceHandlerImpl.java   | 45 ++++++++++++++++----
 2 files changed, 63 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5d256c7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsMemoryResourceHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsMemoryResourceHandlerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsMemoryResourceHandlerImpl.java
index b4d2a9a..d159aad 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsMemoryResourceHandlerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsMemoryResourceHandlerImpl.java
@@ -25,7 +25,9 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
 
@@ -46,6 +48,8 @@ public class CGroupsMemoryResourceHandlerImpl implements MemoryResourceHandler {
       CGroupsMemoryResourceHandlerImpl.class);
   private static final CGroupsHandler.CGroupController MEMORY =
       CGroupsHandler.CGroupController.MEMORY;
+  private static final int OPPORTUNISTIC_SWAPPINESS = 100;
+  private static final int OPPORTUNISTIC_SOFT_LIMIT = 0;
 
   private CGroupsHandler cGroupsHandler;
   private int swappiness = 0;
@@ -85,13 +89,15 @@ public class CGroupsMemoryResourceHandlerImpl implements MemoryResourceHandler {
               + ". Value must be between 0 and 100.");
     }
     float softLimitPerc = conf.getFloat(
-      YarnConfiguration.NM_MEMORY_RESOURCE_CGROUPS_SOFT_LIMIT_PERCENTAGE,
-      YarnConfiguration.DEFAULT_NM_MEMORY_RESOURCE_CGROUPS_SOFT_LIMIT_PERCENTAGE);
+        YarnConfiguration.NM_MEMORY_RESOURCE_CGROUPS_SOFT_LIMIT_PERCENTAGE,
+        YarnConfiguration.
+            DEFAULT_NM_MEMORY_RESOURCE_CGROUPS_SOFT_LIMIT_PERCENTAGE);
     softLimit = softLimitPerc / 100.0f;
     if (softLimitPerc < 0.0f || softLimitPerc > 100.0f) {
       throw new ResourceHandlerException(
           "Illegal value '" + softLimitPerc + "' "
-              + YarnConfiguration.NM_MEMORY_RESOURCE_CGROUPS_SOFT_LIMIT_PERCENTAGE
+              + YarnConfiguration.
+                NM_MEMORY_RESOURCE_CGROUPS_SOFT_LIMIT_PERCENTAGE
               + ". Value must be between 0 and 100.");
     }
     return null;
@@ -122,12 +128,23 @@ public class CGroupsMemoryResourceHandlerImpl implements MemoryResourceHandler {
       cGroupsHandler.updateCGroupParam(MEMORY, cgroupId,
           CGroupsHandler.CGROUP_PARAM_MEMORY_HARD_LIMIT_BYTES,
           String.valueOf(containerHardLimit) + "M");
-      cGroupsHandler.updateCGroupParam(MEMORY, cgroupId,
-          CGroupsHandler.CGROUP_PARAM_MEMORY_SOFT_LIMIT_BYTES,
-          String.valueOf(containerSoftLimit) + "M");
-      cGroupsHandler.updateCGroupParam(MEMORY, cgroupId,
-          CGroupsHandler.CGROUP_PARAM_MEMORY_SWAPPINESS,
-          String.valueOf(swappiness));
+      ContainerTokenIdentifier id = container.getContainerTokenIdentifier();
+      if (id != null && id.getExecutionType() ==
+          ExecutionType.OPPORTUNISTIC) {
+        cGroupsHandler.updateCGroupParam(MEMORY, cgroupId,
+            CGroupsHandler.CGROUP_PARAM_MEMORY_SOFT_LIMIT_BYTES,
+            String.valueOf(OPPORTUNISTIC_SOFT_LIMIT) + "M");
+        cGroupsHandler.updateCGroupParam(MEMORY, cgroupId,
+            CGroupsHandler.CGROUP_PARAM_MEMORY_SWAPPINESS,
+            String.valueOf(OPPORTUNISTIC_SWAPPINESS));
+      } else {
+        cGroupsHandler.updateCGroupParam(MEMORY, cgroupId,
+            CGroupsHandler.CGROUP_PARAM_MEMORY_SOFT_LIMIT_BYTES,
+            String.valueOf(containerSoftLimit) + "M");
+        cGroupsHandler.updateCGroupParam(MEMORY, cgroupId,
+            CGroupsHandler.CGROUP_PARAM_MEMORY_SWAPPINESS,
+            String.valueOf(swappiness));
+      }
     } catch (ResourceHandlerException re) {
       cGroupsHandler.deleteCGroup(MEMORY, cgroupId);
       LOG.warn("Could not update cgroup for container", re);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5d256c7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsMemoryResourceHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsMemoryResourceHandlerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsMemoryResourceHandlerImpl.java
index 180e134..8fd5a9d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsMemoryResourceHandlerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsMemoryResourceHandlerImpl.java
@@ -20,8 +20,10 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resourc
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
 import org.junit.Before;
@@ -32,6 +34,9 @@ import java.util.List;
 
 import static org.mockito.Mockito.*;
 
+/**
+ * Unit test for CGroupsMemoryResourceHandlerImpl.
+ */
 public class TestCGroupsMemoryResourceHandlerImpl {
 
   private CGroupsHandler mockCGroupsHandler;
@@ -60,8 +65,7 @@ public class TestCGroupsMemoryResourceHandlerImpl {
     try {
       cGroupsMemoryResourceHandler.bootstrap(conf);
       Assert.fail("Pmem check should not be allowed to run with cgroups");
-    }
-    catch(ResourceHandlerException re) {
+    } catch(ResourceHandlerException re) {
       // do nothing
     }
     conf.setBoolean(YarnConfiguration.NM_PMEM_CHECK_ENABLED, false);
@@ -69,8 +73,7 @@ public class TestCGroupsMemoryResourceHandlerImpl {
     try {
       cGroupsMemoryResourceHandler.bootstrap(conf);
       Assert.fail("Vmem check should not be allowed to run with cgroups");
-    }
-    catch(ResourceHandlerException re) {
+    } catch(ResourceHandlerException re) {
       // do nothing
     }
   }
@@ -84,8 +87,7 @@ public class TestCGroupsMemoryResourceHandlerImpl {
     try {
       cGroupsMemoryResourceHandler.bootstrap(conf);
       Assert.fail("Negative values for swappiness should not be allowed.");
-    }
-    catch (ResourceHandlerException re) {
+    } catch (ResourceHandlerException re) {
       // do nothing
     }
     try {
@@ -93,8 +95,7 @@ public class TestCGroupsMemoryResourceHandlerImpl {
       cGroupsMemoryResourceHandler.bootstrap(conf);
       Assert.fail("Values greater than 100 for swappiness"
           + " should not be allowed.");
-    }
-    catch (ResourceHandlerException re) {
+    } catch (ResourceHandlerException re) {
       // do nothing
     }
     conf.setInt(YarnConfiguration.NM_MEMORY_RESOURCE_CGROUPS_SWAPPINESS, 60);
@@ -169,4 +170,32 @@ public class TestCGroupsMemoryResourceHandlerImpl {
   public void testTeardown() throws Exception {
     Assert.assertNull(cGroupsMemoryResourceHandler.teardown());
   }
+
+  @Test
+  public void testOpportunistic() throws Exception {
+    Configuration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.NM_PMEM_CHECK_ENABLED, false);
+    conf.setBoolean(YarnConfiguration.NM_VMEM_CHECK_ENABLED, false);
+
+    cGroupsMemoryResourceHandler.bootstrap(conf);
+    ContainerTokenIdentifier tokenId = mock(ContainerTokenIdentifier.class);
+    when(tokenId.getExecutionType()).thenReturn(ExecutionType.OPPORTUNISTIC);
+    Container container = mock(Container.class);
+    String id = "container_01_01";
+    ContainerId mockContainerId = mock(ContainerId.class);
+    when(mockContainerId.toString()).thenReturn(id);
+    when(container.getContainerId()).thenReturn(mockContainerId);
+    when(container.getContainerTokenIdentifier()).thenReturn(tokenId);
+    when(container.getResource()).thenReturn(Resource.newInstance(1024, 2));
+    cGroupsMemoryResourceHandler.preStart(container);
+    verify(mockCGroupsHandler, times(1))
+        .updateCGroupParam(CGroupsHandler.CGroupController.MEMORY, id,
+            CGroupsHandler.CGROUP_PARAM_MEMORY_SOFT_LIMIT_BYTES, "0M");
+    verify(mockCGroupsHandler, times(1))
+        .updateCGroupParam(CGroupsHandler.CGroupController.MEMORY, id,
+            CGroupsHandler.CGROUP_PARAM_MEMORY_SWAPPINESS, "100");
+    verify(mockCGroupsHandler, times(1))
+        .updateCGroupParam(CGroupsHandler.CGroupController.MEMORY, id,
+            CGroupsHandler.CGROUP_PARAM_MEMORY_HARD_LIMIT_BYTES, "1024M");
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[21/51] [abbrv] hadoop git commit: HADOOP-14722. Azure: BlockBlobInputStream position incorrect after seek. Contributed by Thomas Marquardt

Posted by st...@apache.org.
HADOOP-14722. Azure: BlockBlobInputStream position incorrect after seek.
Contributed by Thomas Marquardt


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d91b7a84
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d91b7a84
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d91b7a84

Branch: refs/heads/HADOOP-13345
Commit: d91b7a8451489f97bdde928cea774764155cfe03
Parents: 024c3ec
Author: Steve Loughran <st...@apache.org>
Authored: Sun Aug 6 20:19:23 2017 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Sun Aug 6 20:19:23 2017 +0100

----------------------------------------------------------------------
 .../hadoop/fs/azure/BlockBlobInputStream.java   | 91 +++++++++++++++-----
 .../fs/azure/TestBlockBlobInputStream.java      | 85 ++++++++++++++++--
 2 files changed, 150 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d91b7a84/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobInputStream.java
index 5542415..c37b2be 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobInputStream.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobInputStream.java
@@ -43,11 +43,16 @@ final class BlockBlobInputStream extends InputStream implements Seekable {
   private InputStream blobInputStream = null;
   private int minimumReadSizeInBytes = 0;
   private long streamPositionAfterLastRead = -1;
+  // position of next network read within stream
   private long streamPosition = 0;
+  // length of stream
   private long streamLength = 0;
   private boolean closed = false;
+  // internal buffer, re-used for performance optimization
   private byte[] streamBuffer;
+  // zero-based offset within streamBuffer of current read position
   private int streamBufferPosition;
+  // length of data written to streamBuffer, streamBuffer may be larger
   private int streamBufferLength;
 
   /**
@@ -82,6 +87,16 @@ final class BlockBlobInputStream extends InputStream implements Seekable {
   }
 
   /**
+   * Reset the internal stream buffer but do not release the memory.
+   * The buffer can be reused to avoid frequent memory allocations of
+   * a large buffer.
+   */
+  private void resetStreamBuffer() {
+    streamBufferPosition = 0;
+    streamBufferLength = 0;
+  }
+
+  /**
    * Gets the read position of the stream.
    * @return the zero-based byte offset of the read position.
    * @throws IOException IO failure
@@ -89,7 +104,9 @@ final class BlockBlobInputStream extends InputStream implements Seekable {
   @Override
   public synchronized long getPos() throws IOException {
     checkState();
-    return streamPosition;
+    return (streamBuffer != null)
+        ? streamPosition - streamBufferLength + streamBufferPosition
+        : streamPosition;
   }
 
   /**
@@ -107,21 +124,39 @@ final class BlockBlobInputStream extends InputStream implements Seekable {
       throw new EOFException(
           FSExceptionMessages.CANNOT_SEEK_PAST_EOF + " " + pos);
     }
-    if (pos == getPos()) {
+
+    // calculate offset between the target and current position in the stream
+    long offset = pos - getPos();
+
+    if (offset == 0) {
       // no=op, no state change
       return;
     }
 
+    if (offset > 0) {
+      // forward seek, data can be skipped as an optimization
+      if (skip(offset) != offset) {
+        throw new EOFException(FSExceptionMessages.EOF_IN_READ_FULLY);
+      }
+      return;
+    }
+
+    // reverse seek, offset is negative
     if (streamBuffer != null) {
-      long offset = streamPosition - pos;
-      if (offset > 0 && offset < streamBufferLength) {
-        streamBufferPosition = streamBufferLength - (int) offset;
+      if (streamBufferPosition + offset >= 0) {
+        // target position is inside the stream buffer,
+        // only need to move backwards within the stream buffer
+        streamBufferPosition += offset;
       } else {
-        streamBufferPosition = streamBufferLength;
+        // target position is outside the stream buffer,
+        // need to reset stream buffer and move position for next network read
+        resetStreamBuffer();
+        streamPosition = pos;
       }
+    } else {
+      streamPosition = pos;
     }
 
-    streamPosition = pos;
     // close BlobInputStream after seek is invoked because BlobInputStream
     // does not support seek
     closeBlobInputStream();
@@ -189,8 +224,7 @@ final class BlockBlobInputStream extends InputStream implements Seekable {
         streamBuffer = new byte[(int) Math.min(minimumReadSizeInBytes,
             streamLength)];
       }
-      streamBufferPosition = 0;
-      streamBufferLength = 0;
+      resetStreamBuffer();
       outputStream = new MemoryOutputStream(streamBuffer, streamBufferPosition,
           streamBuffer.length);
       needToCopy = true;
@@ -295,27 +329,44 @@ final class BlockBlobInputStream extends InputStream implements Seekable {
    * @param n the number of bytes to be skipped.
    * @return the actual number of bytes skipped.
    * @throws IOException IO failure
+   * @throws IndexOutOfBoundsException if n is negative or if the sum of n
+   * and the current value of getPos() is greater than the length of the stream.
    */
   @Override
   public synchronized long skip(long n) throws IOException {
     checkState();
 
     if (blobInputStream != null) {
-      return blobInputStream.skip(n);
-    } else {
-      if (n < 0 || streamPosition + n > streamLength) {
-        throw new IndexOutOfBoundsException("skip range");
-      }
+      // blobInput stream is open; delegate the work to it
+      long skipped = blobInputStream.skip(n);
+      // update position to the actual skip value
+      streamPosition += skipped;
+      return skipped;
+    }
 
-      if (streamBuffer != null) {
-        streamBufferPosition = (n < streamBufferLength - streamBufferPosition)
-            ? streamBufferPosition + (int) n
-            : streamBufferLength;
-      }
+    // no blob stream; implement the skip logic directly
+    if (n < 0 || n > streamLength - getPos()) {
+      throw new IndexOutOfBoundsException("skip range");
+    }
 
+    if (streamBuffer != null) {
+      // there's a buffer, so seek with it
+      if (n < streamBufferLength - streamBufferPosition) {
+        // new range is in the buffer, so just update the buffer position
+        // skip within the buffer.
+        streamBufferPosition += (int) n;
+      } else {
+        // skip is out of range, so move position to ne value and reset
+        // the buffer ready for the next read()
+        streamPosition = getPos() + n;
+        resetStreamBuffer();
+      }
+    } else {
+      // no stream buffer; increment the stream position ready for
+      // the next triggered connection & read
       streamPosition += n;
-      return n;
     }
+    return n;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d91b7a84/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlockBlobInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlockBlobInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlockBlobInputStream.java
index 2453584..0ae4012 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlockBlobInputStream.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlockBlobInputStream.java
@@ -155,7 +155,7 @@ public class TestBlockBlobInputStream extends AbstractWasbTestBase {
     }
 
     LOG.info("Creating test file {} of size: {}", TEST_FILE_PATH,
-        TEST_FILE_SIZE );
+        TEST_FILE_SIZE);
     ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
 
     try(FSDataOutputStream outputStream = fs.create(TEST_FILE_PATH)) {
@@ -198,7 +198,7 @@ public class TestBlockBlobInputStream extends AbstractWasbTestBase {
   }
 
   @Test
-  public void test_0200_BasicReadTestV2() throws Exception {
+  public void test_0200_BasicReadTest() throws Exception {
     assumeHugeFileExists();
 
     try (
@@ -214,12 +214,12 @@ public class TestBlockBlobInputStream extends AbstractWasbTestBase {
       // v1 forward seek and read a kilobyte into first kilobyte of bufferV1
       inputStreamV1.seek(5 * MEGABYTE);
       int numBytesReadV1 = inputStreamV1.read(bufferV1, 0, KILOBYTE);
-      assertEquals(numBytesReadV1, KILOBYTE);
+      assertEquals(KILOBYTE, numBytesReadV1);
 
       // v2 forward seek and read a kilobyte into first kilobyte of bufferV2
       inputStreamV2.seek(5 * MEGABYTE);
       int numBytesReadV2 = inputStreamV2.read(bufferV2, 0, KILOBYTE);
-      assertEquals(numBytesReadV2, KILOBYTE);
+      assertEquals(KILOBYTE, numBytesReadV2);
 
       assertArrayEquals(bufferV1, bufferV2);
 
@@ -229,17 +229,90 @@ public class TestBlockBlobInputStream extends AbstractWasbTestBase {
       // v1 reverse seek and read a megabyte into last megabyte of bufferV1
       inputStreamV1.seek(3 * MEGABYTE);
       numBytesReadV1 = inputStreamV1.read(bufferV1, offset, len);
-      assertEquals(numBytesReadV1, len);
+      assertEquals(len, numBytesReadV1);
 
       // v2 reverse seek and read a megabyte into last megabyte of bufferV2
       inputStreamV2.seek(3 * MEGABYTE);
       numBytesReadV2 = inputStreamV2.read(bufferV2, offset, len);
-      assertEquals(numBytesReadV2, len);
+      assertEquals(len, numBytesReadV2);
 
       assertArrayEquals(bufferV1, bufferV2);
     }
   }
 
+  @Test
+  public void test_0201_RandomReadTest() throws Exception {
+    assumeHugeFileExists();
+
+    try (
+        FSDataInputStream inputStreamV1
+            = accountUsingInputStreamV1.getFileSystem().open(TEST_FILE_PATH);
+
+        FSDataInputStream inputStreamV2
+            = accountUsingInputStreamV2.getFileSystem().open(TEST_FILE_PATH);
+    ) {
+      final int bufferSize = 4 * KILOBYTE;
+      byte[] bufferV1 = new byte[bufferSize];
+      byte[] bufferV2 = new byte[bufferV1.length];
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+
+      inputStreamV1.seek(0);
+      inputStreamV2.seek(0);
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+
+      int seekPosition = 2 * KILOBYTE;
+      inputStreamV1.seek(seekPosition);
+      inputStreamV2.seek(seekPosition);
+
+      inputStreamV1.seek(0);
+      inputStreamV2.seek(0);
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+
+      seekPosition = 5 * KILOBYTE;
+      inputStreamV1.seek(seekPosition);
+      inputStreamV2.seek(seekPosition);
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+
+      seekPosition = 10 * KILOBYTE;
+      inputStreamV1.seek(seekPosition);
+      inputStreamV2.seek(seekPosition);
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+
+      seekPosition = 4100 * KILOBYTE;
+      inputStreamV1.seek(seekPosition);
+      inputStreamV2.seek(seekPosition);
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+    }
+  }
+
+  private void verifyConsistentReads(FSDataInputStream inputStreamV1,
+      FSDataInputStream inputStreamV2,
+      byte[] bufferV1,
+      byte[] bufferV2) throws IOException {
+    int size = bufferV1.length;
+    final int numBytesReadV1 = inputStreamV1.read(bufferV1, 0, size);
+    assertEquals("Bytes read from V1 stream", size, numBytesReadV1);
+
+    final int numBytesReadV2 = inputStreamV2.read(bufferV2, 0, size);
+    assertEquals("Bytes read from V2 stream", size, numBytesReadV2);
+
+    assertArrayEquals("Mismatch in read data", bufferV1, bufferV2);
+  }
+
   /**
    * Validates the implementation of InputStream.markSupported.
    * @throws IOException


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[51/51] [abbrv] hadoop git commit: Merge branch 'trunk' into HADOOP-13345

Posted by st...@apache.org.
Merge branch 'trunk' into HADOOP-13345

 Conflicts:
	hadoop-tools/hadoop-aws/pom.xml


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b114f248
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b114f248
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b114f248

Branch: refs/heads/HADOOP-13345
Commit: b114f2488fc1a298d9c993af6ca3575249e6f4d3
Parents: b4c2ab2 63cfcb9
Author: Steve Loughran <st...@apache.org>
Authored: Wed Aug 9 18:23:39 2017 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Aug 9 18:23:39 2017 +0100

----------------------------------------------------------------------
 LICENSE.txt                                     |     1 +
 dev-support/docker/hadoop_env_checks.sh         |     2 +-
 dev-support/findHangingTest.sh                  |     2 +-
 dev-support/verify-xml.sh                       |     2 +-
 .../resources/assemblies/hadoop-yarn-dist.xml   |     1 +
 .../hadoop-client-check-invariants/pom.xml      |     1 -
 .../hadoop-client-check-test-invariants/pom.xml |     1 -
 .../hadoop-client-minicluster/pom.xml           |    43 +
 hadoop-client-modules/hadoop-client/pom.xml     |     4 -
 .../util/TestCertificateUtil.java               |     6 +-
 .../dev-support/findbugsExcludeFile.xml         |     4 +
 hadoop-common-project/hadoop-common/pom.xml     |     7 +-
 .../hadoop-common/src/main/bin/hadoop           |    28 +-
 .../src/main/bin/hadoop-functions.sh            |   161 +-
 .../org/apache/hadoop/conf/Configuration.java   |    63 +-
 .../apache/hadoop/conf/ReconfigurableBase.java  |     7 +-
 .../hadoop/conf/ReconfigurationServlet.java     |     8 +-
 .../hadoop/crypto/JceAesCtrCryptoCodec.java     |     8 +-
 .../hadoop/crypto/OpensslAesCtrCryptoCodec.java |     8 +-
 .../org/apache/hadoop/crypto/OpensslCipher.java |     8 +-
 .../crypto/key/kms/KMSClientProvider.java       |    39 +-
 .../key/kms/LoadBalancingKMSClientProvider.java |    90 +-
 .../crypto/random/OpensslSecureRandom.java      |     8 +-
 .../hadoop/crypto/random/OsSecureRandom.java    |     9 +-
 .../apache/hadoop/fs/AbstractFileSystem.java    |     6 +-
 .../java/org/apache/hadoop/fs/ChecksumFs.java   |     8 +-
 .../fs/CommonConfigurationKeysPublic.java       |    31 +
 .../src/main/java/org/apache/hadoop/fs/DF.java  |     9 +-
 .../hadoop/fs/DelegationTokenRenewer.java       |    10 +-
 .../hadoop/fs/FSDataOutputStreamBuilder.java    |   161 +-
 .../org/apache/hadoop/fs/FSInputChecker.java    |     9 +-
 .../java/org/apache/hadoop/fs/FileContext.java  |    10 +-
 .../apache/hadoop/fs/FileEncryptionInfo.java    |     6 +-
 .../java/org/apache/hadoop/fs/FileStatus.java   |   172 +-
 .../java/org/apache/hadoop/fs/FileSystem.java   |    24 +-
 .../java/org/apache/hadoop/fs/FileUtil.java     |    10 +-
 .../main/java/org/apache/hadoop/fs/FsShell.java |     6 +-
 .../apache/hadoop/fs/FsShellPermissions.java    |     4 +-
 .../org/apache/hadoop/fs/FsUrlConnection.java   |    10 +
 .../hadoop/fs/FsUrlStreamHandlerFactory.java    |    26 +-
 .../main/java/org/apache/hadoop/fs/Globber.java |     7 +-
 .../org/apache/hadoop/fs/HarFileSystem.java     |    11 +-
 .../org/apache/hadoop/fs/LocalDirAllocator.java |     9 +-
 .../org/apache/hadoop/fs/LocatedFileStatus.java |    64 +-
 .../main/java/org/apache/hadoop/fs/Trash.java   |     7 +-
 .../apache/hadoop/fs/TrashPolicyDefault.java    |     8 +-
 .../org/apache/hadoop/fs/ftp/FTPFileSystem.java |     8 +-
 .../hadoop/fs/permission/FsPermission.java      |    18 +-
 .../apache/hadoop/fs/protocolPB/PBHelper.java   |   131 +
 .../hadoop/fs/protocolPB/package-info.java      |    18 +
 .../hadoop/fs/sftp/SFTPConnectionPool.java      |     7 +-
 .../apache/hadoop/fs/sftp/SFTPFileSystem.java   |     7 +-
 .../org/apache/hadoop/fs/shell/Command.java     |     6 +-
 .../apache/hadoop/fs/shell/CopyCommands.java    |   112 +-
 .../apache/hadoop/fs/shell/MoveCommands.java    |     4 +-
 .../java/org/apache/hadoop/fs/shell/Stat.java   |    19 +-
 .../org/apache/hadoop/fs/viewfs/Constants.java  |     2 +
 .../apache/hadoop/fs/viewfs/ViewFileSystem.java |    79 +-
 .../org/apache/hadoop/fs/viewfs/ViewFs.java     |    43 +-
 .../apache/hadoop/ha/ActiveStandbyElector.java  |    15 +-
 .../apache/hadoop/ha/FailoverController.java    |    10 +-
 .../main/java/org/apache/hadoop/ha/HAAdmin.java |     8 +-
 .../org/apache/hadoop/ha/HealthMonitor.java     |     8 +-
 .../java/org/apache/hadoop/ha/NodeFencer.java   |     6 +-
 .../org/apache/hadoop/ha/PowerShellFencer.java  |     7 +-
 .../apache/hadoop/ha/ShellCommandFencer.java    |     7 +-
 .../org/apache/hadoop/ha/SshFenceByTcpPort.java |    18 +-
 .../java/org/apache/hadoop/ha/StreamPumper.java |     8 +-
 .../apache/hadoop/ha/ZKFailoverController.java  |    20 +-
 ...HAServiceProtocolServerSideTranslatorPB.java |     6 +-
 .../org/apache/hadoop/http/HttpServer2.java     |     7 +-
 .../hadoop/http/lib/StaticUserWebFilter.java    |     7 +-
 .../java/org/apache/hadoop/io/BloomMapFile.java |     6 +-
 .../apache/hadoop/io/FastByteComparisons.java   |     7 +-
 .../main/java/org/apache/hadoop/io/IOUtils.java |    10 +-
 .../main/java/org/apache/hadoop/io/MapFile.java |     8 +-
 .../org/apache/hadoop/io/ReadaheadPool.java     |     6 +-
 .../java/org/apache/hadoop/io/SequenceFile.java |     7 +-
 .../main/java/org/apache/hadoop/io/UTF8.java    |     5 +-
 .../apache/hadoop/io/compress/CodecPool.java    |     6 +-
 .../io/compress/CompressionCodecFactory.java    |     8 +-
 .../apache/hadoop/io/compress/DefaultCodec.java |     6 +-
 .../io/compress/bzip2/Bzip2Compressor.java      |     8 +-
 .../io/compress/bzip2/Bzip2Decompressor.java    |     8 +-
 .../hadoop/io/compress/bzip2/Bzip2Factory.java  |     6 +-
 .../hadoop/io/compress/lz4/Lz4Compressor.java   |     8 +-
 .../hadoop/io/compress/lz4/Lz4Decompressor.java |     8 +-
 .../io/compress/snappy/SnappyCompressor.java    |     8 +-
 .../io/compress/snappy/SnappyDecompressor.java  |     8 +-
 .../io/compress/zlib/BuiltInZlibDeflater.java   |     8 +-
 .../hadoop/io/compress/zlib/ZlibCompressor.java |     8 +-
 .../hadoop/io/compress/zlib/ZlibFactory.java    |     8 +-
 .../apache/hadoop/io/erasurecode/CodecUtil.java |    20 +-
 .../apache/hadoop/io/erasurecode/ECSchema.java  |     6 +-
 .../io/erasurecode/ErasureCodeNative.java       |    13 +-
 .../org/apache/hadoop/io/file/tfile/BCFile.java |     6 +-
 .../hadoop/io/file/tfile/Compression.java       |     6 +-
 .../org/apache/hadoop/io/file/tfile/TFile.java  |     8 +-
 .../hadoop/io/file/tfile/TFileDumper.java       |     8 +-
 .../org/apache/hadoop/io/nativeio/NativeIO.java |    16 +-
 .../nativeio/SharedFileDescriptorFactory.java   |     7 +-
 .../hadoop/io/retry/RetryInvocationHandler.java |    11 +
 .../apache/hadoop/io/retry/RetryPolicies.java   |     6 +-
 .../org/apache/hadoop/io/retry/RetryUtils.java  |     6 +-
 .../io/serializer/SerializationFactory.java     |     8 +-
 .../org/apache/hadoop/ipc/CallQueueManager.java |     7 +-
 .../main/java/org/apache/hadoop/ipc/Client.java |    40 +-
 .../org/apache/hadoop/ipc/FairCallQueue.java    |     6 +-
 .../apache/hadoop/ipc/ProtobufRpcEngine.java    |     7 +-
 .../main/java/org/apache/hadoop/ipc/RPC.java    |     6 +-
 .../org/apache/hadoop/ipc/RefreshRegistry.java  |     7 +-
 .../java/org/apache/hadoop/ipc/RetryCache.java  |     6 +-
 .../main/java/org/apache/hadoop/ipc/Server.java |    16 +-
 .../ipc/WeightedRoundRobinMultiplexer.java      |     8 +-
 .../apache/hadoop/ipc/WritableRpcEngine.java    |     6 +-
 .../hadoop/ipc/metrics/RetryCacheMetrics.java   |     6 +-
 .../hadoop/ipc/metrics/RpcDetailedMetrics.java  |     8 +-
 .../apache/hadoop/ipc/metrics/RpcMetrics.java   |     6 +-
 .../org/apache/hadoop/jmx/JMXJsonServlet.java   |     7 +-
 .../hadoop/metrics2/impl/MBeanInfoBuilder.java  |     2 +-
 .../hadoop/metrics2/impl/MetricsConfig.java     |     8 +-
 .../metrics2/impl/MetricsSinkAdapter.java       |     9 +-
 .../metrics2/impl/MetricsSourceAdapter.java     |     7 +-
 .../hadoop/metrics2/impl/MetricsSystemImpl.java |     6 +-
 .../hadoop/metrics2/lib/MethodMetric.java       |     7 +-
 .../metrics2/lib/MetricsSourceBuilder.java      |     7 +-
 .../metrics2/lib/MutableMetricsFactory.java     |     7 +-
 .../hadoop/metrics2/lib/MutableRates.java       |     7 +-
 .../lib/MutableRatesWithAggregation.java        |     7 +-
 .../hadoop/metrics2/sink/GraphiteSink.java      |     7 +-
 .../sink/ganglia/AbstractGangliaSink.java       |    10 +-
 .../metrics2/sink/ganglia/GangliaSink30.java    |     6 +-
 .../metrics2/sink/ganglia/GangliaSink31.java    |     7 +-
 .../org/apache/hadoop/metrics2/util/MBeans.java |     6 +-
 .../hadoop/metrics2/util/MetricsCache.java      |     6 +-
 .../main/java/org/apache/hadoop/net/DNS.java    |     6 +-
 .../java/org/apache/hadoop/net/NetUtils.java    |     6 +-
 .../apache/hadoop/net/ScriptBasedMapping.java   |     8 +-
 .../apache/hadoop/net/SocketIOWithTimeout.java  |     6 +-
 .../org/apache/hadoop/net/TableMapping.java     |     6 +-
 .../apache/hadoop/net/unix/DomainSocket.java    |    10 +-
 .../hadoop/net/unix/DomainSocketWatcher.java    |    12 +-
 .../AuthenticationWithProxyUserFilter.java      |     8 +-
 .../hadoop/security/CompositeGroupsMapping.java |     7 +-
 .../org/apache/hadoop/security/Credentials.java |    10 +-
 .../java/org/apache/hadoop/security/Groups.java |     7 +-
 .../HttpCrossOriginFilterInitializer.java       |     8 +-
 .../security/JniBasedUnixGroupsMapping.java     |     8 +-
 .../JniBasedUnixGroupsMappingWithFallback.java  |     8 +-
 .../JniBasedUnixGroupsNetgroupMapping.java      |     6 +-
 ...edUnixGroupsNetgroupMappingWithFallback.java |     8 +-
 .../hadoop/security/LdapGroupsMapping.java      |     7 +-
 .../apache/hadoop/security/ProviderUtils.java   |     7 +-
 .../apache/hadoop/security/SaslInputStream.java |     7 +-
 .../apache/hadoop/security/SaslRpcClient.java   |     7 +-
 .../apache/hadoop/security/SaslRpcServer.java   |     6 +-
 .../apache/hadoop/security/SecurityUtil.java    |     9 +-
 .../hadoop/security/ShellBasedIdMapping.java    |     8 +-
 .../ShellBasedUnixGroupsNetgroupMapping.java    |     8 +-
 .../hadoop/security/UserGroupInformation.java   |     5 +-
 .../hadoop/security/WhitelistBasedResolver.java |     7 +-
 .../alias/AbstractJavaKeyStoreProvider.java     |     6 +-
 .../alias/CredentialProviderFactory.java        |    10 +
 .../authorize/ServiceAuthorizationManager.java  |     9 +-
 .../hadoop/security/http/CrossOriginFilter.java |     7 +-
 .../security/ssl/FileBasedKeyStoresFactory.java |     8 +-
 .../security/ssl/ReloadingX509TrustManager.java |     7 +-
 .../hadoop/security/token/DtFileOperations.java |     7 +-
 .../hadoop/security/token/DtUtilShell.java      |     6 +-
 .../org/apache/hadoop/security/token/Token.java |     6 +-
 .../AbstractDelegationTokenSecretManager.java   |     8 +-
 .../apache/hadoop/service/AbstractService.java  |     9 +-
 .../apache/hadoop/service/CompositeService.java |     7 +-
 .../service/LoggingStateChangeListener.java     |    11 +-
 .../hadoop/service/ServiceOperations.java       |    25 +-
 .../tracing/TracerConfigurationManager.java     |     8 +-
 .../hadoop/util/ApplicationClassLoader.java     |    10 +-
 .../apache/hadoop/util/AsyncDiskService.java    |     7 +-
 .../apache/hadoop/util/CombinedIPWhiteList.java |     7 +-
 .../apache/hadoop/util/FastNumberFormat.java    |    44 +
 .../org/apache/hadoop/util/FileBasedIPList.java |    11 +-
 .../main/java/org/apache/hadoop/util/GSet.java  |     6 +-
 .../hadoop/util/GenericOptionsParser.java       |     7 +-
 .../org/apache/hadoop/util/GenericsUtil.java    |    15 +
 .../org/apache/hadoop/util/HostsFileReader.java |     9 +-
 .../apache/hadoop/util/IntrusiveCollection.java |     7 +-
 .../org/apache/hadoop/util/JvmPauseMonitor.java |     6 +-
 .../java/org/apache/hadoop/util/LogAdapter.java |     4 +
 .../org/apache/hadoop/util/MachineList.java     |     6 +-
 .../apache/hadoop/util/NativeCodeLoader.java    |     8 +-
 .../hadoop/util/NodeHealthScriptRunner.java     |     7 +-
 .../java/org/apache/hadoop/util/Progress.java   |     6 +-
 .../org/apache/hadoop/util/ReflectionUtils.java |    30 +
 .../apache/hadoop/util/ShutdownHookManager.java |     7 +-
 .../hadoop/util/ShutdownThreadsHelper.java      |     7 +-
 .../org/apache/hadoop/util/SysInfoLinux.java    |     8 +-
 .../org/apache/hadoop/util/SysInfoWindows.java  |     7 +-
 .../java/org/apache/hadoop/util/ThreadUtil.java |     7 +-
 .../org/apache/hadoop/util/VersionInfo.java     |     8 +-
 .../hadoop/util/concurrent/AsyncGetFuture.java  |     7 +-
 .../hadoop/util/concurrent/ExecutorHelper.java  |     8 +-
 .../HadoopScheduledThreadPoolExecutor.java      |     8 +-
 .../concurrent/HadoopThreadPoolExecutor.java    |     8 +-
 .../org/apache/hadoop/crypto/OpensslCipher.c    |    46 +-
 .../hadoop-common/src/main/proto/FSProtos.proto |    69 +
 .../src/main/resources/core-default.xml         |    56 +-
 .../src/site/markdown/FileSystemShell.md        |     4 +-
 .../hadoop-common/src/site/markdown/Metrics.md  |    20 +-
 .../src/site/markdown/SecureMode.md             |     9 +-
 .../src/site/markdown/ServiceLevelAuth.md       |    32 +-
 .../src/site/markdown/UnixShellGuide.md         |     4 +-
 .../src/site/markdown/filesystem/filesystem.md  |    24 +
 .../3.0.0-alpha4/CHANGES.3.0.0-alpha4.md        |   880 +
 .../3.0.0-alpha4/RELEASENOTES.3.0.0-alpha4.md   |   492 +
 .../conf/TestCommonConfigurationFields.java     |     3 +
 .../apache/hadoop/conf/TestConfiguration.java   |   181 +-
 .../apache/hadoop/conf/TestDeprecatedKeys.java  |     2 +-
 .../hadoop/crypto/CryptoStreamsTestBase.java    |     6 +-
 .../apache/hadoop/crypto/TestCryptoCodec.java   |     7 +-
 .../crypto/key/TestKeyProviderFactory.java      |    11 +-
 .../kms/TestLoadBalancingKMSClientProvider.java |   378 +-
 .../apache/hadoop/fs/FCStatisticsBaseTest.java  |     7 +-
 .../fs/FileContextCreateMkdirBaseTest.java      |     2 +-
 .../fs/FileContextMainOperationsBaseTest.java   |    66 +-
 .../hadoop/fs/FileContextPermissionBase.java    |     5 +-
 .../apache/hadoop/fs/FileContextUtilBase.java   |     5 +-
 .../org/apache/hadoop/fs/TestFileContext.java   |     7 +-
 .../org/apache/hadoop/fs/TestFileStatus.java    |     9 +-
 .../java/org/apache/hadoop/fs/TestFileUtil.java |     6 +-
 .../org/apache/hadoop/fs/TestFsShellCopy.java   |     6 +-
 .../apache/hadoop/fs/TestFsShellReturnCode.java |     8 +-
 .../org/apache/hadoop/fs/TestFsShellTouch.java  |     6 +-
 .../org/apache/hadoop/fs/TestHarFileSystem.java |     7 +-
 .../java/org/apache/hadoop/fs/TestHardLink.java |     2 +-
 .../org/apache/hadoop/fs/TestListFiles.java     |     4 +-
 .../apache/hadoop/fs/TestLocalFileSystem.java   |    72 +-
 .../fs/TestLocalFileSystemPermission.java       |     2 +-
 .../fs/contract/AbstractBondedFSContract.java   |     8 +-
 .../fs/contract/AbstractContractAppendTest.java |    33 +-
 .../fs/contract/AbstractContractCreateTest.java |    90 +-
 .../hadoop/fs/contract/ContractTestUtils.java   |   150 +-
 .../hadoop/fs/loadGenerator/LoadGenerator.java  |    10 +-
 .../fs/protocolPB/TestFSSerialization.java      |    85 +
 .../hadoop/fs/shell/TestCopyFromLocal.java      |   173 +
 .../hadoop/fs/shell/TestCopyPreserveFlag.java   |    19 +
 .../fs/viewfs/ViewFileSystemBaseTest.java       |    79 +-
 .../apache/hadoop/fs/viewfs/ViewFsBaseTest.java |    94 +-
 .../hadoop/ha/ActiveStandbyElectorTestUtil.java |     6 +-
 .../org/apache/hadoop/ha/DummyHAService.java    |     7 +-
 .../org/apache/hadoop/ha/MiniZKFCCluster.java   |     7 +-
 .../ha/TestActiveStandbyElectorRealZK.java      |     7 +-
 .../java/org/apache/hadoop/ha/TestHAAdmin.java  |     6 +-
 .../org/apache/hadoop/ha/TestHealthMonitor.java |     6 +-
 .../hadoop/ha/TestShellCommandFencer.java       |    55 +-
 .../apache/hadoop/ha/TestSshFenceByTcpPort.java |     6 +-
 .../hadoop/ha/TestZKFailoverController.java     |     5 +-
 .../apache/hadoop/http/TestGlobalFilter.java    |     6 +-
 .../org/apache/hadoop/http/TestHttpServer.java  |     6 +-
 .../apache/hadoop/http/TestHttpServerLogs.java  |     6 +-
 .../hadoop/http/TestHttpServerWebapps.java      |     9 +-
 .../hadoop/http/TestHttpServerWithSpengo.java   |     7 +-
 .../org/apache/hadoop/http/TestPathFilter.java  |     6 +-
 .../apache/hadoop/http/TestSSLHttpServer.java   |     7 +-
 .../apache/hadoop/http/TestServletFilter.java   |     6 +-
 .../hadoop/http/resource/JerseyResource.java    |     6 +-
 .../org/apache/hadoop/io/TestArrayFile.java     |     7 +-
 .../hadoop/io/TestDefaultStringifier.java       |     8 +-
 .../org/apache/hadoop/io/TestSequenceFile.java  |     7 +-
 .../java/org/apache/hadoop/io/TestSetFile.java  |     7 +-
 .../org/apache/hadoop/io/TestWritableUtils.java |     7 +-
 .../apache/hadoop/io/compress/TestCodec.java    |     8 +-
 .../io/compress/TestCompressionStreamReuse.java |     9 +-
 .../apache/hadoop/io/nativeio/TestNativeIO.java |    10 +-
 .../TestSharedFileDescriptorFactory.java        |     7 +-
 .../io/serializer/TestSerializationFactory.java |     6 +-
 .../org/apache/hadoop/ipc/MiniRPCBenchmark.java |    13 +-
 .../org/apache/hadoop/ipc/TestAsyncIPC.java     |    10 +-
 .../java/org/apache/hadoop/ipc/TestIPC.java     |    98 +-
 .../hadoop/ipc/TestIPCServerResponder.java      |    10 +-
 .../apache/hadoop/ipc/TestMiniRPCBenchmark.java |     2 +-
 .../ipc/TestProtoBufRpcServerHandoff.java       |    12 +-
 .../java/org/apache/hadoop/ipc/TestRPC.java     |     8 +-
 .../apache/hadoop/ipc/TestRPCCompatibility.java |     8 +-
 .../hadoop/ipc/TestRPCServerShutdown.java       |     7 +-
 .../apache/hadoop/ipc/TestRpcServerHandoff.java |     8 +-
 .../java/org/apache/hadoop/ipc/TestSaslRPC.java |    23 +-
 .../java/org/apache/hadoop/ipc/TestServer.java  |     4 +-
 .../ipc/TestWeightedRoundRobinMultiplexer.java  |     7 +-
 .../metrics2/impl/TestGangliaMetrics.java       |     7 +-
 .../hadoop/metrics2/impl/TestMetricsConfig.java |     7 +-
 .../metrics2/impl/TestMetricsSystemImpl.java    |    13 +-
 .../hadoop/metrics2/impl/TestSinkQueue.java     |    10 +-
 .../hadoop/metrics2/lib/TestMutableMetrics.java |     7 +-
 .../hadoop/metrics2/util/TestMetricsCache.java  |     8 +-
 .../org/apache/hadoop/net/ServerSocketUtil.java |     9 +-
 .../java/org/apache/hadoop/net/TestDNS.java     |     6 +-
 .../org/apache/hadoop/net/TestNetUtils.java     |     6 +-
 .../hadoop/net/TestSocketIOWithTimeout.java     |     8 +-
 .../apache/hadoop/net/TestStaticMapping.java    |     7 +-
 .../hadoop/net/unix/TestDomainSocket.java       |     2 +-
 .../net/unix/TestDomainSocketWatcher.java       |    15 +-
 .../security/TestCompositeGroupMapping.java     |     7 +-
 .../hadoop/security/TestDoAsEffectiveUser.java  |     8 +-
 .../hadoop/security/TestGroupFallback.java      |    19 +-
 .../hadoop/security/TestGroupsCaching.java      |     8 +-
 .../TestShellBasedUnixGroupsMapping.java        |     8 +-
 .../hadoop/security/TestUGIWithMiniKdc.java     |     2 +-
 .../security/TestUserGroupInformation.java      |     2 +-
 .../alias/TestCredentialProviderFactory.java    |    17 +-
 .../authorize/TestAccessControlList.java        |    16 +-
 .../security/authorize/TestProxyUsers.java      |     8 +-
 .../hadoop/security/ssl/TestSSLFactory.java     |     2 +-
 .../token/delegation/TestDelegationToken.java   |     7 +-
 .../delegation/web/TestWebDelegationToken.java  |     2 +-
 .../hadoop/service/TestCompositeService.java    |     7 +-
 .../hadoop/service/TestServiceLifecycle.java    |     7 +-
 .../hadoop/service/TestServiceOperations.java   |    65 +
 .../apache/hadoop/test/GenericTestUtils.java    |   104 +-
 .../org/apache/hadoop/test/MetricsAsserts.java  |     6 +-
 .../org/apache/hadoop/test/MockitoMaker.java    |   132 -
 .../hadoop/test/MultithreadedTestUtil.java      |     8 +-
 .../hadoop/test/TestGenericTestUtils.java       |    15 +-
 .../org/apache/hadoop/test/TestJUnitSetup.java  |     7 +-
 .../hadoop/util/Crc32PerformanceTest.java       |    11 +-
 .../hadoop/util/TestAsyncDiskService.java       |     7 +-
 .../org/apache/hadoop/util/TestClasspath.java   |     9 +-
 .../org/apache/hadoop/util/TestDiskChecker.java |    12 +-
 .../hadoop/util/TestFastNumberFormat.java       |    46 +
 .../org/apache/hadoop/util/TestFindClass.java   |     7 +-
 .../apache/hadoop/util/TestGenericsUtil.java    |     5 +
 .../hadoop/util/TestIdentityHashStore.java      |     7 +-
 .../apache/hadoop/util/TestLightWeightGSet.java |     7 +-
 .../util/TestLightWeightResizableGSet.java      |     7 +-
 .../hadoop/util/TestNativeCodeLoader.java       |     6 +-
 .../hadoop/util/TestReadWriteDiskValidator.java |     5 +-
 .../apache/hadoop/util/TestReflectionUtils.java |    19 +-
 .../apache/hadoop/util/TestSignalLogger.java    |    11 +-
 .../org/apache/hadoop/util/TestWinUtils.java    |     6 +-
 .../src/test/resources/testConf.xml             |    54 +-
 .../scripts/hadoop-functions_test_helper.bash   |     2 +-
 .../test/scripts/hadoop_add_array_param.bats    |    37 +
 .../src/test/scripts/hadoop_array_contains.bats |    47 +
 .../src/test/scripts/hadoop_sort_array.bats     |    37 +
 .../main/libexec/shellprofile.d/hadoop-kms.sh   |     4 +-
 .../main/resources/webapps/kms/WEB-INF/web.xml  |    10 +
 .../hadoop/crypto/key/kms/server/TestKMS.java   |    79 +
 hadoop-common-project/hadoop-nfs/pom.xml        |     6 +
 .../org/apache/hadoop/mount/MountdBase.java     |    12 +-
 .../java/org/apache/hadoop/nfs/NfsExports.java  |     6 +-
 .../org/apache/hadoop/nfs/nfs3/FileHandle.java  |     6 +-
 .../org/apache/hadoop/nfs/nfs3/Nfs3Base.java    |    10 +-
 .../hadoop/oncrpc/RegistrationClient.java       |     7 +-
 .../java/org/apache/hadoop/oncrpc/RpcCall.java  |    10 +-
 .../org/apache/hadoop/oncrpc/RpcProgram.java    |     6 +-
 .../java/org/apache/hadoop/oncrpc/RpcUtil.java  |    11 +-
 .../hadoop/oncrpc/SimpleTcpClientHandler.java   |     7 +-
 .../apache/hadoop/oncrpc/SimpleTcpServer.java   |     7 +-
 .../apache/hadoop/oncrpc/SimpleUdpServer.java   |     7 +-
 .../hadoop/oncrpc/security/Credentials.java     |     6 +-
 .../hadoop/oncrpc/security/SecurityHandler.java |     7 +-
 .../java/org/apache/hadoop/portmap/Portmap.java |     8 +-
 .../hadoop/portmap/RpcProgramPortmap.java       |     7 +-
 .../apache/hadoop/oncrpc/TestFrameDecoder.java  |     6 +-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |    26 +-
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |    23 +-
 .../hadoop/hdfs/DistributedFileSystem.java      |     5 +-
 .../hdfs/protocol/ErasureCodingPolicy.java      |     8 +-
 .../hdfs/protocol/FsPermissionExtension.java    |     5 +
 .../hadoop/hdfs/protocol/HdfsConstants.java     |     4 +-
 .../hadoop/hdfs/protocol/HdfsFileStatus.java    |   228 +-
 .../hdfs/protocol/HdfsLocatedFileStatus.java    |    42 +-
 .../protocol/SnapshottableDirectoryStatus.java  |    10 +-
 .../ClientNamenodeProtocolTranslatorPB.java     |     4 +-
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  |    65 +-
 .../apache/hadoop/hdfs/web/JsonUtilClient.java  |    50 +-
 .../hadoop/hdfs/web/WebHdfsConstants.java       |     8 +-
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |    18 +-
 .../hadoop-hdfs-client/src/main/proto/acl.proto |     7 +-
 .../src/main/proto/erasurecoding.proto          |     2 +-
 .../src/main/proto/hdfs.proto                   |    16 +-
 .../src/main/conf/httpfs-env.sh                 |     2 +-
 .../hadoop/fs/http/client/HttpFSFileSystem.java |   134 +-
 .../hadoop/fs/http/server/FSOperations.java     |   157 +-
 .../http/server/HttpFSAuthenticationFilter.java |     6 +-
 .../http/server/HttpFSParametersProvider.java   |    48 +-
 .../hadoop/fs/http/server/HttpFSServer.java     |    57 +-
 .../fs/http/server/HttpFSServerWebServer.java   |    15 +-
 .../libexec/shellprofile.d/hadoop-httpfs.sh     |    35 +-
 .../src/main/resources/httpfs-default.xml       |     8 +-
 .../src/site/markdown/ServerSetup.md.vm         |    10 +-
 .../fs/http/client/BaseTestHttpFSWith.java      |   242 +-
 ...KerberosAuthenticationHandlerForTesting.java |     1 -
 .../hadoop/fs/http/server/TestHttpFSServer.java |   492 +-
 .../fs/http/server/TestHttpFSServerNoACLs.java  |    15 +-
 .../http/server/TestHttpFSServerNoXAttrs.java   |    10 +-
 .../http/server/TestHttpFSServerWebServer.java  |     2 +-
 .../dev-support/findbugsExcludeFile.xml         |    12 +
 .../jdiff/Apache_Hadoop_HDFS_3.0.0-alpha4.xml   |   322 +
 .../hadoop-hdfs/src/main/bin/hdfs               |    62 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |     6 +-
 .../hadoop/hdfs/protocol/SnapshotException.java |     4 +
 .../hadoop/hdfs/protocol/SnapshotInfo.java      |     2 +-
 ...tNamenodeProtocolServerSideTranslatorPB.java |     4 +-
 .../hadoop/hdfs/protocolPB/package-info.java    |    18 +
 .../hadoop/hdfs/qjournal/server/Journal.java    |     3 +-
 .../hdfs/qjournal/server/JournalMetrics.java    |    11 +
 .../hdfs/qjournal/server/JournalNodeSyncer.java |     4 +
 .../hadoop/hdfs/server/balancer/Dispatcher.java |    36 +-
 .../server/blockmanagement/BlockCollection.java |    24 +-
 .../server/blockmanagement/BlockManager.java    |    49 +-
 .../blockmanagement/DatanodeAdminManager.java   |   756 +
 .../blockmanagement/DatanodeDescriptor.java     |    19 +-
 .../server/blockmanagement/DatanodeManager.java |    50 +-
 .../blockmanagement/DecommissionManager.java    |   741 -
 .../hdfs/server/datanode/BPOfferService.java    |    47 +-
 .../hadoop/hdfs/server/datanode/DataNode.java   |    31 +-
 .../hdfs/server/datanode/DataNodeMXBean.java    |     7 +
 .../hdfs/server/datanode/DataXceiver.java       |     9 +-
 .../checker/StorageLocationChecker.java         |     8 +-
 .../erasurecode/ErasureCodingWorker.java        |    15 +-
 .../erasurecode/StripedBlockReconstructor.java  |     3 +-
 .../datanode/erasurecode/StripedReader.java     |    20 +
 .../erasurecode/StripedReconstructionInfo.java  |    15 +
 .../erasurecode/StripedReconstructor.java       |     8 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |    70 +-
 .../hadoop/hdfs/server/namenode/BackupNode.java |     2 +-
 .../ContentSummaryComputationContext.java       |    23 +-
 .../namenode/DirectoryWithQuotaFeature.java     |     4 +-
 .../server/namenode/EncryptionZoneManager.java  |     5 +-
 .../namenode/ErasureCodingPolicyManager.java    |    12 +-
 .../server/namenode/FSDirErasureCodingOp.java   |     5 +-
 .../server/namenode/FSDirStatAndListingOp.java  |    67 +-
 .../hdfs/server/namenode/FSDirectory.java       |    20 +-
 .../hdfs/server/namenode/FSNamesystem.java      |   169 +-
 .../server/namenode/FSPermissionChecker.java    |    32 +
 .../hadoop/hdfs/server/namenode/INode.java      |     9 +-
 .../hdfs/server/namenode/INodeDirectory.java    |    16 +-
 .../hdfs/server/namenode/INodeReference.java    |     3 +-
 .../namenode/INodeWithAdditionalFields.java     |    24 +-
 .../hdfs/server/namenode/INodesInPath.java      |    19 +-
 .../hdfs/server/namenode/LeaseManager.java      |    23 +-
 .../hadoop/hdfs/server/namenode/NameNode.java   |     5 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |    20 +-
 .../hdfs/server/namenode/SecondaryNameNode.java |    10 +
 .../namenode/metrics/ECBlockGroupsMBean.java    |    59 +
 .../metrics/ECBlockGroupsStatsMBean.java        |    59 -
 .../namenode/metrics/ReplicatedBlocksMBean.java |    63 +
 .../metrics/ReplicatedBlocksStatsMBean.java     |    63 -
 .../snapshot/AbstractINodeDiffList.java         |    53 +-
 .../snapshot/DirectorySnapshottableFeature.java |    19 +-
 .../snapshot/DirectoryWithSnapshotFeature.java  |     3 +-
 .../hdfs/server/namenode/snapshot/Snapshot.java |     4 +-
 .../namenode/snapshot/SnapshotManager.java      |     7 +-
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  |    65 +-
 .../org/apache/hadoop/hdfs/tools/ECAdmin.java   |    14 +-
 .../org/apache/hadoop/hdfs/web/JsonUtil.java    |    28 +-
 .../src/main/resources/hdfs-default.xml         |    47 +-
 .../hadoop-hdfs/src/main/webapps/datanode/dn.js |     2 +-
 .../src/site/markdown/HDFSErasureCoding.md      |    27 +
 .../org/apache/hadoop/TestGenericRefresh.java   |    28 +-
 .../fs/TestHDFSFileContextMainOperations.java   |    46 +-
 .../apache/hadoop/fs/TestUrlStreamHandler.java  |    48 +-
 .../fs/viewfs/TestViewFileSystemHdfs.java       |    22 +
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  |     8 +-
 .../hadoop/hdfs/TestDFSClientRetries.java       |     6 +-
 .../hadoop/hdfs/TestDFSClientSocketSize.java    |    20 +-
 .../org/apache/hadoop/hdfs/TestDFSShell.java    |    12 +-
 .../apache/hadoop/hdfs/TestDecommission.java    |    44 +-
 .../hadoop/hdfs/TestDistributedFileSystem.java  |    50 +
 .../apache/hadoop/hdfs/TestEncryptionZones.java |     3 +-
 .../hadoop/hdfs/TestEncryptionZonesWithKMS.java |    19 +-
 .../hadoop/hdfs/TestErasureCodingPolicies.java  |    44 +-
 .../hdfs/TestFileStatusSerialization.java       |   153 +
 .../java/org/apache/hadoop/hdfs/TestLease.java  |     7 +-
 .../hadoop/hdfs/TestReconstructStripedFile.java |    74 +-
 .../hdfs/qjournal/TestJournalNodeSync.java      |   265 -
 .../hdfs/qjournal/server/TestJournalNode.java   |     6 +-
 .../qjournal/server/TestJournalNodeSync.java    |   439 +
 .../blockmanagement/BlockManagerTestUtil.java   |     2 +-
 .../blockmanagement/TestBlockManager.java       |    58 +-
 .../TestPendingReconstruction.java              |    46 +-
 ...constructStripedBlocksWithRackAwareness.java |     5 +-
 .../TestReplicationPolicyConsiderLoad.java      |     2 +-
 .../server/datanode/TestBPOfferService.java     |    29 +
 .../server/datanode/TestDataNodeMXBean.java     |     4 +
 .../TestDataXceiverBackwardsCompat.java         |   212 +
 .../fsdataset/impl/FsDatasetImplTestUtils.java  |    10 +
 .../fsdataset/impl/TestWriteToReplica.java      |    53 +-
 .../hdfs/server/namenode/AclTestHelpers.java    |    13 +-
 .../server/namenode/TestAddStripedBlocks.java   |     4 +-
 .../hdfs/server/namenode/TestDeadDatanode.java  |    53 +
 .../namenode/TestDecommissioningStatus.java     |     6 +-
 .../TestDefaultBlockPlacementPolicy.java        |     4 +-
 .../server/namenode/TestEnabledECPolicies.java  |    10 +-
 .../hadoop/hdfs/server/namenode/TestFsck.java   |    17 +-
 .../TestGetContentSummaryWithPermission.java    |   201 +
 .../hdfs/server/namenode/TestMetaSave.java      |     2 +
 .../namenode/TestNameNodeHttpServerXFrame.java  |    22 +
 .../server/namenode/TestNameNodeMXBean.java     |    14 +-
 .../namenode/TestNameNodeReconfigure.java       |    47 +
 .../namenode/TestNamenodeCapacityReport.java    |     8 +-
 .../namenode/TestReconstructStripedBlocks.java  |     4 +-
 .../server/namenode/TestSnapshotPathINodes.java |     5 +-
 .../namenode/TestTruncateQuotaUpdate.java       |     1 +
 .../namenode/metrics/TestNameNodeMetrics.java   |   134 +-
 .../snapshot/TestOpenFilesWithSnapshot.java     |   126 +
 .../apache/hadoop/hdfs/web/TestJsonUtil.java    |    11 +-
 .../test/resources/testErasureCodingConf.xml    |    35 +
 .../src/test/resources/testHDFSConf.xml         |   125 +-
 hadoop-mapreduce-project/bin/mapred             |    18 +-
 .../jobhistory/JobHistoryEventHandler.java      |    27 +-
 .../hadoop/mapreduce/v2/app/AppContext.java     |     4 +
 .../hadoop/mapreduce/v2/app/MRAppMaster.java    |    14 +-
 .../mapreduce/v2/app/rm/RMCommunicator.java     |     4 +-
 .../mapreduce/v2/app/webapp/AppController.java  |    10 +-
 .../hadoop/mapreduce/v2/app/webapp/AppView.java |     4 +-
 .../mapreduce/v2/app/webapp/ConfBlock.java      |    36 +-
 .../mapreduce/v2/app/webapp/CountersBlock.java  |    38 +-
 .../mapreduce/v2/app/webapp/CountersPage.java   |     4 +-
 .../mapreduce/v2/app/webapp/InfoPage.java       |     2 +-
 .../mapreduce/v2/app/webapp/JobBlock.java       |    95 +-
 .../mapreduce/v2/app/webapp/JobConfPage.java    |     3 +-
 .../hadoop/mapreduce/v2/app/webapp/JobPage.java |     2 +-
 .../mapreduce/v2/app/webapp/JobsBlock.java      |    24 +-
 .../mapreduce/v2/app/webapp/NavBlock.java       |    38 +-
 .../v2/app/webapp/SingleCounterBlock.java       |    28 +-
 .../v2/app/webapp/SingleCounterPage.java        |     3 +-
 .../mapreduce/v2/app/webapp/TaskPage.java       |    20 +-
 .../mapreduce/v2/app/webapp/TasksBlock.java     |    12 +-
 .../mapreduce/v2/app/webapp/TasksPage.java      |     2 +-
 .../jobhistory/TestJobHistoryEventHandler.java  |   102 +
 .../hadoop/mapreduce/v2/app/MockAppContext.java |    10 +
 .../mapreduce/v2/app/TestJobEndNotifier.java    |    16 +-
 .../mapreduce/v2/app/TestRuntimeEstimators.java |    10 +
 .../v2/app/launcher/TestContainerLauncher.java  |     9 +
 .../app/launcher/TestContainerLauncherImpl.java |     9 +
 .../v2/app/metrics/TestMRAppMetrics.java        |     9 +-
 .../v2/app/rm/TestRMContainerAllocator.java     |    10 +-
 .../hadoop/mapreduce/v2/util/MRWebAppUtil.java  |     9 +-
 .../org/apache/hadoop/mapreduce/Cluster.java    |    16 +-
 .../java/org/apache/hadoop/mapreduce/Job.java   |    13 +-
 .../apache/hadoop/mapreduce/JobSubmitter.java   |     2 -
 .../mapreduce/counters/AbstractCounters.java    |     4 -
 .../hadoop/mapreduce/counters/Limits.java       |    12 -
 .../mapreduce/jobhistory/HistoryViewer.java     |    16 -
 .../hadoop/mapreduce/lib/db/DBOutputFormat.java |    15 +-
 .../org/apache/hadoop/mapred/TestQueue.java     |     8 +-
 .../webapp/TestMapReduceTrackingUriPlugin.java  |    26 +-
 .../hadoop/mapreduce/v2/hs/CompletedJob.java    |    15 -
 .../hadoop/mapreduce/v2/hs/JobHistory.java      |    10 +
 .../mapreduce/v2/hs/webapp/HsAboutPage.java     |     7 +-
 .../mapreduce/v2/hs/webapp/HsConfPage.java      |     2 +-
 .../mapreduce/v2/hs/webapp/HsCountersPage.java  |     4 +-
 .../mapreduce/v2/hs/webapp/HsJobBlock.java      |    98 +-
 .../mapreduce/v2/hs/webapp/HsJobPage.java       |     2 +-
 .../mapreduce/v2/hs/webapp/HsJobsBlock.java     |    46 +-
 .../mapreduce/v2/hs/webapp/HsLogsPage.java      |     2 +-
 .../mapreduce/v2/hs/webapp/HsNavBlock.java      |    30 +-
 .../v2/hs/webapp/HsSingleCounterPage.java       |     2 +-
 .../mapreduce/v2/hs/webapp/HsTaskPage.java      |    53 +-
 .../mapreduce/v2/hs/webapp/HsTasksBlock.java    |    57 +-
 .../mapreduce/v2/hs/webapp/HsTasksPage.java     |     2 +-
 .../hadoop/mapreduce/v2/hs/webapp/HsView.java   |     4 +-
 .../java/org/apache/hadoop/fs/TestDFSIO.java    |    14 +-
 .../apache/hadoop/mapred/TestYARNRunner.java    |     8 +-
 .../apache/hadoop/mapreduce/TestCounters.java   |    39 +-
 .../mapreduce/TestMapperReducerCleanup.java     |     4 +
 .../mapreduce/lib/db/TestDBOutputFormat.java    |    45 +
 .../lib/input/TestMRCJCFileInputFormat.java     |    11 +-
 .../mapreduce/lib/input/TestMultipleInputs.java |    10 +-
 .../TestUmbilicalProtocolWithJobToken.java      |    22 +-
 .../apache/hadoop/mapreduce/v2/TestMRJobs.java  |    87 +-
 .../hadoop/mapred/TestShuffleHandler.java       |    32 +-
 hadoop-maven-plugins/pom.xml                    |     5 +-
 hadoop-project-dist/pom.xml                     |     2 +-
 hadoop-project/pom.xml                          |    48 +-
 hadoop-project/src/site/site.xml                |     1 +
 .../main/shellprofile.d/hadoop-archive-logs.sh  |     2 +-
 .../src/main/shellprofile.d/hadoop-archives.sh  |     4 +-
 hadoop-tools/hadoop-aws/pom.xml                 |    22 +-
 .../org/apache/hadoop/fs/adl/AdlFileStatus.java |    69 +
 .../org/apache/hadoop/fs/adl/AdlFileSystem.java |    29 +-
 .../apache/hadoop/fs/adl/TestGetFileStatus.java |    57 +-
 .../apache/hadoop/fs/adl/TestListStatus.java    |     8 +-
 .../fs/azure/AzureNativeFileSystemStore.java    |   324 +-
 .../fs/azure/BlobOperationDescriptor.java       |   222 +
 .../hadoop/fs/azure/BlockBlobInputStream.java   |   447 +
 .../hadoop/fs/azure/CachingAuthorizer.java      |   232 +
 .../fs/azure/ClientThrottlingAnalyzer.java      |   284 +
 .../fs/azure/ClientThrottlingIntercept.java     |   221 +
 .../fs/azure/LocalSASKeyGeneratorImpl.java      |    28 +-
 .../hadoop/fs/azure/NativeAzureFileSystem.java  |   117 +-
 .../fs/azure/NativeAzureFileSystemHelper.java   |    28 +
 .../hadoop/fs/azure/NativeFileSystemStore.java  |     6 +-
 .../fs/azure/RemoteSASKeyGeneratorImpl.java     |   310 +-
 .../fs/azure/RemoteWasbAuthorizerImpl.java      |   267 +-
 .../hadoop/fs/azure/SASKeyGeneratorImpl.java    |     4 +-
 .../fs/azure/SecureStorageInterfaceImpl.java    |     5 +
 .../fs/azure/SecureWasbRemoteCallHelper.java    |   230 +
 .../hadoop/fs/azure/StorageInterface.java       |    11 +-
 .../hadoop/fs/azure/StorageInterfaceImpl.java   |     5 +
 .../hadoop/fs/azure/WasbRemoteCallHelper.java   |   288 +-
 .../hadoop/fs/azure/security/Constants.java     |    33 +-
 .../hadoop/fs/azure/security/JsonUtils.java     |    52 +
 .../RemoteWasbDelegationTokenManager.java       |   169 +
 .../hadoop/fs/azure/security/SecurityUtils.java |    86 -
 .../hadoop/fs/azure/security/SpnegoToken.java   |    49 +
 .../hadoop/fs/azure/security/TokenUtils.java    |    60 +
 .../security/WasbDelegationTokenManager.java    |    54 +
 .../fs/azure/security/WasbTokenRenewer.java     |    77 +-
 .../hadoop-azure/src/site/markdown/index.md     |   107 +-
 .../hadoop/fs/azure/AbstractWasbTestBase.java   |     9 +
 .../fs/azure/AzureBlobStorageTestAccount.java   |    40 +-
 .../hadoop/fs/azure/MockStorageInterface.java   |    36 +-
 .../hadoop/fs/azure/MockWasbAuthorizerImpl.java |    22 +-
 .../azure/TestAzureConcurrentOutOfBandIo.java   |     2 +-
 .../fs/azure/TestBlobOperationDescriptor.java   |   305 +
 .../fs/azure/TestBlockBlobInputStream.java      |   875 +
 .../fs/azure/TestClientThrottlingAnalyzer.java  |   177 +
 .../TestNativeAzureFSAuthorizationCaching.java  |    60 +
 .../TestNativeAzureFileSystemAuthorization.java |   233 +-
 ...veAzureFileSystemAuthorizationWithOwner.java |     2 +-
 .../fs/azure/TestWasbRemoteCallHelper.java      |   281 +-
 .../fs/azure/TestWasbUriAndConfiguration.java   |    48 +
 .../src/test/resources/azure-test.xml           |    13 +-
 .../src/main/shellprofile.d/hadoop-distcp.sh    |     4 +-
 .../src/site/markdown/DistCp.md.vm              |     2 +-
 .../src/main/shellprofile.d/hadoop-extras.sh    |     2 +-
 .../src/main/shellprofile.d/hadoop-gridmix.sh   |     2 +-
 .../src/main/native/pipes/impl/HadoopPipes.cc   |    12 +-
 .../src/main/shellprofile.d/hadoop-rumen.sh     |     4 +-
 .../org/apache/hadoop/yarn/sls/SLSRunner.java   |    64 +-
 .../hadoop/yarn/sls/appmaster/AMSimulator.java  |    39 +-
 .../yarn/sls/appmaster/MRAMSimulator.java       |    11 +-
 .../hadoop/yarn/sls/conf/SLSConfiguration.java  |    15 +
 .../src/site/markdown/SchedulerLoadSimulator.md |     3 +-
 .../yarn/sls/appmaster/TestAMSimulator.java     |     4 +-
 .../src/main/shellprofile.d/hadoop-streaming.sh |     2 +-
 .../MySQL/FederationStateStoreDatabase.sql      |    21 +
 .../MySQL/FederationStateStoreStoredProcs.sql   |   162 +
 .../MySQL/FederationStateStoreTables.sql        |    47 +
 .../MySQL/FederationStateStoreUser.sql          |    25 +
 .../FederationStateStore/MySQL/dropDatabase.sql |    21 +
 .../MySQL/dropStoreProcedures.sql               |    47 +
 .../FederationStateStore/MySQL/dropTables.sql   |    27 +
 .../bin/FederationStateStore/MySQL/dropUser.sql |    21 +
 .../FederationStateStoreStoreProcs.sql          |   511 +
 .../SQLServer/FederationStateStoreTables.sql    |   122 +
 hadoop-yarn-project/hadoop-yarn/bin/yarn        |    47 +-
 hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd    |    13 +-
 .../hadoop-yarn/conf/yarn-env.sh                |    12 +
 .../dev-support/findbugs-exclude.xml            |    22 +-
 .../ams/ApplicationMasterServiceContext.java    |    29 +
 .../ams/ApplicationMasterServiceProcessor.java  |    81 +
 .../yarn/ams/ApplicationMasterServiceUtils.java |    89 +
 .../apache/hadoop/yarn/ams/package-info.java    |    24 +
 .../yarn/api/ContainerManagementProtocol.java   |    23 +
 .../protocolrecords/ContainerUpdateRequest.java |    77 +
 .../ContainerUpdateResponse.java                |    95 +
 .../yarn/api/records/ApplicationAttemptId.java  |    77 +-
 .../hadoop/yarn/api/records/ApplicationId.java  |    54 +-
 .../hadoop/yarn/api/records/ContainerId.java    |   139 +-
 .../hadoop/yarn/api/records/ReservationId.java  |    21 +-
 .../org/apache/hadoop/yarn/conf/HAUtil.java     |    30 +-
 .../hadoop/yarn/conf/YarnConfiguration.java     |   166 +-
 .../proto/containermanagement_protocol.proto    |     1 +
 .../src/main/proto/yarn_service_protos.proto    |     9 +
 .../yarn/conf/TestYarnConfigurationFields.java  |    45 +
 .../yarn/client/api/impl/AMRMClientImpl.java    |     4 +-
 .../yarn/client/api/impl/NMClientImpl.java      |    18 +-
 .../TestFederationRMFailoverProxyProvider.java  |   223 +
 .../yarn/client/api/impl/TestNMClient.java      |    37 +-
 .../yarn/client/api/impl/TestYarnClient.java    |     9 +-
 .../hadoop-yarn/hadoop-yarn-common/pom.xml      |    22 +
 ...ContainerManagementProtocolPBClientImpl.java |    40 +-
 ...ontainerManagementProtocolPBServiceImpl.java |    28 +-
 .../impl/pb/ContainerUpdateRequestPBImpl.java   |   171 +
 .../impl/pb/ContainerUpdateResponsePBImpl.java  |   241 +
 .../impl/pb/ContainerLaunchContextPBImpl.java   |     8 +
 .../hadoop/yarn/client/ClientRMProxy.java       |     4 +-
 .../org/apache/hadoop/yarn/client/RMProxy.java  |    11 +-
 .../api/impl/FileSystemTimelineWriter.java      |    40 +-
 .../yarn/client/api/impl/TimelineWriter.java    |     8 +-
 .../apache/hadoop/yarn/util/AsyncCallback.java  |    35 +
 .../hadoop/yarn/util/LRUCacheHashMap.java       |    49 +
 .../resource/DominantResourceCalculator.java    |    22 +-
 .../apache/hadoop/yarn/webapp/ResponseInfo.java |     4 +-
 .../hadoop/yarn/webapp/example/HelloWorld.java  |     4 +-
 .../hadoop/yarn/webapp/example/MyApp.java       |     6 +-
 .../hadoop/yarn/webapp/hamlet/Hamlet.java       |     4 +
 .../hadoop/yarn/webapp/hamlet/HamletGen.java    |     2 +
 .../hadoop/yarn/webapp/hamlet/HamletImpl.java   |     2 +
 .../hadoop/yarn/webapp/hamlet/HamletSpec.java   |     2 +
 .../hadoop/yarn/webapp/hamlet/package-info.java |     6 +
 .../hadoop/yarn/webapp/hamlet2/Hamlet.java      | 30557 +++++++++++++++++
 .../hadoop/yarn/webapp/hamlet2/HamletGen.java   |   449 +
 .../hadoop/yarn/webapp/hamlet2/HamletImpl.java  |   385 +
 .../hadoop/yarn/webapp/hamlet2/HamletSpec.java  |  3101 ++
 .../yarn/webapp/hamlet2/package-info.java       |    27 +
 .../yarn/webapp/log/AggregatedLogsBlock.java    |    66 +-
 .../yarn/webapp/log/AggregatedLogsNavBlock.java |     4 +-
 .../yarn/webapp/log/AggregatedLogsPage.java     |     2 +-
 .../hadoop/yarn/webapp/util/WebAppUtils.java    |    14 +
 .../hadoop/yarn/webapp/view/ErrorPage.java      |    12 +-
 .../hadoop/yarn/webapp/view/FooterBlock.java    |     2 +-
 .../hadoop/yarn/webapp/view/HeaderBlock.java    |     6 +-
 .../hadoop/yarn/webapp/view/HtmlBlock.java      |     2 +-
 .../hadoop/yarn/webapp/view/HtmlPage.java       |    12 +-
 .../hadoop/yarn/webapp/view/InfoBlock.java      |    28 +-
 .../hadoop/yarn/webapp/view/JQueryUI.java       |    14 +-
 .../hadoop/yarn/webapp/view/LipsumBlock.java    |     4 +-
 .../hadoop/yarn/webapp/view/NavBlock.java       |    10 +-
 .../yarn/webapp/view/TwoColumnCssLayout.java    |    20 +-
 .../yarn/webapp/view/TwoColumnLayout.java       |    20 +-
 .../src/main/resources/yarn-default.xml         |   183 +-
 .../hadoop/yarn/TestContainerLaunchRPC.java     |     9 +
 .../yarn/TestContainerResourceIncreaseRPC.java  |    20 +-
 .../hadoop/yarn/api/TestPBImplRecords.java      |     4 +
 .../TestApplicationClientProtocolRecords.java   |    52 +
 .../api/impl/TestTimelineClientForATS1_5.java   |    81 +-
 .../hadoop/yarn/conf/TestYarnConfiguration.java |     1 -
 .../hadoop/yarn/util/TestLRUCacheHashMap.java   |    74 +
 .../yarn/util/TestProcfsBasedProcessTree.java   |     2 +-
 .../hadoop/yarn/util/TestYarnVersionInfo.java   |    11 +-
 .../apache/hadoop/yarn/webapp/TestSubViews.java |    10 +-
 .../apache/hadoop/yarn/webapp/TestWebApp.java   |    28 +-
 .../hadoop/yarn/webapp/view/TestHtmlBlock.java  |    10 +-
 .../hadoop/yarn/webapp/view/TestHtmlPage.java   |     9 +-
 .../hadoop/yarn/webapp/view/TestInfoBlock.java  |     6 +-
 .../yarn/webapp/view/TestTwoColumnCssPage.java  |     6 +-
 .../client/binding/RegistryPathUtils.java       |     2 +-
 .../hadoop/registry/client/types/Endpoint.java  |     4 +-
 .../registry/client/types/ServiceRecord.java    |     4 +-
 .../pom.xml                                     |     4 -
 .../ApplicationHistoryClientService.java        |     8 +-
 .../ApplicationHistoryManagerImpl.java          |     8 +-
 ...pplicationHistoryManagerOnTimelineStore.java |     8 +-
 .../ApplicationHistoryServer.java               |    10 +-
 .../FileSystemApplicationHistoryStore.java      |    22 +-
 .../webapp/AHSErrorsAndWarningsPage.java        |     2 +-
 .../webapp/AHSLogsPage.java                     |     2 +-
 .../webapp/AHSView.java                         |     4 +-
 .../webapp/AHSWebServices.java                  |     7 +-
 .../webapp/AboutBlock.java                      |     8 +-
 .../webapp/AboutPage.java                       |     5 +-
 .../webapp/AppAttemptPage.java                  |     2 +-
 .../webapp/AppPage.java                         |     2 +-
 .../webapp/ContainerPage.java                   |     2 +-
 .../webapp/NavBlock.java                        |    36 +-
 .../timeline/KeyValueBasedTimelineStore.java    |     8 +-
 .../server/timeline/LeveldbTimelineStore.java   |    35 +-
 .../yarn/server/timeline/RollingLevelDB.java    |    15 +-
 .../timeline/RollingLevelDBTimelineStore.java   |    72 +-
 .../server/timeline/TimelineDataManager.java    |     7 +-
 .../recovery/LeveldbTimelineStateStore.java     |    30 +-
 .../timeline/security/TimelineACLsManager.java  |     7 +-
 ...lineDelegationTokenSecretManagerService.java |     8 +-
 .../timeline/webapp/TimelineWebServices.java    |     7 +-
 .../TestFileSystemApplicationHistoryStore.java  |     8 +-
 .../timeline/TestLeveldbTimelineStore.java      |     2 +-
 .../hadoop-yarn-server-common/pom.xml           |    63 +-
 .../hadoop/yarn/server/api/ServerRMProxy.java   |     4 +-
 .../failover/FederationProxyProviderUtil.java   |   133 +
 .../FederationRMFailoverProxyProvider.java      |   221 +
 .../federation/failover/package-info.java       |    17 +
 .../AbstractConfigurableFederationPolicy.java   |   155 +
 .../policies/ConfigurableFederationPolicy.java  |    41 +
 .../FederationPolicyInitializationContext.java  |   130 +
 ...ionPolicyInitializationContextValidator.java |    76 +
 .../policies/FederationPolicyUtils.java         |   203 +
 .../federation/policies/RouterPolicyFacade.java |   265 +
 .../amrmproxy/AbstractAMRMProxyPolicy.java      |    47 +
 .../amrmproxy/BroadcastAMRMProxyPolicy.java     |    85 +
 .../amrmproxy/FederationAMRMProxyPolicy.java    |    65 +
 .../LocalityMulticastAMRMProxyPolicy.java       |   608 +
 .../amrmproxy/RejectAMRMProxyPolicy.java        |    67 +
 .../policies/amrmproxy/package-info.java        |    19 +
 .../policies/dao/WeightedPolicyInfo.java        |   251 +
 .../federation/policies/dao/package-info.java   |    19 +
 .../exceptions/FederationPolicyException.java   |    33 +
 ...FederationPolicyInitializationException.java |    33 +
 .../NoActiveSubclustersException.java           |    27 +
 .../exceptions/UnknownSubclusterException.java  |    28 +
 .../policies/exceptions/package-info.java       |    19 +
 .../policies/manager/AbstractPolicyManager.java |   190 +
 .../manager/FederationPolicyManager.java        |   118 +
 .../manager/HashBroadcastPolicyManager.java     |    38 +
 .../manager/PriorityBroadcastPolicyManager.java |    66 +
 .../manager/RejectAllPolicyManager.java         |    40 +
 .../manager/UniformBroadcastPolicyManager.java  |    44 +
 .../manager/WeightedLocalityPolicyManager.java  |    67 +
 .../policies/manager/package-info.java          |    19 +
 .../federation/policies/package-info.java       |    19 +
 .../policies/router/AbstractRouterPolicy.java   |    66 +
 .../policies/router/FederationRouterPolicy.java |    52 +
 .../policies/router/HashBasedRouterPolicy.java  |   102 +
 .../policies/router/LoadBasedRouterPolicy.java  |   112 +
 .../policies/router/PriorityRouterPolicy.java   |    72 +
 .../policies/router/RejectRouterPolicy.java     |    76 +
 .../router/UniformRandomRouterPolicy.java       |   104 +
 .../router/WeightedRandomRouterPolicy.java      |    94 +
 .../policies/router/package-info.java           |    19 +
 .../resolver/AbstractSubClusterResolver.java    |    67 +
 .../resolver/DefaultSubClusterResolverImpl.java |   164 +
 .../federation/resolver/SubClusterResolver.java |    58 +
 .../federation/resolver/package-info.java       |    17 +
 ...ederationApplicationHomeSubClusterStore.java |   121 +
 .../store/FederationMembershipStateStore.java   |   117 +
 .../federation/store/FederationPolicyStore.java |    77 +
 .../federation/store/FederationStateStore.java  |    64 +
 .../FederationStateStoreException.java          |    50 +
 ...derationStateStoreInvalidInputException.java |    48 +
 .../FederationStateStoreRetriableException.java |    44 +
 .../store/exception/package-info.java           |    17 +
 .../store/impl/MemoryFederationStateStore.java  |   315 +
 .../store/impl/SQLFederationStateStore.java     |   942 +
 .../federation/store/impl/package-info.java     |    17 +
 .../server/federation/store/package-info.java   |    17 +
 .../AddApplicationHomeSubClusterRequest.java    |    72 +
 .../AddApplicationHomeSubClusterResponse.java   |    65 +
 .../records/ApplicationHomeSubCluster.java      |   124 +
 .../DeleteApplicationHomeSubClusterRequest.java |    65 +
 ...DeleteApplicationHomeSubClusterResponse.java |    43 +
 .../GetApplicationHomeSubClusterRequest.java    |    64 +
 .../GetApplicationHomeSubClusterResponse.java   |    73 +
 .../GetApplicationsHomeSubClusterRequest.java   |    40 +
 .../GetApplicationsHomeSubClusterResponse.java  |    75 +
 .../store/records/GetSubClusterInfoRequest.java |    62 +
 .../records/GetSubClusterInfoResponse.java      |    62 +
 ...SubClusterPoliciesConfigurationsRequest.java |    35 +
 ...ubClusterPoliciesConfigurationsResponse.java |    66 +
 ...GetSubClusterPolicyConfigurationRequest.java |    63 +
 ...etSubClusterPolicyConfigurationResponse.java |    65 +
 .../records/GetSubClustersInfoRequest.java      |    70 +
 .../records/GetSubClustersInfoResponse.java     |    66 +
 ...SetSubClusterPolicyConfigurationRequest.java |    61 +
 ...etSubClusterPolicyConfigurationResponse.java |    36 +
 .../records/SubClusterDeregisterRequest.java    |    89 +
 .../records/SubClusterDeregisterResponse.java   |    42 +
 .../records/SubClusterHeartbeatRequest.java     |   149 +
 .../records/SubClusterHeartbeatResponse.java    |    45 +
 .../federation/store/records/SubClusterId.java  |   100 +
 .../store/records/SubClusterIdInfo.java         |    75 +
 .../store/records/SubClusterInfo.java           |   325 +
 .../records/SubClusterPolicyConfiguration.java  |   162 +
 .../records/SubClusterRegisterRequest.java      |    74 +
 .../records/SubClusterRegisterResponse.java     |    44 +
 .../store/records/SubClusterState.java          |    85 +
 .../UpdateApplicationHomeSubClusterRequest.java |    74 +
 ...UpdateApplicationHomeSubClusterResponse.java |    43 +
 ...dApplicationHomeSubClusterRequestPBImpl.java |   132 +
 ...ApplicationHomeSubClusterResponsePBImpl.java |   117 +
 .../pb/ApplicationHomeSubClusterPBImpl.java     |   167 +
 ...eApplicationHomeSubClusterRequestPBImpl.java |   130 +
 ...ApplicationHomeSubClusterResponsePBImpl.java |    78 +
 ...tApplicationHomeSubClusterRequestPBImpl.java |   139 +
 ...ApplicationHomeSubClusterResponsePBImpl.java |   132 +
 ...ApplicationsHomeSubClusterRequestPBImpl.java |    78 +
 ...pplicationsHomeSubClusterResponsePBImpl.java |   190 +
 .../impl/pb/GetSubClusterInfoRequestPBImpl.java |   125 +
 .../pb/GetSubClusterInfoResponsePBImpl.java     |   134 +
 ...sterPoliciesConfigurationsRequestPBImpl.java |    95 +
 ...terPoliciesConfigurationsResponsePBImpl.java |   191 +
 ...ClusterPolicyConfigurationRequestPBImpl.java |   103 +
 ...lusterPolicyConfigurationResponsePBImpl.java |   143 +
 .../pb/GetSubClustersInfoRequestPBImpl.java     |   108 +
 .../pb/GetSubClustersInfoResponsePBImpl.java    |   184 +
 ...ClusterPolicyConfigurationRequestPBImpl.java |   142 +
 ...lusterPolicyConfigurationResponsePBImpl.java |    93 +
 .../pb/SubClusterDeregisterRequestPBImpl.java   |   156 +
 .../pb/SubClusterDeregisterResponsePBImpl.java  |    77 +
 .../pb/SubClusterHeartbeatRequestPBImpl.java    |   192 +
 .../pb/SubClusterHeartbeatResponsePBImpl.java   |    77 +
 .../records/impl/pb/SubClusterIdPBImpl.java     |    75 +
 .../records/impl/pb/SubClusterInfoPBImpl.java   |   251 +
 .../pb/SubClusterPolicyConfigurationPBImpl.java |   138 +
 .../pb/SubClusterRegisterRequestPBImpl.java     |   134 +
 .../pb/SubClusterRegisterResponsePBImpl.java    |    77 +
 ...eApplicationHomeSubClusterRequestPBImpl.java |   132 +
 ...ApplicationHomeSubClusterResponsePBImpl.java |    78 +
 .../store/records/impl/pb/package-info.java     |    17 +
 .../federation/store/records/package-info.java  |    17 +
 ...cationHomeSubClusterStoreInputValidator.java |   180 +
 ...ationMembershipStateStoreInputValidator.java |   315 +
 .../FederationPolicyStoreInputValidator.java    |   143 +
 .../store/utils/FederationStateStoreUtils.java  |   214 +
 .../federation/store/utils/package-info.java    |    17 +
 .../utils/FederationStateStoreFacade.java       |   602 +
 .../server/federation/utils/package-info.java   |    17 +
 .../yarn/server/uam/UnmanagedAMPoolManager.java |   311 +
 .../server/uam/UnmanagedApplicationManager.java |   607 +
 .../hadoop/yarn/server/uam/package-info.java    |    18 +
 .../yarn/server/utils/AMRMClientUtils.java      |   189 +
 .../server/utils/YarnServerSecurityUtils.java   |    41 +-
 .../yarn/server/webapp/AppAttemptBlock.java     |    31 +-
 .../hadoop/yarn/server/webapp/AppBlock.java     |    80 +-
 .../hadoop/yarn/server/webapp/AppsBlock.java    |    14 +-
 .../yarn/server/webapp/ContainerBlock.java      |    23 +-
 .../server/webapp/ErrorsAndWarningsBlock.java   |    50 +-
 .../hadoop/yarn/server/webapp/dao/AppInfo.java  |    14 +
 .../proto/yarn_server_federation_protos.proto   |   165 +
 .../java/org/apache/hadoop/yarn/TestRPC.java    |     9 +
 .../yarn/server/MockResourceManagerFacade.java  |   855 +
 .../policies/BaseFederationPoliciesTest.java    |   186 +
 ...ionPolicyInitializationContextValidator.java |   138 +
 .../policies/TestRouterPolicyFacade.java        |   222 +
 .../TestBroadcastAMRMProxyFederationPolicy.java |   112 +
 .../TestLocalityMulticastAMRMProxyPolicy.java   |   602 +
 .../amrmproxy/TestRejectAMRMProxyPolicy.java    |    78 +
 .../policies/manager/BasePolicyManagerTest.java |   101 +
 .../TestHashBasedBroadcastPolicyManager.java    |    40 +
 .../TestPriorityBroadcastPolicyManager.java     |    72 +
 .../manager/TestRejectAllPolicyManager.java     |    40 +
 .../TestUniformBroadcastPolicyManager.java      |    40 +
 .../TestWeightedLocalityPolicyManager.java      |    79 +
 .../policies/router/BaseRouterPoliciesTest.java |   118 +
 .../router/TestHashBasedRouterPolicy.java       |    83 +
 .../router/TestLoadBasedRouterPolicy.java       |   106 +
 .../router/TestPriorityRouterPolicy.java        |    85 +
 .../policies/router/TestRejectRouterPolicy.java |    63 +
 .../router/TestUniformRandomRouterPolicy.java   |    64 +
 .../router/TestWeightedRandomRouterPolicy.java  |   137 +
 .../resolver/TestDefaultSubClusterResolver.java |   189 +
 .../impl/FederationStateStoreBaseTest.java      |   578 +
 .../store/impl/HSQLDBFederationStateStore.java  |   252 +
 .../impl/TestMemoryFederationStateStore.java    |    35 +
 .../store/impl/TestSQLFederationStateStore.java |    49 +
 .../records/TestFederationProtocolRecords.java  |   265 +
 .../TestFederationStateStoreInputValidator.java |  1260 +
 .../utils/FederationPoliciesTestUtil.java       |   227 +
 .../utils/FederationStateStoreTestUtil.java     |   181 +
 .../utils/TestFederationStateStoreFacade.java   |   192 +
 .../TestFederationStateStoreFacadeRetry.java    |   146 +
 .../uam/TestUnmanagedApplicationManager.java    |   335 +
 .../src/test/resources/nodes                    |     8 +
 .../src/test/resources/nodes-malformed          |     3 +
 .../hadoop-yarn-server-nodemanager/pom.xml      |    11 +
 .../src/CMakeLists.txt                          |     1 +
 .../nodemanager/LinuxContainerExecutor.java     |    21 +-
 .../nodemanager/NodeStatusUpdaterImpl.java      |    11 +-
 .../amrmproxy/DefaultRequestInterceptor.java    |    30 +-
 .../amrmproxy/FederationInterceptor.java        |  1150 +
 .../containermanager/ContainerManagerImpl.java  |    31 +-
 .../container/ContainerImpl.java                |     4 +
 .../PrivilegedOperationException.java           |    10 +-
 .../CGroupsCpuResourceHandlerImpl.java          |    20 +-
 .../linux/resources/CGroupsHandler.java         |    15 +
 .../linux/resources/CGroupsHandlerImpl.java     |    26 +-
 .../CGroupsMemoryResourceHandlerImpl.java       |    35 +-
 .../linux/resources/ResourceHandlerModule.java  |    58 +-
 .../runtime/DockerLinuxContainerRuntime.java    |    67 +-
 .../linux/runtime/docker/DockerRunCommand.java  |     6 +
 .../localizer/ContainerLocalizer.java           |    52 +-
 .../monitor/ContainerMetrics.java               |     2 +-
 .../runtime/ContainerExecutionException.java    |    10 +-
 ...locationBasedResourceUtilizationTracker.java |    31 +-
 .../scheduler/ContainerScheduler.java           |   144 +-
 .../recovery/NMLeveldbStateStoreService.java    |     6 +-
 .../util/CgroupsLCEResourcesHandler.java        |    53 +-
 .../nodemanager/webapp/AllApplicationsPage.java |    28 +-
 .../nodemanager/webapp/AllContainersPage.java   |    30 +-
 .../nodemanager/webapp/ApplicationPage.java     |    25 +-
 .../nodemanager/webapp/ContainerLogsPage.java   |    17 +-
 .../nodemanager/webapp/ContainerPage.java       |    31 +-
 .../webapp/NMErrorsAndWarningsPage.java         |     2 +-
 .../yarn/server/nodemanager/webapp/NMView.java  |     4 +-
 .../server/nodemanager/webapp/NavBlock.java     |    28 +-
 .../server/nodemanager/webapp/NodePage.java     |    28 +-
 .../impl/container-executor.c                   |   121 +-
 .../impl/container-executor.h                   |    17 +-
 .../impl/utils/string-utils.c                   |    86 +
 .../impl/utils/string-utils.h                   |    32 +
 .../test/test-container-executor.c              |   224 +-
 .../TestLinuxContainerExecutorWithMocks.java    |    89 +
 .../nodemanager/TestNodeManagerResync.java      |    30 +-
 .../amrmproxy/MockRequestInterceptor.java       |    14 +-
 .../amrmproxy/MockResourceManagerFacade.java    |   514 -
 .../amrmproxy/TestAMRMProxyService.java         |     1 +
 .../amrmproxy/TestFederationInterceptor.java    |   496 +
 .../TestableFederationInterceptor.java          |   139 +
 .../containermanager/TestAuxServices.java       |     2 +-
 .../containermanager/TestContainerManager.java  |   145 +-
 .../TestContainerManagerRecovery.java           |    24 +-
 .../TestCGroupsCpuResourceHandlerImpl.java      |    23 +
 .../linux/resources/TestCGroupsHandlerImpl.java |    25 +
 .../TestCGroupsMemoryResourceHandlerImpl.java   |    45 +-
 .../runtime/TestDockerContainerRuntime.java     |    58 +-
 .../localizer/TestContainerLocalizer.java       |    43 +-
 ...locationBasedResourceUtilizationTracker.java |    93 +
 .../TestContainerSchedulerQueuing.java          |    85 +
 .../util/TestCgroupsLCEResourcesHandler.java    |    33 +-
 .../resourcemanager/AMSProcessingChain.java     |   102 +
 ...ActiveStandbyElectorBasedElectorService.java |    12 +-
 .../server/resourcemanager/AdminService.java    |    71 +-
 .../ApplicationMasterService.java               |   443 +-
 .../CuratorBasedElectorService.java             |    10 +-
 .../resourcemanager/DefaultAMSProcessor.java    |   456 +
 ...pportunisticContainerAllocatorAMService.java |   184 +-
 .../resourcemanager/RMActiveServiceContext.java |    36 +-
 .../server/resourcemanager/RMAppManager.java    |    56 +-
 .../yarn/server/resourcemanager/RMContext.java  |     3 +-
 .../server/resourcemanager/RMContextImpl.java   |   312 +-
 .../resourcemanager/RMServiceContext.java       |   162 +
 .../server/resourcemanager/ResourceManager.java |    68 +-
 .../FederationStateStoreHeartbeat.java          |   108 +
 .../federation/FederationStateStoreService.java |   304 +
 .../federation/package-info.java                |    17 +
 .../metrics/TimelineServiceV2Publisher.java     |     6 +-
 .../capacity/FifoCandidatesSelector.java        |     6 +-
 .../ProportionalCapacityPreemptionPolicy.java   |     7 +-
 .../placement/PlacementFactory.java             |    45 +
 .../recovery/FileSystemRMStateStore.java        |     2 +-
 .../recovery/LeveldbRMStateStore.java           |     2 +-
 .../recovery/MemoryRMStateStore.java            |     1 +
 .../resourcemanager/recovery/RMStateStore.java  |    10 +-
 .../recovery/ZKRMStateStore.java                |     2 +-
 .../resource/ResourceWeights.java               |     4 +-
 .../resourcemanager/rmapp/RMAppEvent.java       |    24 -
 .../resourcemanager/rmapp/RMAppEventType.java   |     1 +
 .../server/resourcemanager/rmapp/RMAppImpl.java |     8 +-
 .../rmapp/attempt/RMAppAttemptImpl.java         |    20 +-
 .../scheduler/AbstractYarnScheduler.java        |    11 +-
 .../scheduler/AppSchedulingInfo.java            |     6 +-
 .../scheduler/SchedulerApplicationAttempt.java  |     2 +-
 .../scheduler/SchedulerUtils.java               |    35 +-
 .../scheduler/activities/ActivitiesLogger.java  |    33 +-
 .../scheduler/capacity/AbstractCSQueue.java     |     8 +-
 .../scheduler/capacity/CapacityScheduler.java   |    95 +-
 .../CapacitySchedulerConfiguration.java         |    50 +
 .../scheduler/capacity/LeafQueue.java           |    83 +-
 .../scheduler/capacity/ParentQueue.java         |     7 +-
 .../scheduler/common/fica/FiCaSchedulerApp.java |    13 +
 .../scheduler/fair/FSAppAttempt.java            |    17 +-
 .../scheduler/fair/FSContext.java               |     2 +-
 .../scheduler/fair/FSLeafQueue.java             |     2 -
 .../scheduler/fair/FSPreemptionThread.java      |    28 +-
 .../scheduler/fair/FSSchedulerNode.java         |    68 +-
 .../scheduler/fair/FairScheduler.java           |    20 +-
 .../DominantResourceFairnessPolicy.java         |     2 +
 .../fair/policies/FairSharePolicy.java          |   122 +-
 .../RMTimelineCollectorManager.java             |    10 +-
 .../resourcemanager/webapp/AboutBlock.java      |    20 +-
 .../resourcemanager/webapp/AboutPage.java       |     2 +-
 .../resourcemanager/webapp/AppAttemptPage.java  |     2 +-
 .../webapp/AppLogAggregationStatusPage.java     |     2 +-
 .../server/resourcemanager/webapp/AppPage.java  |     2 +-
 .../webapp/AppsBlockWithMetrics.java            |     5 +-
 .../webapp/CapacitySchedulerPage.java           |   220 +-
 .../resourcemanager/webapp/ContainerPage.java   |     2 +-
 .../webapp/DefaultSchedulerPage.java            |    58 +-
 .../resourcemanager/webapp/ErrorBlock.java      |     2 +-
 .../webapp/FairSchedulerAppsBlock.java          |    18 +-
 .../webapp/FairSchedulerPage.java               |   118 +-
 .../webapp/MetricsOverviewTable.java            |   124 +-
 .../server/resourcemanager/webapp/NavBlock.java |    34 +-
 .../resourcemanager/webapp/NodeIDsInfo.java     |     5 +-
 .../resourcemanager/webapp/NodeLabelsPage.java  |    18 +-
 .../resourcemanager/webapp/NodesPage.java       |    16 +-
 .../webapp/RMAppAttemptBlock.java               |    48 +-
 .../resourcemanager/webapp/RMAppBlock.java      |    28 +-
 .../webapp/RMAppLogAggregationStatusBlock.java  |    34 +-
 .../resourcemanager/webapp/RMAppsBlock.java     |    23 +-
 .../webapp/RMErrorsAndWarningsPage.java         |     2 +-
 .../resourcemanager/webapp/RMWSConsts.java      |    15 +
 .../resourcemanager/webapp/RMWebAppUtil.java    |   224 +
 .../webapp/RMWebServiceProtocol.java            |   133 +-
 .../resourcemanager/webapp/RMWebServices.java   |   183 +-
 .../webapp/RedirectionErrorPage.java            |     2 +-
 .../server/resourcemanager/webapp/RmView.java   |     4 +-
 .../webapp/SchedulerPageUtil.java               |     8 +-
 .../webapp/dao/AppAttemptInfo.java              |     5 +-
 .../resourcemanager/webapp/dao/AppInfo.java     |    13 +
 .../webapp/dao/ClusterMetricsInfo.java          |     5 +-
 .../webapp/dao/FairSchedulerQueueInfo.java      |    23 +
 .../webapp/dao/LabelsToNodesInfo.java           |     6 +-
 .../yarn/server/resourcemanager/MockRM.java     |    16 +-
 .../resourcemanager/MockRMMemoryStateStore.java |    32 +
 .../server/resourcemanager/NodeManager.java     |     9 +
 .../resourcemanager/TestAMAuthorization.java    |     9 +
 .../resourcemanager/TestApplicationCleanup.java |    30 +-
 .../TestApplicationMasterLauncher.java          |    15 +-
 .../TestApplicationMasterService.java           |   163 +-
 .../resourcemanager/TestClientRMService.java    |     4 +-
 .../TestContainerResourceUsage.java             |     8 +-
 .../TestNodeBlacklistingOnAMFailures.java       |     8 +-
 ...pportunisticContainerAllocatorAMService.java |     8 +
 .../resourcemanager/TestRMEmbeddedElector.java  |     8 +-
 .../yarn/server/resourcemanager/TestRMHA.java   |    28 +-
 .../server/resourcemanager/TestRMRestart.java   |   240 +-
 .../TestWorkPreservingRMRestart.java            |   127 +-
 .../applicationsmanager/TestAMRestart.java      |    29 +-
 .../TestFederationRMStateStoreService.java      |   176 +
 .../TestSystemMetricsPublisherForV2.java        |    13 +-
 .../recovery/RMStateStoreTestBase.java          |     8 +-
 .../recovery/TestFSRMStateStore.java            |     1 +
 .../recovery/TestLeveldbRMStateStore.java       |     1 +
 .../recovery/TestZKRMStateStore.java            |    15 +-
 .../TestReservationInputValidator.java          |    10 +-
 .../rmapp/TestApplicationLifetimeMonitor.java   |     7 +-
 .../rmapp/TestRMAppTransitions.java             |    17 +
 .../attempt/TestRMAppAttemptTransitions.java    |    33 +-
 .../scheduler/TestAbstractYarnScheduler.java    |     7 +-
 .../scheduler/TestQueueMetrics.java             |    14 +-
 .../scheduler/TestSchedulerUtils.java           |     4 +-
 .../CapacitySchedulerPreemptionTestBase.java    |     7 +-
 .../capacity/TestApplicationPriority.java       |    18 +-
 .../capacity/TestCapacityScheduler.java         |   455 +-
 .../TestCapacitySchedulerAsyncScheduling.java   |   296 +
 .../TestCapacitySchedulerDynamicBehavior.java   |     2 +-
 ...TestCapacitySchedulerSurgicalPreemption.java |    99 +-
 .../scheduler/capacity/TestLeafQueue.java       |    12 +-
 ...TestWorkPreservingRMRestartForNodeLabel.java |    10 +-
 .../scheduler/fair/TestFSSchedulerNode.java     |    52 +
 .../scheduler/fair/TestFairScheduler.java       |     8 +-
 .../scheduler/fair/TestSchedulingPolicy.java    |    19 +-
 .../security/TestDelegationTokenRenewer.java    |     5 +-
 .../security/TestRMDelegationTokens.java        |     9 +-
 .../webapp/TestRMWebServicesApps.java           |     2 +-
 .../hadoop/yarn/webapp/TestRMWithXFSFilter.java |    14 +-
 .../hadoop-yarn-server-router/pom.xml           |   110 +
 .../hadoop/yarn/server/router/Router.java       |   179 +
 .../yarn/server/router/RouterServerUtil.java    |    63 +
 .../AbstractClientRequestInterceptor.java       |   127 +
 .../clientrm/ClientRequestInterceptor.java      |    65 +
 .../DefaultClientRequestInterceptor.java        |   311 +
 .../clientrm/FederationClientInterceptor.java   |   677 +
 .../router/clientrm/RouterClientRMService.java  |   546 +
 .../server/router/clientrm/package-info.java    |    20 +
 .../hadoop/yarn/server/router/package-info.java |    20 +
 .../AbstractRMAdminRequestInterceptor.java      |    90 +
 .../DefaultRMAdminRequestInterceptor.java       |   215 +
 .../rmadmin/RMAdminRequestInterceptor.java      |    65 +
 .../router/rmadmin/RouterRMAdminService.java    |   423 +
 .../server/router/rmadmin/package-info.java     |    20 +
 .../webapp/AbstractRESTRequestInterceptor.java  |    89 +
 .../webapp/DefaultRequestInterceptorREST.java   |   496 +
 .../yarn/server/router/webapp/HTTPMethods.java  |    34 +
 .../router/webapp/RESTRequestInterceptor.java   |   125 +
 .../yarn/server/router/webapp/RouterWebApp.java |    48 +
 .../router/webapp/RouterWebServiceUtil.java     |   227 +
 .../server/router/webapp/RouterWebServices.java |   876 +
 .../yarn/server/router/webapp/package-info.java |    20 +
 .../router/clientrm/BaseRouterClientRMTest.java |   586 +
 .../clientrm/MockClientRequestInterceptor.java  |    36 +
 .../PassThroughClientRequestInterceptor.java    |   267 +
 .../TestFederationClientInterceptor.java        |   403 +
 .../TestFederationClientInterceptorRetry.java   |   295 +
 .../clientrm/TestRouterClientRMService.java     |   210 +
 .../TestableFederationClientInterceptor.java    |    75 +
 .../router/rmadmin/BaseRouterRMAdminTest.java   |   346 +
 .../rmadmin/MockRMAdminRequestInterceptor.java  |    36 +
 .../PassThroughRMAdminRequestInterceptor.java   |   148 +
 .../rmadmin/TestRouterRMAdminService.java       |   219 +
 .../webapp/BaseRouterWebServicesTest.java       |   601 +
 .../yarn/server/router/webapp/JavaProcess.java  |    52 +
 .../webapp/MockRESTRequestInterceptor.java      |   340 +
 .../PassThroughRESTRequestInterceptor.java      |   339 +
 .../router/webapp/TestRouterWebServices.java    |   269 +
 .../webapp/TestRouterWebServicesREST.java       |  1298 +
 .../src/test/resources/capacity-scheduler.xml   |   111 +
 .../src/test/resources/log4j.properties         |    19 +
 .../src/test/resources/yarn-site.xml            |    30 +
 .../sharedcachemanager/CleanerService.java      |     7 +-
 .../server/sharedcachemanager/CleanerTask.java  |     7 +-
 .../ClientProtocolService.java                  |     7 +-
 .../SCMAdminProtocolService.java                |     8 +-
 .../sharedcachemanager/SharedCacheManager.java  |     9 +-
 .../metrics/CleanerMetrics.java                 |     7 +-
 .../metrics/ClientSCMMetrics.java               |     7 +-
 .../metrics/SharedCacheUploaderMetrics.java     |     8 +-
 .../store/InMemorySCMStore.java                 |     7 +-
 .../webapp/SCMOverviewPage.java                 |    26 +-
 .../sharedcachemanager/webapp/SCMWebServer.java |     7 +-
 .../hadoop/yarn/server/MiniYARNCluster.java     |    13 +-
 .../server/TestContainerManagerSecurity.java    |    74 +-
 .../timeline/EntityGroupFSTimelineStore.java    |    23 +-
 .../timeline/LevelDBCacheTimelineStore.java     |    14 +-
 .../TestEntityGroupFSTimelineStore.java         |    99 +-
 ...stTimelineReaderWebServicesHBaseStorage.java |    30 +-
 .../reader/filter/TimelineFilterUtils.java      |     7 +-
 .../storage/HBaseTimelineReaderImpl.java        |     8 +-
 .../storage/HBaseTimelineWriterImpl.java        |     8 +-
 .../storage/TimelineSchemaCreator.java          |     7 +-
 .../storage/application/ApplicationTable.java   |     7 +-
 .../storage/apptoflow/AppToFlowTable.java       |     7 +-
 .../storage/common/ColumnHelper.java            |     8 +-
 .../common/HBaseTimelineStorageUtils.java       |     8 +-
 .../storage/entity/EntityTable.java             |     7 +-
 .../storage/flow/FlowActivityTable.java         |     7 +-
 .../storage/flow/FlowRunCoprocessor.java        |     7 +-
 .../storage/flow/FlowRunTable.java              |     7 +-
 .../storage/flow/FlowScanner.java               |     7 +-
 .../storage/reader/TimelineEntityReader.java    |     7 +-
 .../collector/AppLevelTimelineCollector.java    |     7 +-
 .../collector/NodeTimelineCollectorManager.java |     8 +-
 .../PerNodeTimelineCollectorsAuxService.java    |    10 +-
 .../collector/TimelineCollector.java            |     7 +-
 .../collector/TimelineCollectorManager.java     |     8 +-
 .../collector/TimelineCollectorWebService.java  |     8 +-
 .../reader/TimelineReaderServer.java            |     9 +-
 .../reader/TimelineReaderWebServices.java       |     8 +-
 .../storage/FileSystemTimelineReaderImpl.java   |     8 +-
 .../storage/common/TimelineStorageUtils.java    |     4 -
 .../hadoop/yarn/server/webproxy/ProxyUtils.java |    16 +-
 .../server/webproxy/WebAppProxyServlet.java     |    14 +-
 .../server/webproxy/amfilter/AmIpFilter.java    |    60 +-
 .../server/webproxy/amfilter/TestAmFilter.java  |    70 +-
 .../hadoop-yarn/hadoop-yarn-server/pom.xml      |     1 +
 .../src/site/markdown/Federation.md             |   289 +
 .../src/site/markdown/GracefulDecommission.md   |    12 +-
 .../src/site/markdown/NodeManagerCgroups.md     |    17 +-
 .../src/site/markdown/ResourceManagerHA.md      |     2 +-
 .../src/site/markdown/TimelineServer.md         |     4 +-
 .../site/markdown/WritingYarnApplications.md    |     4 +-
 .../hadoop-yarn-site/src/site/markdown/YARN.md  |     5 +-
 .../src/site/markdown/registry/yarn-registry.md |    14 +-
 .../resources/images/amrmproxy_architecture.png |   Bin 0 -> 35010 bytes
 .../images/federation_architecture.png          |   Bin 0 -> 46964 bytes
 .../images/federation_sequence_diagram.png      |   Bin 0 -> 47910 bytes
 hadoop-yarn-project/hadoop-yarn/pom.xml         |     2 +-
 hadoop-yarn-project/pom.xml                     |     4 +
 pom.xml                                         |     7 +-
 start-build-env.sh                              |     2 +-
 1223 files changed, 96984 insertions(+), 9279 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b114f248/hadoop-common-project/hadoop-common/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b114f248/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b114f248/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
----------------------------------------------------------------------
diff --cc hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
index 0cd5b84,fef968b..21733b3
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
@@@ -331,17 -331,8 +331,17 @@@ public class FileContext 
            return AbstractFileSystem.get(uri, conf);
          }
        });
 +    } catch (RuntimeException ex) {
 +      // RTEs can wrap other exceptions; if there is an IOException inner,
 +      // throw it direct.
 +      Throwable cause = ex.getCause();
 +      if (cause instanceof IOException) {
 +        throw (IOException) cause;
 +      } else {
 +        throw ex;
 +      }
      } catch (InterruptedException ex) {
-       LOG.error(ex);
+       LOG.error(ex.toString());
        throw new IOException("Failed to get the AbstractFileSystem for path: "
            + uri, ex);
      }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b114f248/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b114f248/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --cc hadoop-project/pom.xml
index 8fcdf0c,8151016..e6a3605
mode 100644,100755..100755
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b114f248/hadoop-tools/hadoop-aws/pom.xml
----------------------------------------------------------------------
diff --cc hadoop-tools/hadoop-aws/pom.xml
index 6cb0202,91e94a6..62371c3
--- a/hadoop-tools/hadoop-aws/pom.xml
+++ b/hadoop-tools/hadoop-aws/pom.xml
@@@ -444,46 -334,6 +444,26 @@@
        <scope>compile</scope>
      </dependency>
      <dependency>
-       <groupId>com.fasterxml.jackson.core</groupId>
-       <artifactId>jackson-core</artifactId>
-     </dependency>
-     <dependency>
-       <groupId>com.fasterxml.jackson.core</groupId>
-       <artifactId>jackson-databind</artifactId>
-     </dependency>
-     <dependency>
-       <groupId>com.fasterxml.jackson.core</groupId>
-       <artifactId>jackson-annotations</artifactId>
-     </dependency>
-     <dependency>
-       <groupId>com.fasterxml.jackson.dataformat</groupId>
-       <artifactId>jackson-dataformat-cbor</artifactId>
-     </dependency>
-     <dependency>
-       <groupId>joda-time</groupId>
-       <artifactId>joda-time</artifactId>
-     </dependency>
-     <dependency>
 +      <groupId>com.amazonaws</groupId>
 +      <artifactId>DynamoDBLocal</artifactId>
 +      <version>${dynamodb.local.version}</version>
 +      <scope>test</scope>
 +      <exclusions>
 +        <exclusion>
 +          <groupId>org.hamcrest</groupId>
 +          <artifactId>hamcrest-core</artifactId>
 +        </exclusion>
 +        <exclusion>
 +          <groupId>org.eclipse.jetty</groupId>
 +          <artifactId>jetty-http</artifactId>
 +        </exclusion>
 +        <exclusion>
 +          <groupId>org.apache.commons</groupId>
 +          <artifactId>commons-lang3</artifactId>
 +        </exclusion>
 +      </exclusions>
 +    </dependency>
 +    <dependency>
        <groupId>junit</groupId>
        <artifactId>junit</artifactId>
        <scope>test</scope>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[14/51] [abbrv] hadoop git commit: HADOOP-14126. Remove jackson, joda and other transient aws SDK dependencies from hadoop-aws. Contributed by Steve Loughran

Posted by st...@apache.org.
HADOOP-14126. Remove jackson, joda and other transient aws SDK dependencies from hadoop-aws.
Contributed by Steve Loughran

(cherry picked from commit ced547d5f0dbea571cbc472c5f55fe89d5900a6f)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7fc324aa
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7fc324aa
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7fc324aa

Branch: refs/heads/HADOOP-13345
Commit: 7fc324aabda8953ddbc296ced767dc9398dd9d6c
Parents: 691bf5e
Author: Steve Loughran <st...@apache.org>
Authored: Fri Aug 4 11:09:08 2017 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Aug 4 11:09:08 2017 +0100

----------------------------------------------------------------------
 hadoop-tools/hadoop-aws/pom.xml | 22 +---------------------
 1 file changed, 1 insertion(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7fc324aa/hadoop-tools/hadoop-aws/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml
index c995ca6..91e94a6 100644
--- a/hadoop-tools/hadoop-aws/pom.xml
+++ b/hadoop-tools/hadoop-aws/pom.xml
@@ -315,7 +315,7 @@
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
-      <scope>compile</scope>
+      <scope>provided</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
@@ -334,26 +334,6 @@
       <scope>compile</scope>
     </dependency>
     <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-core</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-databind</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-annotations</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.fasterxml.jackson.dataformat</groupId>
-      <artifactId>jackson-dataformat-cbor</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>joda-time</groupId>
-      <artifactId>joda-time</artifactId>
-    </dependency>
-    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[40/51] [abbrv] hadoop git commit: HADOOP-14715. TestWasbRemoteCallHelper failing. Contributed by Esfandiar Manii.

Posted by st...@apache.org.
HADOOP-14715. TestWasbRemoteCallHelper failing.
Contributed by Esfandiar Manii.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f4e1aa05
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f4e1aa05
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f4e1aa05

Branch: refs/heads/HADOOP-13345
Commit: f4e1aa0508cadcf9d4ecc4053d8c1cf6ddd6c31b
Parents: 71b8dda
Author: Steve Loughran <st...@apache.org>
Authored: Tue Aug 8 23:37:47 2017 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Tue Aug 8 23:37:47 2017 +0100

----------------------------------------------------------------------
 .../apache/hadoop/fs/azure/TestWasbRemoteCallHelper.java |  7 +++++--
 .../hadoop-azure/src/test/resources/azure-test.xml       | 11 +++++++----
 2 files changed, 12 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4e1aa05/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestWasbRemoteCallHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestWasbRemoteCallHelper.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestWasbRemoteCallHelper.java
index 393dcfd..8aad9e9 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestWasbRemoteCallHelper.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestWasbRemoteCallHelper.java
@@ -282,6 +282,8 @@ public class TestWasbRemoteCallHelper
   @Test
   public void testWhenOneInstanceIsDown() throws Throwable {
 
+    boolean isAuthorizationCachingEnabled = fs.getConf().getBoolean(CachingAuthorizer.KEY_AUTH_SERVICE_CACHING_ENABLE, false);
+
     // set up mocks
     HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
     HttpEntity mockHttpEntity = Mockito.mock(HttpEntity.class);
@@ -356,8 +358,9 @@ public class TestWasbRemoteCallHelper
 
     performop(mockHttpClient);
 
-    Mockito.verify(mockHttpClient, times(2)).execute(Mockito.argThat(new HttpGetForServiceLocal()));
-    Mockito.verify(mockHttpClient, times(2)).execute(Mockito.argThat(new HttpGetForService2()));
+    int expectedNumberOfInvocations = isAuthorizationCachingEnabled ? 1 : 2;
+    Mockito.verify(mockHttpClient, times(expectedNumberOfInvocations)).execute(Mockito.argThat(new HttpGetForServiceLocal()));
+    Mockito.verify(mockHttpClient, times(expectedNumberOfInvocations)).execute(Mockito.argThat(new HttpGetForService2()));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4e1aa05/hadoop-tools/hadoop-azure/src/test/resources/azure-test.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/resources/azure-test.xml b/hadoop-tools/hadoop-azure/src/test/resources/azure-test.xml
index 8c88743..8cea256 100644
--- a/hadoop-tools/hadoop-azure/src/test/resources/azure-test.xml
+++ b/hadoop-tools/hadoop-azure/src/test/resources/azure-test.xml
@@ -29,10 +29,13 @@
     </property>
   -->
 
-  <property>
-    <name>fs.azure.secure.mode</name>
-    <value>true</value>
-  </property>
+  <!-- uncomment to test in Azure secure mode -->
+  <!--
+    <property>
+      <name>fs.azure.secure.mode</name>
+      <value>true</value>
+    </property>
+  -->
 
   <property>
     <name>fs.azure.user.agent.prefix</name>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[28/51] [abbrv] hadoop git commit: YARN-4161. Capacity Scheduler : Assign single or multiple containers per heart beat driven by configuration. (Wei Yan via wangda)

Posted by st...@apache.org.
YARN-4161. Capacity Scheduler : Assign single or multiple containers per heart beat driven by configuration. (Wei Yan via wangda)

Change-Id: Ic441ae4e0bf72e7232411eb54243ec143d5fd0d3


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/adb84f34
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/adb84f34
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/adb84f34

Branch: refs/heads/HADOOP-13345
Commit: adb84f34db7e1cdcd72aa8e3deb464c48da9e353
Parents: a3a9c97
Author: Wangda Tan <wa...@apache.org>
Authored: Mon Aug 7 11:32:12 2017 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Mon Aug 7 11:32:21 2017 -0700

----------------------------------------------------------------------
 .../scheduler/capacity/CapacityScheduler.java   |  53 ++++-
 .../CapacitySchedulerConfiguration.java         |  23 ++
 .../capacity/TestCapacityScheduler.java         | 232 ++++++++++++++++++-
 3 files changed, 289 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/adb84f34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
index 2ccaf63..3286982 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
@@ -94,11 +94,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueInvalidExcep
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesLogger;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesManager;
@@ -163,6 +161,9 @@ public class CapacityScheduler extends
 
   private int offswitchPerHeartbeatLimit;
 
+  private boolean assignMultipleEnabled;
+
+  private int maxAssignPerHeartbeat;
 
   @Override
   public void setConf(Configuration conf) {
@@ -308,6 +309,9 @@ public class CapacityScheduler extends
       asyncScheduleInterval = this.conf.getLong(ASYNC_SCHEDULER_INTERVAL,
           DEFAULT_ASYNC_SCHEDULER_INTERVAL);
 
+      this.assignMultipleEnabled = this.conf.getAssignMultipleEnabled();
+      this.maxAssignPerHeartbeat = this.conf.getMaxAssignPerHeartbeat();
+
       // number of threads for async scheduling
       int maxAsyncSchedulingThreads = this.conf.getInt(
           CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_MAXIMUM_THREAD,
@@ -1109,17 +1113,29 @@ public class CapacityScheduler extends
       .getAssignmentInformation().getReserved());
   }
 
-  private boolean canAllocateMore(CSAssignment assignment, int offswitchCount) {
-    if (null != assignment && Resources.greaterThan(getResourceCalculator(),
-        getClusterResource(), assignment.getResource(), Resources.none())
-        && offswitchCount < offswitchPerHeartbeatLimit) {
-      // And it should not be a reserved container
-      if (assignment.getAssignmentInformation().getNumReservations() == 0) {
-        return true;
-      }
+  private boolean canAllocateMore(CSAssignment assignment, int offswitchCount,
+      int assignedContainers) {
+    // Current assignment shouldn't be empty
+    if (assignment == null
+            || Resources.equals(assignment.getResource(), Resources.none())) {
+      return false;
     }
 
-    return false;
+    // offswitch assignment should be under threshold
+    if (offswitchCount >= offswitchPerHeartbeatLimit) {
+      return false;
+    }
+
+    // And it should not be a reserved container
+    if (assignment.getAssignmentInformation().getNumReservations() > 0) {
+      return false;
+    }
+
+    // assignMultipleEnabled should be ON,
+    // and assignedContainers should be under threshold
+    return assignMultipleEnabled
+        && (maxAssignPerHeartbeat == -1
+            || assignedContainers < maxAssignPerHeartbeat);
   }
 
   /**
@@ -1131,6 +1147,7 @@ public class CapacityScheduler extends
     FiCaSchedulerNode node = getNode(nodeId);
     if (null != node) {
       int offswitchCount = 0;
+      int assignedContainers = 0;
 
       PlacementSet<FiCaSchedulerNode> ps = new SimplePlacementSet<>(node);
       CSAssignment assignment = allocateContainersToNode(ps, withNodeHeartbeat);
@@ -1141,7 +1158,13 @@ public class CapacityScheduler extends
           offswitchCount++;
         }
 
-        while (canAllocateMore(assignment, offswitchCount)) {
+        if (Resources.greaterThan(calculator, getClusterResource(),
+            assignment.getResource(), Resources.none())) {
+          assignedContainers++;
+        }
+
+        while (canAllocateMore(assignment, offswitchCount,
+            assignedContainers)) {
           // Try to see if it is possible to allocate multiple container for
           // the same node heartbeat
           assignment = allocateContainersToNode(ps, true);
@@ -1150,6 +1173,12 @@ public class CapacityScheduler extends
               && assignment.getType() == NodeType.OFF_SWITCH) {
             offswitchCount++;
           }
+
+          if (null != assignment
+              && Resources.greaterThan(calculator, getClusterResource(),
+                  assignment.getResource(), Resources.none())) {
+            assignedContainers++;
+          }
         }
 
         if (offswitchCount >= offswitchPerHeartbeatLimit) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/adb84f34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
index 1e29d50..13b9ff6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
@@ -301,6 +301,21 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   @Private
   public static final boolean DEFAULT_LAZY_PREEMPTION_ENABLED = false;
 
+  @Private
+  public static final String ASSIGN_MULTIPLE_ENABLED = PREFIX
+      + "per-node-heartbeat.multiple-assignments-enabled";
+
+  @Private
+  public static final boolean DEFAULT_ASSIGN_MULTIPLE_ENABLED = true;
+
+  /** Maximum number of containers to assign on each check-in. */
+  @Private
+  public static final String MAX_ASSIGN_PER_HEARTBEAT = PREFIX
+      + "per-node-heartbeat.maximum-container-assignments";
+
+  @Private
+  public static final int DEFAULT_MAX_ASSIGN_PER_HEARTBEAT = -1;
+
   AppPriorityACLConfigurationParser priorityACLConfig = new AppPriorityACLConfigurationParser();
 
   public CapacitySchedulerConfiguration() {
@@ -1473,4 +1488,12 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
     }
     return userWeights;
   }
+
+  public boolean getAssignMultipleEnabled() {
+    return getBoolean(ASSIGN_MULTIPLE_ENABLED, DEFAULT_ASSIGN_MULTIPLE_ENABLED);
+  }
+
+  public int getMaxAssignPerHeartbeat() {
+    return getInt(MAX_ASSIGN_PER_HEARTBEAT, DEFAULT_MAX_ASSIGN_PER_HEARTBEAT);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/adb84f34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
index f51f771..64e0df4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
@@ -233,6 +233,17 @@ public class TestCapacityScheduler {
     }
   }
 
+  private NodeManager registerNode(ResourceManager rm, String hostName,
+      int containerManagerPort, int httpPort, String rackName,
+          Resource capability) throws IOException, YarnException {
+    NodeManager nm = new NodeManager(hostName,
+        containerManagerPort, httpPort, rackName, capability, rm);
+    NodeAddedSchedulerEvent nodeAddEvent1 =
+        new NodeAddedSchedulerEvent(rm.getRMContext().getRMNodes()
+            .get(nm.getNodeId()));
+    rm.getResourceScheduler().handle(nodeAddEvent1);
+    return nm;
+  }
 
   @Test (timeout = 30000)
   public void testConfValidation() throws Exception {
@@ -267,12 +278,12 @@ public class TestCapacityScheduler {
     }
   }
 
-  private org.apache.hadoop.yarn.server.resourcemanager.NodeManager
+  private NodeManager
       registerNode(String hostName, int containerManagerPort, int httpPort,
           String rackName, Resource capability)
           throws IOException, YarnException {
-    org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm =
-        new org.apache.hadoop.yarn.server.resourcemanager.NodeManager(
+    NodeManager nm =
+        new NodeManager(
             hostName, containerManagerPort, httpPort, rackName, capability,
             resourceManager);
     NodeAddedSchedulerEvent nodeAddEvent1 =
@@ -400,8 +411,216 @@ public class TestCapacityScheduler {
     LOG.info("--- END: testCapacityScheduler ---");
   }
 
-  private void nodeUpdate(
-      org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm) {
+  @Test
+  public void testNotAssignMultiple() throws Exception {
+    LOG.info("--- START: testNotAssignMultiple ---");
+    ResourceManager rm = new ResourceManager() {
+      @Override
+      protected RMNodeLabelsManager createNodeLabelManager() {
+        RMNodeLabelsManager mgr = new NullRMNodeLabelsManager();
+        mgr.init(getConfig());
+        return mgr;
+      }
+    };
+    CapacitySchedulerConfiguration csConf =
+        new CapacitySchedulerConfiguration();
+    csConf.setBoolean(
+        CapacitySchedulerConfiguration.ASSIGN_MULTIPLE_ENABLED, false);
+    setupQueueConfiguration(csConf);
+    YarnConfiguration conf = new YarnConfiguration(csConf);
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    rm.init(conf);
+    rm.getRMContext().getContainerTokenSecretManager().rollMasterKey();
+    rm.getRMContext().getNMTokenSecretManager().rollMasterKey();
+    ((AsyncDispatcher) rm.getRMContext().getDispatcher()).start();
+    RMContext mC = mock(RMContext.class);
+    when(mC.getConfigurationProvider()).thenReturn(
+        new LocalConfigurationProvider());
+
+    // Register node1
+    String host0 = "host_0";
+    NodeManager nm0 =
+        registerNode(rm, host0, 1234, 2345, NetworkTopology.DEFAULT_RACK,
+            Resources.createResource(10 * GB, 10));
+
+    // ResourceRequest priorities
+    Priority priority0 = Priority.newInstance(0);
+    Priority priority1 = Priority.newInstance(1);
+
+    // Submit an application
+    Application application0 = new Application("user_0", "a1", rm);
+    application0.submit();
+    application0.addNodeManager(host0, 1234, nm0);
+
+    Resource capability00 = Resources.createResource(1 * GB, 1);
+    application0.addResourceRequestSpec(priority0, capability00);
+
+    Resource capability01 = Resources.createResource(2 * GB, 1);
+    application0.addResourceRequestSpec(priority1, capability01);
+
+    Task task00 =
+        new Task(application0, priority0, new String[] {host0});
+    Task task01 =
+        new Task(application0, priority1, new String[] {host0});
+    application0.addTask(task00);
+    application0.addTask(task01);
+
+    // Submit another application
+    Application application1 = new Application("user_1", "b2", rm);
+    application1.submit();
+    application1.addNodeManager(host0, 1234, nm0);
+
+    Resource capability10 = Resources.createResource(3 * GB, 1);
+    application1.addResourceRequestSpec(priority0, capability10);
+
+    Resource capability11 = Resources.createResource(4 * GB, 1);
+    application1.addResourceRequestSpec(priority1, capability11);
+
+    Task task10 = new Task(application1, priority0, new String[] {host0});
+    Task task11 = new Task(application1, priority1, new String[] {host0});
+    application1.addTask(task10);
+    application1.addTask(task11);
+
+    // Send resource requests to the scheduler
+    application0.schedule();
+
+    application1.schedule();
+
+    // Send a heartbeat to kick the tires on the Scheduler
+    LOG.info("Kick!");
+
+    // task00, used=1G
+    nodeUpdate(rm, nm0);
+
+    // Get allocations from the scheduler
+    application0.schedule();
+    application1.schedule();
+    // 1 Task per heart beat should be scheduled
+    checkNodeResourceUsage(3 * GB, nm0); // task00 (1G)
+    checkApplicationResourceUsage(0 * GB, application0);
+    checkApplicationResourceUsage(3 * GB, application1);
+
+    // Another heartbeat
+    nodeUpdate(rm, nm0);
+    application0.schedule();
+    checkApplicationResourceUsage(1 * GB, application0);
+    application1.schedule();
+    checkApplicationResourceUsage(3 * GB, application1);
+    checkNodeResourceUsage(4 * GB, nm0);
+    LOG.info("--- START: testNotAssignMultiple ---");
+  }
+
+  @Test
+  public void testAssignMultiple() throws Exception {
+    LOG.info("--- START: testAssignMultiple ---");
+    ResourceManager rm = new ResourceManager() {
+      @Override
+      protected RMNodeLabelsManager createNodeLabelManager() {
+        RMNodeLabelsManager mgr = new NullRMNodeLabelsManager();
+        mgr.init(getConfig());
+        return mgr;
+      }
+    };
+    CapacitySchedulerConfiguration csConf =
+        new CapacitySchedulerConfiguration();
+    csConf.setBoolean(
+        CapacitySchedulerConfiguration.ASSIGN_MULTIPLE_ENABLED, true);
+    // Each heartbeat will assign 2 containers at most
+    csConf.setInt(CapacitySchedulerConfiguration.MAX_ASSIGN_PER_HEARTBEAT, 2);
+    setupQueueConfiguration(csConf);
+    YarnConfiguration conf = new YarnConfiguration(csConf);
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    rm.init(conf);
+    rm.getRMContext().getContainerTokenSecretManager().rollMasterKey();
+    rm.getRMContext().getNMTokenSecretManager().rollMasterKey();
+    ((AsyncDispatcher) rm.getRMContext().getDispatcher()).start();
+    RMContext mC = mock(RMContext.class);
+    when(mC.getConfigurationProvider()).thenReturn(
+            new LocalConfigurationProvider());
+
+    // Register node1
+    String host0 = "host_0";
+    NodeManager nm0 =
+        registerNode(rm, host0, 1234, 2345, NetworkTopology.DEFAULT_RACK,
+            Resources.createResource(10 * GB, 10));
+
+    // ResourceRequest priorities
+    Priority priority0 = Priority.newInstance(0);
+    Priority priority1 = Priority.newInstance(1);
+
+    // Submit an application
+    Application application0 = new Application("user_0", "a1", rm);
+    application0.submit();
+    application0.addNodeManager(host0, 1234, nm0);
+
+    Resource capability00 = Resources.createResource(1 * GB, 1);
+    application0.addResourceRequestSpec(priority0, capability00);
+
+    Resource capability01 = Resources.createResource(2 * GB, 1);
+    application0.addResourceRequestSpec(priority1, capability01);
+
+    Task task00 = new Task(application0, priority0, new String[] {host0});
+    Task task01 = new Task(application0, priority1, new String[] {host0});
+    application0.addTask(task00);
+    application0.addTask(task01);
+
+    // Submit another application
+    Application application1 = new Application("user_1", "b2", rm);
+    application1.submit();
+    application1.addNodeManager(host0, 1234, nm0);
+
+    Resource capability10 = Resources.createResource(3 * GB, 1);
+    application1.addResourceRequestSpec(priority0, capability10);
+
+    Resource capability11 = Resources.createResource(4 * GB, 1);
+    application1.addResourceRequestSpec(priority1, capability11);
+
+    Task task10 =
+            new Task(application1, priority0, new String[] {host0});
+    Task task11 =
+            new Task(application1, priority1, new String[] {host0});
+    application1.addTask(task10);
+    application1.addTask(task11);
+
+    // Send resource requests to the scheduler
+    application0.schedule();
+
+    application1.schedule();
+
+    // Send a heartbeat to kick the tires on the Scheduler
+    LOG.info("Kick!");
+
+    // task_0_0, used=1G
+    nodeUpdate(rm, nm0);
+
+    // Get allocations from the scheduler
+    application0.schedule();
+    application1.schedule();
+    // 1 Task per heart beat should be scheduled
+    checkNodeResourceUsage(4 * GB, nm0); // task00 (1G)
+    checkApplicationResourceUsage(1 * GB, application0);
+    checkApplicationResourceUsage(3 * GB, application1);
+
+    // Another heartbeat
+    nodeUpdate(rm, nm0);
+    application0.schedule();
+    checkApplicationResourceUsage(3 * GB, application0);
+    application1.schedule();
+    checkApplicationResourceUsage(7 * GB, application1);
+    checkNodeResourceUsage(10 * GB, nm0);
+    LOG.info("--- START: testAssignMultiple ---");
+  }
+
+  private void nodeUpdate(ResourceManager rm, NodeManager nm) {
+    RMNode node = rm.getRMContext().getRMNodes().get(nm.getNodeId());
+    // Send a heartbeat to kick the tires on the Scheduler
+    NodeUpdateSchedulerEvent nodeUpdate = new NodeUpdateSchedulerEvent(node);
+    rm.getResourceScheduler().handle(nodeUpdate);
+  }
+
+  private void nodeUpdate(NodeManager nm) {
     RMNode node = resourceManager.getRMContext().getRMNodes().get(nm.getNodeId());
     // Send a heartbeat to kick the tires on the Scheduler
     NodeUpdateSchedulerEvent nodeUpdate = new NodeUpdateSchedulerEvent(node);
@@ -699,8 +918,7 @@ public class TestCapacityScheduler {
     Assert.assertEquals(expected, application.getUsedResources().getMemorySize());
   }
 
-  private void checkNodeResourceUsage(int expected,
-      org.apache.hadoop.yarn.server.resourcemanager.NodeManager node) {
+  private void checkNodeResourceUsage(int expected, NodeManager node) {
     Assert.assertEquals(expected, node.getUsed().getMemorySize());
     node.checkResourceUsage();
   }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[03/51] [abbrv] hadoop git commit: HDFS-9388. Decommission related code to support Maintenance State for datanodes.

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/79df1e75/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
index 38c17b7..7cdbde2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
@@ -947,7 +947,8 @@ public class TestFsck {
     DatanodeDescriptor dnDesc0 = dnm.getDatanode(
         cluster.getDataNodes().get(0).getDatanodeId());
 
-    bm.getDatanodeManager().getDecomManager().startDecommission(dnDesc0);
+    bm.getDatanodeManager().getDatanodeAdminManager().startDecommission(
+        dnDesc0);
     final String dn0Name = dnDesc0.getXferAddr();
 
     // check the replica status while decommissioning
@@ -1000,7 +1001,7 @@ public class TestFsck {
         cluster.getDataNodes().get(1).getDatanodeId());
     final String dn1Name = dnDesc1.getXferAddr();
 
-    bm.getDatanodeManager().getDecomManager().startMaintenance(dnDesc1,
+    bm.getDatanodeManager().getDatanodeAdminManager().startMaintenance(dnDesc1,
         Long.MAX_VALUE);
 
     // check the replica status while entering maintenance
@@ -1539,7 +1540,7 @@ public class TestFsck {
       fsn.writeUnlock();
     }
     DatanodeDescriptor dn = bc.getBlocks()[0].getDatanode(0);
-    bm.getDatanodeManager().getDecomManager().startDecommission(dn);
+    bm.getDatanodeManager().getDatanodeAdminManager().startDecommission(dn);
     String dnName = dn.getXferAddr();
 
     //wait for decommission start
@@ -1619,7 +1620,7 @@ public class TestFsck {
     DatanodeManager dnm = bm.getDatanodeManager();
     DatanodeDescriptor dn = dnm.getDatanode(cluster.getDataNodes().get(0)
         .getDatanodeId());
-    bm.getDatanodeManager().getDecomManager().startMaintenance(dn,
+    bm.getDatanodeManager().getDatanodeAdminManager().startMaintenance(dn,
         Long.MAX_VALUE);
     final String dnName = dn.getXferAddr();
 
@@ -1854,7 +1855,7 @@ public class TestFsck {
     }
     DatanodeDescriptor dn = bc.getBlocks()[0]
         .getDatanode(0);
-    bm.getDatanodeManager().getDecomManager().startDecommission(dn);
+    bm.getDatanodeManager().getDatanodeAdminManager().startDecommission(dn);
     String dnName = dn.getXferAddr();
 
     // wait for decommission start
@@ -1933,7 +1934,7 @@ public class TestFsck {
     DatanodeManager dnm = bm.getDatanodeManager();
     DatanodeDescriptor dn = dnm.getDatanode(cluster.getDataNodes().get(0)
         .getDatanodeId());
-    bm.getDatanodeManager().getDecomManager().startMaintenance(dn,
+    bm.getDatanodeManager().getDatanodeAdminManager().startMaintenance(dn,
         Long.MAX_VALUE);
     final String dnName = dn.getXferAddr();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79df1e75/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
index 63f9113..937bb61 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
@@ -370,7 +370,7 @@ public class TestNameNodeMXBean {
           cluster.getDataNodes().get(0).getDisplayName());
       fsn.getBlockManager().getDatanodeManager().refreshNodes(conf);
 
-      // Wait for the DecommissionManager to complete refresh nodes
+      // Wait for the DatanodeAdminManager to complete refresh nodes
       GenericTestUtils.waitFor(new Supplier<Boolean>() {
         @Override
         public Boolean get() {
@@ -399,7 +399,7 @@ public class TestNameNodeMXBean {
       assertEquals(0, fsn.getNumDecomLiveDataNodes());
       assertEquals(0, fsn.getNumDecomDeadDataNodes());
 
-      // Wait for the DecommissionManager to complete check
+      // Wait for the DatanodeAdminManager to complete check
       GenericTestUtils.waitFor(new Supplier<Boolean>() {
         @Override
         public Boolean get() {
@@ -501,7 +501,7 @@ public class TestNameNodeMXBean {
         assertEquals(0, fsn.getNumInMaintenanceDeadDataNodes());
       }
 
-      // Wait for the DecommissionManager to complete check
+      // Wait for the DatanodeAdminManager to complete check
       // and perform state transition
       while (fsn.getNumInMaintenanceLiveDataNodes() != 1) {
         Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79df1e75/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java
index c9fe2c3..b7f0cfc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java
@@ -349,18 +349,18 @@ public class TestNamenodeCapacityReport {
   private void startDecommissionOrMaintenance(DatanodeManager dnm,
       DatanodeDescriptor dnd, boolean decomm) {
     if (decomm) {
-      dnm.getDecomManager().startDecommission(dnd);
+      dnm.getDatanodeAdminManager().startDecommission(dnd);
     } else {
-      dnm.getDecomManager().startMaintenance(dnd, Long.MAX_VALUE);
+      dnm.getDatanodeAdminManager().startMaintenance(dnd, Long.MAX_VALUE);
     }
   }
 
   private void stopDecommissionOrMaintenance(DatanodeManager dnm,
       DatanodeDescriptor dnd, boolean decomm) {
     if (decomm) {
-      dnm.getDecomManager().stopDecommission(dnd);
+      dnm.getDatanodeAdminManager().stopDecommission(dnd);
     } else {
-      dnm.getDecomManager().stopMaintenance(dnd);
+      dnm.getDatanodeAdminManager().stopMaintenance(dnd);
     }
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[29/51] [abbrv] hadoop git commit: HDFS-12264. DataNode uses a deprecated method IoUtils#cleanup. Contributed by Ajay Yadav.

Posted by st...@apache.org.
HDFS-12264. DataNode uses a deprecated method IoUtils#cleanup. Contributed by Ajay Yadav.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/bc206806
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/bc206806
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/bc206806

Branch: refs/heads/HADOOP-13345
Commit: bc206806dadc5dc85f182d98d859307cfb33172b
Parents: adb84f3
Author: Arpit Agarwal <ar...@apache.org>
Authored: Mon Aug 7 15:05:10 2017 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Mon Aug 7 15:05:10 2017 -0700

----------------------------------------------------------------------
 .../hadoop-common/src/main/java/org/apache/hadoop/io/IOUtils.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc206806/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IOUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IOUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IOUtils.java
index 1574431..46ea1c8 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IOUtils.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IOUtils.java
@@ -293,7 +293,7 @@ public class IOUtils {
    */
   public static void closeStream(java.io.Closeable stream) {
     if (stream != null) {
-      cleanup(null, stream);
+      cleanupWithLogger(null, stream);
     }
   }
   


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[43/51] [abbrv] hadoop git commit: HDFS-11975. Provide a system-default EC policy. Contributed by Huichun Lu

Posted by st...@apache.org.
HDFS-11975. Provide a system-default EC policy. Contributed by Huichun Lu


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a53b8b6f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a53b8b6f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a53b8b6f

Branch: refs/heads/HADOOP-13345
Commit: a53b8b6fdce111b1e35ad0dc563eb53d1c58462f
Parents: ad2a350
Author: Kai Zheng <ka...@intel.com>
Authored: Wed Aug 9 10:12:58 2017 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Wed Aug 9 10:12:58 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hdfs/DistributedFileSystem.java      |  2 --
 .../ClientNamenodeProtocolTranslatorPB.java     |  4 ++-
 .../src/main/proto/erasurecoding.proto          |  2 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  4 +++
 ...tNamenodeProtocolServerSideTranslatorPB.java |  4 ++-
 .../namenode/ErasureCodingPolicyManager.java    | 12 +++++--
 .../hdfs/server/namenode/NameNodeRpcServer.java | 14 +++++++-
 .../org/apache/hadoop/hdfs/tools/ECAdmin.java   | 14 ++++----
 .../src/main/resources/hdfs-default.xml         |  8 +++++
 .../src/site/markdown/HDFSErasureCoding.md      |  8 +++++
 .../hadoop/hdfs/TestErasureCodingPolicies.java  | 24 ++++++++++++--
 .../server/namenode/TestEnabledECPolicies.java  | 10 +++---
 .../test/resources/testErasureCodingConf.xml    | 35 ++++++++++++++++++++
 13 files changed, 117 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a53b8b6f/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 13c5eb9..cd368d4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -2515,8 +2515,6 @@ public class DistributedFileSystem extends FileSystem {
   public void setErasureCodingPolicy(final Path path,
       final String ecPolicyName) throws IOException {
     Path absF = fixRelativePart(path);
-    Preconditions.checkNotNull(ecPolicyName, "Erasure coding policy cannot be" +
-        " null.");
     new FileSystemLinkResolver<Void>() {
       @Override
       public Void doCall(final Path p) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a53b8b6f/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index 388788c..aed4117 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -1518,7 +1518,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
     final SetErasureCodingPolicyRequestProto.Builder builder =
         SetErasureCodingPolicyRequestProto.newBuilder();
     builder.setSrc(src);
-    builder.setEcPolicyName(ecPolicyName);
+    if (ecPolicyName != null) {
+      builder.setEcPolicyName(ecPolicyName);
+    }
     SetErasureCodingPolicyRequestProto req = builder.build();
     try {
       rpcProxy.setErasureCodingPolicy(null, req);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a53b8b6f/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto
index 65baab6..9f80350 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto
@@ -25,7 +25,7 @@ import "hdfs.proto";
 
 message SetErasureCodingPolicyRequestProto {
   required string src = 1;
-  required string ecPolicyName = 2;
+  optional string ecPolicyName = 2;
 }
 
 message SetErasureCodingPolicyResponseProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a53b8b6f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index d9568f2..dc9bf76 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -564,6 +564,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_NAMENODE_EC_POLICIES_ENABLED_DEFAULT = "";
   public static final String  DFS_NAMENODE_EC_POLICIES_MAX_CELLSIZE_KEY = "dfs.namenode.ec.policies.max.cellsize";
   public static final int     DFS_NAMENODE_EC_POLICIES_MAX_CELLSIZE_DEFAULT = 4 * 1024 * 1024;
+  public static final String  DFS_NAMENODE_EC_SYSTEM_DEFAULT_POLICY =
+      "dfs.namenode.ec.system.default.policy";
+  public static final String  DFS_NAMENODE_EC_SYSTEM_DEFAULT_POLICY_DEFAULT =
+      "RS-6-3-64k";
   public static final String  DFS_DN_EC_RECONSTRUCTION_STRIPED_READ_THREADS_KEY = "dfs.datanode.ec.reconstruction.stripedread.threads";
   public static final int     DFS_DN_EC_RECONSTRUCTION_STRIPED_READ_THREADS_DEFAULT = 20;
   public static final String  DFS_DN_EC_RECONSTRUCTION_STRIPED_READ_BUFFER_SIZE_KEY = "dfs.datanode.ec.reconstruction.stripedread.buffer.size";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a53b8b6f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index 4ac49fe..38b81c6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@ -1488,7 +1488,9 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       RpcController controller, SetErasureCodingPolicyRequestProto req)
       throws ServiceException {
     try {
-      server.setErasureCodingPolicy(req.getSrc(), req.getEcPolicyName());
+      String ecPolicyName = req.hasEcPolicyName() ?
+          req.getEcPolicyName() : null;
+      server.setErasureCodingPolicy(req.getSrc(), ecPolicyName);
       return SetErasureCodingPolicyResponseProto.newBuilder().build();
     } catch (IOException e) {
       throw new ServiceException(e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a53b8b6f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
index 266d45c..18b8e8a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import org.apache.commons.lang.ArrayUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -92,9 +93,14 @@ public final class ErasureCodingPolicyManager {
 
   public void init(Configuration conf) {
     // Populate the list of enabled policies from configuration
-    final String[] policyNames = conf.getTrimmedStrings(
-        DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_KEY,
-        DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_DEFAULT);
+    final String[] enablePolicyNames = conf.getTrimmedStrings(
+            DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_KEY,
+            DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_DEFAULT);
+    final String defaultPolicyName = conf.getTrimmed(
+            DFSConfigKeys.DFS_NAMENODE_EC_SYSTEM_DEFAULT_POLICY,
+            DFSConfigKeys.DFS_NAMENODE_EC_SYSTEM_DEFAULT_POLICY_DEFAULT);
+    final String[] policyNames =
+            (String[]) ArrayUtils.add(enablePolicyNames, defaultPolicyName);
     this.userPoliciesByID = new TreeMap<>();
     this.userPoliciesByName = new TreeMap<>();
     this.removedPoliciesByName = new TreeMap<>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a53b8b6f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 9265381..d304d3d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -251,13 +251,15 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   
   private final String minimumDataNodeVersion;
 
+  private final String defaultECPolicyName;
+
   public NameNodeRpcServer(Configuration conf, NameNode nn)
       throws IOException {
     this.nn = nn;
     this.namesystem = nn.getNamesystem();
     this.retryCache = namesystem.getRetryCache();
     this.metrics = NameNode.getNameNodeMetrics();
-    
+
     int handlerCount = 
       conf.getInt(DFS_NAMENODE_HANDLER_COUNT_KEY, 
                   DFS_NAMENODE_HANDLER_COUNT_DEFAULT);
@@ -490,6 +492,10 @@ public class NameNodeRpcServer implements NamenodeProtocols {
         DFSConfigKeys.DFS_NAMENODE_MIN_SUPPORTED_DATANODE_VERSION_KEY,
         DFSConfigKeys.DFS_NAMENODE_MIN_SUPPORTED_DATANODE_VERSION_DEFAULT);
 
+    defaultECPolicyName = conf.get(
+        DFSConfigKeys.DFS_NAMENODE_EC_SYSTEM_DEFAULT_POLICY,
+        DFSConfigKeys.DFS_NAMENODE_EC_SYSTEM_DEFAULT_POLICY_DEFAULT);
+
     // Set terse exception whose stack trace won't be logged
     clientRpcServer.addTerseExceptions(SafeModeException.class,
         FileNotFoundException.class,
@@ -2055,6 +2061,12 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     }
     boolean success = false;
     try {
+      if (ecPolicyName == null) {
+        ecPolicyName = defaultECPolicyName;
+        LOG.trace("No policy name is specified, " +
+            "set the default policy name instead");
+      }
+      LOG.trace("Set erasure coding policy " + ecPolicyName + " on " + src);
       namesystem.setErasureCodingPolicy(src, ecPolicyName, cacheEntry != null);
       success = true;
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a53b8b6f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
index 5006b5a..46600a0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
@@ -335,11 +335,6 @@ public class ECAdmin extends Configured implements Tool {
 
       final String ecPolicyName = StringUtils.popOptionWithArgument("-policy",
           args);
-      if (ecPolicyName == null) {
-        System.err.println("Please specify the policy name.\nUsage: " +
-            getLongUsage());
-        return 1;
-      }
 
       if (args.size() > 0) {
         System.err.println(getName() + ": Too many arguments");
@@ -350,8 +345,13 @@ public class ECAdmin extends Configured implements Tool {
       final DistributedFileSystem dfs = AdminHelper.getDFS(p.toUri(), conf);
       try {
         dfs.setErasureCodingPolicy(p, ecPolicyName);
-        System.out.println("Set erasure coding policy " + ecPolicyName +
-            " on " + path);
+        if (ecPolicyName == null){
+          System.out.println("Set default erasure coding policy" +
+              " on " + path);
+        } else {
+          System.out.println("Set erasure coding policy " + ecPolicyName +
+              " on " + path);
+        }
       } catch (Exception e) {
         System.err.println(AdminHelper.prettifyException(e));
         return 2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a53b8b6f/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index bb62359..4942967 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -2976,6 +2976,14 @@
 </property>
 
 <property>
+  <name>dfs.namenode.ec.system.default.policy</name>
+  <value>RS-6-3-64k</value>
+  <description>The default erasure coding policy name will be used
+    on the path if no policy name is passed.
+  </description>
+</property>
+
+<property>
   <name>dfs.namenode.ec.policies.max.cellsize</name>
   <value>4194304</value>
   <description>The maximum cell size of erasure coding policy. Default is 4MB.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a53b8b6f/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
index 88293ba..4a48c2a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
@@ -117,6 +117,11 @@ Deployment
   be more appropriate. If the administrator only cares about node-level fault-tolerance, `RS-10-4-64k` would still be appropriate as long as
   there are at least 14 DataNodes in the cluster.
 
+  A system default EC policy can be configured via 'dfs.namenode.ec.system.default.policy' configuration. With this configuration,
+  the default EC policy will be used when no policy name is passed as an argument in the '-setPolicy' command.
+
+  By default, the 'dfs.namenode.ec.system.default.policy' is "RS-6-3-64k".
+
   The codec implementations for Reed-Solomon and XOR can be configured with the following client and DataNode configuration keys:
   `io.erasurecode.codec.rs.rawcoders` for the default RS codec,
   `io.erasurecode.codec.rs-legacy.rawcoders` for the legacy RS codec,
@@ -167,6 +172,9 @@ Below are the details about each command.
       `path`: An directory in HDFS. This is a mandatory parameter. Setting a policy only affects newly created files, and does not affect existing files.
 
       `policyName`: The erasure coding policy to be used for files under this directory.
+      This parameter can be omitted if a 'dfs.namenode.ec.system.default.policy' configuration is set.
+      The EC policy of the path will be set with the default value in configuration.
+
 
  *  `[-getPolicy -path <path>]`
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a53b8b6f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
index 127dad1..06edb1a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
@@ -209,9 +209,9 @@ public class TestErasureCodingPolicies {
     cluster.restartNameNodes();
     cluster.waitActive();
 
-    // No policies should be enabled after restart
-    Assert.assertTrue("No policies should be enabled after restart",
-        fs.getAllErasureCodingPolicies().isEmpty());
+    // Only default policy should be enabled after restart
+    Assert.assertEquals("Only default policy should be enabled after restart",
+        1, fs.getAllErasureCodingPolicies().size());
 
     // Already set directory-level policies should still be in effect
     Path disabledPolicy = new Path(dir1, "afterDisabled");
@@ -360,6 +360,24 @@ public class TestErasureCodingPolicies {
   }
 
   @Test
+  public void testSetDefaultPolicy()
+          throws IOException {
+    String src = "/ecDir";
+    final Path ecDir = new Path(src);
+    try {
+      fs.mkdir(ecDir, FsPermission.getDirDefault());
+      fs.getClient().setErasureCodingPolicy(src, null);
+      String actualECPolicyName = fs.getClient().
+          getErasureCodingPolicy(src).getName();
+      String expectedECPolicyName =
+          conf.get(DFSConfigKeys.DFS_NAMENODE_EC_SYSTEM_DEFAULT_POLICY,
+          DFSConfigKeys.DFS_NAMENODE_EC_SYSTEM_DEFAULT_POLICY_DEFAULT);
+      assertEquals(expectedECPolicyName, actualECPolicyName);
+    } catch (Exception e) {
+    }
+  }
+
+  @Test
   public void testGetAllErasureCodingPolicies() throws Exception {
     Collection<ErasureCodingPolicy> allECPolicies = fs
         .getAllErasureCodingPolicies();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a53b8b6f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEnabledECPolicies.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEnabledECPolicies.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEnabledECPolicies.java
index fe95734..d769f8b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEnabledECPolicies.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEnabledECPolicies.java
@@ -75,7 +75,7 @@ public class TestEnabledECPolicies {
     String defaultECPolicies = conf.get(
         DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_KEY,
         DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_DEFAULT);
-    expectValidPolicy(defaultECPolicies, 0);
+    expectValidPolicy(defaultECPolicies, 1);
   }
 
   @Test
@@ -98,10 +98,10 @@ public class TestEnabledECPolicies {
     String ecPolicyName = StripedFileTestUtil.getDefaultECPolicy().getName();
     expectValidPolicy(ecPolicyName, 1);
     expectValidPolicy(ecPolicyName + ", ", 1);
-    expectValidPolicy(",", 0);
+    expectValidPolicy(",", 1);
     expectValidPolicy(", " + ecPolicyName, 1);
-    expectValidPolicy(" ", 0);
-    expectValidPolicy(" , ", 0);
+    expectValidPolicy(" ", 1);
+    expectValidPolicy(" , ", 1);
   }
 
   @Test
@@ -147,7 +147,7 @@ public class TestEnabledECPolicies {
       Assert.assertTrue("Did not find specified EC policy " + p.getName(),
           found.contains(p.getName()));
     }
-    Assert.assertEquals(enabledPolicies.length, found.size());
+    Assert.assertEquals(enabledPolicies.length, found.size()-1);
     // Check that getEnabledPolicyByName only returns enabled policies
     for (ErasureCodingPolicy p: SystemErasureCodingPolicies.getPolicies()) {
       if (found.contains(p.getName())) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a53b8b6f/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
index 127effc..c68c6d6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
@@ -553,6 +553,41 @@
     </test>
 
     <test>
+      <description>setPolicy : set erasure coding policy without given a specific policy name</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /ecdir</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -path /ecdir</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rmdir /ecdir</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Set default erasure coding policy on /ecdir</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>getPolicy: get the default policy after setPolicy without given a specific policy name</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /ecdir</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -path /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getPolicy -path /ecdir</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rmdir /ecdir</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>RS-6-3-64k</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
       <description>getPolicy : illegal parameters - path is missing</description>
       <test-commands>
         <ec-admin-command>-fs NAMENODE -getPolicy </ec-admin-command>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[23/51] [abbrv] hadoop git commit: HDFS-12198. Document missing namenode metrics that were added recently. Contributed by Yiqun Lin.

Posted by st...@apache.org.
HDFS-12198. Document missing namenode metrics that were added recently. Contributed by Yiqun Lin.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a4eb7016
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a4eb7016
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a4eb7016

Branch: refs/heads/HADOOP-13345
Commit: a4eb7016cb20dfbc656b831c603136785e62fddc
Parents: 46b7054
Author: Akira Ajisaka <aa...@apache.org>
Authored: Mon Aug 7 18:47:33 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Mon Aug 7 18:47:33 2017 +0900

----------------------------------------------------------------------
 .../hadoop-common/src/site/markdown/Metrics.md              | 9 +++++++--
 1 file changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4eb7016/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
index 852a1e9..4543fac 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
@@ -145,6 +145,9 @@ Each metrics record contains tags such as ProcessName, SessionId, and Hostname a
 | `CreateSymlinkOps` | Total number of createSymlink operations |
 | `GetLinkTargetOps` | Total number of getLinkTarget operations |
 | `FilesInGetListingOps` | Total number of files and directories listed by directory listing operations |
+| `SuccessfulReReplications` | Total number of successful block re-replications |
+| `NumTimesReReplicationNotScheduled` | Total number of times that failed to schedule a block re-replication |
+| `TimeoutReReplications` | Total number of timed out block re-replications |
 | `AllowSnapshotOps` | Total number of allowSnapshot operations |
 | `DisallowSnapshotOps` | Total number of disallowSnapshot operations |
 | `CreateSnapshotOps` | Total number of createSnapshot operations |
@@ -157,8 +160,8 @@ Each metrics record contains tags such as ProcessName, SessionId, and Hostname a
 | `SyncsNumOps` | Total number of Journal syncs |
 | `SyncsAvgTime` | Average time of Journal syncs in milliseconds |
 | `TransactionsBatchedInSync` | Total number of Journal transactions batched in sync |
-| `BlockReportNumOps` | Total number of processing block reports from DataNode |
-| `BlockReportAvgTime` | Average time of processing block reports in milliseconds |
+| `StorageBlockReportNumOps` | Total number of processing block reports from individual storages in DataNode |
+| `StorageBlockReportAvgTime` | Average time of processing block reports in milliseconds |
 | `CacheReportNumOps` | Total number of processing cache reports from DataNode |
 | `CacheReportAvgTime` | Average time of processing cache reports in milliseconds |
 | `SafeModeTime` | The interval between FSNameSystem starts and the last time safemode leaves in milliseconds.  (sometimes not equal to the time in SafeMode, see [HDFS-5156](https://issues.apache.org/jira/browse/HDFS-5156)) |
@@ -176,6 +179,8 @@ Each metrics record contains tags such as ProcessName, SessionId, and Hostname a
 | `GenerateEDEKTimeAvgTime` | Average time of generating EDEK in milliseconds |
 | `WarmUpEDEKTimeNumOps` | Total number of warming up EDEK |
 | `WarmUpEDEKTimeAvgTime` | Average time of warming up EDEK in milliseconds |
+| `ResourceCheckTime`*num*`s(50|75|90|95|99)thPercentileLatency` | The 50/75/90/95/99th percentile of NameNode resource check latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
+| `StorageBlockReport`*num*`s(50|75|90|95|99)thPercentileLatency` | The 50/75/90/95/99th percentile of storage block report latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
 
 FSNamesystem
 ------------


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[42/51] [abbrv] hadoop git commit: YARN-6970. Add PoolInitializationException as retriable exception in FederationFacade. (Giovanni Matteo Fumarola via Subru).

Posted by st...@apache.org.
YARN-6970. Add PoolInitializationException as retriable exception in FederationFacade. (Giovanni Matteo Fumarola via Subru).


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ad2a3506
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ad2a3506
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ad2a3506

Branch: refs/heads/HADOOP-13345
Commit: ad2a3506626728a6be47af0db3ca60610a568734
Parents: 1db4788
Author: Subru Krishnan <su...@apache.org>
Authored: Tue Aug 8 16:48:29 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Aug 8 16:48:29 2017 -0700

----------------------------------------------------------------------
 .../utils/FederationStateStoreFacade.java       |  2 ++
 .../TestFederationStateStoreFacadeRetry.java    | 24 ++++++++++++++++++++
 2 files changed, 26 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad2a3506/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
index 389c769..682eb14 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
@@ -70,6 +70,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.zaxxer.hikari.pool.HikariPool.PoolInitializationException;
 
 /**
  *
@@ -162,6 +163,7 @@ public final class FederationStateStoreFacade {
     exceptionToPolicyMap.put(FederationStateStoreRetriableException.class,
         basePolicy);
     exceptionToPolicyMap.put(CacheLoaderException.class, basePolicy);
+    exceptionToPolicyMap.put(PoolInitializationException.class, basePolicy);
 
     RetryPolicy retryPolicy = RetryPolicies.retryByException(
         RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad2a3506/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacadeRetry.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacadeRetry.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacadeRetry.java
index 304910e..ea43268 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacadeRetry.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacadeRetry.java
@@ -30,6 +30,8 @@ import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateS
 import org.junit.Assert;
 import org.junit.Test;
 
+import com.zaxxer.hikari.pool.HikariPool.PoolInitializationException;
+
 /**
  * Test class to validate FederationStateStoreFacade retry policy.
  */
@@ -119,4 +121,26 @@ public class TestFederationStateStoreFacadeRetry {
         policy.shouldRetry(new CacheLoaderException(""), maxRetries, 0, false);
     Assert.assertEquals(RetryAction.FAIL.action, action.action);
   }
+
+  /*
+   * Test to validate that PoolInitializationException is a retriable exception.
+   */
+  @Test
+  public void testFacadePoolInitRetriableException() throws Exception {
+    // PoolInitializationException is a retriable exception
+    conf = new Configuration();
+    conf.setInt(YarnConfiguration.CLIENT_FAILOVER_RETRIES, maxRetries);
+    RetryPolicy policy = FederationStateStoreFacade.createRetryPolicy(conf);
+    RetryAction action = policy.shouldRetry(
+        new PoolInitializationException(new YarnException()), 0, 0, false);
+    // We compare only the action, delay and the reason are random value
+    // during this test
+    Assert.assertEquals(RetryAction.RETRY.action, action.action);
+
+    // After maxRetries we stop to retry
+    action =
+        policy.shouldRetry(new PoolInitializationException(new YarnException()),
+            maxRetries, 0, false);
+    Assert.assertEquals(RetryAction.FAIL.action, action.action);
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[17/51] [abbrv] hadoop git commit: HDFS-12251. Add document for StreamCapabilities. (Lei (Eddy) Xu)

Posted by st...@apache.org.
HDFS-12251. Add document for StreamCapabilities. (Lei (Eddy) Xu)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/fe334178
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/fe334178
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/fe334178

Branch: refs/heads/HADOOP-13345
Commit: fe3341786a0d61f404127bf21d1afc85b2f21d38
Parents: a6fdeb8
Author: Lei Xu <le...@apache.org>
Authored: Fri Aug 4 11:21:58 2017 -0700
Committer: Lei Xu <le...@apache.org>
Committed: Fri Aug 4 11:21:58 2017 -0700

----------------------------------------------------------------------
 .../src/site/markdown/filesystem/filesystem.md  | 24 ++++++++++++++++++++
 .../src/site/markdown/HDFSErasureCoding.md      | 19 ++++++++++++++++
 2 files changed, 43 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe334178/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
index b56666c..d7e57ce 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
@@ -1210,3 +1210,27 @@ try {
 It is notable that this is *not* done in the Hadoop codebase. This does not imply
 that robust loops are not recommended —more that the concurrency
 problems were not considered during the implementation of these loops.
+
+
+## <a name="StreamCapability"></a> interface `StreamCapabilities`
+
+The `StreamCapabilities` provides a way to programmatically query the
+capabilities that an `OutputStream` supports.
+
+```java
+public interface StreamCapabilities {
+  boolean hasCapability(String capability);
+}
+```
+
+### `boolean hasCapability(capability)`
+
+Return true if the `OutputStream` has the desired capability.
+
+The caller can query the capabilities of a stream using a string value.
+It currently supports to query:
+
+ * `StreamCapabilties.HFLUSH` ("*hflush*"): the capability to flush out the data
+ in client's buffer.
+ * `StreamCapabilities.HSYNC` ("*hsync*"): capability to flush out the data in
+ client's buffer and the disk device.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe334178/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
index 1c0a2de..88293ba 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
@@ -199,3 +199,22 @@ Below are the details about each command.
 *  `[-disablePolicy -policy <policyName>]`
 
      Disable an erasure coding policy.
+
+Limitations
+-----------
+
+Certain HDFS file write operations, i.e., `hflush`, `hsync` and `append`,
+are not supported on erasure coded files due to substantial technical
+challenges.
+
+* `append()` on an erasure coded file will throw `IOException`.
+* `hflush()` and `hsync()` on `DFSStripedOutputStream` are no-op. Thus calling
+`hflush()` or `hsync()` on an erasure coded file can not guarantee data
+being persistent.
+
+A client can use [`StreamCapabilities`](../hadoop-common/filesystem/filesystem.html#interface_StreamCapabilities)
+API to query whether a `OutputStream` supports `hflush()` and `hsync()`.
+If the client desires data persistence via `hflush()` and `hsync()`, the current
+remedy is creating such files as regular 3x replication files in a
+non-erasure-coded directory, or using `FSDataOutputStreamBuilder#replicate()`
+API to create 3x replication files in an erasure-coded directory.


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[36/51] [abbrv] hadoop git commit: MAPREDUCE-6927. MR job should only set tracking url if history was successfully written. Contributed by Eric Badger

Posted by st...@apache.org.
MAPREDUCE-6927. MR job should only set tracking url if history was successfully written. Contributed by Eric Badger


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/735fce5b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/735fce5b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/735fce5b

Branch: refs/heads/HADOOP-13345
Commit: 735fce5bec17f4e1799daf922625c475cf588114
Parents: acf9bd8
Author: Jason Lowe <jl...@yahoo-inc.com>
Authored: Tue Aug 8 14:46:47 2017 -0500
Committer: Jason Lowe <jl...@yahoo-inc.com>
Committed: Tue Aug 8 14:46:47 2017 -0500

----------------------------------------------------------------------
 .../jobhistory/JobHistoryEventHandler.java      |  27 +++--
 .../hadoop/mapreduce/v2/app/AppContext.java     |   4 +
 .../hadoop/mapreduce/v2/app/MRAppMaster.java    |  11 ++
 .../mapreduce/v2/app/rm/RMCommunicator.java     |   4 +-
 .../jobhistory/TestJobHistoryEventHandler.java  | 102 +++++++++++++++++++
 .../hadoop/mapreduce/v2/app/MockAppContext.java |  10 ++
 .../mapreduce/v2/app/TestRuntimeEstimators.java |  10 ++
 .../hadoop/mapreduce/v2/hs/JobHistory.java      |  10 ++
 8 files changed, 168 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/735fce5b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
index 285d36e..53fe055 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
@@ -63,6 +63,7 @@ import org.apache.hadoop.mapreduce.v2.jobhistory.FileNameIndexUtils;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobIndexInfo;
+import org.apache.hadoop.mapreduce.v2.util.MRWebAppUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.util.StringUtils;
@@ -1404,7 +1405,12 @@ public class JobHistoryEventHandler extends AbstractService
         qualifiedDoneFile =
             doneDirFS.makeQualified(new Path(doneDirPrefixPath,
                 doneJobHistoryFileName));
-        moveToDoneNow(qualifiedLogFile, qualifiedDoneFile);
+        if(moveToDoneNow(qualifiedLogFile, qualifiedDoneFile)) {
+          String historyUrl = MRWebAppUtil.getApplicationWebURLOnJHSWithScheme(
+              getConfig(), context.getApplicationID());
+          context.setHistoryUrl(historyUrl);
+          LOG.info("Set historyUrl to " + historyUrl);
+        }
       }
 
       // Move confFile to Done Folder
@@ -1610,7 +1616,7 @@ public class JobHistoryEventHandler extends AbstractService
     }
   }
 
-  private void moveTmpToDone(Path tmpPath) throws IOException {
+  protected void moveTmpToDone(Path tmpPath) throws IOException {
     if (tmpPath != null) {
       String tmpFileName = tmpPath.getName();
       String fileName = getFileNameFromTmpFN(tmpFileName);
@@ -1622,7 +1628,9 @@ public class JobHistoryEventHandler extends AbstractService
   
   // TODO If the FS objects are the same, this should be a rename instead of a
   // copy.
-  private void moveToDoneNow(Path fromPath, Path toPath) throws IOException {
+  protected boolean moveToDoneNow(Path fromPath, Path toPath)
+      throws IOException {
+    boolean success = false;
     // check if path exists, in case of retries it may not exist
     if (stagingDirFS.exists(fromPath)) {
       LOG.info("Copying " + fromPath.toString() + " to " + toPath.toString());
@@ -1631,13 +1639,18 @@ public class JobHistoryEventHandler extends AbstractService
       boolean copied = FileUtil.copy(stagingDirFS, fromPath, doneDirFS, toPath,
           false, getConfig());
 
-      if (copied)
-        LOG.info("Copied to done location: " + toPath);
-      else 
-        LOG.info("copy failed");
       doneDirFS.setPermission(toPath, new FsPermission(
           JobHistoryUtils.HISTORY_INTERMEDIATE_FILE_PERMISSIONS));
+      if (copied) {
+        LOG.info("Copied from: " + fromPath.toString()
+            + " to done location: " + toPath.toString());
+        success = true;
+      } else {
+        LOG.info("Copy failed from: " + fromPath.toString()
+            + " to done location: " + toPath.toString());
+      }
     }
+    return success;
   }
 
   private String getTempFileName(String srcFile) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/735fce5b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/AppContext.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/AppContext.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/AppContext.java
index ddf4fa7..4a21396 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/AppContext.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/AppContext.java
@@ -69,4 +69,8 @@ public interface AppContext {
   String getNMHostname();
 
   TaskAttemptFinishingMonitor getTaskAttemptFinishingMonitor();
+
+  String getHistoryUrl();
+
+  void setHistoryUrl(String historyUrl);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/735fce5b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
index 8c9f605..f511f19 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
@@ -1078,6 +1078,7 @@ public class MRAppMaster extends CompositeService {
     private final ClientToAMTokenSecretManager clientToAMTokenSecretManager;
     private TimelineClient timelineClient = null;
     private TimelineV2Client timelineV2Client = null;
+    private String historyUrl = null;
 
     private final TaskAttemptFinishingMonitor taskAttemptFinishingMonitor;
 
@@ -1197,6 +1198,16 @@ public class MRAppMaster extends CompositeService {
     public TimelineV2Client getTimelineV2Client() {
       return timelineV2Client;
     }
+
+    @Override
+    public String getHistoryUrl() {
+      return historyUrl;
+    }
+
+    @Override
+    public void setHistoryUrl(String historyUrl) {
+      this.historyUrl = historyUrl;
+    }
   }
 
   @SuppressWarnings("unchecked")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/735fce5b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java
index 6cec2f3..a7058e0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java
@@ -215,9 +215,7 @@ public abstract class RMCommunicator extends AbstractService
     }
     LOG.info("Setting job diagnostics to " + sb.toString());
 
-    String historyUrl =
-        MRWebAppUtil.getApplicationWebURLOnJHSWithScheme(getConfig(),
-            context.getApplicationID());
+    String historyUrl = context.getHistoryUrl();
     LOG.info("History url is " + historyUrl);
     FinishApplicationMasterRequest request =
         FinishApplicationMasterRequest.newInstance(finishState,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/735fce5b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
index 6c5e604..caf8c67 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
@@ -21,6 +21,9 @@ package org.apache.hadoop.mapreduce.jobhistory;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
@@ -62,6 +65,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.JobStateInternal;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
 import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
+import org.apache.hadoop.mapreduce.v2.util.MRWebAppUtil;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -920,6 +924,104 @@ public class TestJobHistoryEventHandler {
         jheh.lastEventHandled.getHistoryEvent()
         instanceof JobUnsuccessfulCompletionEvent);
   }
+
+  @Test (timeout=50000)
+  public void testSetTrackingURLAfterHistoryIsWritten() throws Exception {
+    TestParams t = new TestParams(true);
+    Configuration conf = new Configuration();
+
+    JHEvenHandlerForTest realJheh =
+        new JHEvenHandlerForTest(t.mockAppContext, 0, false);
+    JHEvenHandlerForTest jheh = spy(realJheh);
+    jheh.init(conf);
+
+    try {
+      jheh.start();
+      handleEvent(jheh, new JobHistoryEvent(t.jobId, new AMStartedEvent(
+          t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000, -1)));
+      verify(jheh, times(0)).processDoneFiles(any(JobId.class));
+      verify(t.mockAppContext, times(0)).setHistoryUrl(any(String.class));
+
+      // Job finishes and successfully writes history
+      handleEvent(jheh, new JobHistoryEvent(t.jobId, new JobFinishedEvent(
+          TypeConverter.fromYarn(t.jobId), 0, 0, 0, 0, 0, new Counters(),
+          new Counters(), new Counters())));
+
+      verify(jheh, times(1)).processDoneFiles(any(JobId.class));
+      String historyUrl = MRWebAppUtil.getApplicationWebURLOnJHSWithScheme(
+          conf, t.mockAppContext.getApplicationID());
+      verify(t.mockAppContext, times(1)).setHistoryUrl(historyUrl);
+    } finally {
+      jheh.stop();
+    }
+  }
+
+  @Test (timeout=50000)
+  public void testDontSetTrackingURLIfHistoryWriteFailed() throws Exception {
+    TestParams t = new TestParams(true);
+    Configuration conf = new Configuration();
+
+    JHEvenHandlerForTest realJheh =
+        new JHEvenHandlerForTest(t.mockAppContext, 0, false);
+    JHEvenHandlerForTest jheh = spy(realJheh);
+    jheh.init(conf);
+
+    try {
+      jheh.start();
+      doReturn(false).when(jheh).moveToDoneNow(any(Path.class),
+          any(Path.class));
+      doNothing().when(jheh).moveTmpToDone(any(Path.class));
+      handleEvent(jheh, new JobHistoryEvent(t.jobId, new AMStartedEvent(
+          t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000, -1)));
+      verify(jheh, times(0)).processDoneFiles(any(JobId.class));
+      verify(t.mockAppContext, times(0)).setHistoryUrl(any(String.class));
+
+      // Job finishes, but doesn't successfully write history
+      handleEvent(jheh, new JobHistoryEvent(t.jobId, new JobFinishedEvent(
+          TypeConverter.fromYarn(t.jobId), 0, 0, 0, 0, 0, new Counters(),
+          new Counters(), new Counters())));
+      verify(jheh, times(1)).processDoneFiles(any(JobId.class));
+      verify(t.mockAppContext, times(0)).setHistoryUrl(any(String.class));
+
+    } finally {
+      jheh.stop();
+    }
+  }
+  @Test (timeout=50000)
+  public void testDontSetTrackingURLIfHistoryWriteThrows() throws Exception {
+    TestParams t = new TestParams(true);
+    Configuration conf = new Configuration();
+
+    JHEvenHandlerForTest realJheh =
+        new JHEvenHandlerForTest(t.mockAppContext, 0, false);
+    JHEvenHandlerForTest jheh = spy(realJheh);
+    jheh.init(conf);
+
+    try {
+      jheh.start();
+      doThrow(new YarnRuntimeException(new IOException()))
+          .when(jheh).processDoneFiles(any(JobId.class));
+      handleEvent(jheh, new JobHistoryEvent(t.jobId, new AMStartedEvent(
+          t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000, -1)));
+      verify(jheh, times(0)).processDoneFiles(any(JobId.class));
+      verify(t.mockAppContext, times(0)).setHistoryUrl(any(String.class));
+
+      // Job finishes, but doesn't successfully write history
+      try {
+        handleEvent(jheh, new JobHistoryEvent(t.jobId, new JobFinishedEvent(
+            TypeConverter.fromYarn(t.jobId), 0, 0, 0, 0, 0, new Counters(),
+            new Counters(), new Counters())));
+        throw new RuntimeException(
+            "processDoneFiles didn't throw, but should have");
+      } catch (YarnRuntimeException yre) {
+        // Exception expected, do nothing
+      }
+      verify(jheh, times(1)).processDoneFiles(any(JobId.class));
+      verify(t.mockAppContext, times(0)).setHistoryUrl(any(String.class));
+    } finally {
+      jheh.stop();
+    }
+  }
 }
 
 class JHEvenHandlerForTest extends JobHistoryEventHandler {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/735fce5b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockAppContext.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockAppContext.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockAppContext.java
index 4e31b63..0686633 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockAppContext.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockAppContext.java
@@ -154,4 +154,14 @@ public class MockAppContext implements AppContext {
       return null;
   }
 
+  @Override
+  public String getHistoryUrl() {
+    return null;
+  }
+
+  @Override
+  public void setHistoryUrl(String historyUrl) {
+    return;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/735fce5b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java
index 8c7f0db..301d498 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java
@@ -896,5 +896,15 @@ public class TestRuntimeEstimators {
     public TaskAttemptFinishingMonitor getTaskAttemptFinishingMonitor() {
       return null;
     }
+
+    @Override
+    public String getHistoryUrl() {
+      return null;
+    }
+
+    @Override
+    public void setHistoryUrl(String historyUrl) {
+      return;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/735fce5b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java
index c5a40b2..2671df4 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java
@@ -407,4 +407,14 @@ public class JobHistory extends AbstractService implements HistoryContext {
   public TaskAttemptFinishingMonitor getTaskAttemptFinishingMonitor() {
     return null;
   }
+
+  @Override
+  public String getHistoryUrl() {
+    return null;
+  }
+
+  @Override
+  public void setHistoryUrl(String historyUrl) {
+    return;
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[49/51] [abbrv] hadoop git commit: HDFS-12157. Do fsyncDirectory(..) outside of FSDataset lock. Contributed by inayakumar B.

Posted by st...@apache.org.
HDFS-12157. Do fsyncDirectory(..) outside of FSDataset lock. Contributed by inayakumar B.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/69afa26f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/69afa26f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/69afa26f

Branch: refs/heads/HADOOP-13345
Commit: 69afa26f19adad4c630a307c274130eb8b697141
Parents: 1a18d5e
Author: Kihwal Lee <ki...@apache.org>
Authored: Wed Aug 9 09:03:51 2017 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Wed Aug 9 09:03:51 2017 -0500

----------------------------------------------------------------------
 .../datanode/fsdataset/impl/FsDatasetImpl.java  | 46 ++++++++++----------
 1 file changed, 24 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/69afa26f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index 53e2fc6..16df709 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -991,8 +991,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
         replicaInfo, smallBufferSize, conf);
 
     // Finalize the copied files
-    newReplicaInfo = finalizeReplica(block.getBlockPoolId(), newReplicaInfo,
-        false);
+    newReplicaInfo = finalizeReplica(block.getBlockPoolId(), newReplicaInfo);
     try (AutoCloseableLock lock = datasetLock.acquire()) {
       // Increment numBlocks here as this block moved without knowing to BPS
       FsVolumeImpl volume = (FsVolumeImpl) newReplicaInfo.getVolume();
@@ -1295,7 +1294,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           replicaInfo.bumpReplicaGS(newGS);
           // finalize the replica if RBW
           if (replicaInfo.getState() == ReplicaState.RBW) {
-            finalizeReplica(b.getBlockPoolId(), replicaInfo, false);
+            finalizeReplica(b.getBlockPoolId(), replicaInfo);
           }
           return replicaInfo;
         }
@@ -1625,23 +1624,39 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   @Override // FsDatasetSpi
   public void finalizeBlock(ExtendedBlock b, boolean fsyncDir)
       throws IOException {
+    ReplicaInfo replicaInfo = null;
+    ReplicaInfo finalizedReplicaInfo = null;
     try (AutoCloseableLock lock = datasetLock.acquire()) {
       if (Thread.interrupted()) {
         // Don't allow data modifications from interrupted threads
         throw new IOException("Cannot finalize block from Interrupted Thread");
       }
-      ReplicaInfo replicaInfo = getReplicaInfo(b);
+      replicaInfo = getReplicaInfo(b);
       if (replicaInfo.getState() == ReplicaState.FINALIZED) {
         // this is legal, when recovery happens on a file that has
         // been opened for append but never modified
         return;
       }
-      finalizeReplica(b.getBlockPoolId(), replicaInfo, fsyncDir);
+      finalizedReplicaInfo = finalizeReplica(b.getBlockPoolId(), replicaInfo);
+    }
+    /*
+     * Sync the directory after rename from tmp/rbw to Finalized if
+     * configured. Though rename should be atomic operation, sync on both
+     * dest and src directories are done because IOUtils.fsync() calls
+     * directory's channel sync, not the journal itself.
+     */
+    if (fsyncDir && finalizedReplicaInfo instanceof FinalizedReplica
+        && replicaInfo instanceof LocalReplica) {
+      FinalizedReplica finalizedReplica =
+          (FinalizedReplica) finalizedReplicaInfo;
+      finalizedReplica.fsyncDirectory();
+      LocalReplica localReplica = (LocalReplica) replicaInfo;
+      localReplica.fsyncDirectory();
     }
   }
 
-  private ReplicaInfo finalizeReplica(String bpid,
-      ReplicaInfo replicaInfo, boolean fsyncDir) throws IOException {
+  private ReplicaInfo finalizeReplica(String bpid, ReplicaInfo replicaInfo)
+      throws IOException {
     try (AutoCloseableLock lock = datasetLock.acquire()) {
       ReplicaInfo newReplicaInfo = null;
       if (replicaInfo.getState() == ReplicaState.RUR &&
@@ -1656,19 +1671,6 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
 
         newReplicaInfo = v.addFinalizedBlock(
             bpid, replicaInfo, replicaInfo, replicaInfo.getBytesReserved());
-        /*
-         * Sync the directory after rename from tmp/rbw to Finalized if
-         * configured. Though rename should be atomic operation, sync on both
-         * dest and src directories are done because IOUtils.fsync() calls
-         * directory's channel sync, not the journal itself.
-         */
-        if (fsyncDir && newReplicaInfo instanceof FinalizedReplica
-            && replicaInfo instanceof LocalReplica) {
-          FinalizedReplica finalizedReplica = (FinalizedReplica) newReplicaInfo;
-          finalizedReplica.fsyncDirectory();
-          LocalReplica localReplica = (LocalReplica) replicaInfo;
-          localReplica.fsyncDirectory();
-        }
         if (v.isTransientStorage()) {
           releaseLockedMemory(
               replicaInfo.getOriginalBytesReserved()
@@ -2634,11 +2636,11 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
 
         newReplicaInfo.setNumBytes(newlength);
         volumeMap.add(bpid, newReplicaInfo.getReplicaInfo());
-        finalizeReplica(bpid, newReplicaInfo.getReplicaInfo(), false);
+        finalizeReplica(bpid, newReplicaInfo.getReplicaInfo());
       }
     }
     // finalize the block
-    return finalizeReplica(bpid, rur, false);
+    return finalizeReplica(bpid, rur);
   }
 
   @Override // FsDatasetSpi


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[16/51] [abbrv] hadoop git commit: HADOOP-13963. /bin/bash is hard coded in some of the scripts. Contributed by Ajay Yadav.

Posted by st...@apache.org.
HADOOP-13963. /bin/bash is hard coded in some of the scripts. Contributed by Ajay Yadav.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a6fdeb8a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a6fdeb8a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a6fdeb8a

Branch: refs/heads/HADOOP-13345
Commit: a6fdeb8a872d413c76257a32914ade1d0e944583
Parents: 02bf328
Author: Arpit Agarwal <ar...@apache.org>
Authored: Fri Aug 4 10:40:52 2017 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Fri Aug 4 10:40:52 2017 -0700

----------------------------------------------------------------------
 dev-support/docker/hadoop_env_checks.sh                            | 2 +-
 dev-support/findHangingTest.sh                                     | 2 +-
 dev-support/verify-xml.sh                                          | 2 +-
 .../src/test/scripts/hadoop-functions_test_helper.bash             | 2 +-
 start-build-env.sh                                                 | 2 +-
 5 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6fdeb8a/dev-support/docker/hadoop_env_checks.sh
----------------------------------------------------------------------
diff --git a/dev-support/docker/hadoop_env_checks.sh b/dev-support/docker/hadoop_env_checks.sh
index 910c802..5cb4b2b 100755
--- a/dev-support/docker/hadoop_env_checks.sh
+++ b/dev-support/docker/hadoop_env_checks.sh
@@ -1,4 +1,4 @@
-#!/bin/bash
+#!/usr/bin/env bash
 
 # Licensed to the Apache Software Foundation (ASF) under one
 # or more contributor license agreements.  See the NOTICE file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6fdeb8a/dev-support/findHangingTest.sh
----------------------------------------------------------------------
diff --git a/dev-support/findHangingTest.sh b/dev-support/findHangingTest.sh
index f7ebe47..fcda9ff 100644
--- a/dev-support/findHangingTest.sh
+++ b/dev-support/findHangingTest.sh
@@ -1,4 +1,4 @@
-#!/bin/bash
+#!/usr/bin/env bash
 ##
 # Licensed to the Apache Software Foundation (ASF) under one
 # or more contributor license agreements.  See the NOTICE file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6fdeb8a/dev-support/verify-xml.sh
----------------------------------------------------------------------
diff --git a/dev-support/verify-xml.sh b/dev-support/verify-xml.sh
index abab4e6..9ef456a 100755
--- a/dev-support/verify-xml.sh
+++ b/dev-support/verify-xml.sh
@@ -1,4 +1,4 @@
-#!/bin/bash
+#!/usr/bin/env bash
 ##
 # Licensed to the Apache Software Foundation (ASF) under one
 # or more contributor license agreements.  See the NOTICE file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6fdeb8a/hadoop-common-project/hadoop-common/src/test/scripts/hadoop-functions_test_helper.bash
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/scripts/hadoop-functions_test_helper.bash b/hadoop-common-project/hadoop-common/src/test/scripts/hadoop-functions_test_helper.bash
index 86608ed..fa34bdf 100755
--- a/hadoop-common-project/hadoop-common/src/test/scripts/hadoop-functions_test_helper.bash
+++ b/hadoop-common-project/hadoop-common/src/test/scripts/hadoop-functions_test_helper.bash
@@ -1,4 +1,4 @@
-#!/bin/bash
+#!/usr/bin/env bash
 # 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.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6fdeb8a/start-build-env.sh
----------------------------------------------------------------------
diff --git a/start-build-env.sh b/start-build-env.sh
index 18e3a8c..94af7e4 100755
--- a/start-build-env.sh
+++ b/start-build-env.sh
@@ -1,4 +1,4 @@
-#!/bin/bash
+#!/usr/bin/env bash
 
 # Licensed to the Apache Software Foundation (ASF) under one or more
 # contributor license agreements.  See the NOTICE file distributed with


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[47/51] [abbrv] hadoop git commit: HDFS-12117. HttpFS does not seem to support SNAPSHOT related methods for WebHDFS REST Interface. Contributed by Wellington Chevreuil.

Posted by st...@apache.org.
HDFS-12117. HttpFS does not seem to support SNAPSHOT related methods for WebHDFS REST Interface. Contributed by Wellington Chevreuil.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8a4bff02
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8a4bff02
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8a4bff02

Branch: refs/heads/HADOOP-13345
Commit: 8a4bff02c1534c6bf529726f2bbe414ac4c172e8
Parents: 9a3c237
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Tue Aug 8 23:58:53 2017 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Tue Aug 8 23:58:53 2017 -0700

----------------------------------------------------------------------
 .../hadoop/fs/http/client/HttpFSFileSystem.java |  47 ++++++-
 .../hadoop/fs/http/server/FSOperations.java     | 105 ++++++++++++++
 .../http/server/HttpFSParametersProvider.java   |  45 ++++++
 .../hadoop/fs/http/server/HttpFSServer.java     |  36 +++++
 .../fs/http/client/BaseTestHttpFSWith.java      | 110 ++++++++++++++-
 .../hadoop/fs/http/server/TestHttpFSServer.java | 140 ++++++++++++++++++-
 6 files changed, 479 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a4bff02/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
index d139100..1059a02 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
@@ -124,6 +124,8 @@ public class HttpFSFileSystem extends FileSystem
   public static final String POLICY_NAME_PARAM = "storagepolicy";
   public static final String OFFSET_PARAM = "offset";
   public static final String LENGTH_PARAM = "length";
+  public static final String SNAPSHOT_NAME_PARAM = "snapshotname";
+  public static final String OLD_SNAPSHOT_NAME_PARAM = "oldsnapshotname";
 
   public static final Short DEFAULT_PERMISSION = 0755;
   public static final String ACLSPEC_DEFAULT = "";
@@ -144,6 +146,8 @@ public class HttpFSFileSystem extends FileSystem
 
   public static final String UPLOAD_CONTENT_TYPE= "application/octet-stream";
 
+  public static final String SNAPSHOT_JSON = "Path";
+
   public enum FILE_TYPE {
     FILE, DIRECTORY, SYMLINK;
 
@@ -229,7 +233,9 @@ public class HttpFSFileSystem extends FileSystem
     DELETE(HTTP_DELETE), SETXATTR(HTTP_PUT), GETXATTRS(HTTP_GET),
     REMOVEXATTR(HTTP_PUT), LISTXATTRS(HTTP_GET), LISTSTATUS_BATCH(HTTP_GET),
     GETALLSTORAGEPOLICY(HTTP_GET), GETSTORAGEPOLICY(HTTP_GET),
-    SETSTORAGEPOLICY(HTTP_PUT), UNSETSTORAGEPOLICY(HTTP_POST);
+    SETSTORAGEPOLICY(HTTP_PUT), UNSETSTORAGEPOLICY(HTTP_POST),
+    CREATESNAPSHOT(HTTP_PUT), DELETESNAPSHOT(HTTP_DELETE),
+    RENAMESNAPSHOT(HTTP_PUT);
 
     private String httpMethod;
 
@@ -1434,4 +1440,43 @@ public class HttpFSFileSystem extends FileSystem
         Operation.UNSETSTORAGEPOLICY.getMethod(), params, src, true);
     HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
+
+  @Override
+  public final Path createSnapshot(Path path, String snapshotName)
+      throws IOException {
+    Map<String, String> params = new HashMap<String, String>();
+    params.put(OP_PARAM, Operation.CREATESNAPSHOT.toString());
+    if (snapshotName != null) {
+      params.put(SNAPSHOT_NAME_PARAM, snapshotName);
+    }
+    HttpURLConnection conn = getConnection(Operation.CREATESNAPSHOT.getMethod(),
+        params, path, true);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
+    return new Path((String) json.get(SNAPSHOT_JSON));
+  }
+
+  @Override
+  public void renameSnapshot(Path path, String snapshotOldName,
+                             String snapshotNewName) throws IOException {
+    Map<String, String> params = new HashMap<String, String>();
+    params.put(OP_PARAM, Operation.RENAMESNAPSHOT.toString());
+    params.put(SNAPSHOT_NAME_PARAM, snapshotNewName);
+    params.put(OLD_SNAPSHOT_NAME_PARAM, snapshotOldName);
+    HttpURLConnection conn = getConnection(Operation.RENAMESNAPSHOT.getMethod(),
+        params, path, true);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+  }
+
+  @Override
+  public void deleteSnapshot(Path path, String snapshotName)
+      throws IOException {
+    Map<String, String> params = new HashMap<String, String>();
+    params.put(OP_PARAM, Operation.DELETESNAPSHOT.toString());
+    params.put(SNAPSHOT_NAME_PARAM, snapshotName);
+    HttpURLConnection conn = getConnection(Operation.DELETESNAPSHOT.getMethod(),
+        params, path, true);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a4bff02/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
index f1615c3..c008802 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
@@ -1492,4 +1492,109 @@ public class FSOperations {
       return JsonUtil.toJsonMap(locations);
     }
   }
+
+  /**
+   *  Executor that performs a createSnapshot FileSystemAccess operation.
+   */
+  @InterfaceAudience.Private
+  public static class FSCreateSnapshot implements
+      FileSystemAccess.FileSystemExecutor<String> {
+
+    private Path path;
+    private String snapshotName;
+
+    /**
+     * Creates a createSnapshot executor.
+     * @param path directory path to be snapshotted.
+     * @param snapshotName the snapshot name.
+     */
+    public FSCreateSnapshot(String path, String snapshotName) {
+      this.path = new Path(path);
+      this.snapshotName = snapshotName;
+    }
+
+    /**
+     * Executes the filesystem operation.
+     * @param fs filesystem instance to use.
+     * @return <code>Path</code> the complete path for newly created snapshot
+     * @throws IOException thrown if an IO error occurred.
+     */
+    @Override
+    public String execute(FileSystem fs) throws IOException {
+      Path snapshotPath = fs.createSnapshot(path, snapshotName);
+      JSONObject json = toJSON(HttpFSFileSystem.HOME_DIR_JSON,
+          snapshotPath.toString());
+      return json.toJSONString().replaceAll("\\\\", "");
+    }
+  }
+
+  /**
+   *  Executor that performs a deleteSnapshot FileSystemAccess operation.
+   */
+  @InterfaceAudience.Private
+  public static class FSDeleteSnapshot implements
+      FileSystemAccess.FileSystemExecutor<Void> {
+
+    private Path path;
+    private String snapshotName;
+
+    /**
+     * Creates a deleteSnapshot executor.
+     * @param path path for the snapshot to be deleted.
+     * @param snapshotName snapshot name.
+     */
+    public FSDeleteSnapshot(String path, String snapshotName) {
+      this.path = new Path(path);
+      this.snapshotName = snapshotName;
+    }
+
+    /**
+     * Executes the filesystem operation.
+     * @param fs filesystem instance to use.
+     * @return void
+     * @throws IOException thrown if an IO error occurred.
+     */
+    @Override
+    public Void execute(FileSystem fs) throws IOException {
+      fs.deleteSnapshot(path, snapshotName);
+      return null;
+    }
+  }
+
+  /**
+   *  Executor that performs a renameSnapshot FileSystemAccess operation.
+   */
+  @InterfaceAudience.Private
+  public static class FSRenameSnapshot implements
+      FileSystemAccess.FileSystemExecutor<Void> {
+    private Path path;
+    private String oldSnapshotName;
+    private String snapshotName;
+
+    /**
+     * Creates a renameSnapshot executor.
+     * @param path directory path of the snapshot to be renamed.
+     * @param oldSnapshotName current snapshot name.
+     * @param snapshotName new snapshot name to be set.
+     */
+    public FSRenameSnapshot(String path, String oldSnapshotName,
+                            String snapshotName) {
+      this.path = new Path(path);
+      this.oldSnapshotName = oldSnapshotName;
+      this.snapshotName = snapshotName;
+    }
+
+    /**
+     * Executes the filesystem operation.
+     * @param fs filesystem instance to use.
+     * @return void
+     * @throws IOException thrown if an IO error occurred.
+     */
+    @Override
+    public Void execute(FileSystem fs) throws IOException {
+      fs.renameSnapshot(path, oldSnapshotName, snapshotName);
+      return null;
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a4bff02/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java
index 347a747..5f265c0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java
@@ -100,6 +100,13 @@ public class HttpFSParametersProvider extends ParametersProvider {
     PARAMS_DEF.put(Operation.SETSTORAGEPOLICY,
         new Class[] {PolicyNameParam.class});
     PARAMS_DEF.put(Operation.UNSETSTORAGEPOLICY, new Class[] {});
+    PARAMS_DEF.put(Operation.CREATESNAPSHOT,
+            new Class[] {SnapshotNameParam.class});
+    PARAMS_DEF.put(Operation.DELETESNAPSHOT,
+            new Class[] {SnapshotNameParam.class});
+    PARAMS_DEF.put(Operation.RENAMESNAPSHOT,
+            new Class[] {OldSnapshotNameParam.class,
+                SnapshotNameParam.class});
   }
 
   public HttpFSParametersProvider() {
@@ -565,4 +572,42 @@ public class HttpFSParametersProvider extends ParametersProvider {
       super(NAME, null);
     }
   }
+
+  /**
+   * Class for SnapshotName parameter.
+   */
+  public static class SnapshotNameParam extends StringParam {
+
+    /**
+     * Parameter name.
+     */
+    public static final String NAME = HttpFSFileSystem.SNAPSHOT_NAME_PARAM;
+
+    /**
+     * Constructor.
+     */
+    public SnapshotNameParam() {
+      super(NAME, null);
+    }
+
+  }
+
+  /**
+   * Class for OldSnapshotName parameter.
+   */
+  public static class OldSnapshotNameParam extends StringParam {
+
+    /**
+     * Parameter name.
+     */
+    public static final String NAME = HttpFSFileSystem.OLD_SNAPSHOT_NAME_PARAM;
+
+    /**
+     * Constructor.
+     */
+    public OldSnapshotNameParam() {
+      super(NAME, null);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a4bff02/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
index 5c0c9b5..03ccb4c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.LenParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.ModifiedTimeParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.NewLengthParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.OffsetParam;
+import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.OldSnapshotNameParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.OperationParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.OverwriteParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.OwnerParam;
@@ -45,6 +46,7 @@ import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.PolicyNameParam
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.RecursiveParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.ReplicationParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.SourcesParam;
+import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.SnapshotNameParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.XAttrEncodingParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.XAttrNameParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.XAttrSetFlagParam;
@@ -430,6 +432,16 @@ public class HttpFSServer {
         response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
         break;
       }
+      case DELETESNAPSHOT: {
+        String snapshotName = params.get(SnapshotNameParam.NAME,
+            SnapshotNameParam.class);
+        FSOperations.FSDeleteSnapshot command =
+                new FSOperations.FSDeleteSnapshot(path, snapshotName);
+        fsExecute(user, command);
+        AUDIT_LOG.info("[{}] deleted snapshot [{}]", path, snapshotName);
+        response = Response.ok().build();
+        break;
+      }
       default: {
         throw new IOException(
           MessageFormat.format("Invalid HTTP DELETE operation [{0}]",
@@ -602,6 +614,16 @@ public class HttpFSServer {
         }
         break;
       }
+      case CREATESNAPSHOT: {
+        String snapshotName = params.get(SnapshotNameParam.NAME,
+            SnapshotNameParam.class);
+        FSOperations.FSCreateSnapshot command =
+            new FSOperations.FSCreateSnapshot(path, snapshotName);
+        String json = fsExecute(user, command);
+        AUDIT_LOG.info("[{}] snapshot created as [{}]", path, snapshotName);
+        response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
+        break;
+      }
       case SETXATTR: {
         String xattrName = params.get(XAttrNameParam.NAME, 
             XAttrNameParam.class);
@@ -617,6 +639,20 @@ public class HttpFSServer {
         response = Response.ok().build();
         break;
       }
+      case RENAMESNAPSHOT: {
+        String oldSnapshotName = params.get(OldSnapshotNameParam.NAME,
+            OldSnapshotNameParam.class);
+        String snapshotName = params.get(SnapshotNameParam.NAME,
+            SnapshotNameParam.class);
+        FSOperations.FSRenameSnapshot command =
+                new FSOperations.FSRenameSnapshot(path, oldSnapshotName,
+                    snapshotName);
+        fsExecute(user, command);
+        AUDIT_LOG.info("[{}] renamed snapshot [{}] to [{}]", path,
+            oldSnapshotName, snapshotName);
+        response = Response.ok().build();
+        break;
+      }
       case REMOVEXATTR: {
         String xattrName = params.get(XAttrNameParam.NAME, XAttrNameParam.class);
         FSOperations.FSRemoveXAttr command = new FSOperations.FSRemoveXAttr(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a4bff02/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
index ca11c66..553bbce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.AppendTestUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -74,6 +75,7 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
+import java.util.regex.Pattern;
 
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
@@ -1034,11 +1036,12 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
   }
 
   protected enum Operation {
-    GET, OPEN, CREATE, APPEND, TRUNCATE, CONCAT, RENAME, DELETE, LIST_STATUS, 
+    GET, OPEN, CREATE, APPEND, TRUNCATE, CONCAT, RENAME, DELETE, LIST_STATUS,
     WORKING_DIRECTORY, MKDIRS, SET_TIMES, SET_PERMISSION, SET_OWNER,
     SET_REPLICATION, CHECKSUM, CONTENT_SUMMARY, FILEACLS, DIRACLS, SET_XATTR,
     GET_XATTRS, REMOVE_XATTR, LIST_XATTRS, ENCRYPTION, LIST_STATUS_BATCH,
-    GETTRASHROOT, STORAGEPOLICY, ERASURE_CODING, GETFILEBLOCKLOCATIONS
+    GETTRASHROOT, STORAGEPOLICY, ERASURE_CODING, GETFILEBLOCKLOCATIONS,
+    CREATE_SNAPSHOT, RENAME_SNAPSHOT, DELETE_SNAPSHOT
   }
 
   private void operation(Operation op) throws Exception {
@@ -1130,6 +1133,15 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     case GETFILEBLOCKLOCATIONS:
       testGetFileBlockLocations();
       break;
+    case CREATE_SNAPSHOT:
+      testCreateSnapshot();
+      break;
+    case RENAME_SNAPSHOT:
+      testRenameSnapshot();
+      break;
+    case DELETE_SNAPSHOT:
+      testDeleteSnapshot();
+      break;
     }
   }
 
@@ -1257,4 +1269,98 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
           location2.getTopologyPaths());
     }
   }
+
+  private void testCreateSnapshot(String snapshotName) throws Exception {
+    if (!this.isLocalFS()) {
+      Path snapshottablePath = new Path("/tmp/tmp-snap-test");
+      createSnapshotTestsPreconditions(snapshottablePath);
+      //Now get the FileSystem instance that's being tested
+      FileSystem fs = this.getHttpFSFileSystem();
+      if (snapshotName == null) {
+        fs.createSnapshot(snapshottablePath);
+      } else {
+        fs.createSnapshot(snapshottablePath, snapshotName);
+      }
+      Path snapshotsDir = new Path("/tmp/tmp-snap-test/.snapshot");
+      FileStatus[] snapshotItems = fs.listStatus(snapshotsDir);
+      assertTrue("Should have exactly one snapshot.",
+          snapshotItems.length == 1);
+      String resultingSnapName = snapshotItems[0].getPath().getName();
+      if (snapshotName == null) {
+        assertTrue("Snapshot auto generated name not matching pattern",
+            Pattern.matches("(s)(\\d{8})(-)(\\d{6})(\\.)(\\d{3})",
+                resultingSnapName));
+      } else {
+        assertTrue("Snapshot name is not same as passed name.",
+            snapshotName.equals(resultingSnapName));
+      }
+      cleanSnapshotTests(snapshottablePath, resultingSnapName);
+    }
+  }
+
+  private void testCreateSnapshot() throws Exception {
+    testCreateSnapshot(null);
+    testCreateSnapshot("snap-with-name");
+  }
+
+  private void createSnapshotTestsPreconditions(Path snapshottablePath)
+      throws Exception {
+    //Needed to get a DistributedFileSystem instance, in order to
+    //call allowSnapshot on the newly created directory
+    DistributedFileSystem distributedFs = (DistributedFileSystem)
+        FileSystem.get(snapshottablePath.toUri(), this.getProxiedFSConf());
+    distributedFs.mkdirs(snapshottablePath);
+    distributedFs.allowSnapshot(snapshottablePath);
+    Path subdirPath = new Path("/tmp/tmp-snap-test/subdir");
+    distributedFs.mkdirs(subdirPath);
+
+  }
+
+  private void cleanSnapshotTests(Path snapshottablePath,
+                                  String resultingSnapName) throws Exception {
+    DistributedFileSystem distributedFs = (DistributedFileSystem)
+        FileSystem.get(snapshottablePath.toUri(), this.getProxiedFSConf());
+    distributedFs.deleteSnapshot(snapshottablePath, resultingSnapName);
+    distributedFs.delete(snapshottablePath, true);
+  }
+
+  private void testRenameSnapshot() throws Exception {
+    if (!this.isLocalFS()) {
+      Path snapshottablePath = new Path("/tmp/tmp-snap-test");
+      createSnapshotTestsPreconditions(snapshottablePath);
+      //Now get the FileSystem instance that's being tested
+      FileSystem fs = this.getHttpFSFileSystem();
+      fs.createSnapshot(snapshottablePath, "snap-to-rename");
+      fs.renameSnapshot(snapshottablePath, "snap-to-rename",
+          "snap-new-name");
+      Path snapshotsDir = new Path("/tmp/tmp-snap-test/.snapshot");
+      FileStatus[] snapshotItems = fs.listStatus(snapshotsDir);
+      assertTrue("Should have exactly one snapshot.",
+          snapshotItems.length == 1);
+      String resultingSnapName = snapshotItems[0].getPath().getName();
+      assertTrue("Snapshot name is not same as passed name.",
+          "snap-new-name".equals(resultingSnapName));
+      cleanSnapshotTests(snapshottablePath, resultingSnapName);
+    }
+  }
+
+  private void testDeleteSnapshot() throws Exception {
+    if (!this.isLocalFS()) {
+      Path snapshottablePath = new Path("/tmp/tmp-snap-test");
+      createSnapshotTestsPreconditions(snapshottablePath);
+      //Now get the FileSystem instance that's being tested
+      FileSystem fs = this.getHttpFSFileSystem();
+      fs.createSnapshot(snapshottablePath, "snap-to-delete");
+      Path snapshotsDir = new Path("/tmp/tmp-snap-test/.snapshot");
+      FileStatus[] snapshotItems = fs.listStatus(snapshotsDir);
+      assertTrue("Should have exactly one snapshot.",
+          snapshotItems.length == 1);
+      fs.deleteSnapshot(snapshottablePath, "snap-to-delete");
+      snapshotItems = fs.listStatus(snapshotsDir);
+      assertTrue("There should be no snapshot anymore.",
+          snapshotItems.length == 0);
+      fs.delete(snapshottablePath, true);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a4bff02/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java
index 0e1cc20..60e70d2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.fs.http.server;
 
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.security.authentication.util.SignerSecretProvider;
 import org.apache.hadoop.security.authentication.util.StringSignerSecretProviderCreator;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
@@ -71,6 +72,7 @@ import org.eclipse.jetty.webapp.WebAppContext;
 
 import com.google.common.collect.Maps;
 import java.util.Properties;
+import java.util.regex.Pattern;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 
 /**
@@ -465,6 +467,20 @@ public class TestHttpFSServer extends HFSTestCase {
    */
   private void putCmd(String filename, String command,
                       String params) throws Exception {
+    Assert.assertEquals(HttpURLConnection.HTTP_OK,
+            putCmdWithReturn(filename, command, params).getResponseCode());
+  }
+
+  /**
+   * General-purpose http PUT command to the httpfs server,
+   * which returns relted HttpURLConnection instance.
+   * @param filename The file to operate upon
+   * @param command The command to perform (SETACL, etc)
+   * @param params Parameters, like "aclspec=..."
+   * @return HttpURLConnection the HttpURLConnection instance for the given PUT
+   */
+  private HttpURLConnection putCmdWithReturn(String filename, String command,
+                      String params) throws Exception {
     String user = HadoopUsersConfTestHelper.getHadoopUsers()[0];
     // Remove leading / from filename
     if (filename.charAt(0) == '/') {
@@ -478,7 +494,7 @@ public class TestHttpFSServer extends HFSTestCase {
     HttpURLConnection conn = (HttpURLConnection) url.openConnection();
     conn.setRequestMethod("PUT");
     conn.connect();
-    Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
+    return conn;
   }
 
   /**
@@ -882,6 +898,108 @@ public class TestHttpFSServer extends HFSTestCase {
     delegationTokenCommonTests(false);
   }
 
+  private HttpURLConnection snapshotTestPreconditions(String httpMethod,
+                                                      String snapOperation,
+                                                      String additionalParams)
+      throws Exception {
+    String user = HadoopUsersConfTestHelper.getHadoopUsers()[0];
+    URL url = new URL(TestJettyHelper.getJettyURL(), MessageFormat.format(
+        "/webhdfs/v1/tmp/tmp-snap-test/subdir?user.name={0}&op=MKDIRS",
+        user));
+    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    conn.setRequestMethod("PUT");
+    conn.connect();
+
+    Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
+
+    //needed to make the given dir snapshottable
+    Path snapshottablePath = new Path("/tmp/tmp-snap-test");
+    DistributedFileSystem dfs =
+        (DistributedFileSystem) FileSystem.get(snapshottablePath.toUri(),
+        TestHdfsHelper.getHdfsConf());
+    dfs.allowSnapshot(snapshottablePath);
+
+    //Try to create snapshot passing snapshot name
+    url = new URL(TestJettyHelper.getJettyURL(), MessageFormat.format(
+        "/webhdfs/v1/tmp/tmp-snap-test?user.name={0}&op={1}&{2}", user,
+        snapOperation, additionalParams));
+    conn = (HttpURLConnection) url.openConnection();
+    conn.setRequestMethod(httpMethod);
+    conn.connect();
+    return conn;
+  }
+
+  @Test
+  @TestDir
+  @TestJetty
+  @TestHdfs
+  public void testCreateSnapshot() throws Exception {
+    createHttpFSServer(false, false);
+    final HttpURLConnection conn = snapshotTestPreconditions("PUT",
+        "CREATESNAPSHOT",
+        "snapshotname=snap-with-name");
+    Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
+    final BufferedReader reader =
+        new BufferedReader(new InputStreamReader(conn.getInputStream()));
+    String result = reader.readLine();
+    //Validates if the content format is correct
+    Assert.assertTrue(result.
+        equals("{\"Path\":\"/tmp/tmp-snap-test/.snapshot/snap-with-name\"}"));
+    //Validates if the snapshot is properly created under .snapshot folder
+    result = getStatus("/tmp/tmp-snap-test/.snapshot",
+        "LISTSTATUS");
+    Assert.assertTrue(result.contains("snap-with-name"));
+  }
+
+  @Test
+  @TestDir
+  @TestJetty
+  @TestHdfs
+  public void testCreateSnapshotNoSnapshotName() throws Exception {
+    createHttpFSServer(false, false);
+    final HttpURLConnection conn = snapshotTestPreconditions("PUT",
+        "CREATESNAPSHOT",
+        "");
+    Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
+    final BufferedReader reader = new BufferedReader(
+        new InputStreamReader(conn.getInputStream()));
+    String result = reader.readLine();
+    //Validates if the content format is correct
+    Assert.assertTrue(Pattern.matches(
+        "(\\{\\\"Path\\\"\\:\\\"/tmp/tmp-snap-test/.snapshot/s)" +
+            "(\\d{8})(-)(\\d{6})(\\.)(\\d{3})(\\\"\\})", result));
+    //Validates if the snapshot is properly created under .snapshot folder
+    result = getStatus("/tmp/tmp-snap-test/.snapshot",
+        "LISTSTATUS");
+
+    Assert.assertTrue(Pattern.matches("(.+)(\\\"pathSuffix\\\":\\\"s)" +
+            "(\\d{8})(-)(\\d{6})(\\.)(\\d{3})(\\\")(.+)",
+        result));
+  }
+
+  @Test
+  @TestDir
+  @TestJetty
+  @TestHdfs
+  public void testRenameSnapshot() throws Exception {
+    createHttpFSServer(false, false);
+    HttpURLConnection conn = snapshotTestPreconditions("PUT",
+        "CREATESNAPSHOT",
+        "snapshotname=snap-to-rename");
+    Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
+    conn = snapshotTestPreconditions("PUT",
+        "RENAMESNAPSHOT",
+        "oldsnapshotname=snap-to-rename" +
+            "&snapshotname=snap-renamed");
+    Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
+    //Validates the snapshot is properly renamed under .snapshot folder
+    String result = getStatus("/tmp/tmp-snap-test/.snapshot",
+        "LISTSTATUS");
+    Assert.assertTrue(result.contains("snap-renamed"));
+    //There should be no snapshot named snap-to-rename now
+    Assert.assertFalse(result.contains("snap-to-rename"));
+  }
+
   @Test
   @TestDir
   @TestJetty
@@ -890,4 +1008,24 @@ public class TestHttpFSServer extends HFSTestCase {
     createHttpFSServer(true, true);
     delegationTokenCommonTests(true);
   }
+
+  @Test
+  @TestDir
+  @TestJetty
+  @TestHdfs
+  public void testDeleteSnapshot() throws Exception {
+    createHttpFSServer(false, false);
+    HttpURLConnection conn = snapshotTestPreconditions("PUT",
+        "CREATESNAPSHOT",
+        "snapshotname=snap-to-delete");
+    Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
+    conn = snapshotTestPreconditions("DELETE",
+        "DELETESNAPSHOT",
+        "snapshotname=snap-to-delete");
+    Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
+    //Validates the snapshot is not under .snapshot folder anymore
+    String result = getStatus("/tmp/tmp-snap-test/.snapshot",
+        "LISTSTATUS");
+    Assert.assertFalse(result.contains("snap-to-delete"));
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[25/51] [abbrv] hadoop git commit: YARN-6957. Moving logging APIs over to slf4j in hadoop-yarn-server-sharedcachemanager. Contributed by Yeliang Cang.

Posted by st...@apache.org.
YARN-6957. Moving logging APIs over to slf4j in hadoop-yarn-server-sharedcachemanager. Contributed by Yeliang Cang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b0fbf179
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b0fbf179
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b0fbf179

Branch: refs/heads/HADOOP-13345
Commit: b0fbf1796585900a37dc4d6a271c5b5b32e9a9da
Parents: 839e077
Author: Akira Ajisaka <aa...@apache.org>
Authored: Mon Aug 7 19:25:40 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Mon Aug 7 19:25:40 2017 +0900

----------------------------------------------------------------------
 .../yarn/server/sharedcachemanager/CleanerService.java      | 7 ++++---
 .../hadoop/yarn/server/sharedcachemanager/CleanerTask.java  | 7 ++++---
 .../server/sharedcachemanager/ClientProtocolService.java    | 7 ++++---
 .../server/sharedcachemanager/SCMAdminProtocolService.java  | 8 ++++----
 .../yarn/server/sharedcachemanager/SharedCacheManager.java  | 9 +++++----
 .../server/sharedcachemanager/metrics/CleanerMetrics.java   | 7 ++++---
 .../server/sharedcachemanager/metrics/ClientSCMMetrics.java | 7 ++++---
 .../metrics/SharedCacheUploaderMetrics.java                 | 8 ++++----
 .../server/sharedcachemanager/store/InMemorySCMStore.java   | 7 ++++---
 .../yarn/server/sharedcachemanager/webapp/SCMWebServer.java | 7 ++++---
 10 files changed, 41 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0fbf179/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/CleanerService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/CleanerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/CleanerService.java
index 60fc3e5..bcdc46b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/CleanerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/CleanerService.java
@@ -26,8 +26,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
@@ -43,6 +41,8 @@ import org.apache.hadoop.yarn.server.sharedcachemanager.metrics.CleanerMetrics;
 import org.apache.hadoop.yarn.server.sharedcachemanager.store.SCMStore;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The cleaner service that maintains the shared cache area, and cleans up stale
@@ -57,7 +57,8 @@ public class CleanerService extends CompositeService {
    */
   public static final String GLOBAL_CLEANER_PID = ".cleaner_pid";
 
-  private static final Log LOG = LogFactory.getLog(CleanerService.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(CleanerService.class);
 
   private Configuration conf;
   private CleanerMetrics metrics;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0fbf179/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/CleanerTask.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/CleanerTask.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/CleanerTask.java
index a7fdcbd..3e0a62b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/CleanerTask.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/CleanerTask.java
@@ -21,8 +21,6 @@ package org.apache.hadoop.yarn.server.sharedcachemanager;
 import java.io.IOException;
 import java.util.concurrent.locks.Lock;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.conf.Configuration;
@@ -34,6 +32,8 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.sharedcache.SharedCacheUtil;
 import org.apache.hadoop.yarn.server.sharedcachemanager.metrics.CleanerMetrics;
 import org.apache.hadoop.yarn.server.sharedcachemanager.store.SCMStore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The task that runs and cleans up the shared cache area for stale entries and
@@ -44,7 +44,8 @@ import org.apache.hadoop.yarn.server.sharedcachemanager.store.SCMStore;
 @Evolving
 class CleanerTask implements Runnable {
   private static final String RENAMED_SUFFIX = "-renamed";
-  private static final Log LOG = LogFactory.getLog(CleanerTask.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(CleanerTask.class);
 
   private final String location;
   private final long sleepTime;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0fbf179/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/ClientProtocolService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/ClientProtocolService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/ClientProtocolService.java
index 1dcca6c..4275674 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/ClientProtocolService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/ClientProtocolService.java
@@ -21,8 +21,6 @@ package org.apache.hadoop.yarn.server.sharedcachemanager;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.conf.Configuration;
@@ -45,6 +43,8 @@ import org.apache.hadoop.yarn.server.sharedcache.SharedCacheUtil;
 import org.apache.hadoop.yarn.server.sharedcachemanager.metrics.ClientSCMMetrics;
 import org.apache.hadoop.yarn.server.sharedcachemanager.store.SCMStore;
 import org.apache.hadoop.yarn.server.sharedcachemanager.store.SharedCacheResourceReference;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This service handles all rpc calls from the client to the shared cache
@@ -55,7 +55,8 @@ import org.apache.hadoop.yarn.server.sharedcachemanager.store.SharedCacheResourc
 public class ClientProtocolService extends AbstractService implements
     ClientSCMProtocol {
 
-  private static final Log LOG = LogFactory.getLog(ClientProtocolService.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ClientProtocolService.class);
 
   private final RecordFactory recordFactory = RecordFactoryProvider
       .getRecordFactory(null);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0fbf179/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/SCMAdminProtocolService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/SCMAdminProtocolService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/SCMAdminProtocolService.java
index 6f2baf6..e6a885b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/SCMAdminProtocolService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/SCMAdminProtocolService.java
@@ -21,15 +21,12 @@ package org.apache.hadoop.yarn.server.sharedcachemanager;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
 import org.apache.hadoop.yarn.server.api.SCMAdminProtocol;
@@ -41,6 +38,8 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This service handles all SCMAdminProtocol rpc calls from administrators
@@ -51,7 +50,8 @@ import org.apache.hadoop.yarn.ipc.YarnRPC;
 public class SCMAdminProtocolService extends AbstractService implements
     SCMAdminProtocol {
 
-  private static final Log LOG = LogFactory.getLog(SCMAdminProtocolService.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SCMAdminProtocolService.class);
 
   private final RecordFactory recordFactory = RecordFactoryProvider
       .getRecordFactory(null);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0fbf179/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/SharedCacheManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/SharedCacheManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/SharedCacheManager.java
index 331e29e..ca683f2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/SharedCacheManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/SharedCacheManager.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.yarn.server.sharedcachemanager;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -36,6 +34,8 @@ import org.apache.hadoop.yarn.server.sharedcachemanager.store.SCMStore;
 import org.apache.hadoop.yarn.server.sharedcachemanager.webapp.SCMWebServer;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This service maintains the shared cache meta data. It handles claiming and
@@ -51,7 +51,8 @@ public class SharedCacheManager extends CompositeService {
    */
   public static final int SHUTDOWN_HOOK_PRIORITY = 30;
 
-  private static final Log LOG = LogFactory.getLog(SharedCacheManager.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SharedCacheManager.class);
 
   private SCMStore store;
 
@@ -156,7 +157,7 @@ public class SharedCacheManager extends CompositeService {
       sharedCacheManager.init(conf);
       sharedCacheManager.start();
     } catch (Throwable t) {
-      LOG.fatal("Error starting SharedCacheManager", t);
+      LOG.error("Error starting SharedCacheManager", t);
       System.exit(-1);
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0fbf179/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/metrics/CleanerMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/metrics/CleanerMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/metrics/CleanerMetrics.java
index b86a469..55cb074 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/metrics/CleanerMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/metrics/CleanerMetrics.java
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.yarn.server.sharedcachemanager.metrics;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.metrics2.MetricsSource;
@@ -31,6 +29,8 @@ import org.apache.hadoop.metrics2.lib.MetricsRegistry;
 import org.apache.hadoop.metrics2.lib.MetricsSourceBuilder;
 import org.apache.hadoop.metrics2.lib.MutableCounterLong;
 import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This class is for maintaining the various Cleaner activity statistics and
@@ -40,7 +40,8 @@ import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
 @Evolving
 @Metrics(name = "CleanerActivity", about = "Cleaner service metrics", context = "yarn")
 public class CleanerMetrics {
-  public static final Log LOG = LogFactory.getLog(CleanerMetrics.class);
+  public static final Logger LOG =
+      LoggerFactory.getLogger(CleanerMetrics.class);
   private final MetricsRegistry registry = new MetricsRegistry("cleaner");
   private final static CleanerMetrics INSTANCE = create();
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0fbf179/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/metrics/ClientSCMMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/metrics/ClientSCMMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/metrics/ClientSCMMetrics.java
index fe960c6..6b45745 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/metrics/ClientSCMMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/metrics/ClientSCMMetrics.java
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.yarn.server.sharedcachemanager.metrics;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.metrics2.MetricsSystem;
@@ -27,6 +25,8 @@ import org.apache.hadoop.metrics2.annotation.Metrics;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.lib.MetricsRegistry;
 import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This class is for maintaining  client requests metrics
@@ -37,7 +37,8 @@ import org.apache.hadoop.metrics2.lib.MutableCounterLong;
 @Metrics(about="Client SCM metrics", context="yarn")
 public class ClientSCMMetrics {
 
-  private static final Log LOG = LogFactory.getLog(ClientSCMMetrics.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ClientSCMMetrics.class);
   final MetricsRegistry registry;
   private final static ClientSCMMetrics INSTANCE = create();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0fbf179/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/metrics/SharedCacheUploaderMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/metrics/SharedCacheUploaderMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/metrics/SharedCacheUploaderMetrics.java
index 7fff13a..3cf6632 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/metrics/SharedCacheUploaderMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/metrics/SharedCacheUploaderMetrics.java
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.yarn.server.sharedcachemanager.metrics;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.metrics2.MetricsSystem;
@@ -27,6 +25,8 @@ import org.apache.hadoop.metrics2.annotation.Metrics;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.lib.MetricsRegistry;
 import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This class is for maintaining shared cache uploader requests metrics
@@ -37,8 +37,8 @@ import org.apache.hadoop.metrics2.lib.MutableCounterLong;
 @Metrics(about="shared cache upload metrics", context="yarn")
 public class SharedCacheUploaderMetrics {
 
-  static final Log LOG =
-      LogFactory.getLog(SharedCacheUploaderMetrics.class);
+  static final Logger LOG =
+      LoggerFactory.getLogger(SharedCacheUploaderMetrics.class);
   final MetricsRegistry registry;
   private final static SharedCacheUploaderMetrics INSTANCE = create();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0fbf179/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/store/InMemorySCMStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/store/InMemorySCMStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/store/InMemorySCMStore.java
index 7b769a7..d917d9b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/store/InMemorySCMStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/store/InMemorySCMStore.java
@@ -33,8 +33,6 @@ import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
@@ -52,6 +50,8 @@ import org.apache.hadoop.yarn.server.sharedcachemanager.AppChecker;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * A thread safe version of an in-memory SCM store. The thread safety is
@@ -74,7 +74,8 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
 @Private
 @Evolving
 public class InMemorySCMStore extends SCMStore {
-  private static final Log LOG = LogFactory.getLog(InMemorySCMStore.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(InMemorySCMStore.class);
 
   private final Map<String, SharedCacheResource> cachedResources =
       new ConcurrentHashMap<String, SharedCacheResource>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0fbf179/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/webapp/SCMWebServer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/webapp/SCMWebServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/webapp/SCMWebServer.java
index b81ed29..7984090 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/webapp/SCMWebServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/webapp/SCMWebServer.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.yarn.server.sharedcachemanager.webapp;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -28,6 +26,8 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.sharedcachemanager.SharedCacheManager;
 import org.apache.hadoop.yarn.webapp.WebApp;
 import org.apache.hadoop.yarn.webapp.WebApps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * A very simple web interface for the metrics reported by
@@ -37,7 +37,8 @@ import org.apache.hadoop.yarn.webapp.WebApps;
 @Private
 @Unstable
 public class SCMWebServer extends AbstractService {
-  private static final Log LOG = LogFactory.getLog(SCMWebServer.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SCMWebServer.class);
 
   private final SharedCacheManager scm;
   private WebApp webApp;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[07/51] [abbrv] hadoop git commit: YARN-6673 Add cpu cgroup configurations for opportunistic containers. (Miklos Szegedi via Haibo Chen)

Posted by st...@apache.org.
YARN-6673 Add cpu cgroup configurations for opportunistic containers. (Miklos Szegedi via Haibo Chen)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/293c74a8
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/293c74a8
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/293c74a8

Branch: refs/heads/HADOOP-13345
Commit: 293c74a81b407274621751e2104738b2a09e666e
Parents: c5d256c
Author: Haibo Chen <ha...@apache.org>
Authored: Thu Aug 3 09:56:51 2017 -0700
Committer: Haibo Chen <ha...@apache.org>
Committed: Thu Aug 3 09:56:51 2017 -0700

----------------------------------------------------------------------
 .../CGroupsCpuResourceHandlerImpl.java          | 20 ++++++++++++-----
 .../TestCGroupsCpuResourceHandlerImpl.java      | 23 ++++++++++++++++++++
 2 files changed, 38 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/293c74a8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsCpuResourceHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsCpuResourceHandlerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsCpuResourceHandlerImpl.java
index d9cca8f..830782d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsCpuResourceHandlerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsCpuResourceHandlerImpl.java
@@ -26,8 +26,10 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
 import org.apache.hadoop.yarn.server.nodemanager.util.NodeManagerHardwareUtils;
@@ -72,6 +74,7 @@ public class CGroupsCpuResourceHandlerImpl implements CpuResourceHandler {
   static final int MIN_PERIOD_US = 1000;
   @VisibleForTesting
   static final int CPU_DEFAULT_WEIGHT = 1024; // set by kernel
+  static final int CPU_DEFAULT_WEIGHT_OPPORTUNISTIC = 2;
 
   CGroupsCpuResourceHandlerImpl(CGroupsHandler cGroupsHandler) {
     this.cGroupsHandler = cGroupsHandler;
@@ -181,16 +184,23 @@ public class CGroupsCpuResourceHandlerImpl implements CpuResourceHandler {
   @Override
   public List<PrivilegedOperation> preStart(Container container)
       throws ResourceHandlerException {
-
     String cgroupId = container.getContainerId().toString();
     Resource containerResource = container.getResource();
     cGroupsHandler.createCGroup(CPU, cgroupId);
     try {
       int containerVCores = containerResource.getVirtualCores();
-      int cpuShares = CPU_DEFAULT_WEIGHT * containerVCores;
-      cGroupsHandler
-          .updateCGroupParam(CPU, cgroupId, CGroupsHandler.CGROUP_CPU_SHARES,
-              String.valueOf(cpuShares));
+      ContainerTokenIdentifier id = container.getContainerTokenIdentifier();
+      if (id != null && id.getExecutionType() ==
+          ExecutionType.OPPORTUNISTIC) {
+        cGroupsHandler
+            .updateCGroupParam(CPU, cgroupId, CGroupsHandler.CGROUP_CPU_SHARES,
+                String.valueOf(CPU_DEFAULT_WEIGHT_OPPORTUNISTIC));
+      } else {
+        int cpuShares = CPU_DEFAULT_WEIGHT * containerVCores;
+        cGroupsHandler
+            .updateCGroupParam(CPU, cgroupId, CGroupsHandler.CGROUP_CPU_SHARES,
+                String.valueOf(cpuShares));
+      }
       if (strictResourceUsageMode) {
         if (nodeVCores != containerVCores) {
           float containerCPU =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/293c74a8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsCpuResourceHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsCpuResourceHandlerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsCpuResourceHandlerImpl.java
index 674cd71..006b060 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsCpuResourceHandlerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsCpuResourceHandlerImpl.java
@@ -21,8 +21,10 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resourc
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
 import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
@@ -294,4 +296,25 @@ public class TestCGroupsCpuResourceHandlerImpl {
   public void testStrictResourceUsage() throws Exception {
     Assert.assertNull(cGroupsCpuResourceHandler.teardown());
   }
+
+  @Test
+  public void testOpportunistic() throws Exception {
+    Configuration conf = new YarnConfiguration();
+
+    cGroupsCpuResourceHandler.bootstrap(plugin, conf);
+    ContainerTokenIdentifier tokenId = mock(ContainerTokenIdentifier.class);
+    when(tokenId.getExecutionType()).thenReturn(ExecutionType.OPPORTUNISTIC);
+    Container container = mock(Container.class);
+    String id = "container_01_01";
+    ContainerId mockContainerId = mock(ContainerId.class);
+    when(mockContainerId.toString()).thenReturn(id);
+    when(container.getContainerId()).thenReturn(mockContainerId);
+    when(container.getContainerTokenIdentifier()).thenReturn(tokenId);
+    when(container.getResource()).thenReturn(Resource.newInstance(1024, 2));
+    cGroupsCpuResourceHandler.preStart(container);
+    verify(mockCGroupsHandler, times(1))
+        .updateCGroupParam(CGroupsHandler.CGroupController.CPU, id,
+            CGroupsHandler.CGROUP_CPU_SHARES, "2");
+  }
+
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[15/51] [abbrv] hadoop git commit: HADOOP-14471. Upgrade Jetty to latest 9.3 version. Contributed by John Zhuge.

Posted by st...@apache.org.
HADOOP-14471. Upgrade Jetty to latest 9.3 version. Contributed by John Zhuge.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/02bf328e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/02bf328e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/02bf328e

Branch: refs/heads/HADOOP-13345
Commit: 02bf328e0fcad07565805420d652f6847927b346
Parents: 7fc324a
Author: John Zhuge <jz...@apache.org>
Authored: Thu Jun 1 16:12:20 2017 -0700
Committer: John Zhuge <jz...@apache.org>
Committed: Fri Aug 4 09:40:45 2017 -0700

----------------------------------------------------------------------
 hadoop-project/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/02bf328e/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 61d7c9b..5aabdc7 100755
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -35,7 +35,7 @@
 
     <failIfNoTests>false</failIfNoTests>
     <maven.test.redirectTestOutputToFile>true</maven.test.redirectTestOutputToFile>
-    <jetty.version>9.3.11.v20160721</jetty.version>
+    <jetty.version>9.3.19.v20170502</jetty.version>
     <test.exclude>_</test.exclude>
     <test.exclude.pattern>_</test.exclude.pattern>
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[10/51] [abbrv] hadoop git commit: YARN-6932. Fix TestFederationRMFailoverProxyProvider test case failure.

Posted by st...@apache.org.
YARN-6932. Fix TestFederationRMFailoverProxyProvider test case failure.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/0542e6f8
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/0542e6f8
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/0542e6f8

Branch: refs/heads/HADOOP-13345
Commit: 0542e6f86e6734f8ceea00960bebe719827136db
Parents: b8e8241
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Aug 3 14:18:03 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Thu Aug 3 14:18:03 2017 -0700

----------------------------------------------------------------------
 .../FederationRMFailoverProxyProvider.java      | 20 ++++++++++----------
 1 file changed, 10 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0542e6f8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationRMFailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationRMFailoverProxyProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationRMFailoverProxyProvider.java
index e00f8d1..c631208 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationRMFailoverProxyProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationRMFailoverProxyProvider.java
@@ -72,8 +72,7 @@ public class FederationRMFailoverProxyProvider<T>
     this.rmProxy = proxy;
     this.protocol = proto;
     this.rmProxy.checkAllowedProtocols(this.protocol);
-    String clusterId =
-        configuration.get(YarnConfiguration.RM_CLUSTER_ID);
+    String clusterId = configuration.get(YarnConfiguration.RM_CLUSTER_ID);
     Preconditions.checkNotNull(clusterId, "Missing RM ClusterId");
     this.subClusterId = SubClusterId.newInstance(clusterId);
     this.facade = facade.getInstance();
@@ -197,16 +196,17 @@ public class FederationRMFailoverProxyProvider<T>
   }
 
   private void closeInternal(T currentProxy) {
-    if ((currentProxy != null) && (currentProxy instanceof Closeable)) {
-      try {
-        ((Closeable) currentProxy).close();
-      } catch (IOException e) {
-        LOG.warn("Exception while trying to close proxy", e);
+    if (currentProxy != null) {
+      if (currentProxy instanceof Closeable) {
+        try {
+          ((Closeable) currentProxy).close();
+        } catch (IOException e) {
+          LOG.warn("Exception while trying to close proxy", e);
+        }
+      } else {
+        RPC.stopProxy(currentProxy);
       }
-    } else {
-      RPC.stopProxy(currentProxy);
     }
-
   }
 
   /**


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[41/51] [abbrv] hadoop git commit: HADOOP-14598. Blacklist Http/HttpsFileSystem in FsUrlStreamHandlerFactory. Contributed by Steve Loughran.

Posted by st...@apache.org.
HADOOP-14598. Blacklist Http/HttpsFileSystem in FsUrlStreamHandlerFactory. Contributed by Steve Loughran.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/1db4788b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1db4788b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1db4788b

Branch: refs/heads/HADOOP-13345
Commit: 1db4788b7d22e57f91520e4a6971774ef84ffab9
Parents: f4e1aa0
Author: Haohui Mai <wh...@apache.org>
Authored: Tue Aug 8 16:27:23 2017 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Tue Aug 8 16:33:18 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/FsUrlConnection.java   | 10 ++++
 .../hadoop/fs/FsUrlStreamHandlerFactory.java    | 26 ++++++++++-
 .../apache/hadoop/fs/TestUrlStreamHandler.java  | 48 +++++++++++++++-----
 3 files changed, 72 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1db4788b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsUrlConnection.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsUrlConnection.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsUrlConnection.java
index 90e75b0..03c7aed 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsUrlConnection.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsUrlConnection.java
@@ -23,6 +23,10 @@ import java.net.URISyntaxException;
 import java.net.URL;
 import java.net.URLConnection;
 
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -33,6 +37,8 @@ import org.apache.hadoop.conf.Configuration;
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 class FsUrlConnection extends URLConnection {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FsUrlConnection.class);
 
   private Configuration conf;
 
@@ -40,12 +46,16 @@ class FsUrlConnection extends URLConnection {
 
   FsUrlConnection(Configuration conf, URL url) {
     super(url);
+    Preconditions.checkArgument(conf != null, "null conf argument");
+    Preconditions.checkArgument(url != null, "null url argument");
     this.conf = conf;
   }
 
   @Override
   public void connect() throws IOException {
+    Preconditions.checkState(is == null, "Already connected");
     try {
+      LOG.debug("Connecting to {}", url);
       FileSystem fs = FileSystem.get(url.toURI(), conf);
       is = fs.open(new Path(url.getPath()));
     } catch (URISyntaxException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1db4788b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsUrlStreamHandlerFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsUrlStreamHandlerFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsUrlStreamHandlerFactory.java
index 91a527d..751b955 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsUrlStreamHandlerFactory.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsUrlStreamHandlerFactory.java
@@ -22,6 +22,9 @@ import java.net.URLStreamHandlerFactory;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -41,6 +44,18 @@ import org.apache.hadoop.conf.Configuration;
 public class FsUrlStreamHandlerFactory implements
     URLStreamHandlerFactory {
 
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FsUrlStreamHandlerFactory.class);
+
+  /**
+   * These are the protocols with MUST NOT be exported, as doing so
+   * would conflict with the standard URL handlers registered by
+   * the JVM. Many things will break.
+   */
+  public static final String[] UNEXPORTED_PROTOCOLS = {
+      "http", "https"
+  };
+
   // The configuration holds supported FS implementation class names.
   private Configuration conf;
 
@@ -64,14 +79,20 @@ public class FsUrlStreamHandlerFactory implements
       throw new RuntimeException(io);
     }
     this.handler = new FsUrlStreamHandler(this.conf);
+    for (String protocol : UNEXPORTED_PROTOCOLS) {
+      protocols.put(protocol, false);
+    }
   }
 
   @Override
   public java.net.URLStreamHandler createURLStreamHandler(String protocol) {
+    LOG.debug("Creating handler for protocol {}", protocol);
     if (!protocols.containsKey(protocol)) {
       boolean known = true;
       try {
-        FileSystem.getFileSystemClass(protocol, conf);
+        Class<? extends FileSystem> impl
+            = FileSystem.getFileSystemClass(protocol, conf);
+        LOG.debug("Found implementation of {}: {}", protocol, impl);
       }
       catch (IOException ex) {
         known = false;
@@ -79,9 +100,12 @@ public class FsUrlStreamHandlerFactory implements
       protocols.put(protocol, known);
     }
     if (protocols.get(protocol)) {
+      LOG.debug("Using handler for protocol {}", protocol);
       return handler;
     } else {
       // FileSystem does not know the protocol, let the VM handle this
+      LOG.debug("Unknown protocol {}, delegating to default implementation",
+          protocol);
       return null;
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1db4788b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUrlStreamHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUrlStreamHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUrlStreamHandler.java
index 6fc97a2..5a04f67 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUrlStreamHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUrlStreamHandler.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.fs;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
 
 import java.io.File;
 import java.io.IOException;
@@ -32,6 +33,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.test.PathUtils;
+
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 /**
@@ -39,8 +42,22 @@ import org.junit.Test;
  */
 public class TestUrlStreamHandler {
 
-  private static final File TEST_ROOT_DIR = PathUtils.getTestDir(TestUrlStreamHandler.class);
-    
+  private static final File TEST_ROOT_DIR =
+      PathUtils.getTestDir(TestUrlStreamHandler.class);
+
+  private static final FsUrlStreamHandlerFactory HANDLER_FACTORY
+      = new FsUrlStreamHandlerFactory();
+
+  @BeforeClass
+  public static void setupHandler() {
+
+    // Setup our own factory
+    // setURLStreamHandlerFactor is can be set at most once in the JVM
+    // the new URLStreamHandler is valid for all tests cases
+    // in TestStreamHandler
+    URL.setURLStreamHandlerFactory(HANDLER_FACTORY);
+  }
+
   /**
    * Test opening and reading from an InputStream through a hdfs:// URL.
    * <p>
@@ -55,15 +72,6 @@ public class TestUrlStreamHandler {
     Configuration conf = new HdfsConfiguration();
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     FileSystem fs = cluster.getFileSystem();
-
-    // Setup our own factory
-    // setURLSteramHandlerFactor is can be set at most once in the JVM
-    // the new URLStreamHandler is valid for all tests cases 
-    // in TestStreamHandler
-    FsUrlStreamHandlerFactory factory =
-        new org.apache.hadoop.fs.FsUrlStreamHandlerFactory();
-    java.net.URL.setURLStreamHandlerFactory(factory);
-
     Path filePath = new Path("/thefile");
 
     try {
@@ -156,4 +164,22 @@ public class TestUrlStreamHandler {
 
   }
 
+  @Test
+  public void testHttpDefaultHandler() throws Throwable {
+    assertNull("Handler for HTTP is the Hadoop one",
+        HANDLER_FACTORY.createURLStreamHandler("http"));
+  }
+
+  @Test
+  public void testHttpsDefaultHandler() throws Throwable {
+    assertNull("Handler for HTTPS is the Hadoop one",
+        HANDLER_FACTORY.createURLStreamHandler("https"));
+  }
+
+  @Test
+  public void testUnknownProtocol() throws Throwable {
+    assertNull("Unknown protocols are not handled",
+        HANDLER_FACTORY.createURLStreamHandler("gopher"));
+  }
+
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[38/51] [abbrv] hadoop git commit: YARN-6879. TestLeafQueue.testDRFUserLimits() has commented out code (Contributed by Angela Wang via Daniel Templeton)

Posted by st...@apache.org.
YARN-6879. TestLeafQueue.testDRFUserLimits() has commented out code
(Contributed by Angela Wang via Daniel Templeton)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e0c24145
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e0c24145
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e0c24145

Branch: refs/heads/HADOOP-13345
Commit: e0c24145d2c2a7d2cf10864fb4800cb1dcbc2977
Parents: 1794de3
Author: Daniel Templeton <te...@apache.org>
Authored: Tue Aug 8 13:35:22 2017 -0700
Committer: Daniel Templeton <te...@apache.org>
Committed: Tue Aug 8 13:35:22 2017 -0700

----------------------------------------------------------------------
 .../server/resourcemanager/scheduler/capacity/TestLeafQueue.java   | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0c24145/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
index 2864d7f..d45f756 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
@@ -820,8 +820,6 @@ public class TestLeafQueue {
       applyCSAssignment(clusterResource, assign, b, nodes, apps);
     } while (assign.getResource().getMemorySize() > 0 &&
         assign.getAssignmentInformation().getNumReservations() == 0);
-    //LOG.info("user_0: " + queueUser0.getUsed());
-    //LOG.info("user_1: " + queueUser1.getUsed());
 
     assertTrue("Verify user_0 got resources ", queueUser0.getUsed()
         .getMemorySize() > 0);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[32/51] [abbrv] hadoop git commit: HADOOP-14730. Support protobuf FileStatus in AdlFileSystem.

Posted by st...@apache.org.
HADOOP-14730. Support protobuf FileStatus in AdlFileSystem.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/55a181f8
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/55a181f8
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/55a181f8

Branch: refs/heads/HADOOP-13345
Commit: 55a181f845adcdcc9b008e9906ade1544fc220e4
Parents: 8d3fd81
Author: Chris Douglas <cd...@apache.org>
Authored: Mon Aug 7 21:31:28 2017 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Mon Aug 7 21:31:28 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/adl/AdlFileStatus.java | 69 ++++++++++++++++++++
 .../org/apache/hadoop/fs/adl/AdlFileSystem.java | 27 ++------
 .../apache/hadoop/fs/adl/TestGetFileStatus.java | 57 ++++++++--------
 .../apache/hadoop/fs/adl/TestListStatus.java    |  8 ++-
 4 files changed, 105 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/55a181f8/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileStatus.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileStatus.java
new file mode 100644
index 0000000..70c005d
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileStatus.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.fs.adl;
+
+import com.microsoft.azure.datalake.store.DirectoryEntry;
+import com.microsoft.azure.datalake.store.DirectoryEntryType;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+import static org.apache.hadoop.fs.adl.AdlConfKeys.ADL_BLOCK_SIZE;
+import static org.apache.hadoop.fs.adl.AdlConfKeys.ADL_REPLICATION_FACTOR;
+
+/**
+ * Shim class supporting linking against 2.x clients.
+ */
+class AdlFileStatus extends FileStatus {
+
+  private static final long serialVersionUID = 0x01fcbe5e;
+
+  private boolean hasAcl = false;
+
+  AdlFileStatus(DirectoryEntry entry, Path path, boolean hasAcl) {
+    this(entry, path, entry.user, entry.group, hasAcl);
+  }
+
+  AdlFileStatus(DirectoryEntry entry, Path path,
+                String owner, String group, boolean hasAcl) {
+    super(entry.length, DirectoryEntryType.DIRECTORY == entry.type,
+        ADL_REPLICATION_FACTOR, ADL_BLOCK_SIZE,
+        entry.lastModifiedTime.getTime(), entry.lastAccessTime.getTime(),
+        new AdlPermission(hasAcl, Short.parseShort(entry.permission, 8)),
+        owner, group, null, path);
+    this.hasAcl = hasAcl;
+  }
+
+  @Override
+  public boolean hasAcl() {
+    return hasAcl;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    // satisfy findbugs
+    return super.equals(o);
+  }
+
+  @Override
+  public int hashCode() {
+    // satisfy findbugs
+    return super.hashCode();
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55a181f8/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
index 0de538e..76ce43e 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
@@ -29,7 +29,6 @@ import com.google.common.annotations.VisibleForTesting;
 import com.microsoft.azure.datalake.store.ADLStoreClient;
 import com.microsoft.azure.datalake.store.ADLStoreOptions;
 import com.microsoft.azure.datalake.store.DirectoryEntry;
-import com.microsoft.azure.datalake.store.DirectoryEntryType;
 import com.microsoft.azure.datalake.store.IfExists;
 import com.microsoft.azure.datalake.store.LatencyTracker;
 import com.microsoft.azure.datalake.store.UserGroupRepresentation;
@@ -606,30 +605,12 @@ public class AdlFileSystem extends FileSystem {
   }
 
   private FileStatus toFileStatus(final DirectoryEntry entry, final Path f) {
-    boolean isDirectory = entry.type == DirectoryEntryType.DIRECTORY;
-    long lastModificationData = entry.lastModifiedTime.getTime();
-    long lastAccessTime = entry.lastAccessTime.getTime();
-    // set aclBit from ADLS backend response if
-    // ADL_SUPPORT_ACL_BIT_IN_FSPERMISSION is true.
-    final boolean aclBit = aclBitStatus ? entry.aclBit : false;
-
-    FsPermission permission = new AdlPermission(aclBit,
-        Short.valueOf(entry.permission, 8));
-    String user = entry.user;
-    String group = entry.group;
-
-    FileStatus status;
+    Path p = makeQualified(f);
+    boolean aclBit = aclBitStatus ? entry.aclBit : false;
     if (overrideOwner) {
-      status = new FileStatus(entry.length, isDirectory, ADL_REPLICATION_FACTOR,
-          ADL_BLOCK_SIZE, lastModificationData, lastAccessTime, permission,
-          userName, "hdfs", this.makeQualified(f));
-    } else {
-      status = new FileStatus(entry.length, isDirectory, ADL_REPLICATION_FACTOR,
-          ADL_BLOCK_SIZE, lastModificationData, lastAccessTime, permission,
-          user, group, this.makeQualified(f));
+      return new AdlFileStatus(entry, p, userName, "hdfs", aclBit);
     }
-
-    return status;
+    return new AdlFileStatus(entry, p, aclBit);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55a181f8/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestGetFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestGetFileStatus.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestGetFileStatus.java
index 0ea4b86..d9e22db 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestGetFileStatus.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestGetFileStatus.java
@@ -42,8 +42,8 @@ import static org.apache.hadoop.fs.adl.AdlConfKeys.ADL_BLOCK_SIZE;
  * org.apache.hadoop.fs.adl.live testing package.
  */
 public class TestGetFileStatus extends AdlMockWebServer {
-  private static final Logger LOG = LoggerFactory
-      .getLogger(TestGetFileStatus.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestGetFileStatus.class);
 
   @Test
   public void getFileStatusReturnsAsExpected()
@@ -72,33 +72,30 @@ public class TestGetFileStatus extends AdlMockWebServer {
         fileStatus.isErasureCoded());
   }
 
-    @Test
-    public void getFileStatusAclBit()
-            throws URISyntaxException, IOException {
-        // With ACLBIT set to true
-        getMockServer().enqueue(new MockResponse().setResponseCode(200)
-                .setBody(TestADLResponseData.getGetFileStatusJSONResponse(true)));
-        long startTime = Time.monotonicNow();
-        FileStatus fileStatus = getMockAdlFileSystem()
-                .getFileStatus(new Path("/test1/test2"));
-        long endTime = Time.monotonicNow();
-        LOG.debug("Time : " + (endTime - startTime));
-        Assert.assertTrue(fileStatus.isFile());
-        Assert.assertEquals(true, fileStatus.getPermission().getAclBit());
-        Assert.assertEquals(fileStatus.hasAcl(),
-            fileStatus.getPermission().getAclBit());
+  @Test
+  public void getFileStatusAclBit() throws URISyntaxException, IOException {
+    // With ACLBIT set to true
+    getMockServer().enqueue(new MockResponse().setResponseCode(200)
+            .setBody(TestADLResponseData.getGetFileStatusJSONResponse(true)));
+    long startTime = Time.monotonicNow();
+    FileStatus fileStatus = getMockAdlFileSystem()
+            .getFileStatus(new Path("/test1/test2"));
+    long endTime = Time.monotonicNow();
+    LOG.debug("Time : " + (endTime - startTime));
+    Assert.assertTrue(fileStatus.isFile());
+    Assert.assertTrue(fileStatus.hasAcl());
+    Assert.assertTrue(fileStatus.getPermission().getAclBit());
 
-        // With ACLBIT set to false
-        getMockServer().enqueue(new MockResponse().setResponseCode(200)
-                .setBody(TestADLResponseData.getGetFileStatusJSONResponse(false)));
-        startTime = Time.monotonicNow();
-        fileStatus = getMockAdlFileSystem()
-                .getFileStatus(new Path("/test1/test2"));
-        endTime = Time.monotonicNow();
-        LOG.debug("Time : " + (endTime - startTime));
-        Assert.assertTrue(fileStatus.isFile());
-        Assert.assertEquals(false, fileStatus.getPermission().getAclBit());
-        Assert.assertEquals(fileStatus.hasAcl(),
-            fileStatus.getPermission().getAclBit());
-    }
+    // With ACLBIT set to false
+    getMockServer().enqueue(new MockResponse().setResponseCode(200)
+            .setBody(TestADLResponseData.getGetFileStatusJSONResponse(false)));
+    startTime = Time.monotonicNow();
+    fileStatus = getMockAdlFileSystem()
+            .getFileStatus(new Path("/test1/test2"));
+    endTime = Time.monotonicNow();
+    LOG.debug("Time : " + (endTime - startTime));
+    Assert.assertTrue(fileStatus.isFile());
+    Assert.assertFalse(fileStatus.hasAcl());
+    Assert.assertFalse(fileStatus.getPermission().getAclBit());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55a181f8/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestListStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestListStatus.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestListStatus.java
index dac8886..db32476 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestListStatus.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestListStatus.java
@@ -102,7 +102,7 @@ public class TestListStatus extends AdlMockWebServer {
   }
 
   @Test
-  public void listStatusAclBit()
+  public void listStatusAcl()
           throws URISyntaxException, IOException {
     // With ACLBIT set to true
     getMockServer().enqueue(new MockResponse().setResponseCode(200)
@@ -115,7 +115,8 @@ public class TestListStatus extends AdlMockWebServer {
     LOG.debug("Time : " + (endTime - startTime));
     for (int i = 0; i < ls.length; i++) {
       Assert.assertTrue(ls[i].isDirectory());
-      Assert.assertEquals(true, ls[i].getPermission().getAclBit());
+      Assert.assertTrue(ls[i].hasAcl());
+      Assert.assertTrue(ls[i].getPermission().getAclBit());
     }
 
     // With ACLBIT set to false
@@ -129,7 +130,8 @@ public class TestListStatus extends AdlMockWebServer {
     LOG.debug("Time : " + (endTime - startTime));
     for (int i = 0; i < ls.length; i++) {
       Assert.assertTrue(ls[i].isDirectory());
-      Assert.assertEquals(false, ls[i].getPermission().getAclBit());
+      Assert.assertFalse(ls[i].hasAcl());
+      Assert.assertFalse(ls[i].getPermission().getAclBit());
     }
   }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[19/51] [abbrv] hadoop git commit: YARN-6811. [ATS1.5] All history logs should be kept under its own User Directory. Contributed by Rohith Sharma K S.

Posted by st...@apache.org.
YARN-6811. [ATS1.5] All history logs should be kept under its own User Directory. Contributed by Rohith Sharma K S.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f44b349b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f44b349b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f44b349b

Branch: refs/heads/HADOOP-13345
Commit: f44b349b813508f0f6d99ca10bddba683dedf6c4
Parents: bbc6d25
Author: Junping Du <ju...@apache.org>
Authored: Fri Aug 4 16:03:56 2017 -0700
Committer: Junping Du <ju...@apache.org>
Committed: Fri Aug 4 16:03:56 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |  4 +
 .../api/impl/FileSystemTimelineWriter.java      | 40 ++++++--
 .../src/main/resources/yarn-default.xml         | 10 ++
 .../api/impl/TestTimelineClientForATS1_5.java   | 81 ++++++++++++----
 .../timeline/EntityGroupFSTimelineStore.java    | 23 ++++-
 .../TestEntityGroupFSTimelineStore.java         | 99 ++++++++++++++++++--
 6 files changed, 224 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f44b349b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index d608df8..71a7134 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -2069,6 +2069,10 @@ public class YarnConfiguration extends Configuration {
       = TIMELINE_SERVICE_PREFIX
       + "entity-file.fs-support-append";
 
+  public static final String
+      TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_WITH_USER_DIR =
+      TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_PREFIX + "with-user-dir";
+
   /**
    * Settings for timeline service v2.0
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f44b349b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
index fc3385b..b7bb48e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
@@ -145,9 +145,12 @@ public class FileSystemTimelineWriter extends TimelineWriter{
         new LogFDsCache(flushIntervalSecs, cleanIntervalSecs, ttl,
             timerTaskTTL);
 
-    this.isAppendSupported =
-        conf.getBoolean(
-            YarnConfiguration.TIMELINE_SERVICE_ENTITYFILE_FS_SUPPORT_APPEND, true);
+    this.isAppendSupported = conf.getBoolean(
+        YarnConfiguration.TIMELINE_SERVICE_ENTITYFILE_FS_SUPPORT_APPEND, true);
+
+    boolean storeInsideUserDir = conf.getBoolean(
+        YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_WITH_USER_DIR,
+        false);
 
     objMapper = createObjectMapper();
 
@@ -157,8 +160,8 @@ public class FileSystemTimelineWriter extends TimelineWriter{
         YarnConfiguration
             .DEFAULT_TIMELINE_SERVICE_CLIENT_INTERNAL_ATTEMPT_DIR_CACHE_SIZE);
 
-    attemptDirCache =
-        new AttemptDirCache(attemptDirCacheSize, fs, activePath);
+    attemptDirCache = new AttemptDirCache(attemptDirCacheSize, fs, activePath,
+        authUgi, storeInsideUserDir);
 
     if (LOG.isDebugEnabled()) {
       StringBuilder debugMSG = new StringBuilder();
@@ -171,6 +174,8 @@ public class FileSystemTimelineWriter extends TimelineWriter{
               + "=" + ttl + ", " +
           YarnConfiguration.TIMELINE_SERVICE_ENTITYFILE_FS_SUPPORT_APPEND
               + "=" + isAppendSupported + ", " +
+          YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_WITH_USER_DIR
+              + "=" + storeInsideUserDir + ", " +
           YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_ACTIVE_DIR
               + "=" + activePath);
 
@@ -946,8 +951,11 @@ public class FileSystemTimelineWriter extends TimelineWriter{
     private final Map<ApplicationAttemptId, Path> attemptDirCache;
     private final FileSystem fs;
     private final Path activePath;
+    private final UserGroupInformation authUgi;
+    private final boolean storeInsideUserDir;
 
-    public AttemptDirCache(int cacheSize, FileSystem fs, Path activePath) {
+    public AttemptDirCache(int cacheSize, FileSystem fs, Path activePath,
+        UserGroupInformation ugi, boolean storeInsideUserDir) {
       this.attemptDirCacheSize = cacheSize;
       this.attemptDirCache =
           new LinkedHashMap<ApplicationAttemptId, Path>(
@@ -961,6 +969,8 @@ public class FileSystemTimelineWriter extends TimelineWriter{
           };
       this.fs = fs;
       this.activePath = activePath;
+      this.authUgi = ugi;
+      this.storeInsideUserDir = storeInsideUserDir;
     }
 
     public Path getAppAttemptDir(ApplicationAttemptId attemptId)
@@ -993,8 +1003,8 @@ public class FileSystemTimelineWriter extends TimelineWriter{
     }
 
     private Path createApplicationDir(ApplicationId appId) throws IOException {
-      Path appDir =
-          new Path(activePath, appId.toString());
+      Path appRootDir = getAppRootDir(authUgi.getShortUserName());
+      Path appDir = new Path(appRootDir, appId.toString());
       if (FileSystem.mkdirs(fs, appDir,
           new FsPermission(APP_LOG_DIR_PERMISSIONS))) {
         if (LOG.isDebugEnabled()) {
@@ -1003,5 +1013,19 @@ public class FileSystemTimelineWriter extends TimelineWriter{
       }
       return appDir;
     }
+
+    private Path getAppRootDir(String user) throws IOException {
+      if (!storeInsideUserDir) {
+        return activePath;
+      }
+      Path userDir = new Path(activePath, user);
+      if (FileSystem.mkdirs(fs, userDir,
+          new FsPermission(APP_LOG_DIR_PERMISSIONS))) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("New user directory created - " + userDir);
+        }
+      }
+      return userDir;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f44b349b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index 564a451..95b8a88 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -3244,4 +3244,14 @@
     <value>0.0.0.0:8091</value>
   </property>
 
+  <property>
+    <description>
+       It is TimelineClient 1.5 configuration whether to store active
+       application’s timeline data with in user directory i.e
+       ${yarn.timeline-service.entity-group-fs-store.active-dir}/${user.name}
+    </description>
+    <name>yarn.timeline-service.entity-group-fs-store.with-user-dir</name>
+    <value>false</value>
+  </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f44b349b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientForATS1_5.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientForATS1_5.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientForATS1_5.java
index d3826e1..8573033 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientForATS1_5.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientForATS1_5.java
@@ -59,25 +59,30 @@ public class TestTimelineClientForATS1_5 {
   private static FileContext localFS;
   private static File localActiveDir;
   private TimelineWriter spyTimelineWriter;
+  private UserGroupInformation authUgi;
 
   @Before
   public void setup() throws Exception {
     localFS = FileContext.getLocalFSFileContext();
     localActiveDir =
         new File("target", this.getClass().getSimpleName() + "-activeDir")
-          .getAbsoluteFile();
+            .getAbsoluteFile();
     localFS.delete(new Path(localActiveDir.getAbsolutePath()), true);
     localActiveDir.mkdir();
     LOG.info("Created activeDir in " + localActiveDir.getAbsolutePath());
+    authUgi = UserGroupInformation.getCurrentUser();
+  }
+
+  private YarnConfiguration getConfigurations() {
     YarnConfiguration conf = new YarnConfiguration();
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
     conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 1.5f);
     conf.set(YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_ACTIVE_DIR,
-      localActiveDir.getAbsolutePath());
+        localActiveDir.getAbsolutePath());
     conf.set(
-      YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_SUMMARY_ENTITY_TYPES,
-      "summary_type");
-    client = createTimelineClient(conf);
+        YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_SUMMARY_ENTITY_TYPES,
+        "summary_type");
+    return conf;
   }
 
   @After
@@ -90,6 +95,21 @@ public class TestTimelineClientForATS1_5 {
 
   @Test
   public void testPostEntities() throws Exception {
+    client = createTimelineClient(getConfigurations());
+    verifyForPostEntities(false);
+  }
+
+  @Test
+  public void testPostEntitiesToKeepUnderUserDir() throws Exception {
+    YarnConfiguration conf = getConfigurations();
+    conf.setBoolean(
+        YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_WITH_USER_DIR,
+        true);
+    client = createTimelineClient(conf);
+    verifyForPostEntities(true);
+  }
+
+  private void verifyForPostEntities(boolean storeInsideUserDir) {
     ApplicationId appId =
         ApplicationId.newInstance(System.currentTimeMillis(), 1);
     TimelineEntityGroupId groupId =
@@ -118,7 +138,8 @@ public class TestTimelineClientForATS1_5 {
       entityTDB[0] = entities[0];
       verify(spyTimelineWriter, times(1)).putEntities(entityTDB);
       Assert.assertTrue(localFS.util().exists(
-        new Path(getAppAttemptDir(attemptId1), "summarylog-"
+          new Path(getAppAttemptDir(attemptId1, storeInsideUserDir),
+              "summarylog-"
             + attemptId1.toString())));
       reset(spyTimelineWriter);
 
@@ -132,13 +153,16 @@ public class TestTimelineClientForATS1_5 {
       verify(spyTimelineWriter, times(0)).putEntities(
         any(TimelineEntity[].class));
       Assert.assertTrue(localFS.util().exists(
-        new Path(getAppAttemptDir(attemptId2), "summarylog-"
+          new Path(getAppAttemptDir(attemptId2, storeInsideUserDir),
+              "summarylog-"
             + attemptId2.toString())));
       Assert.assertTrue(localFS.util().exists(
-        new Path(getAppAttemptDir(attemptId2), "entitylog-"
+          new Path(getAppAttemptDir(attemptId2, storeInsideUserDir),
+              "entitylog-"
             + groupId.toString())));
       Assert.assertTrue(localFS.util().exists(
-        new Path(getAppAttemptDir(attemptId2), "entitylog-"
+          new Path(getAppAttemptDir(attemptId2, storeInsideUserDir),
+              "entitylog-"
             + groupId2.toString())));
       reset(spyTimelineWriter);
     } catch (Exception e) {
@@ -148,6 +172,21 @@ public class TestTimelineClientForATS1_5 {
 
   @Test
   public void testPutDomain() {
+    client = createTimelineClient(getConfigurations());
+    verifyForPutDomain(false);
+  }
+
+  @Test
+  public void testPutDomainToKeepUnderUserDir() {
+    YarnConfiguration conf = getConfigurations();
+    conf.setBoolean(
+        YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_WITH_USER_DIR,
+        true);
+    client = createTimelineClient(conf);
+    verifyForPutDomain(true);
+  }
+
+  private void verifyForPutDomain(boolean storeInsideUserDir) {
     ApplicationId appId =
         ApplicationId.newInstance(System.currentTimeMillis(), 1);
     ApplicationAttemptId attemptId1 =
@@ -161,23 +200,33 @@ public class TestTimelineClientForATS1_5 {
 
       client.putDomain(attemptId1, domain);
       verify(spyTimelineWriter, times(0)).putDomain(domain);
-      Assert.assertTrue(localFS.util().exists(
-        new Path(getAppAttemptDir(attemptId1), "domainlog-"
-            + attemptId1.toString())));
+      Assert.assertTrue(localFS.util()
+          .exists(new Path(getAppAttemptDir(attemptId1, storeInsideUserDir),
+              "domainlog-" + attemptId1.toString())));
       reset(spyTimelineWriter);
     } catch (Exception e) {
       Assert.fail("Exception is not expected." + e);
     }
   }
 
-  private Path getAppAttemptDir(ApplicationAttemptId appAttemptId) {
-    Path appDir =
-        new Path(localActiveDir.getAbsolutePath(), appAttemptId
-          .getApplicationId().toString());
+  private Path getAppAttemptDir(ApplicationAttemptId appAttemptId,
+      boolean storeInsideUserDir) {
+    Path userDir = getUserDir(appAttemptId, storeInsideUserDir);
+    Path appDir = new Path(userDir, appAttemptId.getApplicationId().toString());
     Path attemptDir = new Path(appDir, appAttemptId.toString());
     return attemptDir;
   }
 
+  private Path getUserDir(ApplicationAttemptId appAttemptId,
+      boolean storeInsideUserDir) {
+    if (!storeInsideUserDir) {
+      return new Path(localActiveDir.getAbsolutePath());
+    }
+    Path userDir =
+        new Path(localActiveDir.getAbsolutePath(), authUgi.getShortUserName());
+    return userDir;
+  }
+
   private static TimelineEntity generateEntity(String type) {
     TimelineEntity entity = new TimelineEntity();
     entity.setEntityId("entity id");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f44b349b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityGroupFSTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityGroupFSTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityGroupFSTimelineStore.java
index 1675a48..80baf89 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityGroupFSTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityGroupFSTimelineStore.java
@@ -356,7 +356,13 @@ public class EntityGroupFSTimelineStore extends CompositeService
   @VisibleForTesting
   int scanActiveLogs() throws IOException {
     long startTime = Time.monotonicNow();
-    RemoteIterator<FileStatus> iter = list(activeRootPath);
+    int logsToScanCount = scanActiveLogs(activeRootPath);
+    metrics.addActiveLogDirScanTime(Time.monotonicNow() - startTime);
+    return logsToScanCount;
+  }
+
+  int scanActiveLogs(Path dir) throws IOException {
+    RemoteIterator<FileStatus> iter = list(dir);
     int logsToScanCount = 0;
     while (iter.hasNext()) {
       FileStatus stat = iter.next();
@@ -368,10 +374,9 @@ public class EntityGroupFSTimelineStore extends CompositeService
         AppLogs logs = getAndSetActiveLog(appId, stat.getPath());
         executor.execute(new ActiveLogParser(logs));
       } else {
-        LOG.debug("Unable to parse entry {}", name);
+        logsToScanCount += scanActiveLogs(stat.getPath());
       }
     }
-    metrics.addActiveLogDirScanTime(Time.monotonicNow() - startTime);
     return logsToScanCount;
   }
 
@@ -418,6 +423,18 @@ public class EntityGroupFSTimelineStore extends CompositeService
         appDirPath = getActiveAppPath(applicationId);
         if (fs.exists(appDirPath)) {
           appState = AppState.ACTIVE;
+        } else {
+          // check for user directory inside active path
+          RemoteIterator<FileStatus> iter = list(activeRootPath);
+          while (iter.hasNext()) {
+            Path child = new Path(iter.next().getPath().getName(),
+                applicationId.toString());
+            appDirPath = new Path(activeRootPath, child);
+            if (fs.exists(appDirPath)) {
+              appState = AppState.ACTIVE;
+              break;
+            }
+          }
         }
       }
       if (appState != AppState.UNKNOWN) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f44b349b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestEntityGroupFSTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestEntityGroupFSTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestEntityGroupFSTimelineStore.java
index 8540d45..0458722 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestEntityGroupFSTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestEntityGroupFSTimelineStore.java
@@ -37,6 +37,8 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.timeline.EntityGroupFSTimelineStore.AppState;
+import org.apache.hadoop.yarn.server.timeline.TimelineReader.Field;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -58,7 +60,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import static org.apache.hadoop.yarn.server.timeline.EntityGroupFSTimelineStore.AppState;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
@@ -91,6 +92,7 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
   private static ApplicationId mainTestAppId;
   private static Path mainTestAppDirPath;
   private static Path testDoneDirPath;
+  private static Path testActiveDirPath;
   private static String mainEntityLogFileName;
 
   private EntityGroupFSTimelineStore store;
@@ -125,23 +127,28 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
               + i);
       sampleAppIds.add(appId);
     }
+    testActiveDirPath = getTestRootPath("active");
     // Among all sample applicationIds, choose the first one for most of the
     // tests.
     mainTestAppId = sampleAppIds.get(0);
-    mainTestAppDirPath = getTestRootPath(mainTestAppId.toString());
+    mainTestAppDirPath = new Path(testActiveDirPath, mainTestAppId.toString());
     mainEntityLogFileName = EntityGroupFSTimelineStore.ENTITY_LOG_PREFIX
           + EntityGroupPlugInForTest.getStandardTimelineGroupId(mainTestAppId);
 
     testDoneDirPath = getTestRootPath("done");
     config.set(YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_DONE_DIR,
         testDoneDirPath.toString());
+    config.set(
+        YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_ACTIVE_DIR,
+        testActiveDirPath.toString());
   }
 
   @Before
   public void setup() throws Exception {
     for (ApplicationId appId : sampleAppIds) {
-      Path attemotDirPath = new Path(getTestRootPath(appId.toString()),
-          getAttemptDirName(appId));
+      Path attemotDirPath =
+          new Path(new Path(testActiveDirPath, appId.toString()),
+              getAttemptDirName(appId));
       createTestFiles(appId, attemotDirPath);
     }
 
@@ -178,7 +185,7 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
   public void tearDown() throws Exception {
     store.stop();
     for (ApplicationId appId : sampleAppIds) {
-      fs.delete(getTestRootPath(appId.toString()), true);
+      fs.delete(new Path(testActiveDirPath,appId.toString()), true);
     }
     if (testJar != null) {
       testJar.delete();
@@ -414,8 +421,88 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
 
   }
 
+  @Test
+  public void testGetEntityPluginRead() throws Exception {
+    EntityGroupFSTimelineStore store = null;
+    ApplicationId appId =
+        ApplicationId.fromString("application_1501509265053_0001");
+    String user = UserGroupInformation.getCurrentUser().getShortUserName();
+    Path userBase = new Path(testActiveDirPath, user);
+    Path userAppRoot = new Path(userBase, appId.toString());
+    Path attemotDirPath = new Path(userAppRoot, getAttemptDirName(appId));
+
+    try {
+      store = createAndStartTimelineStore(AppState.ACTIVE);
+      String logFileName = EntityGroupFSTimelineStore.ENTITY_LOG_PREFIX
+          + EntityGroupPlugInForTest.getStandardTimelineGroupId(appId);
+      createTestFiles(appId, attemotDirPath, logFileName);
+      TimelineEntity entity = store.getEntity(entityNew.getEntityId(),
+          entityNew.getEntityType(), EnumSet.allOf(Field.class));
+      assertNotNull(entity);
+      assertEquals(entityNew.getEntityId(), entity.getEntityId());
+      assertEquals(entityNew.getEntityType(), entity.getEntityType());
+    } finally {
+      if (store != null) {
+        store.stop();
+      }
+      fs.delete(userBase, true);
+    }
+  }
+
+  @Test
+  public void testScanActiveLogsAndMoveToDonePluginRead() throws Exception {
+    EntityGroupFSTimelineStore store = null;
+    ApplicationId appId =
+        ApplicationId.fromString("application_1501509265053_0002");
+    String user = UserGroupInformation.getCurrentUser().getShortUserName();
+    Path userBase = new Path(testActiveDirPath, user);
+    Path userAppRoot = new Path(userBase, appId.toString());
+    Path attemotDirPath = new Path(userAppRoot, getAttemptDirName(appId));
+
+    try {
+      store = createAndStartTimelineStore(AppState.COMPLETED);
+      String logFileName = EntityGroupFSTimelineStore.ENTITY_LOG_PREFIX
+          + EntityGroupPlugInForTest.getStandardTimelineGroupId(appId);
+      createTestFiles(appId, attemotDirPath, logFileName);
+      store.scanActiveLogs();
+
+      TimelineEntity entity = store.getEntity(entityNew.getEntityId(),
+          entityNew.getEntityType(), EnumSet.allOf(Field.class));
+      assertNotNull(entity);
+      assertEquals(entityNew.getEntityId(), entity.getEntityId());
+      assertEquals(entityNew.getEntityType(), entity.getEntityType());
+    } finally {
+      if (store != null) {
+        store.stop();
+      }
+      fs.delete(userBase, true);
+    }
+  }
+
+  private EntityGroupFSTimelineStore createAndStartTimelineStore(
+      AppState appstate) {
+    // stop before creating new store to get the lock
+    store.stop();
+    
+    EntityGroupFSTimelineStore newStore = new EntityGroupFSTimelineStore() {
+      @Override
+      protected AppState getAppState(ApplicationId appId) throws IOException {
+        return appstate;
+      }
+    };
+    newStore.init(config);
+    newStore.setFs(fs);
+    newStore.start();
+    return newStore;
+  }
+
   private void createTestFiles(ApplicationId appId, Path attemptDirPath)
       throws IOException {
+    createTestFiles(appId, attemptDirPath, mainEntityLogFileName);
+  }
+
+  private void createTestFiles(ApplicationId appId, Path attemptDirPath,
+      String logPath) throws IOException {
     TimelineEntities entities = PluginStoreTestUtils.generateTestEntities();
     PluginStoreTestUtils.writeEntities(entities,
         new Path(attemptDirPath, TEST_SUMMARY_LOG_FILE_NAME), fs);
@@ -429,7 +516,7 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
     TimelineEntities entityList = new TimelineEntities();
     entityList.addEntity(entityNew);
     PluginStoreTestUtils.writeEntities(entityList,
-        new Path(attemptDirPath, mainEntityLogFileName), fs);
+        new Path(attemptDirPath, logPath), fs);
 
     FSDataOutputStream out = fs.create(
         new Path(attemptDirPath, TEST_DOMAIN_LOG_FILE_NAME));


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[04/51] [abbrv] hadoop git commit: HDFS-9388. Decommission related code to support Maintenance State for datanodes.

Posted by st...@apache.org.
HDFS-9388. Decommission related code to support Maintenance State for datanodes.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/79df1e75
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/79df1e75
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/79df1e75

Branch: refs/heads/HADOOP-13345
Commit: 79df1e750ef558afed6d166ce225a23061b36aed
Parents: 12e44e7
Author: Manoj Govindassamy <ma...@apache.org>
Authored: Wed Aug 2 14:22:41 2017 -0700
Committer: Manoj Govindassamy <ma...@apache.org>
Committed: Wed Aug 2 14:22:46 2017 -0700

----------------------------------------------------------------------
 .../blockmanagement/DatanodeAdminManager.java   | 756 +++++++++++++++++++
 .../server/blockmanagement/DatanodeManager.java |  30 +-
 .../blockmanagement/DecommissionManager.java    | 741 ------------------
 .../hadoop/hdfs/server/namenode/BackupNode.java |   2 +-
 .../src/main/resources/hdfs-default.xml         |  21 +-
 .../apache/hadoop/hdfs/TestDecommission.java    |  44 +-
 .../blockmanagement/BlockManagerTestUtil.java   |   2 +-
 ...constructStripedBlocksWithRackAwareness.java |   5 +-
 .../TestReplicationPolicyConsiderLoad.java      |   2 +-
 .../namenode/TestDecommissioningStatus.java     |   6 +-
 .../TestDefaultBlockPlacementPolicy.java        |   4 +-
 .../hadoop/hdfs/server/namenode/TestFsck.java   |  13 +-
 .../server/namenode/TestNameNodeMXBean.java     |   6 +-
 .../namenode/TestNamenodeCapacityReport.java    |   8 +-
 14 files changed, 833 insertions(+), 807 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/79df1e75/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminManager.java
new file mode 100644
index 0000000..928036a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminManager.java
@@ -0,0 +1,756 @@
+/**
+ * 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 static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.hadoop.util.Time.monotonicNow;
+
+import java.util.AbstractList;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.TreeMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.server.namenode.INodeId;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.Namesystem;
+import org.apache.hadoop.hdfs.util.CyclicIteration;
+import org.apache.hadoop.util.ChunkedArrayList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * Manages decommissioning and maintenance state for DataNodes. A background
+ * monitor thread periodically checks the status of DataNodes that are
+ * decommissioning or entering maintenance state.
+ * <p/>
+ * A DataNode can be decommissioned in a few situations:
+ * <ul>
+ * <li>If a DN is dead, it is decommissioned immediately.</li>
+ * <li>If a DN is alive, it is decommissioned after all of its blocks
+ * are sufficiently replicated. Merely under-replicated blocks do not
+ * block decommissioning as long as they are above a replication
+ * threshold.</li>
+ * </ul>
+ * In the second case, the DataNode transitions to a DECOMMISSION_INPROGRESS
+ * state and is tracked by the monitor thread. The monitor periodically scans
+ * through the list of insufficiently replicated blocks on these DataNodes to
+ * determine if they can be DECOMMISSIONED. The monitor also prunes this list
+ * as blocks become replicated, so monitor scans will become more efficient
+ * over time.
+ * <p/>
+ * DECOMMISSION_INPROGRESS nodes that become dead do not progress to
+ * DECOMMISSIONED until they become live again. This prevents potential
+ * durability loss for singly-replicated blocks (see HDFS-6791).
+ * <p/>
+ * DataNodes can also be put under maintenance state for any short duration
+ * maintenance operations. Unlike decommissioning, blocks are not always
+ * re-replicated for the DataNodes to enter maintenance state. When the
+ * blocks are replicated at least dfs.namenode.maintenance.replication.min,
+ * DataNodes transition to IN_MAINTENANCE state. Otherwise, just like
+ * decommissioning, DataNodes transition to ENTERING_MAINTENANCE state and
+ * wait for the blocks to be sufficiently replicated and then transition to
+ * IN_MAINTENANCE state. The block replication factor is relaxed for a maximum
+ * of maintenance expiry time. When DataNodes don't transition or join the
+ * cluster back by expiry time, blocks are re-replicated just as in
+ * decommissioning case as to avoid read or write performance degradation.
+ * <p/>
+ * This class depends on the FSNamesystem lock for synchronization.
+ */
+@InterfaceAudience.Private
+public class DatanodeAdminManager {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DatanodeAdminManager.class);
+  private final Namesystem namesystem;
+  private final BlockManager blockManager;
+  private final HeartbeatManager hbManager;
+  private final ScheduledExecutorService executor;
+
+  /**
+   * Map containing the DECOMMISSION_INPROGRESS or ENTERING_MAINTENANCE
+   * datanodes that are being tracked so they can be be marked as
+   * DECOMMISSIONED or IN_MAINTENANCE. Even after the node is marked as
+   * IN_MAINTENANCE, the node remains in the map until
+   * maintenance expires checked during a monitor tick.
+   * <p/>
+   * This holds a set of references to the under-replicated blocks on the DN at
+   * the time the DN is added to the map, i.e. the blocks that are preventing
+   * the node from being marked as decommissioned. During a monitor tick, this
+   * list is pruned as blocks becomes replicated.
+   * <p/>
+   * Note also that the reference to the list of under-replicated blocks
+   * will be null on initial add
+   * <p/>
+   * However, this map can become out-of-date since it is not updated by block
+   * reports or other events. Before being finally marking as decommissioned,
+   * another check is done with the actual block map.
+   */
+  private final TreeMap<DatanodeDescriptor, AbstractList<BlockInfo>>
+      outOfServiceNodeBlocks;
+
+  /**
+   * Tracking a node in outOfServiceNodeBlocks consumes additional memory. To
+   * limit the impact on NN memory consumption, we limit the number of nodes in
+   * outOfServiceNodeBlocks. Additional nodes wait in pendingNodes.
+   */
+  private final Queue<DatanodeDescriptor> pendingNodes;
+  private Monitor monitor = null;
+
+  DatanodeAdminManager(final Namesystem namesystem,
+      final BlockManager blockManager, final HeartbeatManager hbManager) {
+    this.namesystem = namesystem;
+    this.blockManager = blockManager;
+    this.hbManager = hbManager;
+
+    executor = Executors.newScheduledThreadPool(1,
+        new ThreadFactoryBuilder().setNameFormat("DatanodeAdminMonitor-%d")
+            .setDaemon(true).build());
+    outOfServiceNodeBlocks = new TreeMap<>();
+    pendingNodes = new LinkedList<>();
+  }
+
+  /**
+   * Start the DataNode admin monitor thread.
+   * @param conf
+   */
+  void activate(Configuration conf) {
+    final int intervalSecs = (int) conf.getTimeDuration(
+        DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY,
+        DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_DEFAULT,
+        TimeUnit.SECONDS);
+    checkArgument(intervalSecs >= 0, "Cannot set a negative " +
+        "value for " + DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY);
+
+    int blocksPerInterval = conf.getInt(
+        DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_BLOCKS_PER_INTERVAL_KEY,
+        DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_BLOCKS_PER_INTERVAL_DEFAULT);
+
+    final String deprecatedKey =
+        "dfs.namenode.decommission.nodes.per.interval";
+    final String strNodes = conf.get(deprecatedKey);
+    if (strNodes != null) {
+      LOG.warn("Deprecated configuration key {} will be ignored.",
+          deprecatedKey);
+      LOG.warn("Please update your configuration to use {} instead.",
+          DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_BLOCKS_PER_INTERVAL_KEY);
+    }
+
+    checkArgument(blocksPerInterval > 0,
+        "Must set a positive value for "
+        + DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_BLOCKS_PER_INTERVAL_KEY);
+
+    final int maxConcurrentTrackedNodes = conf.getInt(
+        DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_MAX_CONCURRENT_TRACKED_NODES,
+        DFSConfigKeys
+            .DFS_NAMENODE_DECOMMISSION_MAX_CONCURRENT_TRACKED_NODES_DEFAULT);
+    checkArgument(maxConcurrentTrackedNodes >= 0, "Cannot set a negative " +
+        "value for "
+        + DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_MAX_CONCURRENT_TRACKED_NODES);
+
+    monitor = new Monitor(blocksPerInterval, maxConcurrentTrackedNodes);
+    executor.scheduleAtFixedRate(monitor, intervalSecs, intervalSecs,
+        TimeUnit.SECONDS);
+
+    LOG.debug("Activating DatanodeAdminManager with interval {} seconds, " +
+            "{} max blocks per interval, " +
+            "{} max concurrently tracked nodes.", intervalSecs,
+        blocksPerInterval, maxConcurrentTrackedNodes);
+  }
+
+  /**
+   * Stop the admin monitor thread, waiting briefly for it to terminate.
+   */
+  void close() {
+    executor.shutdownNow();
+    try {
+      executor.awaitTermination(3000, TimeUnit.MILLISECONDS);
+    } catch (InterruptedException e) {}
+  }
+
+  /**
+   * Start decommissioning the specified datanode.
+   * @param node
+   */
+  @VisibleForTesting
+  public void startDecommission(DatanodeDescriptor node) {
+    if (!node.isDecommissionInProgress() && !node.isDecommissioned()) {
+      // Update DN stats maintained by HeartbeatManager
+      hbManager.startDecommission(node);
+      // hbManager.startDecommission will set dead node to decommissioned.
+      if (node.isDecommissionInProgress()) {
+        for (DatanodeStorageInfo storage : node.getStorageInfos()) {
+          LOG.info("Starting decommission of {} {} with {} blocks",
+              node, storage, storage.numBlocks());
+        }
+        node.getLeavingServiceStatus().setStartTime(monotonicNow());
+        pendingNodes.add(node);
+      }
+    } else {
+      LOG.trace("startDecommission: Node {} in {}, nothing to do." +
+          node, node.getAdminState());
+    }
+  }
+
+  /**
+   * Stop decommissioning the specified datanode.
+   * @param node
+   */
+  @VisibleForTesting
+  public void stopDecommission(DatanodeDescriptor node) {
+    if (node.isDecommissionInProgress() || node.isDecommissioned()) {
+      // Update DN stats maintained by HeartbeatManager
+      hbManager.stopDecommission(node);
+      // extra redundancy blocks will be detected and processed when
+      // the dead node comes back and send in its full block report.
+      if (node.isAlive()) {
+        blockManager.processExtraRedundancyBlocksOnInService(node);
+      }
+      // Remove from tracking in DatanodeAdminManager
+      pendingNodes.remove(node);
+      outOfServiceNodeBlocks.remove(node);
+    } else {
+      LOG.trace("stopDecommission: Node {} in {}, nothing to do." +
+          node, node.getAdminState());
+    }
+  }
+
+  /**
+   * Start maintenance of the specified datanode.
+   * @param node
+   */
+  @VisibleForTesting
+  public void startMaintenance(DatanodeDescriptor node,
+      long maintenanceExpireTimeInMS) {
+    // Even if the node is already in maintenance, we still need to adjust
+    // the expiration time.
+    node.setMaintenanceExpireTimeInMS(maintenanceExpireTimeInMS);
+    if (!node.isMaintenance()) {
+      // Update DN stats maintained by HeartbeatManager
+      hbManager.startMaintenance(node);
+      // hbManager.startMaintenance will set dead node to IN_MAINTENANCE.
+      if (node.isEnteringMaintenance()) {
+        for (DatanodeStorageInfo storage : node.getStorageInfos()) {
+          LOG.info("Starting maintenance of {} {} with {} blocks",
+              node, storage, storage.numBlocks());
+        }
+        node.getLeavingServiceStatus().setStartTime(monotonicNow());
+      }
+      // Track the node regardless whether it is ENTERING_MAINTENANCE or
+      // IN_MAINTENANCE to support maintenance expiration.
+      pendingNodes.add(node);
+    } else {
+      LOG.trace("startMaintenance: Node {} in {}, nothing to do." +
+          node, node.getAdminState());
+    }
+  }
+
+
+  /**
+   * Stop maintenance of the specified datanode.
+   * @param node
+   */
+  @VisibleForTesting
+  public void stopMaintenance(DatanodeDescriptor node) {
+    if (node.isMaintenance()) {
+      // Update DN stats maintained by HeartbeatManager
+      hbManager.stopMaintenance(node);
+
+      // extra redundancy blocks will be detected and processed when
+      // the dead node comes back and send in its full block report.
+      if (!node.isAlive()) {
+        // The node became dead when it was in maintenance, at which point
+        // the replicas weren't removed from block maps.
+        // When the node leaves maintenance, the replicas should be removed
+        // from the block maps to trigger the necessary replication to
+        // maintain the safety property of "# of live replicas + maintenance
+        // replicas" >= the expected redundancy.
+        blockManager.removeBlocksAssociatedTo(node);
+      } else {
+        // Even though putting nodes in maintenance node doesn't cause live
+        // replicas to match expected replication factor, it is still possible
+        // to have over replicated when the node leaves maintenance node.
+        // First scenario:
+        // a. Node became dead when it is at AdminStates.NORMAL, thus
+        //    block is replicated so that 3 replicas exist on other nodes.
+        // b. Admins put the dead node into maintenance mode and then
+        //    have the node rejoin the cluster.
+        // c. Take the node out of maintenance mode.
+        // Second scenario:
+        // a. With replication factor 3, set one replica to maintenance node,
+        //    thus block has 1 maintenance replica and 2 live replicas.
+        // b. Change the replication factor to 2. The block will still have
+        //    1 maintenance replica and 2 live replicas.
+        // c. Take the node out of maintenance mode.
+        blockManager.processExtraRedundancyBlocksOnInService(node);
+      }
+
+      // Remove from tracking in DatanodeAdminManager
+      pendingNodes.remove(node);
+      outOfServiceNodeBlocks.remove(node);
+    } else {
+      LOG.trace("stopMaintenance: Node {} in {}, nothing to do." +
+          node, node.getAdminState());
+    }
+  }
+
+  private void setDecommissioned(DatanodeDescriptor dn) {
+    dn.setDecommissioned();
+    LOG.info("Decommissioning complete for node {}", dn);
+  }
+
+  private void setInMaintenance(DatanodeDescriptor dn) {
+    dn.setInMaintenance();
+    LOG.info("Node {} has entered maintenance mode.", dn);
+  }
+
+  /**
+   * Checks whether a block is sufficiently replicated/stored for
+   * DECOMMISSION_INPROGRESS or ENTERING_MAINTENANCE datanodes. For replicated
+   * blocks or striped blocks, full-strength replication or storage is not
+   * always necessary, hence "sufficient".
+   * @return true if sufficient, else false.
+   */
+  private boolean isSufficient(BlockInfo block, BlockCollection bc,
+      NumberReplicas numberReplicas, boolean isDecommission) {
+    if (blockManager.hasEnoughEffectiveReplicas(block, numberReplicas, 0)) {
+      // Block has enough replica, skip
+      LOG.trace("Block {} does not need replication.", block);
+      return true;
+    }
+
+    final int numExpected = blockManager.getExpectedLiveRedundancyNum(block,
+        numberReplicas);
+    final int numLive = numberReplicas.liveReplicas();
+
+    // Block is under-replicated
+    LOG.trace("Block {} numExpected={}, numLive={}", block, numExpected,
+        numLive);
+    if (isDecommission && numExpected > numLive) {
+      if (bc.isUnderConstruction() && block.equals(bc.getLastBlock())) {
+        // Can decom a UC block as long as there will still be minReplicas
+        if (blockManager.hasMinStorage(block, numLive)) {
+          LOG.trace("UC block {} sufficiently-replicated since numLive ({}) "
+              + ">= minR ({})", block, numLive,
+              blockManager.getMinStorageNum(block));
+          return true;
+        } else {
+          LOG.trace("UC block {} insufficiently-replicated since numLive "
+              + "({}) < minR ({})", block, numLive,
+              blockManager.getMinStorageNum(block));
+        }
+      } else {
+        // Can decom a non-UC as long as the default replication is met
+        if (numLive >= blockManager.getDefaultStorageNum(block)) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  private void logBlockReplicationInfo(BlockInfo block,
+      BlockCollection bc,
+      DatanodeDescriptor srcNode, NumberReplicas num,
+      Iterable<DatanodeStorageInfo> storages) {
+    if (!NameNode.blockStateChangeLog.isInfoEnabled()) {
+      return;
+    }
+
+    int curReplicas = num.liveReplicas();
+    int curExpectedRedundancy = blockManager.getExpectedRedundancyNum(block);
+    StringBuilder nodeList = new StringBuilder();
+    for (DatanodeStorageInfo storage : storages) {
+      final DatanodeDescriptor node = storage.getDatanodeDescriptor();
+      nodeList.append(node);
+      nodeList.append(" ");
+    }
+    NameNode.blockStateChangeLog.info(
+        "Block: " + block + ", Expected Replicas: "
+        + curExpectedRedundancy + ", live replicas: " + curReplicas
+        + ", corrupt replicas: " + num.corruptReplicas()
+        + ", decommissioned replicas: " + num.decommissioned()
+        + ", decommissioning replicas: " + num.decommissioning()
+        + ", maintenance replicas: " + num.maintenanceReplicas()
+        + ", live entering maintenance replicas: "
+        + num.liveEnteringMaintenanceReplicas()
+        + ", excess replicas: " + num.excessReplicas()
+        + ", Is Open File: " + bc.isUnderConstruction()
+        + ", Datanodes having this block: " + nodeList + ", Current Datanode: "
+        + srcNode + ", Is current datanode decommissioning: "
+        + srcNode.isDecommissionInProgress() +
+        ", Is current datanode entering maintenance: "
+        + srcNode.isEnteringMaintenance());
+  }
+
+  @VisibleForTesting
+  public int getNumPendingNodes() {
+    return pendingNodes.size();
+  }
+
+  @VisibleForTesting
+  public int getNumTrackedNodes() {
+    return outOfServiceNodeBlocks.size();
+  }
+
+  @VisibleForTesting
+  public int getNumNodesChecked() {
+    return monitor.numNodesChecked;
+  }
+
+  /**
+   * Checks to see if datanodes have finished DECOMMISSION_INPROGRESS or
+   * ENTERING_MAINTENANCE state.
+   * <p/>
+   * Since this is done while holding the namesystem lock,
+   * the amount of work per monitor tick is limited.
+   */
+  private class Monitor implements Runnable {
+    /**
+     * The maximum number of blocks to check per tick.
+     */
+    private final int numBlocksPerCheck;
+    /**
+     * The maximum number of nodes to track in outOfServiceNodeBlocks.
+     * A value of 0 means no limit.
+     */
+    private final int maxConcurrentTrackedNodes;
+    /**
+     * The number of blocks that have been checked on this tick.
+     */
+    private int numBlocksChecked = 0;
+    /**
+     * The number of blocks checked after (re)holding lock.
+     */
+    private int numBlocksCheckedPerLock = 0;
+    /**
+     * The number of nodes that have been checked on this tick. Used for
+     * statistics.
+     */
+    private int numNodesChecked = 0;
+    /**
+     * The last datanode in outOfServiceNodeBlocks that we've processed.
+     */
+    private DatanodeDescriptor iterkey = new DatanodeDescriptor(
+        new DatanodeID("", "", "", 0, 0, 0, 0));
+
+    Monitor(int numBlocksPerCheck, int maxConcurrentTrackedNodes) {
+      this.numBlocksPerCheck = numBlocksPerCheck;
+      this.maxConcurrentTrackedNodes = maxConcurrentTrackedNodes;
+    }
+
+    private boolean exceededNumBlocksPerCheck() {
+      LOG.trace("Processed {} blocks so far this tick", numBlocksChecked);
+      return numBlocksChecked >= numBlocksPerCheck;
+    }
+
+    @Override
+    public void run() {
+      if (!namesystem.isRunning()) {
+        LOG.info("Namesystem is not running, skipping " +
+            "decommissioning/maintenance checks.");
+        return;
+      }
+      // Reset the checked count at beginning of each iteration
+      numBlocksChecked = 0;
+      numBlocksCheckedPerLock = 0;
+      numNodesChecked = 0;
+      // Check decommission or maintenance progress.
+      namesystem.writeLock();
+      try {
+        processPendingNodes();
+        check();
+      } finally {
+        namesystem.writeUnlock();
+      }
+      if (numBlocksChecked + numNodesChecked > 0) {
+        LOG.info("Checked {} blocks and {} nodes this tick", numBlocksChecked,
+            numNodesChecked);
+      }
+    }
+
+    /**
+     * Pop datanodes off the pending list and into decomNodeBlocks,
+     * subject to the maxConcurrentTrackedNodes limit.
+     */
+    private void processPendingNodes() {
+      while (!pendingNodes.isEmpty() &&
+          (maxConcurrentTrackedNodes == 0 ||
+          outOfServiceNodeBlocks.size() < maxConcurrentTrackedNodes)) {
+        outOfServiceNodeBlocks.put(pendingNodes.poll(), null);
+      }
+    }
+
+    private void check() {
+      final Iterator<Map.Entry<DatanodeDescriptor, AbstractList<BlockInfo>>>
+          it = new CyclicIteration<>(outOfServiceNodeBlocks,
+              iterkey).iterator();
+      final LinkedList<DatanodeDescriptor> toRemove = new LinkedList<>();
+
+      while (it.hasNext() && !exceededNumBlocksPerCheck() && namesystem
+          .isRunning()) {
+        numNodesChecked++;
+        final Map.Entry<DatanodeDescriptor, AbstractList<BlockInfo>>
+            entry = it.next();
+        final DatanodeDescriptor dn = entry.getKey();
+        AbstractList<BlockInfo> blocks = entry.getValue();
+        boolean fullScan = false;
+        if (dn.isMaintenance() && dn.maintenanceExpired()) {
+          // If maintenance expires, stop tracking it.
+          stopMaintenance(dn);
+          toRemove.add(dn);
+          continue;
+        }
+        if (dn.isInMaintenance()) {
+          // The dn is IN_MAINTENANCE and the maintenance hasn't expired yet.
+          continue;
+        }
+        if (blocks == null) {
+          // This is a newly added datanode, run through its list to schedule
+          // under-replicated blocks for replication and collect the blocks
+          // that are insufficiently replicated for further tracking
+          LOG.debug("Newly-added node {}, doing full scan to find " +
+              "insufficiently-replicated blocks.", dn);
+          blocks = handleInsufficientlyStored(dn);
+          outOfServiceNodeBlocks.put(dn, blocks);
+          fullScan = true;
+        } else {
+          // This is a known datanode, check if its # of insufficiently
+          // replicated blocks has dropped to zero and if it can move
+          // to the next state.
+          LOG.debug("Processing {} node {}", dn.getAdminState(), dn);
+          pruneReliableBlocks(dn, blocks);
+        }
+        if (blocks.size() == 0) {
+          if (!fullScan) {
+            // If we didn't just do a full scan, need to re-check with the
+            // full block map.
+            //
+            // We've replicated all the known insufficiently replicated
+            // blocks. Re-check with the full block map before finally
+            // marking the datanode as DECOMMISSIONED or IN_MAINTENANCE.
+            LOG.debug("Node {} has finished replicating current set of "
+                + "blocks, checking with the full block map.", dn);
+            blocks = handleInsufficientlyStored(dn);
+            outOfServiceNodeBlocks.put(dn, blocks);
+          }
+          // If the full scan is clean AND the node liveness is okay,
+          // we can finally mark as DECOMMISSIONED or IN_MAINTENANCE.
+          final boolean isHealthy =
+              blockManager.isNodeHealthyForDecommissionOrMaintenance(dn);
+          if (blocks.size() == 0 && isHealthy) {
+            if (dn.isDecommissionInProgress()) {
+              setDecommissioned(dn);
+              toRemove.add(dn);
+            } else if (dn.isEnteringMaintenance()) {
+              // IN_MAINTENANCE node remains in the outOfServiceNodeBlocks to
+              // to track maintenance expiration.
+              setInMaintenance(dn);
+            } else {
+              Preconditions.checkState(false,
+                  "A node is in an invalid state!");
+            }
+            LOG.debug("Node {} is sufficiently replicated and healthy, "
+                + "marked as {}.", dn.getAdminState());
+          } else {
+            LOG.debug("Node {} {} healthy."
+                + " It needs to replicate {} more blocks."
+                + " {} is still in progress.", dn,
+                isHealthy? "is": "isn't", blocks.size(), dn.getAdminState());
+          }
+        } else {
+          LOG.debug("Node {} still has {} blocks to replicate "
+              + "before it is a candidate to finish {}.",
+              dn, blocks.size(), dn.getAdminState());
+        }
+        iterkey = dn;
+      }
+      // Remove the datanodes that are DECOMMISSIONED or in service after
+      // maintenance expiration.
+      for (DatanodeDescriptor dn : toRemove) {
+        Preconditions.checkState(dn.isDecommissioned() || dn.isInService(),
+            "Removing a node that is not yet decommissioned or in service!");
+        outOfServiceNodeBlocks.remove(dn);
+      }
+    }
+
+    /**
+     * Removes reliable blocks from the block list of a datanode.
+     */
+    private void pruneReliableBlocks(final DatanodeDescriptor datanode,
+        AbstractList<BlockInfo> blocks) {
+      processBlocksInternal(datanode, blocks.iterator(), null, true);
+    }
+
+    /**
+     * Returns a list of blocks on a datanode that are insufficiently
+     * replicated or require recovery, i.e. requiring recovery and
+     * should prevent decommission or maintenance.
+     * <p/>
+     * As part of this, it also schedules replication/recovery work.
+     *
+     * @return List of blocks requiring recovery
+     */
+    private AbstractList<BlockInfo> handleInsufficientlyStored(
+        final DatanodeDescriptor datanode) {
+      AbstractList<BlockInfo> insufficient = new ChunkedArrayList<>();
+      processBlocksInternal(datanode, datanode.getBlockIterator(),
+          insufficient, false);
+      return insufficient;
+    }
+
+    /**
+     * Used while checking if DECOMMISSION_INPROGRESS datanodes can be
+     * marked as DECOMMISSIONED or ENTERING_MAINTENANCE datanodes can be
+     * marked as IN_MAINTENANCE. Combines shared logic of pruneReliableBlocks
+     * and handleInsufficientlyStored.
+     *
+     * @param datanode                    Datanode
+     * @param it                          Iterator over the blocks on the
+     *                                    datanode
+     * @param insufficientList            Return parameter. If it's not null,
+     *                                    will contain the insufficiently
+     *                                    replicated-blocks from the list.
+     * @param pruneReliableBlocks         whether to remove blocks reliable
+     *                                    enough from the iterator
+     */
+    private void processBlocksInternal(
+        final DatanodeDescriptor datanode,
+        final Iterator<BlockInfo> it,
+        final List<BlockInfo> insufficientList,
+        boolean pruneReliableBlocks) {
+      boolean firstReplicationLog = true;
+      // Low redundancy in UC Blocks only
+      int lowRedundancyInOpenFiles = 0;
+      // All low redundancy blocks. Includes lowRedundancyInOpenFiles.
+      int lowRedundancyBlocks = 0;
+      // All maintenance and decommission replicas.
+      int outOfServiceOnlyReplicas = 0;
+      while (it.hasNext()) {
+        if (insufficientList == null
+            && numBlocksCheckedPerLock >= numBlocksPerCheck) {
+          // During fullscan insufficientlyReplicated will NOT be null, iterator
+          // will be DN's iterator. So should not yield lock, otherwise
+          // ConcurrentModificationException could occur.
+          // Once the fullscan done, iterator will be a copy. So can yield the
+          // lock.
+          // Yielding is required in case of block number is greater than the
+          // configured per-iteration-limit.
+          namesystem.writeUnlock();
+          try {
+            LOG.debug("Yielded lock during decommission/maintenance check");
+            Thread.sleep(0, 500);
+          } catch (InterruptedException ignored) {
+            return;
+          }
+          // reset
+          numBlocksCheckedPerLock = 0;
+          namesystem.writeLock();
+        }
+        numBlocksChecked++;
+        numBlocksCheckedPerLock++;
+        final BlockInfo block = it.next();
+        // Remove the block from the list if it's no longer in the block map,
+        // e.g. the containing file has been deleted
+        if (blockManager.blocksMap.getStoredBlock(block) == null) {
+          LOG.trace("Removing unknown block {}", block);
+          it.remove();
+          continue;
+        }
+
+        long bcId = block.getBlockCollectionId();
+        if (bcId == INodeId.INVALID_INODE_ID) {
+          // Orphan block, will be invalidated eventually. Skip.
+          continue;
+        }
+
+        final BlockCollection bc = blockManager.getBlockCollection(block);
+        final NumberReplicas num = blockManager.countNodes(block);
+        final int liveReplicas = num.liveReplicas();
+
+        // Schedule low redundancy blocks for reconstruction
+        // if not already pending.
+        boolean isDecommission = datanode.isDecommissionInProgress();
+        boolean neededReconstruction = isDecommission ?
+            blockManager.isNeededReconstruction(block, num) :
+            blockManager.isNeededReconstructionForMaintenance(block, num);
+        if (neededReconstruction) {
+          if (!blockManager.neededReconstruction.contains(block) &&
+              blockManager.pendingReconstruction.getNumReplicas(block) == 0 &&
+              blockManager.isPopulatingReplQueues()) {
+            // Process these blocks only when active NN is out of safe mode.
+            blockManager.neededReconstruction.add(block,
+                liveReplicas, num.readOnlyReplicas(),
+                num.outOfServiceReplicas(),
+                blockManager.getExpectedRedundancyNum(block));
+          }
+        }
+
+        // Even if the block is without sufficient redundancy,
+        // it might not block decommission/maintenance if it
+        // has sufficient redundancy.
+        if (isSufficient(block, bc, num, isDecommission)) {
+          if (pruneReliableBlocks) {
+            it.remove();
+          }
+          continue;
+        }
+
+        // We've found a block without sufficient redundancy.
+        if (insufficientList != null) {
+          insufficientList.add(block);
+        }
+        // Log if this is our first time through
+        if (firstReplicationLog) {
+          logBlockReplicationInfo(block, bc, datanode, num,
+              blockManager.blocksMap.getStorages(block));
+          firstReplicationLog = false;
+        }
+        // Update various counts
+        lowRedundancyBlocks++;
+        if (bc.isUnderConstruction()) {
+          lowRedundancyInOpenFiles++;
+        }
+        if ((liveReplicas == 0) && (num.outOfServiceReplicas() > 0)) {
+          outOfServiceOnlyReplicas++;
+        }
+      }
+
+      datanode.getLeavingServiceStatus().set(lowRedundancyInOpenFiles,
+          lowRedundancyBlocks, outOfServiceOnlyReplicas);
+    }
+  }
+
+  @VisibleForTesting
+  void runMonitorForTest() throws ExecutionException, InterruptedException {
+    executor.submit(monitor).get();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79df1e75/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index 2c5779a..d705fec 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -75,7 +75,7 @@ public class DatanodeManager {
 
   private final Namesystem namesystem;
   private final BlockManager blockManager;
-  private final DecommissionManager decomManager;
+  private final DatanodeAdminManager datanodeAdminManager;
   private final HeartbeatManager heartbeatManager;
   private final FSClusterStats fsClusterStats;
 
@@ -223,9 +223,10 @@ public class DatanodeManager {
       networktopology = NetworkTopology.getInstance(conf);
     }
 
-    this.heartbeatManager = new HeartbeatManager(namesystem, blockManager, conf);
-    this.decomManager = new DecommissionManager(namesystem, blockManager,
-        heartbeatManager);
+    this.heartbeatManager = new HeartbeatManager(namesystem,
+        blockManager, conf);
+    this.datanodeAdminManager = new DatanodeAdminManager(namesystem,
+        blockManager, heartbeatManager);
     this.fsClusterStats = newFSClusterStats();
     this.dataNodePeerStatsEnabled = conf.getBoolean(
         DFSConfigKeys.DFS_DATANODE_PEER_STATS_ENABLED_KEY,
@@ -372,12 +373,12 @@ public class DatanodeManager {
   }
   
   void activate(final Configuration conf) {
-    decomManager.activate(conf);
+    datanodeAdminManager.activate(conf);
     heartbeatManager.activate();
   }
 
   void close() {
-    decomManager.close();
+    datanodeAdminManager.close();
     heartbeatManager.close();
   }
 
@@ -392,8 +393,8 @@ public class DatanodeManager {
   }
 
   @VisibleForTesting
-  public DecommissionManager getDecomManager() {
-    return decomManager;
+  public DatanodeAdminManager getDatanodeAdminManager() {
+    return datanodeAdminManager;
   }
 
   public HostConfigManager getHostConfigManager() {
@@ -991,9 +992,9 @@ public class DatanodeManager {
         hostConfigManager.getMaintenanceExpirationTimeInMS(nodeReg);
     // If the registered node is in exclude list, then decommission it
     if (getHostConfigManager().isExcluded(nodeReg)) {
-      decomManager.startDecommission(nodeReg);
+      datanodeAdminManager.startDecommission(nodeReg);
     } else if (nodeReg.maintenanceNotExpired(maintenanceExpireTimeInMS)) {
-      decomManager.startMaintenance(nodeReg, maintenanceExpireTimeInMS);
+      datanodeAdminManager.startMaintenance(nodeReg, maintenanceExpireTimeInMS);
     }
   }
 
@@ -1219,12 +1220,13 @@ public class DatanodeManager {
         long maintenanceExpireTimeInMS =
             hostConfigManager.getMaintenanceExpirationTimeInMS(node);
         if (node.maintenanceNotExpired(maintenanceExpireTimeInMS)) {
-          decomManager.startMaintenance(node, maintenanceExpireTimeInMS);
+          datanodeAdminManager.startMaintenance(
+              node, maintenanceExpireTimeInMS);
         } else if (hostConfigManager.isExcluded(node)) {
-          decomManager.startDecommission(node);
+          datanodeAdminManager.startDecommission(node);
         } else {
-          decomManager.stopMaintenance(node);
-          decomManager.stopDecommission(node);
+          datanodeAdminManager.stopMaintenance(node);
+          datanodeAdminManager.stopDecommission(node);
         }
       }
       node.setUpgradeDomain(hostConfigManager.getUpgradeDomain(node));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79df1e75/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
deleted file mode 100644
index ae79826..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
+++ /dev/null
@@ -1,741 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.blockmanagement;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static org.apache.hadoop.util.Time.monotonicNow;
-
-import java.util.AbstractList;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Queue;
-import java.util.TreeMap;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.server.namenode.INodeId;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.server.namenode.Namesystem;
-import org.apache.hadoop.hdfs.util.CyclicIteration;
-import org.apache.hadoop.util.ChunkedArrayList;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-/**
- * Manages datanode decommissioning. A background monitor thread 
- * periodically checks the status of datanodes that are in-progress of 
- * decommissioning.
- * <p/>
- * A datanode can be decommissioned in a few situations:
- * <ul>
- * <li>If a DN is dead, it is decommissioned immediately.</li>
- * <li>If a DN is alive, it is decommissioned after all of its blocks 
- * are sufficiently replicated. Merely under-replicated blocks do not 
- * block decommissioning as long as they are above a replication 
- * threshold.</li>
- * </ul>
- * In the second case, the datanode transitions to a 
- * decommission-in-progress state and is tracked by the monitor thread. The 
- * monitor periodically scans through the list of insufficiently replicated
- * blocks on these datanodes to 
- * determine if they can be decommissioned. The monitor also prunes this list 
- * as blocks become replicated, so monitor scans will become more efficient 
- * over time.
- * <p/>
- * Decommission-in-progress nodes that become dead do not progress to 
- * decommissioned until they become live again. This prevents potential 
- * durability loss for singly-replicated blocks (see HDFS-6791).
- * <p/>
- * This class depends on the FSNamesystem lock for synchronization.
- */
-@InterfaceAudience.Private
-public class DecommissionManager {
-  private static final Logger LOG = LoggerFactory.getLogger(DecommissionManager
-      .class);
-
-  private final Namesystem namesystem;
-  private final BlockManager blockManager;
-  private final HeartbeatManager hbManager;
-  private final ScheduledExecutorService executor;
-
-  /**
-   * Map containing the DECOMMISSION_INPROGRESS or ENTERING_MAINTENANCE
-   * datanodes that are being tracked so they can be be marked as
-   * DECOMMISSIONED or IN_MAINTENANCE. Even after the node is marked as
-   * IN_MAINTENANCE, the node remains in the map until
-   * maintenance expires checked during a monitor tick.
-   * <p/>
-   * This holds a set of references to the under-replicated blocks on the DN at
-   * the time the DN is added to the map, i.e. the blocks that are preventing
-   * the node from being marked as decommissioned. During a monitor tick, this
-   * list is pruned as blocks becomes replicated.
-   * <p/>
-   * Note also that the reference to the list of under-replicated blocks 
-   * will be null on initial add
-   * <p/>
-   * However, this map can become out-of-date since it is not updated by block
-   * reports or other events. Before being finally marking as decommissioned,
-   * another check is done with the actual block map.
-   */
-  private final TreeMap<DatanodeDescriptor, AbstractList<BlockInfo>>
-      outOfServiceNodeBlocks;
-
-  /**
-   * Tracking a node in outOfServiceNodeBlocks consumes additional memory. To
-   * limit the impact on NN memory consumption, we limit the number of nodes in
-   * outOfServiceNodeBlocks. Additional nodes wait in pendingNodes.
-   */
-  private final Queue<DatanodeDescriptor> pendingNodes;
-
-  private Monitor monitor = null;
-
-  DecommissionManager(final Namesystem namesystem,
-      final BlockManager blockManager, final HeartbeatManager hbManager) {
-    this.namesystem = namesystem;
-    this.blockManager = blockManager;
-    this.hbManager = hbManager;
-
-    executor = Executors.newScheduledThreadPool(1,
-        new ThreadFactoryBuilder().setNameFormat("DecommissionMonitor-%d")
-            .setDaemon(true).build());
-    outOfServiceNodeBlocks = new TreeMap<>();
-    pendingNodes = new LinkedList<>();
-  }
-
-  /**
-   * Start the decommission monitor thread.
-   * @param conf
-   */
-  void activate(Configuration conf) {
-    final int intervalSecs = (int) conf.getTimeDuration(
-        DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY,
-        DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_DEFAULT,
-        TimeUnit.SECONDS);
-    checkArgument(intervalSecs >= 0, "Cannot set a negative " +
-        "value for " + DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY);
-
-    int blocksPerInterval = conf.getInt(
-        DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_BLOCKS_PER_INTERVAL_KEY,
-        DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_BLOCKS_PER_INTERVAL_DEFAULT);
-
-    final String deprecatedKey =
-        "dfs.namenode.decommission.nodes.per.interval";
-    final String strNodes = conf.get(deprecatedKey);
-    if (strNodes != null) {
-      LOG.warn("Deprecated configuration key {} will be ignored.",
-          deprecatedKey);
-      LOG.warn("Please update your configuration to use {} instead.", 
-          DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_BLOCKS_PER_INTERVAL_KEY);
-    }
-
-    checkArgument(blocksPerInterval > 0,
-        "Must set a positive value for "
-        + DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_BLOCKS_PER_INTERVAL_KEY);
-
-    final int maxConcurrentTrackedNodes = conf.getInt(
-        DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_MAX_CONCURRENT_TRACKED_NODES,
-        DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_MAX_CONCURRENT_TRACKED_NODES_DEFAULT);
-    checkArgument(maxConcurrentTrackedNodes >= 0, "Cannot set a negative " +
-        "value for "
-        + DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_MAX_CONCURRENT_TRACKED_NODES);
-
-    monitor = new Monitor(blocksPerInterval, maxConcurrentTrackedNodes);
-    executor.scheduleAtFixedRate(monitor, intervalSecs, intervalSecs,
-        TimeUnit.SECONDS);
-
-    LOG.debug("Activating DecommissionManager with interval {} seconds, " +
-            "{} max blocks per interval, " +
-            "{} max concurrently tracked nodes.", intervalSecs,
-        blocksPerInterval, maxConcurrentTrackedNodes);
-  }
-
-  /**
-   * Stop the decommission monitor thread, waiting briefly for it to terminate.
-   */
-  void close() {
-    executor.shutdownNow();
-    try {
-      executor.awaitTermination(3000, TimeUnit.MILLISECONDS);
-    } catch (InterruptedException e) {}
-  }
-
-  /**
-   * Start decommissioning the specified datanode. 
-   * @param node
-   */
-  @VisibleForTesting
-  public void startDecommission(DatanodeDescriptor node) {
-    if (!node.isDecommissionInProgress() && !node.isDecommissioned()) {
-      // Update DN stats maintained by HeartbeatManager
-      hbManager.startDecommission(node);
-      // hbManager.startDecommission will set dead node to decommissioned.
-      if (node.isDecommissionInProgress()) {
-        for (DatanodeStorageInfo storage : node.getStorageInfos()) {
-          LOG.info("Starting decommission of {} {} with {} blocks",
-              node, storage, storage.numBlocks());
-        }
-        node.getLeavingServiceStatus().setStartTime(monotonicNow());
-        pendingNodes.add(node);
-      }
-    } else {
-      LOG.trace("startDecommission: Node {} in {}, nothing to do." +
-          node, node.getAdminState());
-    }
-  }
-
-  /**
-   * Stop decommissioning the specified datanode. 
-   * @param node
-   */
-  @VisibleForTesting
-  public void stopDecommission(DatanodeDescriptor node) {
-    if (node.isDecommissionInProgress() || node.isDecommissioned()) {
-      // Update DN stats maintained by HeartbeatManager
-      hbManager.stopDecommission(node);
-      // extra redundancy blocks will be detected and processed when
-      // the dead node comes back and send in its full block report.
-      if (node.isAlive()) {
-        blockManager.processExtraRedundancyBlocksOnInService(node);
-      }
-      // Remove from tracking in DecommissionManager
-      pendingNodes.remove(node);
-      outOfServiceNodeBlocks.remove(node);
-    } else {
-      LOG.trace("stopDecommission: Node {} in {}, nothing to do." +
-          node, node.getAdminState());
-    }
-  }
-
-  /**
-   * Start maintenance of the specified datanode.
-   * @param node
-   */
-  @VisibleForTesting
-  public void startMaintenance(DatanodeDescriptor node,
-      long maintenanceExpireTimeInMS) {
-    // Even if the node is already in maintenance, we still need to adjust
-    // the expiration time.
-    node.setMaintenanceExpireTimeInMS(maintenanceExpireTimeInMS);
-    if (!node.isMaintenance()) {
-      // Update DN stats maintained by HeartbeatManager
-      hbManager.startMaintenance(node);
-      // hbManager.startMaintenance will set dead node to IN_MAINTENANCE.
-      if (node.isEnteringMaintenance()) {
-        for (DatanodeStorageInfo storage : node.getStorageInfos()) {
-          LOG.info("Starting maintenance of {} {} with {} blocks",
-              node, storage, storage.numBlocks());
-        }
-        node.getLeavingServiceStatus().setStartTime(monotonicNow());
-      }
-      // Track the node regardless whether it is ENTERING_MAINTENANCE or
-      // IN_MAINTENANCE to support maintenance expiration.
-      pendingNodes.add(node);
-    } else {
-      LOG.trace("startMaintenance: Node {} in {}, nothing to do." +
-          node, node.getAdminState());
-    }
-  }
-
-
-  /**
-   * Stop maintenance of the specified datanode.
-   * @param node
-   */
-  @VisibleForTesting
-  public void stopMaintenance(DatanodeDescriptor node) {
-    if (node.isMaintenance()) {
-      // Update DN stats maintained by HeartbeatManager
-      hbManager.stopMaintenance(node);
-
-      // extra redundancy blocks will be detected and processed when
-      // the dead node comes back and send in its full block report.
-      if (!node.isAlive()) {
-        // The node became dead when it was in maintenance, at which point
-        // the replicas weren't removed from block maps.
-        // When the node leaves maintenance, the replicas should be removed
-        // from the block maps to trigger the necessary replication to
-        // maintain the safety property of "# of live replicas + maintenance
-        // replicas" >= the expected redundancy.
-        blockManager.removeBlocksAssociatedTo(node);
-      } else {
-        // Even though putting nodes in maintenance node doesn't cause live
-        // replicas to match expected replication factor, it is still possible
-        // to have over replicated when the node leaves maintenance node.
-        // First scenario:
-        // a. Node became dead when it is at AdminStates.NORMAL, thus
-        //    block is replicated so that 3 replicas exist on other nodes.
-        // b. Admins put the dead node into maintenance mode and then
-        //    have the node rejoin the cluster.
-        // c. Take the node out of maintenance mode.
-        // Second scenario:
-        // a. With replication factor 3, set one replica to maintenance node,
-        //    thus block has 1 maintenance replica and 2 live replicas.
-        // b. Change the replication factor to 2. The block will still have
-        //    1 maintenance replica and 2 live replicas.
-        // c. Take the node out of maintenance mode.
-        blockManager.processExtraRedundancyBlocksOnInService(node);
-      }
-
-      // Remove from tracking in DecommissionManager
-      pendingNodes.remove(node);
-      outOfServiceNodeBlocks.remove(node);
-    } else {
-      LOG.trace("stopMaintenance: Node {} in {}, nothing to do." +
-          node, node.getAdminState());
-    }
-  }
-
-  private void setDecommissioned(DatanodeDescriptor dn) {
-    dn.setDecommissioned();
-    LOG.info("Decommissioning complete for node {}", dn);
-  }
-
-  private void setInMaintenance(DatanodeDescriptor dn) {
-    dn.setInMaintenance();
-    LOG.info("Node {} has entered maintenance mode.", dn);
-  }
-
-  /**
-   * Checks whether a block is sufficiently replicated/stored for
-   * decommissioning. For replicated blocks or striped blocks, full-strength
-   * replication or storage is not always necessary, hence "sufficient".
-   * @return true if sufficient, else false.
-   */
-  private boolean isSufficient(BlockInfo block, BlockCollection bc,
-      NumberReplicas numberReplicas, boolean isDecommission) {
-    if (blockManager.hasEnoughEffectiveReplicas(block, numberReplicas, 0)) {
-      // Block has enough replica, skip
-      LOG.trace("Block {} does not need replication.", block);
-      return true;
-    }
-
-    final int numExpected = blockManager.getExpectedLiveRedundancyNum(block,
-        numberReplicas);
-    final int numLive = numberReplicas.liveReplicas();
-
-    // Block is under-replicated
-    LOG.trace("Block {} numExpected={}, numLive={}", block, numExpected,
-        numLive);
-    if (isDecommission && numExpected > numLive) {
-      if (bc.isUnderConstruction() && block.equals(bc.getLastBlock())) {
-        // Can decom a UC block as long as there will still be minReplicas
-        if (blockManager.hasMinStorage(block, numLive)) {
-          LOG.trace("UC block {} sufficiently-replicated since numLive ({}) "
-              + ">= minR ({})", block, numLive,
-              blockManager.getMinStorageNum(block));
-          return true;
-        } else {
-          LOG.trace("UC block {} insufficiently-replicated since numLive "
-              + "({}) < minR ({})", block, numLive,
-              blockManager.getMinStorageNum(block));
-        }
-      } else {
-        // Can decom a non-UC as long as the default replication is met
-        if (numLive >= blockManager.getDefaultStorageNum(block)) {
-          return true;
-        }
-      }
-    }
-    return false;
-  }
-
-  private void logBlockReplicationInfo(BlockInfo block,
-      BlockCollection bc,
-      DatanodeDescriptor srcNode, NumberReplicas num,
-      Iterable<DatanodeStorageInfo> storages) {
-    if (!NameNode.blockStateChangeLog.isInfoEnabled()) {
-      return;
-    }
-
-    int curReplicas = num.liveReplicas();
-    int curExpectedRedundancy = blockManager.getExpectedRedundancyNum(block);
-    StringBuilder nodeList = new StringBuilder();
-    for (DatanodeStorageInfo storage : storages) {
-      final DatanodeDescriptor node = storage.getDatanodeDescriptor();
-      nodeList.append(node);
-      nodeList.append(" ");
-    }
-    NameNode.blockStateChangeLog.info(
-        "Block: " + block + ", Expected Replicas: "
-        + curExpectedRedundancy + ", live replicas: " + curReplicas
-        + ", corrupt replicas: " + num.corruptReplicas()
-        + ", decommissioned replicas: " + num.decommissioned()
-        + ", decommissioning replicas: " + num.decommissioning()
-        + ", maintenance replicas: " + num.maintenanceReplicas()
-        + ", live entering maintenance replicas: "
-        + num.liveEnteringMaintenanceReplicas()
-        + ", excess replicas: " + num.excessReplicas()
-        + ", Is Open File: " + bc.isUnderConstruction()
-        + ", Datanodes having this block: " + nodeList + ", Current Datanode: "
-        + srcNode + ", Is current datanode decommissioning: "
-        + srcNode.isDecommissionInProgress() +
-        ", Is current datanode entering maintenance: "
-        + srcNode.isEnteringMaintenance());
-  }
-
-  @VisibleForTesting
-  public int getNumPendingNodes() {
-    return pendingNodes.size();
-  }
-
-  @VisibleForTesting
-  public int getNumTrackedNodes() {
-    return outOfServiceNodeBlocks.size();
-  }
-
-  @VisibleForTesting
-  public int getNumNodesChecked() {
-    return monitor.numNodesChecked;
-  }
-
-  /**
-   * Checks to see if DNs have finished decommissioning.
-   * <p/>
-   * Since this is done while holding the namesystem lock, 
-   * the amount of work per monitor tick is limited.
-   */
-  private class Monitor implements Runnable {
-    /**
-     * The maximum number of blocks to check per tick.
-     */
-    private final int numBlocksPerCheck;
-    /**
-     * The maximum number of nodes to track in outOfServiceNodeBlocks.
-     * A value of 0 means no limit.
-     */
-    private final int maxConcurrentTrackedNodes;
-    /**
-     * The number of blocks that have been checked on this tick.
-     */
-    private int numBlocksChecked = 0;
-    /**
-     * The number of blocks checked after (re)holding lock.
-     */
-    private int numBlocksCheckedPerLock = 0;
-    /**
-     * The number of nodes that have been checked on this tick. Used for 
-     * statistics.
-     */
-    private int numNodesChecked = 0;
-    /**
-     * The last datanode in outOfServiceNodeBlocks that we've processed
-     */
-    private DatanodeDescriptor iterkey = new DatanodeDescriptor(new 
-        DatanodeID("", "", "", 0, 0, 0, 0));
-
-    Monitor(int numBlocksPerCheck, int maxConcurrentTrackedNodes) {
-      this.numBlocksPerCheck = numBlocksPerCheck;
-      this.maxConcurrentTrackedNodes = maxConcurrentTrackedNodes;
-    }
-
-    private boolean exceededNumBlocksPerCheck() {
-      LOG.trace("Processed {} blocks so far this tick", numBlocksChecked);
-      return numBlocksChecked >= numBlocksPerCheck;
-    }
-
-    @Override
-    public void run() {
-      if (!namesystem.isRunning()) {
-        LOG.info("Namesystem is not running, skipping decommissioning checks"
-            + ".");
-        return;
-      }
-      // Reset the checked count at beginning of each iteration
-      numBlocksChecked = 0;
-      numBlocksCheckedPerLock = 0;
-      numNodesChecked = 0;
-      // Check decommission or maintenance progress.
-      namesystem.writeLock();
-      try {
-        processPendingNodes();
-        check();
-      } finally {
-        namesystem.writeUnlock();
-      }
-      if (numBlocksChecked + numNodesChecked > 0) {
-        LOG.info("Checked {} blocks and {} nodes this tick", numBlocksChecked,
-            numNodesChecked);
-      }
-    }
-
-    /**
-     * Pop datanodes off the pending list and into decomNodeBlocks, 
-     * subject to the maxConcurrentTrackedNodes limit.
-     */
-    private void processPendingNodes() {
-      while (!pendingNodes.isEmpty() &&
-          (maxConcurrentTrackedNodes == 0 ||
-          outOfServiceNodeBlocks.size() < maxConcurrentTrackedNodes)) {
-        outOfServiceNodeBlocks.put(pendingNodes.poll(), null);
-      }
-    }
-
-    private void check() {
-      final Iterator<Map.Entry<DatanodeDescriptor, AbstractList<BlockInfo>>>
-          it = new CyclicIteration<>(outOfServiceNodeBlocks,
-              iterkey).iterator();
-      final LinkedList<DatanodeDescriptor> toRemove = new LinkedList<>();
-
-      while (it.hasNext() && !exceededNumBlocksPerCheck() && namesystem
-          .isRunning()) {
-        numNodesChecked++;
-        final Map.Entry<DatanodeDescriptor, AbstractList<BlockInfo>>
-            entry = it.next();
-        final DatanodeDescriptor dn = entry.getKey();
-        AbstractList<BlockInfo> blocks = entry.getValue();
-        boolean fullScan = false;
-        if (dn.isMaintenance() && dn.maintenanceExpired()) {
-          // If maintenance expires, stop tracking it.
-          stopMaintenance(dn);
-          toRemove.add(dn);
-          continue;
-        }
-        if (dn.isInMaintenance()) {
-          // The dn is IN_MAINTENANCE and the maintenance hasn't expired yet.
-          continue;
-        }
-        if (blocks == null) {
-          // This is a newly added datanode, run through its list to schedule 
-          // under-replicated blocks for replication and collect the blocks 
-          // that are insufficiently replicated for further tracking
-          LOG.debug("Newly-added node {}, doing full scan to find " +
-              "insufficiently-replicated blocks.", dn);
-          blocks = handleInsufficientlyStored(dn);
-          outOfServiceNodeBlocks.put(dn, blocks);
-          fullScan = true;
-        } else {
-          // This is a known datanode, check if its # of insufficiently 
-          // replicated blocks has dropped to zero and if it can be decommed
-          LOG.debug("Processing {} node {}", dn.getAdminState(), dn);
-          pruneReliableBlocks(dn, blocks);
-        }
-        if (blocks.size() == 0) {
-          if (!fullScan) {
-            // If we didn't just do a full scan, need to re-check with the 
-            // full block map.
-            //
-            // We've replicated all the known insufficiently replicated 
-            // blocks. Re-check with the full block map before finally 
-            // marking the datanode as decommissioned 
-            LOG.debug("Node {} has finished replicating current set of "
-                + "blocks, checking with the full block map.", dn);
-            blocks = handleInsufficientlyStored(dn);
-            outOfServiceNodeBlocks.put(dn, blocks);
-          }
-          // If the full scan is clean AND the node liveness is okay, 
-          // we can finally mark as decommissioned.
-          final boolean isHealthy =
-              blockManager.isNodeHealthyForDecommissionOrMaintenance(dn);
-          if (blocks.size() == 0 && isHealthy) {
-            if (dn.isDecommissionInProgress()) {
-              setDecommissioned(dn);
-              toRemove.add(dn);
-            } else if (dn.isEnteringMaintenance()) {
-              // IN_MAINTENANCE node remains in the outOfServiceNodeBlocks to
-              // to track maintenance expiration.
-              setInMaintenance(dn);
-            } else {
-              Preconditions.checkState(false,
-                  "A node is in an invalid state!");
-            }
-            LOG.debug("Node {} is sufficiently replicated and healthy, "
-                + "marked as {}.", dn.getAdminState());
-          } else {
-            LOG.debug("Node {} {} healthy."
-                + " It needs to replicate {} more blocks."
-                + " {} is still in progress.", dn,
-                isHealthy? "is": "isn't", blocks.size(), dn.getAdminState());
-          }
-        } else {
-          LOG.debug("Node {} still has {} blocks to replicate "
-              + "before it is a candidate to finish {}.",
-              dn, blocks.size(), dn.getAdminState());
-        }
-        iterkey = dn;
-      }
-      // Remove the datanodes that are decommissioned or in service after
-      // maintenance expiration.
-      for (DatanodeDescriptor dn : toRemove) {
-        Preconditions.checkState(dn.isDecommissioned() || dn.isInService(),
-            "Removing a node that is not yet decommissioned or in service!");
-        outOfServiceNodeBlocks.remove(dn);
-      }
-    }
-
-    /**
-     * Removes reliable blocks from the block list of a datanode.
-     */
-    private void pruneReliableBlocks(final DatanodeDescriptor datanode,
-        AbstractList<BlockInfo> blocks) {
-      processBlocksInternal(datanode, blocks.iterator(), null, true);
-    }
-
-    /**
-     * Returns a list of blocks on a datanode that are insufficiently replicated
-     * or require recovery, i.e. requiring recovery and should prevent
-     * decommission.
-     * <p/>
-     * As part of this, it also schedules replication/recovery work.
-     *
-     * @return List of blocks requiring recovery
-     */
-    private AbstractList<BlockInfo> handleInsufficientlyStored(
-        final DatanodeDescriptor datanode) {
-      AbstractList<BlockInfo> insufficient = new ChunkedArrayList<>();
-      processBlocksInternal(datanode, datanode.getBlockIterator(),
-          insufficient, false);
-      return insufficient;
-    }
-
-    /**
-     * Used while checking if decommission-in-progress datanodes can be marked
-     * as decommissioned. Combines shared logic of 
-     * pruneReliableBlocks and handleInsufficientlyStored.
-     *
-     * @param datanode                    Datanode
-     * @param it                          Iterator over the blocks on the
-     *                                    datanode
-     * @param insufficientList            Return parameter. If it's not null,
-     *                                    will contain the insufficiently
-     *                                    replicated-blocks from the list.
-     * @param pruneReliableBlocks         whether to remove blocks reliable
-     *                                    enough from the iterator
-     */
-    private void processBlocksInternal(
-        final DatanodeDescriptor datanode,
-        final Iterator<BlockInfo> it,
-        final List<BlockInfo> insufficientList,
-        boolean pruneReliableBlocks) {
-      boolean firstReplicationLog = true;
-      // Low redundancy in UC Blocks only
-      int lowRedundancyInOpenFiles = 0;
-      // All low redundancy blocks. Includes lowRedundancyInOpenFiles.
-      int lowRedundancyBlocks = 0;
-      // All maintenance and decommission replicas.
-      int outOfServiceOnlyReplicas = 0;
-      while (it.hasNext()) {
-        if (insufficientList == null
-            && numBlocksCheckedPerLock >= numBlocksPerCheck) {
-          // During fullscan insufficientlyReplicated will NOT be null, iterator
-          // will be DN's iterator. So should not yield lock, otherwise
-          // ConcurrentModificationException could occur.
-          // Once the fullscan done, iterator will be a copy. So can yield the
-          // lock.
-          // Yielding is required in case of block number is greater than the
-          // configured per-iteration-limit.
-          namesystem.writeUnlock();
-          try {
-            LOG.debug("Yielded lock during decommission check");
-            Thread.sleep(0, 500);
-          } catch (InterruptedException ignored) {
-            return;
-          }
-          // reset
-          numBlocksCheckedPerLock = 0;
-          namesystem.writeLock();
-        }
-        numBlocksChecked++;
-        numBlocksCheckedPerLock++;
-        final BlockInfo block = it.next();
-        // Remove the block from the list if it's no longer in the block map,
-        // e.g. the containing file has been deleted
-        if (blockManager.blocksMap.getStoredBlock(block) == null) {
-          LOG.trace("Removing unknown block {}", block);
-          it.remove();
-          continue;
-        }
-
-        long bcId = block.getBlockCollectionId();
-        if (bcId == INodeId.INVALID_INODE_ID) {
-          // Orphan block, will be invalidated eventually. Skip.
-          continue;
-        }
-
-        final BlockCollection bc = blockManager.getBlockCollection(block);
-        final NumberReplicas num = blockManager.countNodes(block);
-        final int liveReplicas = num.liveReplicas();
-
-        // Schedule low redundancy blocks for reconstruction if not already
-        // pending
-        boolean isDecommission = datanode.isDecommissionInProgress();
-        boolean neededReconstruction = isDecommission ?
-            blockManager.isNeededReconstruction(block, num) :
-            blockManager.isNeededReconstructionForMaintenance(block, num);
-        if (neededReconstruction) {
-          if (!blockManager.neededReconstruction.contains(block) &&
-              blockManager.pendingReconstruction.getNumReplicas(block) == 0 &&
-              blockManager.isPopulatingReplQueues()) {
-            // Process these blocks only when active NN is out of safe mode.
-            blockManager.neededReconstruction.add(block,
-                liveReplicas, num.readOnlyReplicas(),
-                num.outOfServiceReplicas(),
-                blockManager.getExpectedRedundancyNum(block));
-          }
-        }
-
-        // Even if the block is without sufficient redundancy,
-        // it doesn't block decommission if has sufficient redundancy
-        if (isSufficient(block, bc, num, isDecommission)) {
-          if (pruneReliableBlocks) {
-            it.remove();
-          }
-          continue;
-        }
-
-        // We've found a block without sufficient redundancy.
-        if (insufficientList != null) {
-          insufficientList.add(block);
-        }
-        // Log if this is our first time through
-        if (firstReplicationLog) {
-          logBlockReplicationInfo(block, bc, datanode, num,
-              blockManager.blocksMap.getStorages(block));
-          firstReplicationLog = false;
-        }
-        // Update various counts
-        lowRedundancyBlocks++;
-        if (bc.isUnderConstruction()) {
-          lowRedundancyInOpenFiles++;
-        }
-        if ((liveReplicas == 0) && (num.outOfServiceReplicas() > 0)) {
-          outOfServiceOnlyReplicas++;
-        }
-      }
-
-      datanode.getLeavingServiceStatus().set(lowRedundancyInOpenFiles,
-          lowRedundancyBlocks, outOfServiceOnlyReplicas);
-    }
-  }
-
-  @VisibleForTesting
-  void runMonitorForTest() throws ExecutionException, InterruptedException {
-    executor.submit(monitor).get();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79df1e75/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
index b859148..318d8e0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
@@ -471,7 +471,7 @@ public class BackupNode extends NameNode {
      * {@link LeaseManager.Monitor} protected by SafeMode.
      * {@link BlockManager.RedundancyMonitor} protected by SafeMode.
      * {@link HeartbeatManager.Monitor} protected by SafeMode.
-     * {@link DecommissionManager.Monitor} need to prohibit refreshNodes().
+     * {@link DatanodeAdminManager.Monitor} need to prohibit refreshNodes().
      * {@link PendingReconstructionBlocks.PendingReconstructionMonitor}
      * harmless, because RedundancyMonitor is muted.
      */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79df1e75/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 4caee9e..8bf2b8c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -960,17 +960,17 @@
 <property>
   <name>dfs.namenode.decommission.interval</name>
   <value>30s</value>
-  <description>Namenode periodicity in seconds to check if decommission is 
-  complete. Support multiple time unit suffix(case insensitive), as described
-  in dfs.heartbeat.interval.
+  <description>Namenode periodicity in seconds to check if
+    decommission or maintenance is complete. Support multiple time unit
+    suffix(case insensitive), as described in dfs.heartbeat.interval.
   </description>
 </property>
 
 <property>
   <name>dfs.namenode.decommission.blocks.per.interval</name>
   <value>500000</value>
-  <description>The approximate number of blocks to process per 
-      decommission interval, as defined in dfs.namenode.decommission.interval.
+  <description>The approximate number of blocks to process per decommission
+    or maintenance interval, as defined in dfs.namenode.decommission.interval.
   </description>
 </property>
 
@@ -978,11 +978,12 @@
   <name>dfs.namenode.decommission.max.concurrent.tracked.nodes</name>
   <value>100</value>
   <description>
-    The maximum number of decommission-in-progress datanodes nodes that will be
-    tracked at one time by the namenode. Tracking a decommission-in-progress
-    datanode consumes additional NN memory proportional to the number of blocks
-    on the datnode. Having a conservative limit reduces the potential impact
-    of decomissioning a large number of nodes at once.
+    The maximum number of decommission-in-progress or
+    entering-maintenance datanodes nodes that will be tracked at one time by
+    the namenode. Tracking these datanode consumes additional NN memory
+    proportional to the number of blocks on the datnode. Having a conservative
+    limit reduces the potential impact of decommissioning or maintenance of
+    a large number of nodes at once.
       
     A value of 0 means no limit will be enforced.
   </description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79df1e75/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
index c2c6be1..ac14a2a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
@@ -51,7 +51,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
-import org.apache.hadoop.hdfs.server.blockmanagement.DecommissionManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeAdminManager;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
@@ -256,9 +256,10 @@ public class TestDecommission extends AdminStatesBaseTest {
 
     startSimpleHACluster(3);
 
-    // Step 1, create a cluster with 4 DNs. Blocks are stored on the first 3 DNs.
-    // The last DN is empty. Also configure the last DN to have slow heartbeat
-    // so that it will be chosen as excess replica candidate during recommission.
+    // Step 1, create a cluster with 4 DNs. Blocks are stored on the
+    // first 3 DNs. The last DN is empty. Also configure the last DN to have
+    // slow heartbeat so that it will be chosen as excess replica candidate
+    // during recommission.
 
     // Step 1.a, copy blocks to the first 3 DNs. Given the replica count is the
     // same as # of DNs, each DN will have a replica for any block.
@@ -290,9 +291,9 @@ public class TestDecommission extends AdminStatesBaseTest {
 
     // Step 3, recommission the first DN on SBN and ANN to create excess replica
     // It recommissions the node on SBN first to create potential
-    // inconsistent state. In production cluster, such insistent state can happen
-    // even if recommission command was issued on ANN first given the async nature
-    // of the system.
+    // inconsistent state. In production cluster, such insistent state can
+    // happen even if recommission command was issued on ANN first given the
+    // async nature of the system.
 
     // Step 3.a, ask SBN to recomm the first DN.
     // SBN has been fixed so that it no longer invalidates excess replica during
@@ -301,10 +302,10 @@ public class TestDecommission extends AdminStatesBaseTest {
     //    1. the last DN would have been chosen as excess replica, given its
     //    heartbeat is considered old.
     //    Please refer to BlockPlacementPolicyDefault#chooseReplicaToDelete
-    //    2. After recommissionNode finishes, SBN has 3 live replicas ( 0, 1, 2 )
+    //    2. After recommissionNode finishes, SBN has 3 live replicas (0, 1, 2)
     //    and one excess replica ( 3 )
     // After the fix,
-    //    After recommissionNode finishes, SBN has 4 live replicas ( 0, 1, 2, 3 )
+    //    After recommissionNode finishes, SBN has 4 live replicas (0, 1, 2, 3)
     Thread.sleep(slowHeartbeatDNwaitTime);
     putNodeInService(1, decomNodeFromSBN);
 
@@ -561,7 +562,8 @@ public class TestDecommission extends AdminStatesBaseTest {
    * federated cluster.
    */
   @Test(timeout=360000)
-  public void testHostsFileFederation() throws IOException, InterruptedException {
+  public void testHostsFileFederation()
+      throws IOException, InterruptedException {
     // Test for 3 namenode federated cluster
     testHostsFile(3);
   }
@@ -598,7 +600,8 @@ public class TestDecommission extends AdminStatesBaseTest {
   }
   
   @Test(timeout=120000)
-  public void testDecommissionWithOpenfile() throws IOException, InterruptedException {
+  public void testDecommissionWithOpenfile()
+      throws IOException, InterruptedException {
     LOG.info("Starting test testDecommissionWithOpenfile");
     
     //At most 4 nodes will be decommissioned
@@ -742,14 +745,15 @@ public class TestDecommission extends AdminStatesBaseTest {
 
     // make sure the two datanodes remain in decomm in progress state
     BlockManagerTestUtil.recheckDecommissionState(dm);
-    assertTrackedAndPending(dm.getDecomManager(), 2, 0);
+    assertTrackedAndPending(dm.getDatanodeAdminManager(), 2, 0);
   }
   
   /**
    * Tests restart of namenode while datanode hosts are added to exclude file
    **/
   @Test(timeout=360000)
-  public void testDecommissionWithNamenodeRestart()throws IOException, InterruptedException {
+  public void testDecommissionWithNamenodeRestart()
+      throws IOException, InterruptedException {
     LOG.info("Starting test testDecommissionWithNamenodeRestart");
     int numNamenodes = 1;
     int numDatanodes = 1;
@@ -914,7 +918,7 @@ public class TestDecommission extends AdminStatesBaseTest {
   
   @Test(timeout=120000)
   public void testBlocksPerInterval() throws Exception {
-    org.apache.log4j.Logger.getLogger(DecommissionManager.class)
+    org.apache.log4j.Logger.getLogger(DatanodeAdminManager.class)
         .setLevel(Level.TRACE);
     // Turn the blocks per interval way down
     getConf().setInt(
@@ -927,7 +931,8 @@ public class TestDecommission extends AdminStatesBaseTest {
     final FileSystem fs = getCluster().getFileSystem();
     final DatanodeManager datanodeManager =
         getCluster().getNamesystem().getBlockManager().getDatanodeManager();
-    final DecommissionManager decomManager = datanodeManager.getDecomManager();
+    final DatanodeAdminManager decomManager =
+        datanodeManager.getDatanodeAdminManager();
 
     // Write a 3 block file, so each node has one block. Should scan 3 nodes.
     DFSTestUtil.createFile(fs, new Path("/file1"), 64, (short) 3, 0xBAD1DEA);
@@ -944,7 +949,7 @@ public class TestDecommission extends AdminStatesBaseTest {
   }
 
   private void doDecomCheck(DatanodeManager datanodeManager,
-      DecommissionManager decomManager, int expectedNumCheckedNodes)
+      DatanodeAdminManager decomManager, int expectedNumCheckedNodes)
       throws IOException, ExecutionException, InterruptedException {
     // Decom all nodes
     ArrayList<DatanodeInfo> decommissionedNodes = Lists.newArrayList();
@@ -965,7 +970,7 @@ public class TestDecommission extends AdminStatesBaseTest {
 
   @Test(timeout=120000)
   public void testPendingNodes() throws Exception {
-    org.apache.log4j.Logger.getLogger(DecommissionManager.class)
+    org.apache.log4j.Logger.getLogger(DatanodeAdminManager.class)
         .setLevel(Level.TRACE);
     // Only allow one node to be decom'd at a time
     getConf().setInt(
@@ -978,7 +983,8 @@ public class TestDecommission extends AdminStatesBaseTest {
     final FileSystem fs = getCluster().getFileSystem();
     final DatanodeManager datanodeManager =
         getCluster().getNamesystem().getBlockManager().getDatanodeManager();
-    final DecommissionManager decomManager = datanodeManager.getDecomManager();
+    final DatanodeAdminManager decomManager =
+        datanodeManager.getDatanodeAdminManager();
 
     // Keep a file open to prevent decom from progressing
     HdfsDataOutputStream open1 =
@@ -1014,7 +1020,7 @@ public class TestDecommission extends AdminStatesBaseTest {
     assertTrackedAndPending(decomManager, 1, 0);
   }
 
-  private void assertTrackedAndPending(DecommissionManager decomManager,
+  private void assertTrackedAndPending(DatanodeAdminManager decomManager,
       int tracked, int pending) {
     assertEquals("Unexpected number of tracked nodes", tracked,
         decomManager.getNumTrackedNodes());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79df1e75/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
index 77e2ffb..7ee766f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
@@ -328,7 +328,7 @@ public class BlockManagerTestUtil {
    */
   public static void recheckDecommissionState(DatanodeManager dm)
       throws ExecutionException, InterruptedException {
-    dm.getDecomManager().runMonitorForTest();
+    dm.getDatanodeAdminManager().runMonitorForTest();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79df1e75/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReconstructStripedBlocksWithRackAwareness.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReconstructStripedBlocksWithRackAwareness.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReconstructStripedBlocksWithRackAwareness.java
index 4ecfd50..aaa4899 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReconstructStripedBlocksWithRackAwareness.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReconstructStripedBlocksWithRackAwareness.java
@@ -330,8 +330,9 @@ public class TestReconstructStripedBlocksWithRackAwareness {
     // start decommissioning h9
     boolean satisfied = bm.isPlacementPolicySatisfied(blockInfo);
     Assert.assertFalse(satisfied);
-    final DecommissionManager decomManager =
-        (DecommissionManager) Whitebox.getInternalState(dm, "decomManager");
+    final DatanodeAdminManager decomManager =
+        (DatanodeAdminManager) Whitebox.getInternalState(
+            dm, "datanodeAdminManager");
     cluster.getNamesystem().writeLock();
     try {
       dn9.stopDecommission();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79df1e75/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyConsiderLoad.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyConsiderLoad.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyConsiderLoad.java
index bcd8245..fef0b45 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyConsiderLoad.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyConsiderLoad.java
@@ -100,7 +100,7 @@ public class TestReplicationPolicyConsiderLoad
       // returns false
       for (int i = 0; i < 3; i++) {
         DatanodeDescriptor d = dataNodes[i];
-        dnManager.getDecomManager().startDecommission(d);
+        dnManager.getDatanodeAdminManager().startDecommission(d);
         d.setDecommissioned();
       }
       assertEquals((double)load/3, dnManager.getFSClusterStats()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79df1e75/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
index 11d7431..cfebff7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
@@ -50,7 +50,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
-import org.apache.hadoop.hdfs.server.blockmanagement.DecommissionManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeAdminManager;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
@@ -100,7 +100,7 @@ public class TestDecommissioningStatus {
     fileSys = cluster.getFileSystem();
     cluster.getNamesystem().getBlockManager().getDatanodeManager()
         .setHeartbeatExpireInterval(3000);
-    Logger.getLogger(DecommissionManager.class).setLevel(Level.DEBUG);
+    Logger.getLogger(DatanodeAdminManager.class).setLevel(Level.DEBUG);
     LOG = Logger.getLogger(TestDecommissioningStatus.class);
   }
 
@@ -344,7 +344,7 @@ public class TestDecommissioningStatus {
    */
   @Test(timeout=120000)
   public void testDecommissionDeadDN() throws Exception {
-    Logger log = Logger.getLogger(DecommissionManager.class);
+    Logger log = Logger.getLogger(DatanodeAdminManager.class);
     log.setLevel(Level.DEBUG);
     DatanodeID dnID = cluster.getDataNodes().get(0).getDatanodeId();
     String dnName = dnID.getXferAddr();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79df1e75/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDefaultBlockPlacementPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDefaultBlockPlacementPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDefaultBlockPlacementPolicy.java
index eab1199..205593f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDefaultBlockPlacementPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDefaultBlockPlacementPolicy.java
@@ -156,11 +156,11 @@ public class TestDefaultBlockPlacementPolicy {
     DatanodeDescriptor dnd3 = dnm.getDatanode(
         cluster.getDataNodes().get(3).getDatanodeId());
     assertEquals(dnd3.getNetworkLocation(), clientRack);
-    dnm.getDecomManager().startDecommission(dnd3);
+    dnm.getDatanodeAdminManager().startDecommission(dnd3);
     try {
       testPlacement(clientMachine, clientRack, false);
     } finally {
-      dnm.getDecomManager().stopDecommission(dnd3);
+      dnm.getDatanodeAdminManager().stopDecommission(dnd3);
     }
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[44/51] [abbrv] hadoop git commit: HADOOP-14628. Upgrade maven enforcer plugin to 3.0.0-M1.

Posted by st...@apache.org.
HADOOP-14628. Upgrade maven enforcer plugin to 3.0.0-M1.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ebabc709
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ebabc709
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ebabc709

Branch: refs/heads/HADOOP-13345
Commit: ebabc7094c6bcbd9063744331c69e3fba615fa62
Parents: a53b8b6
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Aug 9 13:16:31 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed Aug 9 13:18:16 2017 +0900

----------------------------------------------------------------------
 hadoop-client-modules/hadoop-client-check-invariants/pom.xml      | 1 -
 hadoop-client-modules/hadoop-client-check-test-invariants/pom.xml | 1 -
 pom.xml                                                           | 2 +-
 3 files changed, 1 insertion(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ebabc709/hadoop-client-modules/hadoop-client-check-invariants/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client-check-invariants/pom.xml b/hadoop-client-modules/hadoop-client-check-invariants/pom.xml
index e495a69..2f31fa6 100644
--- a/hadoop-client-modules/hadoop-client-check-invariants/pom.xml
+++ b/hadoop-client-modules/hadoop-client-check-invariants/pom.xml
@@ -46,7 +46,6 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-enforcer-plugin</artifactId>
-        <version>1.4</version>
         <dependencies>
           <dependency>
             <groupId>org.codehaus.mojo</groupId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ebabc709/hadoop-client-modules/hadoop-client-check-test-invariants/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client-check-test-invariants/pom.xml b/hadoop-client-modules/hadoop-client-check-test-invariants/pom.xml
index 68d1f5b..0e23db9 100644
--- a/hadoop-client-modules/hadoop-client-check-test-invariants/pom.xml
+++ b/hadoop-client-modules/hadoop-client-check-test-invariants/pom.xml
@@ -50,7 +50,6 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-enforcer-plugin</artifactId>
-        <version>1.4</version>
         <dependencies>
           <dependency>
             <groupId>org.codehaus.mojo</groupId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ebabc709/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index d82cd9f..22a4b59 100644
--- a/pom.xml
+++ b/pom.xml
@@ -97,7 +97,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
     <maven-antrun-plugin.version>1.7</maven-antrun-plugin.version>
     <maven-assembly-plugin.version>2.4</maven-assembly-plugin.version>
     <maven-dependency-plugin.version>2.10</maven-dependency-plugin.version>
-    <maven-enforcer-plugin.version>1.4.1</maven-enforcer-plugin.version>
+    <maven-enforcer-plugin.version>3.0.0-M1</maven-enforcer-plugin.version>
     <maven-javadoc-plugin.version>2.10.4</maven-javadoc-plugin.version>
     <maven-gpg-plugin.version>1.5</maven-gpg-plugin.version>
     <maven-remote-resources-plugin.version>1.5</maven-remote-resources-plugin.version>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org