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 zj...@apache.org on 2015/05/04 22:04:20 UTC

[01/50] [abbrv] hadoop git commit: HADOOP-11881. test-patch.sh javac result is wildly wrong (Kengo Seki via aw)

Repository: hadoop
Updated Branches:
  refs/heads/YARN-2928 b689f5d43 -> 0b1e184cc


HADOOP-11881. test-patch.sh javac result is wildly wrong (Kengo Seki via aw)


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

Branch: refs/heads/YARN-2928
Commit: 972d09d9cbe18415f00a715d40453bfbcf741caf
Parents: 8eb0efe
Author: Allen Wittenauer <aw...@apache.org>
Authored: Tue Apr 28 10:32:32 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:52 2015 -0700

----------------------------------------------------------------------
 dev-support/test-patch.sh                       | 2 +-
 hadoop-common-project/hadoop-common/CHANGES.txt | 3 +++
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/972d09d9/dev-support/test-patch.sh
----------------------------------------------------------------------
diff --git a/dev-support/test-patch.sh b/dev-support/test-patch.sh
index e331deb..ae21837 100755
--- a/dev-support/test-patch.sh
+++ b/dev-support/test-patch.sh
@@ -1552,7 +1552,7 @@ function check_javac
         > "${PATCH_DIR}/diffJavacWarnings.txt"
 
         add_jira_table -1 javac "The applied patch generated "\
-        "$((patchJavacWarnings-${PATCH_BRANCH}JavacWarnings))" \
+        "$((patchJavacWarnings-branchJavacWarnings))" \
         " additional warning messages."
 
         add_jira_footer javac "@@BASE@@/diffJavacWarnings.txt"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/972d09d9/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 1783e8d..39d14c9 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -576,6 +576,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11870. [JDK8] AuthenticationFilter, CertificateUtil,
     SignerSecretProviders, KeyAuthorizationKeyProvider Javadoc issues (rkanter)
 
+    HADOOP-11881. test-patch.sh javac result is wildly wrong (Kengo Seki via
+    aw)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[05/50] [abbrv] hadoop git commit: HDFS-8269. getBlockLocations() does not resolve the .reserved path and generates incorrect edit logs when updating the atime. Contributed by Haohui Mai.

Posted by zj...@apache.org.
HDFS-8269. getBlockLocations() does not resolve the .reserved path and generates incorrect edit logs when updating the atime. Contributed by Haohui Mai.


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

Branch: refs/heads/YARN-2928
Commit: c1e8813a2af24ae41d8a934da1ebd027a4ab7b3d
Parents: b5227c9
Author: Haohui Mai <wh...@apache.org>
Authored: Wed Apr 29 11:12:45 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:53 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hdfs/server/namenode/FSNamesystem.java      |  63 ++++++---
 .../hdfs/server/namenode/NamenodeFsck.java      |   4 +-
 .../hadoop/hdfs/server/namenode/TestFsck.java   |   8 +-
 .../server/namenode/TestGetBlockLocations.java  | 133 +++++++++++++++++++
 5 files changed, 188 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c1e8813a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 2dde356..a3f219b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -635,6 +635,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-8273. FSNamesystem#Delete() should not call logSync() when holding the
     lock. (wheat9)
 
+    HDFS-8269. getBlockLocations() does not resolve the .reserved path and
+    generates incorrect edit logs when updating the atime. (wheat9)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c1e8813a/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 0ec81d8..b938263 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
@@ -1697,13 +1697,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   static class GetBlockLocationsResult {
-    final INodesInPath iip;
+    final boolean updateAccessTime;
     final LocatedBlocks blocks;
     boolean updateAccessTime() {
-      return iip != null;
+      return updateAccessTime;
     }
-    private GetBlockLocationsResult(INodesInPath iip, LocatedBlocks blocks) {
-      this.iip = iip;
+    private GetBlockLocationsResult(
+        boolean updateAccessTime, LocatedBlocks blocks) {
+      this.updateAccessTime = updateAccessTime;
       this.blocks = blocks;
     }
   }
@@ -1712,34 +1713,58 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * Get block locations within the specified range.
    * @see ClientProtocol#getBlockLocations(String, long, long)
    */
-  LocatedBlocks getBlockLocations(String clientMachine, String src,
+  LocatedBlocks getBlockLocations(String clientMachine, String srcArg,
       long offset, long length) throws IOException {
     checkOperation(OperationCategory.READ);
     GetBlockLocationsResult res = null;
+    FSPermissionChecker pc = getPermissionChecker();
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      res = getBlockLocations(src, offset, length, true, true);
+      res = getBlockLocations(pc, srcArg, offset, length, true, true);
     } catch (AccessControlException e) {
-      logAuditEvent(false, "open", src);
+      logAuditEvent(false, "open", srcArg);
       throw e;
     } finally {
       readUnlock();
     }
 
-    logAuditEvent(true, "open", src);
+    logAuditEvent(true, "open", srcArg);
 
     if (res.updateAccessTime()) {
+      byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(
+          srcArg);
+      String src = srcArg;
       writeLock();
       final long now = now();
       try {
         checkOperation(OperationCategory.WRITE);
-        INode inode = res.iip.getLastINode();
-        boolean updateAccessTime = now > inode.getAccessTime() +
-            getAccessTimePrecision();
+        /**
+         * Resolve the path again and update the atime only when the file
+         * exists.
+         *
+         * XXX: Races can still occur even after resolving the path again.
+         * For example:
+         *
+         * <ul>
+         *   <li>Get the block location for "/a/b"</li>
+         *   <li>Rename "/a/b" to "/c/b"</li>
+         *   <li>The second resolution still points to "/a/b", which is
+         *   wrong.</li>
+         * </ul>
+         *
+         * The behavior is incorrect but consistent with the one before
+         * HDFS-7463. A better fix is to change the edit log of SetTime to
+         * use inode id instead of a path.
+         */
+        src = dir.resolvePath(pc, srcArg, pathComponents);
+        final INodesInPath iip = dir.getINodesInPath(src, true);
+        INode inode = iip.getLastINode();
+        boolean updateAccessTime = inode != null &&
+            now > inode.getAccessTime() + getAccessTimePrecision();
         if (!isInSafeMode() && updateAccessTime) {
           boolean changed = FSDirAttrOp.setTimes(dir,
-              inode, -1, now, false, res.iip.getLatestSnapshotId());
+              inode, -1, now, false, iip.getLatestSnapshotId());
           if (changed) {
             getEditLog().logTimes(src, -1, now);
           }
@@ -1773,8 +1798,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @throws IOException
    */
   GetBlockLocationsResult getBlockLocations(
-      String src, long offset, long length, boolean needBlockToken,
-      boolean checkSafeMode) throws IOException {
+      FSPermissionChecker pc, String src, long offset, long length,
+      boolean needBlockToken, boolean checkSafeMode) throws IOException {
     if (offset < 0) {
       throw new HadoopIllegalArgumentException(
           "Negative offset is not supported. File: " + src);
@@ -1784,7 +1809,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           "Negative length is not supported. File: " + src);
     }
     final GetBlockLocationsResult ret = getBlockLocationsInt(
-        src, offset, length, needBlockToken);
+        pc, src, offset, length, needBlockToken);
 
     if (checkSafeMode && isInSafeMode()) {
       for (LocatedBlock b : ret.blocks.getLocatedBlocks()) {
@@ -1805,12 +1830,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   private GetBlockLocationsResult getBlockLocationsInt(
-      final String srcArg, long offset, long length, boolean needBlockToken)
+      FSPermissionChecker pc, final String srcArg, long offset, long length,
+      boolean needBlockToken)
       throws IOException {
     String src = srcArg;
-    FSPermissionChecker pc = getPermissionChecker();
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
-    src = dir.resolvePath(pc, src, pathComponents);
+    src = dir.resolvePath(pc, srcArg, pathComponents);
     final INodesInPath iip = dir.getINodesInPath(src, true);
     final INodeFile inode = INodeFile.valueOf(iip.getLastINode(), src);
     if (isPermissionEnabled) {
@@ -1846,7 +1871,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     boolean updateAccessTime = isAccessTimeSupported() && !isInSafeMode()
         && !iip.isSnapshot()
         && now > inode.getAccessTime() + getAccessTimePrecision();
-    return new GetBlockLocationsResult(updateAccessTime ? iip : null, blocks);
+    return new GetBlockLocationsResult(updateAccessTime, blocks);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c1e8813a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index 23fea12..0cfe31a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -484,7 +484,9 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
     FSNamesystem fsn = namenode.getNamesystem();
     fsn.readLock();
     try {
-      blocks = fsn.getBlockLocations(path, 0, fileLen, false, false).blocks;
+      blocks = fsn.getBlockLocations(
+          fsn.getPermissionChecker(), path, 0, fileLen, false, false)
+          .blocks;
     } catch (FileNotFoundException fnfe) {
       blocks = null;
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c1e8813a/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 4c3fa9c..8fe273b 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
@@ -24,6 +24,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyBoolean;
 import static org.mockito.Matchers.anyLong;
 import static org.mockito.Matchers.anyString;
@@ -1154,10 +1155,11 @@ public class TestFsck {
     FSNamesystem fsName = mock(FSNamesystem.class);
     BlockManager blockManager = mock(BlockManager.class);
     DatanodeManager dnManager = mock(DatanodeManager.class);
-    
+
     when(namenode.getNamesystem()).thenReturn(fsName);
-    when(fsName.getBlockLocations(
-        anyString(), anyLong(), anyLong(), anyBoolean(), anyBoolean()))
+    when(fsName.getBlockLocations(any(FSPermissionChecker.class), anyString(),
+                                  anyLong(), anyLong(),
+                                  anyBoolean(), anyBoolean()))
         .thenThrow(new FileNotFoundException());
     when(fsName.getBlockManager()).thenReturn(blockManager);
     when(blockManager.getDatanodeManager()).thenReturn(dnManager);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c1e8813a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java
new file mode 100644
index 0000000..a19eb1d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java
@@ -0,0 +1,133 @@
+/**
+ * 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.namenode;
+
+import org.apache.commons.io.Charsets;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.IOException;
+import java.util.ArrayList;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY;
+import static org.apache.hadoop.util.Time.now;
+import static org.mockito.Mockito.anyLong;
+import static org.mockito.Mockito.anyString;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class TestGetBlockLocations {
+  private static final String FILE_NAME = "foo";
+  private static final String FILE_PATH = "/" + FILE_NAME;
+  private static final long MOCK_INODE_ID = 16386;
+  private static final String RESERVED_PATH =
+      "/.reserved/.inodes/" + MOCK_INODE_ID;
+
+  @Test(timeout = 30000)
+  public void testResolveReservedPath() throws IOException {
+    FSNamesystem fsn = setupFileSystem();
+    FSEditLog editlog = fsn.getEditLog();
+    fsn.getBlockLocations("dummy", RESERVED_PATH, 0, 1024);
+    verify(editlog).logTimes(eq(FILE_PATH), anyLong(), anyLong());
+    fsn.close();
+  }
+
+  @Test(timeout = 30000)
+  public void testGetBlockLocationsRacingWithDelete() throws IOException {
+    FSNamesystem fsn = spy(setupFileSystem());
+    final FSDirectory fsd = fsn.getFSDirectory();
+    FSEditLog editlog = fsn.getEditLog();
+
+    doAnswer(new Answer<Void>() {
+
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        INodesInPath iip = fsd.getINodesInPath(FILE_PATH, true);
+        FSDirDeleteOp.delete(fsd, iip, new INode.BlocksMapUpdateInfo(),
+                             new ArrayList<INode>(), now());
+        invocation.callRealMethod();
+        return null;
+      }
+    }).when(fsn).writeLock();
+    fsn.getBlockLocations("dummy", RESERVED_PATH, 0, 1024);
+
+    verify(editlog, never()).logTimes(anyString(), anyLong(), anyLong());
+    fsn.close();
+  }
+
+  @Test(timeout = 30000)
+  public void testGetBlockLocationsRacingWithRename() throws IOException {
+    FSNamesystem fsn = spy(setupFileSystem());
+    final FSDirectory fsd = fsn.getFSDirectory();
+    FSEditLog editlog = fsn.getEditLog();
+    final String DST_PATH = "/bar";
+    final boolean[] renamed = new boolean[1];
+
+    doAnswer(new Answer<Void>() {
+
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        invocation.callRealMethod();
+        if (!renamed[0]) {
+          FSDirRenameOp.renameTo(fsd, fsd.getPermissionChecker(), FILE_PATH,
+                                 DST_PATH, new INode.BlocksMapUpdateInfo(),
+                                 false);
+          renamed[0] = true;
+        }
+        return null;
+      }
+    }).when(fsn).writeLock();
+    fsn.getBlockLocations("dummy", RESERVED_PATH, 0, 1024);
+
+    verify(editlog).logTimes(eq(DST_PATH), anyLong(), anyLong());
+    fsn.close();
+  }
+
+  private static FSNamesystem setupFileSystem() throws IOException {
+    Configuration conf = new Configuration();
+    conf.setLong(DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, 1L);
+    FSEditLog editlog = mock(FSEditLog.class);
+    FSImage image = mock(FSImage.class);
+    when(image.getEditLog()).thenReturn(editlog);
+    final FSNamesystem fsn = new FSNamesystem(conf, image, true);
+
+    final FSDirectory fsd = fsn.getFSDirectory();
+    INodesInPath iip = fsd.getINodesInPath("/", true);
+    PermissionStatus perm = new PermissionStatus(
+        "hdfs", "supergroup",
+        FsPermission.createImmutable((short) 0x1ff));
+    final INodeFile file = new INodeFile(
+        MOCK_INODE_ID, FILE_NAME.getBytes(Charsets.UTF_8),
+        perm, 1, 1, new BlockInfoContiguous[] {}, (short) 1,
+        DFS_BLOCK_SIZE_DEFAULT);
+    fsn.getFSDirectory().addINode(iip, file);
+    return fsn;
+  }
+
+}


[06/50] [abbrv] hadoop git commit: HDFS-8283. DataStreamer cleanup and some minor improvement. Contributed by Tsz Wo Nicholas Sze.

Posted by zj...@apache.org.
HDFS-8283. DataStreamer cleanup and some minor improvement. Contributed by Tsz Wo Nicholas Sze.


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

Branch: refs/heads/YARN-2928
Commit: b5227c940794fe43ef5c27905fca4e481f0b0cd0
Parents: 19dd6d9
Author: Jing Zhao <ji...@apache.org>
Authored: Wed Apr 29 10:41:46 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:53 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/io/MultipleIOException.java   |  26 ++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |  30 +--
 .../org/apache/hadoop/hdfs/DataStreamer.java    | 235 ++++++++++---------
 .../apache/hadoop/hdfs/TestDFSOutputStream.java |   3 +-
 5 files changed, 162 insertions(+), 135 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b5227c94/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MultipleIOException.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MultipleIOException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MultipleIOException.java
index 5e584c9c..66c1ab1 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MultipleIOException.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MultipleIOException.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.io;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -51,4 +52,29 @@ public class MultipleIOException extends IOException {
     }
     return new MultipleIOException(exceptions);
   }
+
+  /**
+   * Build an {@link IOException} using {@link MultipleIOException}
+   * if there are more than one.
+   */
+  public static class Builder {
+    private List<IOException> exceptions;
+    
+    /** Add the given {@link Throwable} to the exception list. */
+    public void add(Throwable t) {
+      if (exceptions == null) {
+        exceptions = new ArrayList<>();
+      }
+      exceptions.add(t instanceof IOException? (IOException)t
+          : new IOException(t));
+    }
+
+    /**
+     * @return null if nothing is added to this builder;
+     *         otherwise, return an {@link IOException}
+     */
+    public IOException build() {
+      return createIOException(exceptions);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b5227c94/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index e7fa8fd..2dde356 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -480,6 +480,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8280. Code Cleanup in DFSInputStream. (Jing Zhao via wheat9)
 
+    HDFS-8283. DataStreamer cleanup and some minor improvement. (szetszwo via
+    jing9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b5227c94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index d9b8ee7..4646b60 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@ -139,8 +139,7 @@ public class DFSOutputStream extends FSOutputSummer
   @Override
   protected void checkClosed() throws IOException {
     if (isClosed()) {
-      IOException e = streamer.getLastException().get();
-      throw e != null ? e : new ClosedChannelException();
+      streamer.getLastException().throwException4Close();
     }
   }
 
@@ -216,10 +215,7 @@ public class DFSOutputStream extends FSOutputSummer
     computePacketChunkSize(dfsClient.getConf().getWritePacketSize(), bytesPerChecksum);
 
     streamer = new DataStreamer(stat, null, dfsClient, src, progress, checksum,
-        cachingStrategy, byteArrayManager);
-    if (favoredNodes != null && favoredNodes.length != 0) {
-      streamer.setFavoredNodes(favoredNodes);
-    }
+        cachingStrategy, byteArrayManager, favoredNodes);
   }
 
   static DFSOutputStream newStreamForCreate(DFSClient dfsClient, String src,
@@ -282,7 +278,8 @@ public class DFSOutputStream extends FSOutputSummer
   /** Construct a new output stream for append. */
   private DFSOutputStream(DFSClient dfsClient, String src,
       EnumSet<CreateFlag> flags, Progressable progress, LocatedBlock lastBlock,
-      HdfsFileStatus stat, DataChecksum checksum) throws IOException {
+      HdfsFileStatus stat, DataChecksum checksum, String[] favoredNodes)
+          throws IOException {
     this(dfsClient, src, progress, stat, checksum);
     initialFileSize = stat.getLen(); // length of file when opened
     this.shouldSyncBlock = flags.contains(CreateFlag.SYNC_BLOCK);
@@ -303,7 +300,8 @@ public class DFSOutputStream extends FSOutputSummer
       computePacketChunkSize(dfsClient.getConf().getWritePacketSize(),
           bytesPerChecksum);
       streamer = new DataStreamer(stat, lastBlock != null ? lastBlock.getBlock() : null,
-          dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager);
+          dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager,
+          favoredNodes);
     }
   }
 
@@ -351,10 +349,7 @@ public class DFSOutputStream extends FSOutputSummer
         dfsClient.getPathTraceScope("newStreamForAppend", src);
     try {
       final DFSOutputStream out = new DFSOutputStream(dfsClient, src, flags,
-          progress, lastBlock, stat, checksum);
-      if (favoredNodes != null && favoredNodes.length != 0) {
-        out.streamer.setFavoredNodes(favoredNodes);
-      }
+          progress, lastBlock, stat, checksum, favoredNodes);
       out.start();
       return out;
     } finally {
@@ -653,7 +648,7 @@ public class DFSOutputStream extends FSOutputSummer
       DFSClient.LOG.warn("Error while syncing", e);
       synchronized (this) {
         if (!isClosed()) {
-          streamer.getLastException().set(new IOException("IOException flush: " + e));
+          streamer.getLastException().set(e);
           closeThreads(true);
         }
       }
@@ -720,7 +715,7 @@ public class DFSOutputStream extends FSOutputSummer
     if (isClosed()) {
       return;
     }
-    streamer.setLastException(new IOException("Lease timeout of "
+    streamer.getLastException().set(new IOException("Lease timeout of "
         + (dfsClient.getConf().getHdfsTimeout()/1000) + " seconds expired."));
     closeThreads(true);
     dfsClient.endFileLease(fileId);
@@ -767,11 +762,8 @@ public class DFSOutputStream extends FSOutputSummer
 
   protected synchronized void closeImpl() throws IOException {
     if (isClosed()) {
-      IOException e = streamer.getLastException().getAndSet(null);
-      if (e == null)
-        return;
-      else
-        throw e;
+      streamer.getLastException().check();
+      return;
     }
 
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b5227c94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
index 5f0c9ac..3727d20 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
@@ -41,6 +41,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
@@ -73,6 +75,7 @@ import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
 import org.apache.hadoop.hdfs.util.ByteArrayManager;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.MultipleIOException;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
@@ -88,6 +91,7 @@ import org.apache.htrace.Trace;
 import org.apache.htrace.TraceInfo;
 import org.apache.htrace.TraceScope;
 
+import com.google.common.base.Preconditions;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
@@ -117,6 +121,7 @@ import com.google.common.cache.RemovalNotification;
 
 @InterfaceAudience.Private
 class DataStreamer extends Daemon {
+  static final Log LOG = LogFactory.getLog(DataStreamer.class);
   /**
    * Create a socket for a write pipeline
    *
@@ -129,8 +134,8 @@ class DataStreamer extends Daemon {
       final int length, final DFSClient client) throws IOException {
     final DfsClientConf conf = client.getConf();
     final String dnAddr = first.getXferAddr(conf.isConnectToDnViaHostname());
-    if (DFSClient.LOG.isDebugEnabled()) {
-      DFSClient.LOG.debug("Connecting to datanode " + dnAddr);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Connecting to datanode " + dnAddr);
     }
     final InetSocketAddress isa = NetUtils.createSocketAddr(dnAddr);
     final Socket sock = client.socketFactory.createSocket();
@@ -138,8 +143,8 @@ class DataStreamer extends Daemon {
     NetUtils.connect(sock, isa, client.getRandomLocalInterfaceAddr(), conf.getSocketTimeout());
     sock.setSoTimeout(timeout);
     sock.setSendBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
-    if(DFSClient.LOG.isDebugEnabled()) {
-      DFSClient.LOG.debug("Send buf size " + sock.getSendBufferSize());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Send buf size " + sock.getSendBufferSize());
     }
     return sock;
   }
@@ -168,6 +173,34 @@ class DataStreamer extends Daemon {
     }
     packets.clear();
   }
+  
+  static class LastException {
+    private Throwable thrown;
+
+    synchronized void set(Throwable t) {
+      Preconditions.checkNotNull(t);
+      Preconditions.checkState(thrown == null);
+      this.thrown = t;
+    }
+
+    synchronized void clear() {
+      thrown = null;
+    }
+
+    /** Check if there already is an exception. */
+    synchronized void check() throws IOException {
+      if (thrown != null) {
+        throw new IOException(thrown);
+      }
+    }
+
+    synchronized void throwException4Close() throws IOException {
+      check();
+      final IOException ioe = new ClosedChannelException();
+      thrown = ioe;
+      throw ioe;
+    }
+  }
 
   private volatile boolean streamerClosed = false;
   private ExtendedBlock block; // its length is number of bytes acked
@@ -178,7 +211,6 @@ class DataStreamer extends Daemon {
   private volatile DatanodeInfo[] nodes = null; // list of targets for current block
   private volatile StorageType[] storageTypes = null;
   private volatile String[] storageIDs = null;
-  private String[] favoredNodes;
   volatile boolean hasError = false;
   volatile int errorIndex = -1;
   // Restarting node index
@@ -196,13 +228,13 @@ class DataStreamer extends Daemon {
   /** Has the current block been hflushed? */
   private boolean isHflushed = false;
   /** Append on an existing block? */
-  private boolean isAppend;
+  private final boolean isAppend;
 
   private long currentSeqno = 0;
   private long lastQueuedSeqno = -1;
   private long lastAckedSeqno = -1;
   private long bytesCurBlock = 0; // bytes written in current block
-  private final AtomicReference<IOException> lastException = new AtomicReference<>();
+  private final LastException lastException = new LastException();
   private Socket s;
 
   private final DFSClient dfsClient;
@@ -227,18 +259,20 @@ class DataStreamer extends Daemon {
   private long artificialSlowdown = 0;
   // List of congested data nodes. The stream will back off if the DataNodes
   // are congested
-  private final ArrayList<DatanodeInfo> congestedNodes = new ArrayList<>();
+  private final List<DatanodeInfo> congestedNodes = new ArrayList<>();
   private static final int CONGESTION_BACKOFF_MEAN_TIME_IN_MS = 5000;
   private static final int CONGESTION_BACK_OFF_MAX_TIME_IN_MS =
       CONGESTION_BACKOFF_MEAN_TIME_IN_MS * 10;
   private int lastCongestionBackoffTime;
 
   private final LoadingCache<DatanodeInfo, DatanodeInfo> excludedNodes;
+  private final String[] favoredNodes;
 
   private DataStreamer(HdfsFileStatus stat, DFSClient dfsClient, String src,
                        Progressable progress, DataChecksum checksum,
                        AtomicReference<CachingStrategy> cachingStrategy,
-                       ByteArrayManager byteArrayManage){
+                       ByteArrayManager byteArrayManage,
+                       boolean isAppend, String[] favoredNodes) {
     this.dfsClient = dfsClient;
     this.src = src;
     this.progress = progress;
@@ -246,10 +280,12 @@ class DataStreamer extends Daemon {
     this.checksum4WriteBlock = checksum;
     this.cachingStrategy = cachingStrategy;
     this.byteArrayManager = byteArrayManage;
-    isLazyPersistFile = isLazyPersist(stat);
+    this.isLazyPersistFile = isLazyPersist(stat);
     this.dfsclientSlowLogThresholdMs =
         dfsClient.getConf().getSlowIoWarningThresholdMs();
-    excludedNodes = initExcludedNodes();
+    this.excludedNodes = initExcludedNodes();
+    this.isAppend = isAppend;
+    this.favoredNodes = favoredNodes;
   }
 
   /**
@@ -258,10 +294,9 @@ class DataStreamer extends Daemon {
   DataStreamer(HdfsFileStatus stat, ExtendedBlock block, DFSClient dfsClient,
                String src, Progressable progress, DataChecksum checksum,
                AtomicReference<CachingStrategy> cachingStrategy,
-               ByteArrayManager byteArrayManage) {
+               ByteArrayManager byteArrayManage, String[] favoredNodes) {
     this(stat, dfsClient, src, progress, checksum, cachingStrategy,
-        byteArrayManage);
-    isAppend = false;
+        byteArrayManage, false, favoredNodes);
     this.block = block;
     stage = BlockConstructionStage.PIPELINE_SETUP_CREATE;
   }
@@ -277,8 +312,7 @@ class DataStreamer extends Daemon {
                AtomicReference<CachingStrategy> cachingStrategy,
                ByteArrayManager byteArrayManage) throws IOException {
     this(stat, dfsClient, src, progress, checksum, cachingStrategy,
-        byteArrayManage);
-    isAppend = true;
+        byteArrayManage, true, null);
     stage = BlockConstructionStage.PIPELINE_SETUP_APPEND;
     block = lastBlock.getBlock();
     bytesSent = block.getNumBytes();
@@ -314,15 +348,6 @@ class DataStreamer extends Daemon {
   }
 
   /**
-   * Set favored nodes
-   *
-   * @param favoredNodes favored nodes
-   */
-  void setFavoredNodes(String[] favoredNodes) {
-    this.favoredNodes = favoredNodes;
-  }
-
-  /**
    * Initialize for data streaming
    */
   private void initDataStreaming() {
@@ -334,8 +359,8 @@ class DataStreamer extends Daemon {
   }
 
   private void endBlock() {
-    if(DFSClient.LOG.isDebugEnabled()) {
-      DFSClient.LOG.debug("Closing old block " + block);
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("Closing old block " + block);
     }
     this.setName("DataStreamer for file " + src);
     closeResponder();
@@ -360,7 +385,7 @@ class DataStreamer extends Daemon {
           response.join();
           response = null;
         } catch (InterruptedException  e) {
-          DFSClient.LOG.warn("Caught exception ", e);
+          LOG.warn("Caught exception", e);
         }
       }
 
@@ -388,7 +413,7 @@ class DataStreamer extends Daemon {
             try {
               dataQueue.wait(timeout);
             } catch (InterruptedException  e) {
-              DFSClient.LOG.warn("Caught exception ", e);
+              LOG.warn("Caught exception", e);
             }
             doSleep = false;
             now = Time.monotonicNow();
@@ -404,7 +429,7 @@ class DataStreamer extends Daemon {
             try {
               backOffIfNecessary();
             } catch (InterruptedException e) {
-              DFSClient.LOG.warn("Caught exception ", e);
+              LOG.warn("Caught exception", e);
             }
             one = dataQueue.getFirst(); // regular data packet
             long parents[] = one.getTraceParents();
@@ -419,14 +444,14 @@ class DataStreamer extends Daemon {
 
         // get new block from namenode.
         if (stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
-          if(DFSClient.LOG.isDebugEnabled()) {
-            DFSClient.LOG.debug("Allocating new block");
+          if(LOG.isDebugEnabled()) {
+            LOG.debug("Allocating new block");
           }
           setPipeline(nextBlockOutputStream());
           initDataStreaming();
         } else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) {
-          if(DFSClient.LOG.isDebugEnabled()) {
-            DFSClient.LOG.debug("Append to block " + block);
+          if(LOG.isDebugEnabled()) {
+            LOG.debug("Append to block " + block);
           }
           setupPipelineForAppendOrRecovery();
           initDataStreaming();
@@ -450,7 +475,7 @@ class DataStreamer extends Daemon {
                 // wait for acks to arrive from datanodes
                 dataQueue.wait(1000);
               } catch (InterruptedException  e) {
-                DFSClient.LOG.warn("Caught exception ", e);
+                LOG.warn("Caught exception", e);
               }
             }
           }
@@ -473,8 +498,8 @@ class DataStreamer extends Daemon {
           }
         }
 
-        if (DFSClient.LOG.isDebugEnabled()) {
-          DFSClient.LOG.debug("DataStreamer block " + block +
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("DataStreamer block " + block +
               " sending packet " + one);
         }
 
@@ -534,16 +559,12 @@ class DataStreamer extends Daemon {
           // Since their messages are descriptive enough, do not always
           // log a verbose stack-trace WARN for quota exceptions.
           if (e instanceof QuotaExceededException) {
-            DFSClient.LOG.debug("DataStreamer Quota Exception", e);
+            LOG.debug("DataStreamer Quota Exception", e);
           } else {
-            DFSClient.LOG.warn("DataStreamer Exception", e);
+            LOG.warn("DataStreamer Exception", e);
           }
         }
-        if (e instanceof IOException) {
-          setLastException((IOException)e);
-        } else {
-          setLastException(new IOException("DataStreamer Exception: ",e));
-        }
+        lastException.set(e);
         hasError = true;
         if (errorIndex == -1 && restartingNodeIndex.get() == -1) {
           // Not a datanode issue
@@ -586,8 +607,8 @@ class DataStreamer extends Daemon {
   void waitForAckedSeqno(long seqno) throws IOException {
     TraceScope scope = Trace.startSpan("waitForAckedSeqno", Sampler.NEVER);
     try {
-      if (DFSClient.LOG.isDebugEnabled()) {
-        DFSClient.LOG.debug("Waiting for ack for: " + seqno);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Waiting for ack for: " + seqno);
       }
       long begin = Time.monotonicNow();
       try {
@@ -611,7 +632,7 @@ class DataStreamer extends Daemon {
       }
       long duration = Time.monotonicNow() - begin;
       if (duration > dfsclientSlowLogThresholdMs) {
-        DFSClient.LOG.warn("Slow waitForAckedSeqno took " + duration
+        LOG.warn("Slow waitForAckedSeqno took " + duration
             + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms)");
       }
     } finally {
@@ -688,8 +709,7 @@ class DataStreamer extends Daemon {
 
   private void checkClosed() throws IOException {
     if (streamerClosed) {
-      IOException e = lastException.get();
-      throw e != null ? e : new ClosedChannelException();
+      lastException.throwException4Close();
     }
   }
 
@@ -699,7 +719,7 @@ class DataStreamer extends Daemon {
         response.close();
         response.join();
       } catch (InterruptedException  e) {
-        DFSClient.LOG.warn("Caught exception ", e);
+        LOG.warn("Caught exception", e);
       } finally {
         response = null;
       }
@@ -707,11 +727,13 @@ class DataStreamer extends Daemon {
   }
 
   private void closeStream() {
+    final MultipleIOException.Builder b = new MultipleIOException.Builder();
+
     if (blockStream != null) {
       try {
         blockStream.close();
       } catch (IOException e) {
-        setLastException(e);
+        b.add(e);
       } finally {
         blockStream = null;
       }
@@ -720,7 +742,7 @@ class DataStreamer extends Daemon {
       try {
         blockReplyStream.close();
       } catch (IOException e) {
-        setLastException(e);
+        b.add(e);
       } finally {
         blockReplyStream = null;
       }
@@ -729,11 +751,16 @@ class DataStreamer extends Daemon {
       try {
         s.close();
       } catch (IOException e) {
-        setLastException(e);
+        b.add(e);
       } finally {
         s = null;
       }
     }
+
+    final IOException ioe = b.build();
+    if (ioe != null) {
+      lastException.set(ioe);
+    }
   }
 
   // The following synchronized methods are used whenever
@@ -825,12 +852,11 @@ class DataStreamer extends Daemon {
           long duration = Time.monotonicNow() - begin;
           if (duration > dfsclientSlowLogThresholdMs
               && ack.getSeqno() != DFSPacket.HEART_BEAT_SEQNO) {
-            DFSClient.LOG
-                .warn("Slow ReadProcessor read fields took " + duration
-                    + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms); ack: "
-                    + ack + ", targets: " + Arrays.asList(targets));
-          } else if (DFSClient.LOG.isDebugEnabled()) {
-            DFSClient.LOG.debug("DFSClient " + ack);
+            LOG.warn("Slow ReadProcessor read fields took " + duration
+                + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms); ack: "
+                + ack + ", targets: " + Arrays.asList(targets));
+          } else if (LOG.isDebugEnabled()) {
+            LOG.debug("DFSClient " + ack);
           }
 
           long seqno = ack.getSeqno();
@@ -851,7 +877,7 @@ class DataStreamer extends Daemon {
                   + Time.monotonicNow();
               setRestartingNodeIndex(i);
               String message = "A datanode is restarting: " + targets[i];
-              DFSClient.LOG.info(message);
+              LOG.info(message);
               throw new IOException(message);
             }
             // node error
@@ -917,9 +943,7 @@ class DataStreamer extends Daemon {
           }
         } catch (Exception e) {
           if (!responderClosed) {
-            if (e instanceof IOException) {
-              setLastException((IOException)e);
-            }
+            lastException.set(e);
             hasError = true;
             // If no explicit error report was received, mark the primary
             // node as failed.
@@ -928,8 +952,7 @@ class DataStreamer extends Daemon {
               dataQueue.notifyAll();
             }
             if (restartingNodeIndex.get() == -1) {
-              DFSClient.LOG.warn("DataStreamer ResponseProcessor exception "
-                  + " for block " + block, e);
+              LOG.warn("Exception for " + block, e);
             }
             responderClosed = true;
           }
@@ -951,7 +974,7 @@ class DataStreamer extends Daemon {
   //
   private boolean processDatanodeError() throws IOException {
     if (response != null) {
-      DFSClient.LOG.info("Error Recovery for " + block +
+      LOG.info("Error Recovery for " + block +
           " waiting for responder to exit. ");
       return true;
     }
@@ -972,7 +995,7 @@ class DataStreamer extends Daemon {
       // same packet, this client likely has corrupt data or corrupting
       // during transmission.
       if (++pipelineRecoveryCount > 5) {
-        DFSClient.LOG.warn("Error recovering pipeline for writing " +
+        LOG.warn("Error recovering pipeline for writing " +
             block + ". Already retried 5 times for the same packet.");
         lastException.set(new IOException("Failing write. Tried pipeline " +
             "recovery 5 times without success."));
@@ -1147,8 +1170,8 @@ class DataStreamer extends Daemon {
     if (nodes == null || nodes.length == 0) {
       String msg = "Could not get block locations. " + "Source file \""
           + src + "\" - Aborting...";
-      DFSClient.LOG.warn(msg);
-      setLastException(new IOException(msg));
+      LOG.warn(msg);
+      lastException.set(new IOException(msg));
       streamerClosed = true;
       return false;
     }
@@ -1193,7 +1216,7 @@ class DataStreamer extends Daemon {
           streamerClosed = true;
           return false;
         }
-        DFSClient.LOG.warn("Error Recovery for block " + block +
+        LOG.warn("Error Recovery for block " + block +
             " in pipeline " + pipelineMsg +
             ": bad datanode " + nodes[errorIndex]);
         failed.add(nodes[errorIndex]);
@@ -1227,7 +1250,7 @@ class DataStreamer extends Daemon {
         if (restartingNodeIndex.get() == -1) {
           hasError = false;
         }
-        lastException.set(null);
+        lastException.clear();
         errorIndex = -1;
       }
 
@@ -1240,7 +1263,7 @@ class DataStreamer extends Daemon {
           if (!dfsClient.dtpReplaceDatanodeOnFailure.isBestEffort()) {
             throw ioe;
           }
-          DFSClient.LOG.warn("Failed to replace datanode."
+          LOG.warn("Failed to replace datanode."
               + " Continue with the remaining datanodes since "
               + HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY
               + " is set to true.", ioe);
@@ -1281,7 +1304,7 @@ class DataStreamer extends Daemon {
         restartDeadline = 0;
         int expiredNodeIndex = restartingNodeIndex.get();
         restartingNodeIndex.set(-1);
-        DFSClient.LOG.warn("Datanode did not restart in time: " +
+        LOG.warn("Datanode did not restart in time: " +
             nodes[expiredNodeIndex]);
         // Mark the restarting node as failed. If there is any other failed
         // node during the last pipeline construction attempt, it will not be
@@ -1321,7 +1344,7 @@ class DataStreamer extends Daemon {
     ExtendedBlock oldBlock = block;
     do {
       hasError = false;
-      lastException.set(null);
+      lastException.clear();
       errorIndex = -1;
       success = false;
 
@@ -1344,11 +1367,11 @@ class DataStreamer extends Daemon {
       success = createBlockOutputStream(nodes, storageTypes, 0L, false);
 
       if (!success) {
-        DFSClient.LOG.info("Abandoning " + block);
+        LOG.info("Abandoning " + block);
         dfsClient.namenode.abandonBlock(block, stat.getFileId(), src,
             dfsClient.clientName);
         block = null;
-        DFSClient.LOG.info("Excluding datanode " + nodes[errorIndex]);
+        LOG.info("Excluding datanode " + nodes[errorIndex]);
         excludedNodes.put(nodes[errorIndex], nodes[errorIndex]);
       }
     } while (!success && --count >= 0);
@@ -1365,17 +1388,14 @@ class DataStreamer extends Daemon {
   private boolean createBlockOutputStream(DatanodeInfo[] nodes,
       StorageType[] nodeStorageTypes, long newGS, boolean recoveryFlag) {
     if (nodes.length == 0) {
-      DFSClient.LOG.info("nodes are empty for write pipeline of block "
-          + block);
+      LOG.info("nodes are empty for write pipeline of " + block);
       return false;
     }
     Status pipelineStatus = SUCCESS;
     String firstBadLink = "";
     boolean checkRestart = false;
-    if (DFSClient.LOG.isDebugEnabled()) {
-      for (int i = 0; i < nodes.length; i++) {
-        DFSClient.LOG.debug("pipeline = " + nodes[i]);
-      }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("pipeline = " + Arrays.asList(nodes));
     }
 
     // persist blocks on namenode on next flush
@@ -1447,10 +1467,10 @@ class DataStreamer extends Daemon {
         hasError = false;
       } catch (IOException ie) {
         if (restartingNodeIndex.get() == -1) {
-          DFSClient.LOG.info("Exception in createBlockOutputStream", ie);
+          LOG.info("Exception in createBlockOutputStream", ie);
         }
         if (ie instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
-          DFSClient.LOG.info("Will fetch a new encryption key and retry, "
+          LOG.info("Will fetch a new encryption key and retry, "
               + "encryption key was invalid when connecting to "
               + nodes[0] + " : " + ie);
           // The encryption key used is invalid.
@@ -1480,11 +1500,11 @@ class DataStreamer extends Daemon {
               + Time.monotonicNow();
           restartingNodeIndex.set(errorIndex);
           errorIndex = -1;
-          DFSClient.LOG.info("Waiting for the datanode to be restarted: " +
+          LOG.info("Waiting for the datanode to be restarted: " +
               nodes[restartingNodeIndex.get()]);
         }
         hasError = true;
-        setLastException(ie);
+        lastException.set(ie);
         result =  false;  // error
       } finally {
         if (!result) {
@@ -1509,18 +1529,16 @@ class DataStreamer extends Daemon {
           new HashSet<String>(Arrays.asList(favoredNodes));
       for (int i = 0; i < nodes.length; i++) {
         pinnings[i] = favoredSet.remove(nodes[i].getXferAddrWithHostname());
-        if (DFSClient.LOG.isDebugEnabled()) {
-          DFSClient.LOG.debug(nodes[i].getXferAddrWithHostname() +
-              " was chosen by name node (favored=" + pinnings[i] +
-              ").");
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(nodes[i].getXferAddrWithHostname() +
+              " was chosen by name node (favored=" + pinnings[i] + ").");
         }
       }
       if (shouldLog && !favoredSet.isEmpty()) {
         // There is one or more favored nodes that were not allocated.
-        DFSClient.LOG.warn(
-            "These favored nodes were specified but not chosen: " +
-                favoredSet +
-                " Specified favored nodes: " + Arrays.toString(favoredNodes));
+        LOG.warn("These favored nodes were specified but not chosen: "
+            + favoredSet + " Specified favored nodes: "
+            + Arrays.toString(favoredNodes));
 
       }
       return pinnings;
@@ -1557,19 +1575,19 @@ class DataStreamer extends Daemon {
               throw e;
             } else {
               --retries;
-              DFSClient.LOG.info("Exception while adding a block", e);
+              LOG.info("Exception while adding a block", e);
               long elapsed = Time.monotonicNow() - localstart;
               if (elapsed > 5000) {
-                DFSClient.LOG.info("Waiting for replication for "
+                LOG.info("Waiting for replication for "
                     + (elapsed / 1000) + " seconds");
               }
               try {
-                DFSClient.LOG.warn("NotReplicatedYetException sleeping " + src
+                LOG.warn("NotReplicatedYetException sleeping " + src
                     + " retries left " + retries);
                 Thread.sleep(sleeptime);
                 sleeptime *= 2;
               } catch (InterruptedException ie) {
-                DFSClient.LOG.warn("Caught exception ", ie);
+                LOG.warn("Caught exception", ie);
               }
             }
           } else {
@@ -1606,7 +1624,7 @@ class DataStreamer extends Daemon {
                      (int)(base + Math.random() * range));
         lastCongestionBackoffTime = t;
         sb.append(" are congested. Backing off for ").append(t).append(" ms");
-        DFSClient.LOG.info(sb.toString());
+        LOG.info(sb.toString());
         congestedNodes.clear();
       }
     }
@@ -1643,15 +1661,6 @@ class DataStreamer extends Daemon {
   }
 
   /**
-   * set last exception
-   *
-   * @param e an exception
-   */
-  void setLastException(IOException e) {
-    lastException.compareAndSet(null, e);
-  }
-
-  /**
    * Put a packet to the data queue
    *
    * @param packet the packet to be put into the data queued
@@ -1662,8 +1671,8 @@ class DataStreamer extends Daemon {
       packet.addTraceParent(Trace.currentSpan());
       dataQueue.addLast(packet);
       lastQueuedSeqno = packet.getSeqno();
-      if (DFSClient.LOG.isDebugEnabled()) {
-        DFSClient.LOG.debug("Queued packet " + packet.getSeqno());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Queued packet " + packet.getSeqno());
       }
       dataQueue.notifyAll();
     }
@@ -1686,7 +1695,7 @@ class DataStreamer extends Daemon {
           @Override
           public void onRemoval(
               RemovalNotification<DatanodeInfo, DatanodeInfo> notification) {
-            DFSClient.LOG.info("Removing node " + notification.getKey()
+            LOG.info("Removing node " + notification.getKey()
                 + " from the excluded nodes list");
           }
         }).build(new CacheLoader<DatanodeInfo, DatanodeInfo>() {
@@ -1730,11 +1739,9 @@ class DataStreamer extends Daemon {
   }
 
   /**
-   * get the last exception
-   *
    * @return the last exception
    */
-  AtomicReference<IOException> getLastException(){
+  LastException getLastException(){
     return lastException;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b5227c94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
index 478f7e5..eac1fcd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
@@ -62,7 +62,6 @@ public class TestDFSOutputStream {
     FSDataOutputStream os = fs.create(new Path("/test"));
     DFSOutputStream dos = (DFSOutputStream) Whitebox.getInternalState(os,
         "wrappedStream");
-    @SuppressWarnings("unchecked")
     DataStreamer streamer = (DataStreamer) Whitebox
         .getInternalState(dos, "streamer");
     @SuppressWarnings("unchecked")
@@ -122,7 +121,7 @@ public class TestDFSOutputStream {
         mock(HdfsFileStatus.class),
         mock(ExtendedBlock.class),
         client,
-        "foo", null, null, null, null);
+        "foo", null, null, null, null, null);
 
     DataOutputStream blockStream = mock(DataOutputStream.class);
     doThrow(new IOException()).when(blockStream).flush();


[09/50] [abbrv] hadoop git commit: HADOOP-11821. Fix findbugs warnings in hadoop-sls. Contributed by Brahma Reddy Battula.

Posted by zj...@apache.org.
HADOOP-11821. Fix findbugs warnings in hadoop-sls. Contributed by Brahma Reddy Battula.


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

Branch: refs/heads/YARN-2928
Commit: 5e30ec96af7dba894429f2d70b16c774584677c1
Parents: 69f6468
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Apr 30 19:34:41 2015 +0900
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:54 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   3 +
 .../hadoop/yarn/sls/RumenToSLSConverter.java    |  47 ++++----
 .../org/apache/hadoop/yarn/sls/SLSRunner.java   |  41 +++----
 .../sls/scheduler/ResourceSchedulerWrapper.java |  13 ++-
 .../sls/scheduler/SLSCapacityScheduler.java     | 112 ++++++++-----------
 .../apache/hadoop/yarn/sls/utils/SLSUtils.java  |  27 +++--
 .../apache/hadoop/yarn/sls/web/SLSWebApp.java   |  31 +++--
 7 files changed, 132 insertions(+), 142 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e30ec96/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 39d14c9..373b33e 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -579,6 +579,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11881. test-patch.sh javac result is wildly wrong (Kengo Seki via
     aw)
 
+    HADOOP-11821. Fix findbugs warnings in hadoop-sls.
+    (Brahma Reddy Battula via aajisaka)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e30ec96/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RumenToSLSConverter.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RumenToSLSConverter.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RumenToSLSConverter.java
index 2d4b4ae..63d022b 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RumenToSLSConverter.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RumenToSLSConverter.java
@@ -17,22 +17,12 @@
  */
 package org.apache.hadoop.yarn.sls;
 
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.CommandLineParser;
-import org.apache.commons.cli.GnuParser;
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.codehaus.jackson.JsonFactory;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.map.ObjectWriter;
-
-import org.apache.hadoop.yarn.sls.utils.SLSUtils;
-
 import java.io.File;
-import java.io.FileReader;
-import java.io.FileWriter;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
 import java.io.Reader;
 import java.io.Writer;
 import java.util.ArrayList;
@@ -44,6 +34,17 @@ import java.util.Set;
 import java.util.TreeMap;
 import java.util.TreeSet;
 
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.sls.utils.SLSUtils;
+import org.codehaus.jackson.JsonFactory;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.map.ObjectWriter;
+
 @Private
 @Unstable
 public class RumenToSLSConverter {
@@ -119,10 +120,10 @@ public class RumenToSLSConverter {
 
   private static void generateSLSLoadFile(String inputFile, String outputFile)
           throws IOException {
-    Reader input = new FileReader(inputFile);
-    try {
-      Writer output = new FileWriter(outputFile);
-      try {
+    try (Reader input =
+        new InputStreamReader(new FileInputStream(inputFile), "UTF-8")) {
+      try (Writer output =
+          new OutputStreamWriter(new FileOutputStream(outputFile), "UTF-8")) {
         ObjectMapper mapper = new ObjectMapper();
         ObjectWriter writer = mapper.writerWithDefaultPrettyPrinter();
         Iterator<Map> i = mapper.readValues(
@@ -131,19 +132,15 @@ public class RumenToSLSConverter {
           Map m = i.next();
           output.write(writer.writeValueAsString(createSLSJob(m)) + EOL);
         }
-      } finally {
-        output.close();
       }
-    } finally {
-      input.close();
     }
   }
 
   @SuppressWarnings("unchecked")
   private static void generateSLSNodeFile(String outputFile)
           throws IOException {
-    Writer output = new FileWriter(outputFile);
-    try {
+    try (Writer output =
+        new OutputStreamWriter(new FileOutputStream(outputFile), "UTF-8")) {
       ObjectMapper mapper = new ObjectMapper();
       ObjectWriter writer = mapper.writerWithDefaultPrettyPrinter();
       for (Map.Entry<String, Set<String>> entry : rackNodeMap.entrySet()) {
@@ -158,8 +155,6 @@ public class RumenToSLSConverter {
         rack.put("nodes", nodes);
         output.write(writer.writeValueAsString(rack) + EOL);
       }
-    } finally {
-      output.close();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e30ec96/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java
index 9baa736..b36edc9 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java
@@ -18,51 +18,51 @@
 package org.apache.hadoop.yarn.sls;
 
 import java.io.File;
-import java.io.FileReader;
+import java.io.FileInputStream;
 import java.io.IOException;
+import java.io.InputStreamReader;
 import java.io.Reader;
 import java.text.MessageFormat;
-import java.util.Map;
+import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
-import java.util.Set;
 import java.util.HashSet;
-import java.util.List;
-import java.util.ArrayList;
 import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
 import java.util.Random;
-import java.util.Arrays;
+import java.util.Set;
 
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.Options;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.tools.rumen.JobTraceReader;
 import org.apache.hadoop.tools.rumen.LoggedJob;
 import org.apache.hadoop.tools.rumen.LoggedTask;
 import org.apache.hadoop.tools.rumen.LoggedTaskAttempt;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.sls.appmaster.AMSimulator;
 import org.apache.hadoop.yarn.sls.conf.SLSConfiguration;
 import org.apache.hadoop.yarn.sls.nodemanager.NMSimulator;
 import org.apache.hadoop.yarn.sls.scheduler.ContainerSimulator;
 import org.apache.hadoop.yarn.sls.scheduler.ResourceSchedulerWrapper;
 import org.apache.hadoop.yarn.sls.scheduler.SLSCapacityScheduler;
+import org.apache.hadoop.yarn.sls.scheduler.SchedulerWrapper;
 import org.apache.hadoop.yarn.sls.scheduler.TaskRunner;
-import  org.apache.hadoop.yarn.sls.scheduler.SchedulerWrapper;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.CommandLineParser;
-import org.apache.commons.cli.GnuParser;
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
-import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.sls.utils.SLSUtils;
 import org.apache.log4j.Logger;
 import org.codehaus.jackson.JsonFactory;
@@ -277,7 +277,8 @@ public class SLSRunner {
     JsonFactory jsonF = new JsonFactory();
     ObjectMapper mapper = new ObjectMapper();
     for (String inputTrace : inputTraces) {
-      Reader input = new FileReader(inputTrace);
+      Reader input =
+          new InputStreamReader(new FileInputStream(inputTrace), "UTF-8");
       try {
         Iterator<Map> i = mapper.readValues(jsonF.createJsonParser(input),
                 Map.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e30ec96/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java
index 0a80291..08cb1e6 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java
@@ -19,8 +19,9 @@ package org.apache.hadoop.yarn.sls.scheduler;
 
 import java.io.BufferedWriter;
 import java.io.File;
-import java.io.FileWriter;
+import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.OutputStreamWriter;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -487,8 +488,9 @@ final public class ResourceSchedulerWrapper
             TimeUnit.MILLISECONDS);
 
     // application running information
-    jobRuntimeLogBW = new BufferedWriter(
-            new FileWriter(metricsOutputDir + "/jobruntime.csv"));
+    jobRuntimeLogBW =
+        new BufferedWriter(new OutputStreamWriter(new FileOutputStream(
+            metricsOutputDir + "/jobruntime.csv"), "UTF-8"));
     jobRuntimeLogBW.write("JobID,real_start_time,real_end_time," +
             "simulate_start_time,simulate_end_time" + EOL);
     jobRuntimeLogBW.flush();
@@ -692,8 +694,9 @@ final public class ResourceSchedulerWrapper
     private boolean firstLine = true;
     public MetricsLogRunnable() {
       try {
-        metricsLogBW = new BufferedWriter(
-                new FileWriter(metricsOutputDir + "/realtimetrack.json"));
+        metricsLogBW =
+            new BufferedWriter(new OutputStreamWriter(new FileOutputStream(
+                metricsOutputDir + "/realtimetrack.json"), "UTF-8"));
         metricsLogBW.write("[");
       } catch (IOException e) {
         e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e30ec96/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java
index 06addfb..a4416db 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java
@@ -17,90 +17,66 @@
  */
 package org.apache.hadoop.yarn.sls.scheduler;
 
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.util.ShutdownHookManager;
-import org.apache.hadoop.yarn.sls.SLSRunner;
-import org.apache.hadoop.yarn.sls.conf.SLSConfiguration;
-import org.apache.hadoop.yarn.sls.web.SLSWebApp;
-import com.codahale.metrics.Counter;
-import com.codahale.metrics.CsvReporter;
-import com.codahale.metrics.Gauge;
-import com.codahale.metrics.Histogram;
-import com.codahale.metrics.MetricRegistry;
-import com.codahale.metrics.SlidingWindowReservoir;
-import com.codahale.metrics.Timer;
-
-import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 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.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.QueueInfo;
-import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
-import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
-import org.apache.hadoop.yarn.server.resourcemanager.rmnode
-        .UpdatedContainerInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler
-        .ResourceScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler
-        .SchedulerAppReport;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler
-        .SchedulerNodeReport;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
-        .CapacityScheduler;
-
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
 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.resourcemanager.scheduler.event
-        .SchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event
-        .SchedulerEventType;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair
-        .FairScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo
-        .FifoScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
+import org.apache.hadoop.yarn.sls.SLSRunner;
+import org.apache.hadoop.yarn.sls.conf.SLSConfiguration;
+import org.apache.hadoop.yarn.sls.web.SLSWebApp;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.log4j.Logger;
 
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.CsvReporter;
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.SlidingWindowReservoir;
+import com.codahale.metrics.Timer;
 
 @Private
 @Unstable
@@ -490,8 +466,9 @@ public class SLSCapacityScheduler extends CapacityScheduler implements
             TimeUnit.MILLISECONDS);
 
     // application running information
-    jobRuntimeLogBW = new BufferedWriter(
-            new FileWriter(metricsOutputDir + "/jobruntime.csv"));
+    jobRuntimeLogBW =
+        new BufferedWriter(new OutputStreamWriter(new FileOutputStream(
+            metricsOutputDir + "/jobruntime.csv"), "UTF-8"));
     jobRuntimeLogBW.write("JobID,real_start_time,real_end_time," +
             "simulate_start_time,simulate_end_time" + EOL);
     jobRuntimeLogBW.flush();
@@ -695,8 +672,9 @@ public class SLSCapacityScheduler extends CapacityScheduler implements
     private boolean firstLine = true;
     public MetricsLogRunnable() {
       try {
-        metricsLogBW = new BufferedWriter(
-                new FileWriter(metricsOutputDir + "/realtimetrack.json"));
+        metricsLogBW =
+            new BufferedWriter(new OutputStreamWriter(new FileOutputStream(
+                metricsOutputDir + "/realtimetrack.json"), "UTF-8"));
         metricsLogBW.write("[");
       } catch (IOException e) {
         e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e30ec96/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java
index f62f024..922370c 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java
@@ -17,6 +17,17 @@
  */
 package org.apache.hadoop.yarn.sls.utils;
 
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -28,16 +39,6 @@ import org.apache.hadoop.tools.rumen.LoggedTaskAttempt;
 import org.codehaus.jackson.JsonFactory;
 import org.codehaus.jackson.map.ObjectMapper;
 
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.io.Reader;
-import java.util.Set;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.List;
-import java.util.Iterator;
-
 @Private
 @Unstable
 public class SLSUtils {
@@ -89,7 +90,8 @@ public class SLSUtils {
     Set<String> nodeSet = new HashSet<String>();
     JsonFactory jsonF = new JsonFactory();
     ObjectMapper mapper = new ObjectMapper();
-    Reader input = new FileReader(jobTrace);
+    Reader input =
+        new InputStreamReader(new FileInputStream(jobTrace), "UTF-8");
     try {
       Iterator<Map> i = mapper.readValues(
               jsonF.createJsonParser(input), Map.class);
@@ -116,7 +118,8 @@ public class SLSUtils {
     Set<String> nodeSet = new HashSet<String>();
     JsonFactory jsonF = new JsonFactory();
     ObjectMapper mapper = new ObjectMapper();
-    Reader input = new FileReader(nodeFile);
+    Reader input =
+        new InputStreamReader(new FileInputStream(nodeFile), "UTF-8");
     try {
       Iterator<Map> i = mapper.readValues(
               jsonF.createJsonParser(input), Map.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e30ec96/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/web/SLSWebApp.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/web/SLSWebApp.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/web/SLSWebApp.java
index 45301a1..e152696 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/web/SLSWebApp.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/web/SLSWebApp.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.sls.web;
 
 import java.io.File;
 import java.io.IOException;
+import java.io.ObjectInputStream;
 import java.text.MessageFormat;
 import java.util.HashMap;
 import java.util.Map;
@@ -32,24 +33,21 @@ import javax.servlet.http.HttpServletResponse;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event
-        .SchedulerEventType;
-import org.mortbay.jetty.Handler;
-import org.mortbay.jetty.Server;
-import org.mortbay.jetty.handler.AbstractHandler;
-import org.mortbay.jetty.Request;
-
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
 import org.apache.hadoop.yarn.sls.SLSRunner;
 import org.apache.hadoop.yarn.sls.scheduler.FairSchedulerMetrics;
-import org.apache.hadoop.yarn.sls.scheduler.ResourceSchedulerWrapper;
 import org.apache.hadoop.yarn.sls.scheduler.SchedulerMetrics;
 import org.apache.hadoop.yarn.sls.scheduler.SchedulerWrapper;
+import org.mortbay.jetty.Handler;
+import org.mortbay.jetty.Request;
+import org.mortbay.jetty.Server;
+import org.mortbay.jetty.handler.AbstractHandler;
+import org.mortbay.jetty.handler.ResourceHandler;
 
 import com.codahale.metrics.Counter;
 import com.codahale.metrics.Gauge;
 import com.codahale.metrics.Histogram;
 import com.codahale.metrics.MetricRegistry;
-import org.mortbay.jetty.handler.ResourceHandler;
 
 @Private
 @Unstable
@@ -71,9 +69,10 @@ public class SLSWebApp extends HttpServlet {
   private transient Gauge availableVCoresGauge;
   private transient Histogram allocateTimecostHistogram;
   private transient Histogram handleTimecostHistogram;
-  private Map<SchedulerEventType, Histogram> handleOperTimecostHistogramMap;
-  private Map<String, Counter> queueAllocatedMemoryCounterMap;
-  private Map<String, Counter> queueAllocatedVCoresCounterMap;
+  private transient Map<SchedulerEventType, Histogram>
+     handleOperTimecostHistogramMap;
+  private transient Map<String, Counter> queueAllocatedMemoryCounterMap;
+  private transient Map<String, Counter> queueAllocatedVCoresCounterMap;
   private int port;
   private int ajaxUpdateTimeMS = 1000;
   // html page templates
@@ -96,6 +95,14 @@ public class SLSWebApp extends HttpServlet {
     }
   }
 
+  private void readObject(ObjectInputStream in) throws IOException,
+      ClassNotFoundException {
+    in.defaultReadObject();
+    handleOperTimecostHistogramMap = new HashMap<>();
+    queueAllocatedMemoryCounterMap = new HashMap<>();
+    queueAllocatedVCoresCounterMap = new HashMap<>();
+  }
+
   public SLSWebApp(SchedulerWrapper wrapper, int metricsAddressPort) {
     this.wrapper = wrapper;
     metrics = wrapper.getMetrics();


[32/50] [abbrv] hadoop git commit: YARN-3006. Improve the error message when attempting manual failover with auto-failover enabled. (Akira AJISAKA via wangda)

Posted by zj...@apache.org.
YARN-3006. Improve the error message when attempting manual failover with auto-failover enabled. (Akira AJISAKA via wangda)


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

Branch: refs/heads/YARN-2928
Commit: 3660b7f9a4b4f04d84366244410c58155d49452a
Parents: c1c22df
Author: Wangda Tan <wa...@apache.org>
Authored: Fri May 1 16:40:44 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:58 2015 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/ha/HAAdmin.java          | 8 +++++++-
 hadoop-yarn-project/CHANGES.txt                              | 3 +++
 2 files changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3660b7f9/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java
index 9c28eb9..aa19ee2 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java
@@ -279,7 +279,13 @@ public abstract class HAAdmin extends Configured implements Tool {
             "supported with auto-failover enabled.");
         return -1;
       }
-      return gracefulFailoverThroughZKFCs(toNode);
+      try {
+        return gracefulFailoverThroughZKFCs(toNode);
+      } catch (UnsupportedOperationException e){
+        errOut.println("Failover command is not supported with " +
+            "auto-failover enabled: " + e.getLocalizedMessage());
+        return -1;
+      }
     }
     
     FailoverController fc = new FailoverController(getConf(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3660b7f9/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index a66024a..6c1a2e1 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -253,6 +253,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3469. ZKRMStateStore: Avoid setting watches that are not required. 
     (Jun Gong via kasha)
 
+    YARN-3006. Improve the error message when attempting manual failover with 
+    auto-failover enabled. (Akira AJISAKA via wangda)
+
   BUG FIXES
 
     YARN-3197. Confusing log generated by CapacityScheduler. (Varun Saxena 


[29/50] [abbrv] hadoop git commit: HADOOP-11491. HarFs incorrectly declared as requiring an authority. (Brahma Reddy Battula via gera)

Posted by zj...@apache.org.
HADOOP-11491. HarFs incorrectly declared as requiring an authority. (Brahma Reddy Battula via gera)


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

Branch: refs/heads/YARN-2928
Commit: ad8c70a98e693c73bfdc3530a155a777d1ab2b48
Parents: 9b14ff8
Author: Gera Shegalov <ge...@apache.org>
Authored: Fri May 1 15:44:36 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:58 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt               | 3 +++
 .../src/main/java/org/apache/hadoop/fs/HarFs.java             | 2 +-
 .../java/org/apache/hadoop/fs/TestHarFileSystemBasics.java    | 7 +++++++
 3 files changed, 11 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad8c70a9/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 0a53396..d00e3ef 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -585,6 +585,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11866. increase readability and reliability of checkstyle,
     shellcheck, and whitespace reports (aw)
 
+    HADOOP-11491. HarFs incorrectly declared as requiring an authority.
+    (Brahma Reddy Battula via gera)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad8c70a9/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFs.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFs.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFs.java
index a2369e3..4f5fde8 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFs.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFs.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.conf.Configuration;
 public class HarFs extends DelegateToFileSystem {
   HarFs(final URI theUri, final Configuration conf)
       throws IOException, URISyntaxException {
-    super(theUri, new HarFileSystem(), conf, "har", true);
+    super(theUri, new HarFileSystem(), conf, "har", false);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad8c70a9/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystemBasics.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystemBasics.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystemBasics.java
index 577abfd..53507b9 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystemBasics.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystemBasics.java
@@ -398,4 +398,11 @@ public class TestHarFileSystemBasics {
     }
   }
 
+  @Test
+  public void testHarFsWithoutAuthority() throws Exception {
+    final URI uri = harFileSystem.getUri();
+    Assert.assertNull("har uri authority not null: " + uri, uri.getAuthority());
+    FileContext.getFileContext(uri, conf);
+  }
+
 }


[08/50] [abbrv] hadoop git commit: YARN-3485. FairScheduler headroom calculation doesn't consider maxResources for Fifo and FairShare policies. (kasha)

Posted by zj...@apache.org.
YARN-3485. FairScheduler headroom calculation doesn't consider maxResources for Fifo and FairShare policies. (kasha)


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

Branch: refs/heads/YARN-2928
Commit: 19dd6d93513d77e8be3b56cc7f6ba6774a2a6915
Parents: a742c10
Author: Karthik Kambatla <ka...@apache.org>
Authored: Tue Apr 28 21:00:35 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:53 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../scheduler/fair/FSAppAttempt.java            | 20 ++++--
 .../scheduler/fair/SchedulingPolicy.java        |  4 +-
 .../DominantResourceFairnessPolicy.java         |  6 +-
 .../fair/policies/FairSharePolicy.java          |  6 +-
 .../scheduler/fair/policies/FifoPolicy.java     |  6 +-
 .../scheduler/fair/TestFSAppAttempt.java        | 66 ++++++++++++++------
 7 files changed, 76 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/19dd6d93/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 0f81fbc..7d90477 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -372,6 +372,9 @@ Release 2.7.1 - UNRELEASED
 
     YARN-3464. Race condition in LocalizerRunner kills localizer before 
     localizing all resources. (Zhihai Xu via kasha)
+    
+    YARN-3485. FairScheduler headroom calculation doesn't consider 
+    maxResources for Fifo and FairShare policies. (kasha)
 
 Release 2.7.0 - 2015-04-20
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19dd6d93/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.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/fair/FSAppAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
index f0d1ed1..6287deb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
@@ -172,6 +172,10 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
         + this.attemptResourceUsage.getReserved());
   }
 
+  /**
+   * Headroom depends on resources in the cluster, current usage of the
+   * queue, queue's fair-share and queue's max-resources.
+   */
   @Override
   public Resource getHeadroom() {
     final FSQueue queue = (FSQueue) this.queue;
@@ -182,18 +186,22 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
     Resource clusterResource = this.scheduler.getClusterResource();
     Resource clusterUsage = this.scheduler.getRootQueueMetrics()
         .getAllocatedResources();
-    Resource clusterAvailableResource = Resources.subtract(clusterResource,
-        clusterUsage);
+
+    Resource clusterAvailableResources =
+        Resources.subtract(clusterResource, clusterUsage);
+    Resource queueMaxAvailableResources =
+        Resources.subtract(queue.getMaxShare(), queueUsage);
+    Resource maxAvailableResource = Resources.componentwiseMin(
+        clusterAvailableResources, queueMaxAvailableResources);
+
     Resource headroom = policy.getHeadroom(queueFairShare,
-        queueUsage, clusterAvailableResource);
+        queueUsage, maxAvailableResource);
     if (LOG.isDebugEnabled()) {
       LOG.debug("Headroom calculation for " + this.getName() + ":" +
           "Min(" +
           "(queueFairShare=" + queueFairShare +
           " - queueUsage=" + queueUsage + ")," +
-          " clusterAvailableResource=" + clusterAvailableResource +
-          "(clusterResource=" + clusterResource +
-          " - clusterUsage=" + clusterUsage + ")" +
+          " maxAvailableResource=" + maxAvailableResource +
           "Headroom=" + headroom);
     }
     return headroom;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19dd6d93/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.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/fair/SchedulingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java
index bf2a25b..abdc834 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java
@@ -185,10 +185,10 @@ public abstract class SchedulingPolicy {
    *
    * @param queueFairShare fairshare in the queue
    * @param queueUsage resources used in the queue
-   * @param clusterAvailable available resource in cluster
+   * @param maxAvailable available resource in cluster for this queue
    * @return calculated headroom
    */
   public abstract Resource getHeadroom(Resource queueFairShare,
-      Resource queueUsage, Resource clusterAvailable);
+      Resource queueUsage, Resource maxAvailable);
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19dd6d93/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.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/fair/policies/DominantResourceFairnessPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
index 3f6cbd1..86d503b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
@@ -90,15 +90,15 @@ public class DominantResourceFairnessPolicy extends SchedulingPolicy {
 
   @Override
   public Resource getHeadroom(Resource queueFairShare, Resource queueUsage,
-                              Resource clusterAvailable) {
+                              Resource maxAvailable) {
     int queueAvailableMemory =
         Math.max(queueFairShare.getMemory() - queueUsage.getMemory(), 0);
     int queueAvailableCPU =
         Math.max(queueFairShare.getVirtualCores() - queueUsage
             .getVirtualCores(), 0);
     Resource headroom = Resources.createResource(
-        Math.min(clusterAvailable.getMemory(), queueAvailableMemory),
-        Math.min(clusterAvailable.getVirtualCores(),
+        Math.min(maxAvailable.getMemory(), queueAvailableMemory),
+        Math.min(maxAvailable.getVirtualCores(),
             queueAvailableCPU));
     return headroom;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19dd6d93/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.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/fair/policies/FairSharePolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
index 97669cb..918db9d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
@@ -116,12 +116,12 @@ public class FairSharePolicy extends SchedulingPolicy {
 
   @Override
   public Resource getHeadroom(Resource queueFairShare,
-                              Resource queueUsage, Resource clusterAvailable) {
+                              Resource queueUsage, Resource maxAvailable) {
     int queueAvailableMemory = Math.max(
         queueFairShare.getMemory() - queueUsage.getMemory(), 0);
     Resource headroom = Resources.createResource(
-        Math.min(clusterAvailable.getMemory(), queueAvailableMemory),
-        clusterAvailable.getVirtualCores());
+        Math.min(maxAvailable.getMemory(), queueAvailableMemory),
+        maxAvailable.getVirtualCores());
     return headroom;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19dd6d93/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FifoPolicy.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/fair/policies/FifoPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FifoPolicy.java
index a2e17ec..7d88933 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FifoPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FifoPolicy.java
@@ -109,12 +109,12 @@ public class FifoPolicy extends SchedulingPolicy {
 
   @Override
   public Resource getHeadroom(Resource queueFairShare,
-                              Resource queueUsage, Resource clusterAvailable) {
+                              Resource queueUsage, Resource maxAvailable) {
     int queueAvailableMemory = Math.max(
         queueFairShare.getMemory() - queueUsage.getMemory(), 0);
     Resource headroom = Resources.createResource(
-        Math.min(clusterAvailable.getMemory(), queueAvailableMemory),
-        clusterAvailable.getVirtualCores());
+        Math.min(maxAvailable.getMemory(), queueAvailableMemory),
+        maxAvailable.getVirtualCores());
     return headroom;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19dd6d93/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.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/fair/TestFSAppAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.java
index f560690..43fe186 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.java
@@ -198,18 +198,24 @@ public class TestFSAppAttempt extends FairSchedulerTestBase {
     Mockito.when(mockScheduler.getClock()).thenReturn(scheduler.getClock());
 
     final FSLeafQueue mockQueue = Mockito.mock(FSLeafQueue.class);
-    final Resource queueFairShare = Resources.createResource(4096, 4);
-    final Resource queueUsage = Resource.newInstance(1024, 1);
+
+    final Resource queueMaxResources = Resource.newInstance(5 * 1024, 3);
+    final Resource queueFairShare = Resources.createResource(4096, 2);
+    final Resource queueUsage = Resource.newInstance(2048, 2);
+
+    final Resource queueStarvation =
+        Resources.subtract(queueFairShare, queueUsage);
+    final Resource queueMaxResourcesAvailable =
+        Resources.subtract(queueMaxResources, queueUsage);
+
     final Resource clusterResource = Resources.createResource(8192, 8);
-    final Resource clusterUsage = Resources.createResource(6144, 2);
-    final QueueMetrics fakeRootQueueMetrics = Mockito.mock(QueueMetrics.class);
+    final Resource clusterUsage = Resources.createResource(2048, 2);
+    final Resource clusterAvailable =
+        Resources.subtract(clusterResource, clusterUsage);
 
-    ApplicationAttemptId applicationAttemptId = createAppAttemptId(1, 1);
-    RMContext rmContext = resourceManager.getRMContext();
-    FSAppAttempt schedulerApp =
-        new FSAppAttempt(mockScheduler, applicationAttemptId, "user1", mockQueue ,
-            null, rmContext);
+    final QueueMetrics fakeRootQueueMetrics = Mockito.mock(QueueMetrics.class);
 
+    Mockito.when(mockQueue.getMaxShare()).thenReturn(queueMaxResources);
     Mockito.when(mockQueue.getFairShare()).thenReturn(queueFairShare);
     Mockito.when(mockQueue.getResourceUsage()).thenReturn(queueUsage);
     Mockito.when(mockScheduler.getClusterResource()).thenReturn
@@ -219,27 +225,51 @@ public class TestFSAppAttempt extends FairSchedulerTestBase {
     Mockito.when(mockScheduler.getRootQueueMetrics()).thenReturn
         (fakeRootQueueMetrics);
 
-    int minClusterAvailableMemory = 2048;
-    int minClusterAvailableCPU = 6;
-    int minQueueAvailableCPU = 3;
+    ApplicationAttemptId applicationAttemptId = createAppAttemptId(1, 1);
+    RMContext rmContext = resourceManager.getRMContext();
+    FSAppAttempt schedulerApp =
+        new FSAppAttempt(mockScheduler, applicationAttemptId, "user1", mockQueue ,
+            null, rmContext);
 
     // Min of Memory and CPU across cluster and queue is used in
     // DominantResourceFairnessPolicy
     Mockito.when(mockQueue.getPolicy()).thenReturn(SchedulingPolicy
         .getInstance(DominantResourceFairnessPolicy.class));
-    verifyHeadroom(schedulerApp, minClusterAvailableMemory,
-        minQueueAvailableCPU);
+    verifyHeadroom(schedulerApp,
+        min(queueStarvation.getMemory(),
+            clusterAvailable.getMemory(),
+            queueMaxResourcesAvailable.getMemory()),
+        min(queueStarvation.getVirtualCores(),
+            clusterAvailable.getVirtualCores(),
+            queueMaxResourcesAvailable.getVirtualCores())
+    );
 
     // Fair and Fifo ignore CPU of queue, so use cluster available CPU
     Mockito.when(mockQueue.getPolicy()).thenReturn(SchedulingPolicy
         .getInstance(FairSharePolicy.class));
-    verifyHeadroom(schedulerApp, minClusterAvailableMemory,
-        minClusterAvailableCPU);
+    verifyHeadroom(schedulerApp,
+        min(queueStarvation.getMemory(),
+            clusterAvailable.getMemory(),
+            queueMaxResourcesAvailable.getMemory()),
+        Math.min(
+            clusterAvailable.getVirtualCores(),
+            queueMaxResourcesAvailable.getVirtualCores())
+    );
 
     Mockito.when(mockQueue.getPolicy()).thenReturn(SchedulingPolicy
         .getInstance(FifoPolicy.class));
-    verifyHeadroom(schedulerApp, minClusterAvailableMemory,
-        minClusterAvailableCPU);
+    verifyHeadroom(schedulerApp,
+        min(queueStarvation.getMemory(),
+            clusterAvailable.getMemory(),
+            queueMaxResourcesAvailable.getMemory()),
+        Math.min(
+            clusterAvailable.getVirtualCores(),
+            queueMaxResourcesAvailable.getVirtualCores())
+    );
+  }
+
+  private static int min(int value1, int value2, int value3) {
+    return Math.min(Math.min(value1, value2), value3);
   }
 
   protected void verifyHeadroom(FSAppAttempt schedulerApp,


[13/50] [abbrv] hadoop git commit: YARN-3533. Test: Fix launchAM in MockRM to wait for attempt to be scheduled. Contributed by Anubhav Dhoot

Posted by zj...@apache.org.
YARN-3533. Test: Fix launchAM in MockRM to wait for attempt to be scheduled. Contributed by Anubhav Dhoot


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

Branch: refs/heads/YARN-2928
Commit: a49bcaae56e7d5a75efb2b5dfd459e558b5c207e
Parents: 6d0351c
Author: Jian He <ji...@apache.org>
Authored: Wed Apr 29 14:50:01 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:54 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                              | 3 +++
 .../apache/hadoop/yarn/server/resourcemanager/MockRM.java    | 8 +++++++-
 2 files changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a49bcaae/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 5ea419c..10eba67 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -335,6 +335,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3517. RM web ui for dumping scheduler logs should be for admins only
     (Varun Vasudev via tgraves)
 
+    YARN-3533. Test: Fix launchAM in MockRM to wait for attempt to be scheduled.
+    (Anubhav Dhoot via jianhe)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a49bcaae/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.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/MockRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
index f2b1d86..63d6557 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
@@ -164,6 +164,8 @@ public class MockRM extends ResourceManager {
       nm.nodeHeartbeat(true);
       Thread.sleep(200);
     }
+    Assert.assertNotNull("Failed in waiting for " + containerId + " " +
+        "allocation.", getResourceScheduler().getRMContainer(containerId));
   }
 
   public void waitForContainerToComplete(RMAppAttempt attempt,
@@ -662,7 +664,7 @@ public class MockRM extends ResourceManager {
     am.waitForState(RMAppAttemptState.FINISHED);
     rm.waitForState(rmApp.getApplicationId(), RMAppState.FINISHED);
   }
-  
+
   @SuppressWarnings("rawtypes")
   private static void waitForSchedulerAppAttemptAdded(
       ApplicationAttemptId attemptId, MockRM rm) throws InterruptedException {
@@ -677,6 +679,9 @@ public class MockRM extends ResourceManager {
       }
       tick++;
     }
+    Assert.assertNotNull("Timed out waiting for SchedulerApplicationAttempt=" +
+      attemptId + " to be added.", ((AbstractYarnScheduler)
+        rm.getResourceScheduler()).getApplicationAttempt(attemptId));
   }
 
   public static MockAM launchAM(RMApp app, MockRM rm, MockNM nm)
@@ -684,6 +689,7 @@ public class MockRM extends ResourceManager {
     rm.waitForState(app.getApplicationId(), RMAppState.ACCEPTED);
     RMAppAttempt attempt = app.getCurrentAppAttempt();
     waitForSchedulerAppAttemptAdded(attempt.getAppAttemptId(), rm);
+    rm.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.SCHEDULED);
     System.out.println("Launch AM " + attempt.getAppAttemptId());
     nm.nodeHeartbeat(true);
     MockAM am = rm.sendAMLaunched(attempt.getAppAttemptId());


[48/50] [abbrv] hadoop git commit: HADOOP-11328. ZKFailoverController does not log Exception when doRun raises errors. Contributed by Tianyin Xu.

Posted by zj...@apache.org.
HADOOP-11328. ZKFailoverController does not log Exception when doRun raises errors. Contributed by Tianyin Xu.


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

Branch: refs/heads/YARN-2928
Commit: 54e28874d2a0866b3ef151158ab8b98fbc766f9c
Parents: 9229247
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Mon May 4 17:47:37 2015 +0900
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:59:01 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                   | 3 +++
 .../src/main/java/org/apache/hadoop/ha/ZKFailoverController.java  | 1 +
 2 files changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/54e28874/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index bb8f900..2bf790a 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -525,6 +525,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11357. Print information of the build enviornment in test-patch.sh
     (aw)
 
+    HADOOP-11328. ZKFailoverController does not log Exception when doRun raises
+    errors. (Tianyin Xu via ozawa)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54e28874/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
index f58c3f4..9eb1ff8 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
@@ -180,6 +180,7 @@ public abstract class ZKFailoverController {
         }
       });
     } catch (RuntimeException rte) {
+      LOG.fatal("The failover controller encounters runtime error: " + rte);
       throw (Exception)rte.getCause();
     }
   }


[16/50] [abbrv] hadoop git commit: YARN-3544. Got back AM logs link on the RM web UI for a completed app. Contributed by Xuan Gong.

Posted by zj...@apache.org.
YARN-3544. Got back AM logs link on the RM web UI for a completed app. Contributed by Xuan Gong.


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

Branch: refs/heads/YARN-2928
Commit: c2390c9058d5117144da076b28c088f2e29a5402
Parents: 71580ae
Author: Zhijie Shen <zj...@apache.org>
Authored: Wed Apr 29 17:12:52 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:55 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../resourcemanager/webapp/RMAppBlock.java      | 83 +++++++-------------
 2 files changed, 30 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2390c90/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index bb00ee6..27b4fe8 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -385,6 +385,9 @@ Release 2.7.1 - UNRELEASED
     YARN-3485. FairScheduler headroom calculation doesn't consider 
     maxResources for Fifo and FairShare policies. (kasha)
 
+    YARN-3544. Got back AM logs link on the RM web UI for a completed app.
+    (Xuan Gong via zjshen)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2390c90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.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/webapp/RMAppBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java
index 3779b91..43e26be 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java
@@ -25,28 +25,24 @@ 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.yarn.api.protocolrecords.GetContainerReportRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.exceptions.ContainerNotFoundException;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMetrics;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptMetrics;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptInfo;
 import org.apache.hadoop.yarn.server.webapp.AppBlock;
-import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo;
-import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
-import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
+import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.apache.hadoop.yarn.webapp.view.InfoBlock;
 
 import com.google.inject.Inject;
 
-import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
 import java.util.Set;
 
@@ -54,11 +50,13 @@ public class RMAppBlock extends AppBlock{
 
   private static final Log LOG = LogFactory.getLog(RMAppBlock.class);
   private final ResourceManager rm;
+  private final Configuration conf;
 
 
   @Inject
   RMAppBlock(ViewContext ctx, Configuration conf, ResourceManager rm) {
     super(rm.getClientRMService(), ctx, conf);
+    this.conf = conf;
     this.rm = rm;
   }
 
@@ -120,65 +118,38 @@ public class RMAppBlock extends AppBlock{
             .th(".started", "Started").th(".node", "Node").th(".logs", "Logs")
             .th(".blacklistednodes", "Blacklisted Nodes")._()._().tbody();
 
+    RMApp rmApp = this.rm.getRMContext().getRMApps().get(this.appID);
+    if (rmApp == null) {
+      return;
+    }
     StringBuilder attemptsTableData = new StringBuilder("[\n");
     for (final ApplicationAttemptReport appAttemptReport : attempts) {
-      AppAttemptInfo appAttempt = new AppAttemptInfo(appAttemptReport);
-      ContainerReport containerReport = null;
-      try {
-        // AM container is always the first container of the attempt
-        final GetContainerReportRequest request =
-            GetContainerReportRequest.newInstance(ContainerId.newContainerId(
-                appAttemptReport.getApplicationAttemptId(), 1));
-        if (callerUGI == null) {
-          containerReport =
-              appBaseProt.getContainerReport(request).getContainerReport();
-        } else {
-          containerReport = callerUGI.doAs(
-              new PrivilegedExceptionAction<ContainerReport>() {
-                @Override
-                public ContainerReport run() throws Exception {
-                  ContainerReport report = null;
-                  try {
-                    report = appBaseProt.getContainerReport(request)
-                        .getContainerReport();
-                  } catch (ContainerNotFoundException ex) {
-                    LOG.warn(ex.getMessage());
-                  }
-                  return report;
-                }
-              });
-        }
-      } catch (Exception e) {
-        String message =
-            "Failed to read the AM container of the application attempt "
-                + appAttemptReport.getApplicationAttemptId() + ".";
-        LOG.error(message, e);
-        html.p()._(message)._();
-        return;
-      }
-      long startTime = 0L;
-      String logsLink = null;
-      String nodeLink = null;
-      if (containerReport != null) {
-        ContainerInfo container = new ContainerInfo(containerReport);
-        startTime = container.getStartedTime();
-        logsLink = containerReport.getLogUrl();
-        nodeLink = containerReport.getNodeHttpAddress();
+      RMAppAttempt rmAppAttempt =
+          rmApp.getRMAppAttempt(appAttemptReport.getApplicationAttemptId());
+      if (rmAppAttempt == null) {
+        continue;
       }
+      AppAttemptInfo attemptInfo =
+          new AppAttemptInfo(this.rm, rmAppAttempt, rmApp.getUser());
       String blacklistedNodesCount = "N/A";
-      Set<String> nodes = RMAppAttemptBlock.getBlacklistedNodes(rm,
-          ConverterUtils.toApplicationAttemptId(appAttempt.getAppAttemptId()));
+      Set<String> nodes =
+          RMAppAttemptBlock.getBlacklistedNodes(rm,
+            rmAppAttempt.getAppAttemptId());
       if(nodes != null) {
         blacklistedNodesCount = String.valueOf(nodes.size());
       }
-
+      String nodeLink = attemptInfo.getNodeHttpAddress();
+      if (nodeLink != null) {
+        nodeLink = WebAppUtils.getHttpSchemePrefix(conf) + nodeLink;
+      }
+      String logsLink = attemptInfo.getLogsLink();
       attemptsTableData
           .append("[\"<a href='")
-          .append(url("appattempt", appAttempt.getAppAttemptId()))
+          .append(url("appattempt", rmAppAttempt.getAppAttemptId().toString()))
           .append("'>")
-          .append(appAttempt.getAppAttemptId())
+          .append(String.valueOf(rmAppAttempt.getAppAttemptId()))
           .append("</a>\",\"")
-          .append(startTime)
+          .append(attemptInfo.getStartTime())
           .append("\",\"<a ")
           .append(nodeLink == null ? "#" : "href='" + nodeLink)
           .append("'>")


[03/50] [abbrv] hadoop git commit: MAPREDUCE-6334. Fetcher#copyMapOutput is leaking usedMemory upon IOException during InMemoryMapOutput shuffle handler. Contributed by Eric Payne

Posted by zj...@apache.org.
MAPREDUCE-6334. Fetcher#copyMapOutput is leaking usedMemory upon IOException during InMemoryMapOutput shuffle handler. Contributed by Eric Payne


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

Branch: refs/heads/YARN-2928
Commit: b42a6c54b70159b973c2f3df985d0040a98860a7
Parents: c18ec25
Author: Jason Lowe <jl...@apache.org>
Authored: Tue Apr 28 20:17:52 2015 +0000
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:52 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |  3 ++
 .../hadoop/mapreduce/task/reduce/Fetcher.java   |  6 ++--
 .../mapreduce/task/reduce/TestFetcher.java      | 34 ++++++++++++++++++++
 3 files changed, 41 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b42a6c54/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index cd79e2a..f8e713d 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -394,6 +394,9 @@ Release 2.7.1 - UNRELEASED
     MAPREDUCE-6252. JobHistoryServer should not fail when encountering a 
     missing directory. (Craig Welch via devaraj)
 
+    MAPREDUCE-6334. Fetcher#copyMapOutput is leaking usedMemory upon
+    IOException during InMemoryMapOutput shuffle handler (Eric Payne via jlowe)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b42a6c54/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java
index d867e4b..4b80dc9 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java
@@ -553,7 +553,10 @@ class Fetcher<K,V> extends Thread {
       metrics.successFetch();
       return null;
     } catch (IOException ioe) {
-      
+      if (mapOutput != null) {
+        mapOutput.abort();
+      }
+
       if (canRetry) {
         checkTimeoutOrRetry(host, ioe);
       } 
@@ -574,7 +577,6 @@ class Fetcher<K,V> extends Thread {
                " from " + host.getHostName(), ioe); 
 
       // Inform the shuffle-scheduler
-      mapOutput.abort();
       metrics.failedFetch();
       return new TaskAttemptID[] {mapId};
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b42a6c54/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestFetcher.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestFetcher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestFetcher.java
index 723df17..a9cd33e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestFetcher.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestFetcher.java
@@ -628,6 +628,40 @@ public class TestFetcher {
     verify(odmo).abort();
   }
 
+  @SuppressWarnings("unchecked")
+  @Test(timeout=10000)
+  public void testCopyFromHostWithRetryUnreserve() throws Exception {
+    InMemoryMapOutput<Text, Text> immo = mock(InMemoryMapOutput.class);
+    Fetcher<Text,Text> underTest = new FakeFetcher<Text,Text>(jobWithRetry,
+        id, ss, mm, r, metrics, except, key, connection);
+
+    String replyHash = SecureShuffleUtils.generateHash(encHash.getBytes(), key);
+
+    when(connection.getResponseCode()).thenReturn(200);
+    when(connection.getHeaderField(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH))
+        .thenReturn(replyHash);
+    ShuffleHeader header = new ShuffleHeader(map1ID.toString(), 10, 10, 1);
+    ByteArrayOutputStream bout = new ByteArrayOutputStream();
+    header.write(new DataOutputStream(bout));
+    ByteArrayInputStream in = new ByteArrayInputStream(bout.toByteArray());
+    when(connection.getInputStream()).thenReturn(in);
+    when(connection.getHeaderField(ShuffleHeader.HTTP_HEADER_NAME))
+        .thenReturn(ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
+    when(connection.getHeaderField(ShuffleHeader.HTTP_HEADER_VERSION))
+        .thenReturn(ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
+
+    // Verify that unreserve occurs if an exception happens after shuffle
+    // buffer is reserved.
+    when(mm.reserve(any(TaskAttemptID.class), anyLong(), anyInt()))
+        .thenReturn(immo);
+    doThrow(new IOException("forced error")).when(immo).shuffle(
+        any(MapHost.class), any(InputStream.class), anyLong(),
+        anyLong(), any(ShuffleClientMetrics.class), any(Reporter.class));
+
+    underTest.copyFromHost(host);
+    verify(immo).abort();
+  }
+
   public static class FakeFetcher<K,V> extends Fetcher<K,V> {
 
     // If connection need to be reopen.


[41/50] [abbrv] hadoop git commit: YARN-1993. Cross-site scripting vulnerability in TextView.java. Contributed byKenji Kikushima.

Posted by zj...@apache.org.
YARN-1993. Cross-site scripting vulnerability in TextView.java. Contributed byKenji Kikushima.


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

Branch: refs/heads/YARN-2928
Commit: 4b999c74cee3aabb5d4e7aff9f4fb953dcce7eac
Parents: b125d0d
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Sun May 3 10:51:17 2015 +0900
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:59:00 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                 | 4 ++++
 .../main/java/org/apache/hadoop/yarn/webapp/view/TextView.java  | 5 ++++-
 2 files changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b999c74/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 511ddb7..fde0168 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -357,6 +357,10 @@ Release 2.8.0 - UNRELEASED
     YARN-2454. Fix compareTo of variable UNBOUNDED in o.a.h.y.util.resource.Resources.
     (Xu Yang via junping_du)
 
+    YARN-1993. Cross-site scripting vulnerability in TextView.java. (Kenji Kikushima
+    via ozawa)
+
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b999c74/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/TextView.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/TextView.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/TextView.java
index 16efa4e..4983dac 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/TextView.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/TextView.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.webapp.view;
 
 import java.io.PrintWriter;
 
+import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.yarn.webapp.View;
 
@@ -45,7 +46,9 @@ public abstract class TextView extends View {
   public void echo(Object... args) {
     PrintWriter out = writer();
     for (Object s : args) {
-      out.print(s);
+      String escapedString = StringEscapeUtils.escapeJavaScript(
+          StringEscapeUtils.escapeHtml(s.toString()));
+      out.print(escapedString);
     }
   }
 


[36/50] [abbrv] hadoop git commit: MAPREDUCE-6345. Documentation fix for when CRLA is enabled for MRAppMaster logs. (Rohit Agarwal via gera)

Posted by zj...@apache.org.
MAPREDUCE-6345. Documentation fix for when CRLA is enabled for MRAppMaster logs. (Rohit Agarwal via gera)


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

Branch: refs/heads/YARN-2928
Commit: a0e3a4c450983205426e34f4701be8ab7c59d056
Parents: 1fde5be
Author: Gera Shegalov <ge...@apache.org>
Authored: Fri May 1 16:13:56 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:59 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                              | 3 +++
 .../src/main/resources/mapred-default.xml                         | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a0e3a4c4/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 1acbca0..1b23ee2 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -374,6 +374,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6341. Fix typo in mapreduce tutorial. (John Michael Luy
     via ozawa)
 
+    MAPREDUCE-6345. Documentation fix for when CRLA is enabled for MRAppMaster
+    logs. (Rohit Agarwal via gera)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a0e3a4c4/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
index 2b25b59..fb761ba 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
@@ -746,7 +746,7 @@
     org.apache.log4j.RollingFileAppender.maxBackupIndex. By default,
     ContainerLogAppender (CLA) is used, and container logs are not rolled. CRLA
     is enabled for the ApplicationMaster when both
-    mapreduce.task.userlog.limit.kb and
+    yarn.app.mapreduce.am.container.log.limit.kb and
     yarn.app.mapreduce.am.container.log.backups are greater than zero.
   </description>
 </property>


[26/50] [abbrv] hadoop git commit: HDFS-8213. DFSClient should use hdfs.client.htrace HTrace configuration prefix rather than hadoop.htrace (cmccabe)

Posted by zj...@apache.org.
HDFS-8213. DFSClient should use hdfs.client.htrace HTrace configuration prefix rather than hadoop.htrace (cmccabe)


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

Branch: refs/heads/YARN-2928
Commit: fe95065abfb481c9b38fc32b0a0c15dbb137aed8
Parents: 45f0120
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Fri May 1 11:19:40 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:57 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/tracing/SpanReceiverHost.java | 61 ++++++++++----------
 .../org/apache/hadoop/tracing/TraceUtils.java   | 14 ++---
 .../apache/hadoop/tracing/TestTraceUtils.java   | 10 ++--
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  5 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  7 +++
 .../hadoop/hdfs/server/datanode/DataNode.java   |  3 +-
 .../hadoop/hdfs/server/namenode/NameNode.java   |  3 +-
 .../apache/hadoop/tracing/TestTraceAdmin.java   |  4 +-
 .../org/apache/hadoop/tracing/TestTracing.java  | 10 +---
 .../TestTracingShortCircuitLocalRead.java       |  4 +-
 11 files changed, 69 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe95065a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/SpanReceiverHost.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/SpanReceiverHost.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/SpanReceiverHost.java
index f2de0a0..bf9479b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/SpanReceiverHost.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/SpanReceiverHost.java
@@ -25,6 +25,7 @@ import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
@@ -52,41 +53,36 @@ import org.apache.htrace.Trace;
  */
 @InterfaceAudience.Private
 public class SpanReceiverHost implements TraceAdminProtocol {
-  public static final String SPAN_RECEIVERS_CONF_KEY =
-    "hadoop.htrace.spanreceiver.classes";
+  public static final String SPAN_RECEIVERS_CONF_SUFFIX =
+    "spanreceiver.classes";
   private static final Log LOG = LogFactory.getLog(SpanReceiverHost.class);
+  private static final HashMap<String, SpanReceiverHost> hosts =
+      new HashMap<String, SpanReceiverHost>(1);
   private final TreeMap<Long, SpanReceiver> receivers =
       new TreeMap<Long, SpanReceiver>();
+  private final String confPrefix;
   private Configuration config;
   private boolean closed = false;
   private long highestId = 1;
 
-  private final static String LOCAL_FILE_SPAN_RECEIVER_PATH =
-      "hadoop.htrace.local-file-span-receiver.path";
+  private final static String LOCAL_FILE_SPAN_RECEIVER_PATH_SUFFIX =
+      "local-file-span-receiver.path";
 
-  private static enum SingletonHolder {
-    INSTANCE;
-    Object lock = new Object();
-    SpanReceiverHost host = null;
-  }
-
-  public static SpanReceiverHost getInstance(Configuration conf) {
-    if (SingletonHolder.INSTANCE.host != null) {
-      return SingletonHolder.INSTANCE.host;
-    }
-    synchronized (SingletonHolder.INSTANCE.lock) {
-      if (SingletonHolder.INSTANCE.host != null) {
-        return SingletonHolder.INSTANCE.host;
+  public static SpanReceiverHost get(Configuration conf, String confPrefix) {
+    synchronized (SpanReceiverHost.class) {
+      SpanReceiverHost host = hosts.get(confPrefix);
+      if (host != null) {
+        return host;
       }
-      SpanReceiverHost host = new SpanReceiverHost();
-      host.loadSpanReceivers(conf);
-      SingletonHolder.INSTANCE.host = host;
+      final SpanReceiverHost newHost = new SpanReceiverHost(confPrefix);
+      newHost.loadSpanReceivers(conf);
       ShutdownHookManager.get().addShutdownHook(new Runnable() {
           public void run() {
-            SingletonHolder.INSTANCE.host.closeReceivers();
+            newHost.closeReceivers();
           }
         }, 0);
-      return SingletonHolder.INSTANCE.host;
+      hosts.put(confPrefix, newHost);
+      return newHost;
     }
   }
 
@@ -119,6 +115,10 @@ public class SpanReceiverHost implements TraceAdminProtocol {
     return new File(tmp, nonce).getAbsolutePath();
   }
 
+  private SpanReceiverHost(String confPrefix) {
+    this.confPrefix = confPrefix;
+  }
+
   /**
    * Reads the names of classes specified in the
    * "hadoop.htrace.spanreceiver.classes" property and instantiates and registers
@@ -131,22 +131,22 @@ public class SpanReceiverHost implements TraceAdminProtocol {
    */
   public synchronized void loadSpanReceivers(Configuration conf) {
     config = new Configuration(conf);
-    String[] receiverNames =
-        config.getTrimmedStrings(SPAN_RECEIVERS_CONF_KEY);
+    String receiverKey = confPrefix + SPAN_RECEIVERS_CONF_SUFFIX;
+    String[] receiverNames = config.getTrimmedStrings(receiverKey);
     if (receiverNames == null || receiverNames.length == 0) {
       if (LOG.isTraceEnabled()) {
-        LOG.trace("No span receiver names found in " +
-                  SPAN_RECEIVERS_CONF_KEY + ".");
+        LOG.trace("No span receiver names found in " + receiverKey + ".");
       }
       return;
     }
     // It's convenient to have each daemon log to a random trace file when
     // testing.
-    if (config.get(LOCAL_FILE_SPAN_RECEIVER_PATH) == null) {
+    String pathKey = confPrefix + LOCAL_FILE_SPAN_RECEIVER_PATH_SUFFIX;
+    if (config.get(pathKey) == null) {
       String uniqueFile = getUniqueLocalTraceFileName();
-      config.set(LOCAL_FILE_SPAN_RECEIVER_PATH, uniqueFile);
+      config.set(pathKey, uniqueFile);
       if (LOG.isTraceEnabled()) {
-        LOG.trace("Set " + LOCAL_FILE_SPAN_RECEIVER_PATH + " to " +  uniqueFile);
+        LOG.trace("Set " + pathKey + " to " + uniqueFile);
       }
     }
     for (String className : receiverNames) {
@@ -164,7 +164,8 @@ public class SpanReceiverHost implements TraceAdminProtocol {
   private synchronized SpanReceiver loadInstance(String className,
       List<ConfigurationPair> extraConfig) throws IOException {
     SpanReceiverBuilder builder =
-        new SpanReceiverBuilder(TraceUtils.wrapHadoopConf(config, extraConfig));
+        new SpanReceiverBuilder(TraceUtils.
+            wrapHadoopConf(confPrefix, config, extraConfig));
     SpanReceiver rcvr = builder.spanReceiverClass(className.trim()).build();
     if (rcvr == null) {
       throw new IOException("Failed to load SpanReceiver " + className);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe95065a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceUtils.java
index 11797e6..fa52ac6 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceUtils.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceUtils.java
@@ -31,15 +31,15 @@ import org.apache.htrace.HTraceConfiguration;
  */
 @InterfaceAudience.Private
 public class TraceUtils {
-  public static final String HTRACE_CONF_PREFIX = "hadoop.htrace.";
   private static List<ConfigurationPair> EMPTY = Collections.emptyList();
 
-  public static HTraceConfiguration wrapHadoopConf(final Configuration conf) {
-    return wrapHadoopConf(conf, EMPTY);
+  public static HTraceConfiguration wrapHadoopConf(final String prefix,
+        final Configuration conf) {
+    return wrapHadoopConf(prefix, conf, EMPTY);
   }
 
-  public static HTraceConfiguration wrapHadoopConf(final Configuration conf,
-          List<ConfigurationPair> extraConfig) {
+  public static HTraceConfiguration wrapHadoopConf(final String prefix,
+        final Configuration conf, List<ConfigurationPair> extraConfig) {
     final HashMap<String, String> extraMap = new HashMap<String, String>();
     for (ConfigurationPair pair : extraConfig) {
       extraMap.put(pair.getKey(), pair.getValue());
@@ -50,7 +50,7 @@ public class TraceUtils {
         if (extraMap.containsKey(key)) {
           return extraMap.get(key);
         }
-        return conf.get(HTRACE_CONF_PREFIX + key, "");
+        return conf.get(prefix + key, "");
       }
 
       @Override
@@ -58,7 +58,7 @@ public class TraceUtils {
         if (extraMap.containsKey(key)) {
           return extraMap.get(key);
         }
-        return conf.get(HTRACE_CONF_PREFIX + key, defaultValue);
+        return conf.get(prefix + key, defaultValue);
       }
     };
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe95065a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tracing/TestTraceUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tracing/TestTraceUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tracing/TestTraceUtils.java
index 9ef3483..80d64b1 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tracing/TestTraceUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tracing/TestTraceUtils.java
@@ -25,13 +25,15 @@ import org.apache.htrace.HTraceConfiguration;
 import org.junit.Test;
 
 public class TestTraceUtils {
+  private static String TEST_PREFIX = "test.prefix.htrace.";
+
   @Test
   public void testWrappedHadoopConf() {
     String key = "sampler";
     String value = "ProbabilitySampler";
     Configuration conf = new Configuration();
-    conf.set(TraceUtils.HTRACE_CONF_PREFIX + key, value);
-    HTraceConfiguration wrapped = TraceUtils.wrapHadoopConf(conf);
+    conf.set(TEST_PREFIX + key, value);
+    HTraceConfiguration wrapped = TraceUtils.wrapHadoopConf(TEST_PREFIX, conf);
     assertEquals(value, wrapped.get(key));
   }
 
@@ -41,11 +43,11 @@ public class TestTraceUtils {
     String oldValue = "old value";
     String newValue = "new value";
     Configuration conf = new Configuration();
-    conf.set(TraceUtils.HTRACE_CONF_PREFIX + key, oldValue);
+    conf.set(TEST_PREFIX + key, oldValue);
     LinkedList<ConfigurationPair> extraConfig =
         new LinkedList<ConfigurationPair>();
     extraConfig.add(new ConfigurationPair(key, newValue));
-    HTraceConfiguration wrapped = TraceUtils.wrapHadoopConf(conf, extraConfig);
+    HTraceConfiguration wrapped = TraceUtils.wrapHadoopConf(TEST_PREFIX, conf, extraConfig);
     assertEquals(newValue, wrapped.get(key));
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe95065a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 9accdc0..16094a2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -617,6 +617,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-7770. Need document for storage type label of data node storage
     locations under dfs.data.dir. (Xiaoyu Yao via aajisaka)
 
+    HDFS-8213. DFSClient should use hdfs.client.htrace HTrace configuration
+    prefix rather than hadoop.htrace (cmccabe)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe95065a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 8fc9e77..d47992b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -299,8 +299,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode,
       Configuration conf, FileSystem.Statistics stats)
     throws IOException {
-    SpanReceiverHost.getInstance(conf);
-    traceSampler = new SamplerBuilder(TraceUtils.wrapHadoopConf(conf)).build();
+    SpanReceiverHost.get(conf, DFSConfigKeys.DFS_CLIENT_HTRACE_PREFIX);
+    traceSampler = new SamplerBuilder(TraceUtils.
+        wrapHadoopConf(DFSConfigKeys.DFS_CLIENT_HTRACE_PREFIX, conf)).build();
     // Copy only the required DFSClient configuration
     this.dfsClientConf = new DfsClientConf(conf);
     this.conf = conf;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe95065a/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 1e3a5b6..4356b9b 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
@@ -55,6 +55,13 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT =
       HdfsClientConfigKeys.DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT;
 
+  // HDFS HTrace configuration is controlled by dfs.htrace.spanreceiver.classes,
+  // etc.
+  public static final String  DFS_SERVER_HTRACE_PREFIX = "dfs.htrace.";
+
+  // HDFS client HTrace configuration.
+  public static final String  DFS_CLIENT_HTRACE_PREFIX = "dfs.client.htrace.";
+
   // HA related configuration
   public static final String  DFS_DATANODE_RESTART_REPLICA_EXPIRY_KEY = "dfs.datanode.restart.replica.expiration";
   public static final long    DFS_DATANODE_RESTART_REPLICA_EXPIRY_DEFAULT = 50;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe95065a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 2401d9c..f042dff 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -1099,7 +1099,8 @@ public class DataNode extends ReconfigurableBase
     this.dnConf = new DNConf(conf);
     checkSecureConfig(dnConf, conf, resources);
 
-    this.spanReceiverHost = SpanReceiverHost.getInstance(conf);
+    this.spanReceiverHost =
+      SpanReceiverHost.get(conf, DFSConfigKeys.DFS_SERVER_HTRACE_PREFIX);
 
     if (dnConf.maxLockedMemory > 0) {
       if (!NativeIO.POSIX.getCacheManipulator().verifyCanMlock()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe95065a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
index 1e94923..132b93e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
@@ -639,7 +639,8 @@ public class NameNode implements NameNodeStatusMXBean {
       startHttpServer(conf);
     }
 
-    this.spanReceiverHost = SpanReceiverHost.getInstance(conf);
+    this.spanReceiverHost =
+      SpanReceiverHost.get(conf, DFSConfigKeys.DFS_SERVER_HTRACE_PREFIX);
 
     loadNamesystem(conf);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe95065a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTraceAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTraceAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTraceAdmin.java
index 7b3568d..4a102a3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTraceAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTraceAdmin.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.tracing;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
 import org.junit.Assert;
@@ -57,7 +58,8 @@ public class TestTraceAdmin {
   public void testCreateAndDestroySpanReceiver() throws Exception {
     Configuration conf = new Configuration();
     conf = new Configuration();
-    conf.set(SpanReceiverHost.SPAN_RECEIVERS_CONF_KEY, "");
+    conf.set(DFSConfigKeys.DFS_SERVER_HTRACE_PREFIX  +
+        SpanReceiverHost.SPAN_RECEIVERS_CONF_SUFFIX, "");
     MiniDFSCluster cluster =
         new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
     cluster.waitActive();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe95065a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracing.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracing.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracing.java
index f6fef5a..59d1238 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracing.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracing.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -53,14 +54,9 @@ public class TestTracing {
   private static Configuration conf;
   private static MiniDFSCluster cluster;
   private static DistributedFileSystem dfs;
-  private static SpanReceiverHost spanReceiverHost;
 
   @Test
   public void testTracing() throws Exception {
-    // getting instance already loaded.
-    Assert.assertEquals(spanReceiverHost,
-        SpanReceiverHost.getInstance(new Configuration()));
-
     // write and read without tracing started
     String fileName = "testTracingDisabled.dat";
     writeTestFile(fileName);
@@ -196,9 +192,9 @@ public class TestTracing {
   public static void setup() throws IOException {
     conf = new Configuration();
     conf.setLong("dfs.blocksize", 100 * 1024);
-    conf.set(SpanReceiverHost.SPAN_RECEIVERS_CONF_KEY,
+    conf.set(DFSConfigKeys.DFS_CLIENT_HTRACE_PREFIX +
+        SpanReceiverHost.SPAN_RECEIVERS_CONF_SUFFIX,
         SetSpanReceiver.class.getName());
-    spanReceiverHost = SpanReceiverHost.getInstance(conf);
   }
 
   @Before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe95065a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracingShortCircuitLocalRead.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracingShortCircuitLocalRead.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracingShortCircuitLocalRead.java
index 5d6db16..09ab350 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracingShortCircuitLocalRead.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracingShortCircuitLocalRead.java
@@ -64,7 +64,8 @@ public class TestTracingShortCircuitLocalRead {
   public void testShortCircuitTraceHooks() throws IOException {
     assumeTrue(NativeCodeLoader.isNativeCodeLoaded() && !Path.WINDOWS);
     conf = new Configuration();
-    conf.set(SpanReceiverHost.SPAN_RECEIVERS_CONF_KEY,
+    conf.set(DFSConfigKeys.DFS_CLIENT_HTRACE_PREFIX +
+        SpanReceiverHost.SPAN_RECEIVERS_CONF_SUFFIX,
         TestTracing.SetSpanReceiver.class.getName());
     conf.setLong("dfs.blocksize", 100 * 1024);
     conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
@@ -78,7 +79,6 @@ public class TestTracingShortCircuitLocalRead {
     dfs = cluster.getFileSystem();
 
     try {
-      spanReceiverHost = SpanReceiverHost.getInstance(conf);
       DFSTestUtil.createFile(dfs, TEST_PATH, TEST_LENGTH, (short)1, 5678L);
 
       TraceScope ts = Trace.startSpan("testShortCircuitTraceHooks", Sampler.ALWAYS);


[50/50] [abbrv] hadoop git commit: HDFS-7397. Add more detail to the documentation for the conf key "dfs.client.read.shortcircuit.streams.cache.size" (Brahma Reddy Battula via Colin P. McCabe)

Posted by zj...@apache.org.
HDFS-7397. Add more detail to the documentation for the conf key "dfs.client.read.shortcircuit.streams.cache.size" (Brahma Reddy Battula via Colin P. McCabe)


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

Branch: refs/heads/YARN-2928
Commit: 0b1e184ccb8540ff5216a21ff580314ecc92ac74
Parents: aa5c6f4
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Mon May 4 12:50:29 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:59:02 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                      | 4 ++++
 .../hadoop-hdfs/src/main/resources/hdfs-default.xml              | 2 +-
 2 files changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b1e184c/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index a365b86..21d73ba 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -500,6 +500,10 @@ Release 2.8.0 - UNRELEASED
     HDFS-8249. Separate HdfsConstants into the client and the server side
     class. (wheat9)
 
+    HDFS-7397. Add more detail to the documentation for the conf key
+    "dfs.client.read.shortcircuit.streams.cache.size" (Brahma Reddy Battula via
+    Colin P. McCabe)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b1e184c/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 f03c0fb..e3e4323 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
@@ -2063,7 +2063,7 @@
   <value>256</value>
   <description>
     The DFSClient maintains a cache of recently opened file descriptors.
-    This parameter controls the size of that cache.
+    This parameter controls the maximum number of file descriptors in the cache.
     Setting this higher will use more file descriptors,
     but potentially provide better performance on workloads
     involving lots of seeks.


[33/50] [abbrv] hadoop git commit: HDFS-8229. LAZY_PERSIST file gets deleted after NameNode restart. (Contributed by Surendra Singh Lilhore)

Posted by zj...@apache.org.
HDFS-8229. LAZY_PERSIST file gets deleted after NameNode restart. (Contributed by Surendra Singh Lilhore)


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

Branch: refs/heads/YARN-2928
Commit: 41ef21456d47f3872ddbf944504c127c46f02ea7
Parents: 3660b7f
Author: Arpit Agarwal <ar...@apache.org>
Authored: Fri May 1 16:30:51 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:58 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../hdfs/server/namenode/FSNamesystem.java      |  9 +++++-
 .../fsdataset/impl/LazyPersistTestCase.java     |  1 +
 .../fsdataset/impl/TestLazyPersistFiles.java    | 31 ++++++++++++++++++++
 4 files changed, 43 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/41ef2145/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 179fe7e..1882df5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -601,6 +601,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8276. LazyPersistFileScrubber should be disabled if scrubber interval
     configured zero. (Surendra Singh Lilhore via Arpit Agarwal)
 
+    HDFS-8229. LAZY_PERSIST file gets deleted after NameNode restart.
+    (Surendra Singh Lilhore via Arpit Agarwal) 
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/41ef2145/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 d5ff80e..809d594 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
@@ -4767,7 +4767,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     public void run() {
       while (fsRunning && shouldRun) {
         try {
-          clearCorruptLazyPersistFiles();
+          if (!isInSafeMode()) {
+            clearCorruptLazyPersistFiles();
+          } else {
+            if (FSNamesystem.LOG.isDebugEnabled()) {
+              FSNamesystem.LOG
+                  .debug("Namenode is in safemode, skipping scrubbing of corrupted lazy-persist files.");
+            }
+          }
           Thread.sleep(scrubIntervalSec * 1000);
         } catch (InterruptedException e) {
           FSNamesystem.LOG.info(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/41ef2145/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
index 93cd23a..7e1aa81 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
@@ -258,6 +258,7 @@ public abstract class LazyPersistTestCase {
                 LAZY_WRITER_INTERVAL_SEC);
     conf.setLong(DFS_DATANODE_RAM_DISK_LOW_WATERMARK_BYTES,
                 evictionLowWatermarkReplicas * BLOCK_SIZE);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_MIN_DATANODES_KEY, 1);
 
     if (useSCR) {
       conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/41ef2145/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java
index 60cc8fe..950e9dc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java
@@ -16,6 +16,7 @@
  * limitations under the License.
  */
 package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
+import com.google.common.collect.Iterators;
 import com.google.common.util.concurrent.Uninterruptibles;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.Path;
@@ -145,6 +146,36 @@ public class TestLazyPersistFiles extends LazyPersistTestCase {
     Assert.assertTrue(fs.exists(path1));
 
   }
+
+ /**
+  * If NN restarted then lazyPersist files should not deleted
+  */
+  @Test
+  public void testFileShouldNotDiscardedIfNNRestarted() throws IOException,
+      InterruptedException {
+    getClusterBuilder().setRamDiskReplicaCapacity(2).build();
+    final String METHOD_NAME = GenericTestUtils.getMethodName();
+    Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
+    makeTestFile(path1, BLOCK_SIZE, true);
+    ensureFileReplicasOnStorageType(path1, RAM_DISK);
+
+    cluster.shutdownDataNodes();
+
+    cluster.restartNameNodes();
+
+    // wait for the replication monitor to mark the file as corrupt
+    Thread.sleep(2 * DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT * 1000);
+
+    Long corruptBlkCount = (long) Iterators.size(cluster.getNameNode()
+        .getNamesystem().getBlockManager().getCorruptReplicaBlockIterator());
+
+    // Check block detected as corrupted
+    assertThat(corruptBlkCount, is(1L));
+
+    // Ensure path1 exist.
+    Assert.assertTrue(fs.exists(path1));
+  }
+
   /**
    * Concurrent read from the same node and verify the contents.
    */


[23/50] [abbrv] hadoop git commit: HDFS-8300. Fix unit test failures and findbugs warning caused by HDFS-8283. Contributed by Jing Zhao.

Posted by zj...@apache.org.
HDFS-8300. Fix unit test failures and findbugs warning caused by HDFS-8283. Contributed by Jing Zhao.


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

Branch: refs/heads/YARN-2928
Commit: 960b4e985cf978f67ab19d480fd0095ef8907ba7
Parents: 17a96fa
Author: Jing Zhao <ji...@apache.org>
Authored: Thu Apr 30 19:27:07 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:56 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |  2 +-
 .../org/apache/hadoop/hdfs/DataStreamer.java    | 32 ++++++++++++--------
 .../apache/hadoop/hdfs/TestDFSOutputStream.java |  9 ++++--
 4 files changed, 29 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/960b4e98/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 7e4a6f2..3bee852 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -592,6 +592,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8214. Secondary NN Web UI shows wrong date for Last Checkpoint. (clamb via wang)
 
+    HDFS-8300. Fix unit test failures and findbugs warning caused by HDFS-8283.
+    (jing9)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/960b4e98/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index 4646b60..ae5d3eb 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@ -762,7 +762,7 @@ public class DFSOutputStream extends FSOutputSummer
 
   protected synchronized void closeImpl() throws IOException {
     if (isClosed()) {
-      streamer.getLastException().check();
+      streamer.getLastException().check(true);
       return;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/960b4e98/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
index 3727d20..8e874eb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
@@ -174,13 +174,13 @@ class DataStreamer extends Daemon {
     packets.clear();
   }
   
-  static class LastException {
-    private Throwable thrown;
+  static class LastExceptionInStreamer {
+    private IOException thrown;
 
     synchronized void set(Throwable t) {
-      Preconditions.checkNotNull(t);
-      Preconditions.checkState(thrown == null);
-      this.thrown = t;
+      assert t != null;
+      this.thrown = t instanceof IOException ?
+          (IOException) t : new IOException(t);
     }
 
     synchronized void clear() {
@@ -188,17 +188,23 @@ class DataStreamer extends Daemon {
     }
 
     /** Check if there already is an exception. */
-    synchronized void check() throws IOException {
+    synchronized void check(boolean resetToNull) throws IOException {
       if (thrown != null) {
-        throw new IOException(thrown);
+        if (LOG.isTraceEnabled()) {
+          // wrap and print the exception to know when the check is called
+          LOG.trace("Got Exception while checking", new Throwable(thrown));
+        }
+        final IOException e = thrown;
+        if (resetToNull) {
+          thrown = null;
+        }
+        throw e;
       }
     }
 
     synchronized void throwException4Close() throws IOException {
-      check();
-      final IOException ioe = new ClosedChannelException();
-      thrown = ioe;
-      throw ioe;
+      check(false);
+      throw new ClosedChannelException();
     }
   }
 
@@ -234,7 +240,7 @@ class DataStreamer extends Daemon {
   private long lastQueuedSeqno = -1;
   private long lastAckedSeqno = -1;
   private long bytesCurBlock = 0; // bytes written in current block
-  private final LastException lastException = new LastException();
+  private final LastExceptionInStreamer lastException = new LastExceptionInStreamer();
   private Socket s;
 
   private final DFSClient dfsClient;
@@ -1741,7 +1747,7 @@ class DataStreamer extends Daemon {
   /**
    * @return the last exception
    */
-  LastException getLastException(){
+  LastExceptionInStreamer getLastException(){
     return lastException;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/960b4e98/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
index eac1fcd..e627455 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
@@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DataStreamer.LastExceptionInStreamer;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -65,9 +66,10 @@ public class TestDFSOutputStream {
     DataStreamer streamer = (DataStreamer) Whitebox
         .getInternalState(dos, "streamer");
     @SuppressWarnings("unchecked")
-    AtomicReference<IOException> ex = (AtomicReference<IOException>) Whitebox
+    LastExceptionInStreamer ex = (LastExceptionInStreamer) Whitebox
         .getInternalState(streamer, "lastException");
-    Assert.assertEquals(null, ex.get());
+    Throwable thrown = (Throwable) Whitebox.getInternalState(ex, "thrown");
+    Assert.assertNull(thrown);
 
     dos.close();
 
@@ -78,7 +80,8 @@ public class TestDFSOutputStream {
     } catch (IOException e) {
       Assert.assertEquals(e, dummy);
     }
-    Assert.assertEquals(null, ex.get());
+    thrown = (Throwable) Whitebox.getInternalState(ex, "thrown");
+    Assert.assertNull(thrown);
     dos.close();
   }
 


[38/50] [abbrv] hadoop git commit: HDFS-8249. Separate HdfsConstants into the client and the server side class. Contributed by Haohui Mai.

Posted by zj...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
index cf0325e..409967e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -511,7 +510,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    */
   public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps) {
     final byte storagePolicyId = isSymlink() ?
-        HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED : getStoragePolicyID();
+        HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED : getStoragePolicyID();
     return computeQuotaUsage(bsps, storagePolicyId,
         new QuotaCounts.Builder().build(), true, Snapshot.CURRENT_STATE_ID);
   }
@@ -555,7 +554,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   public final QuotaCounts computeQuotaUsage(
     BlockStoragePolicySuite bsps, QuotaCounts counts, boolean useCache) {
     final byte storagePolicyId = isSymlink() ?
-        HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED : getStoragePolicyID();
+        HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED : getStoragePolicyID();
     return computeQuotaUsage(bsps, storagePolicyId, counts,
         useCache, Snapshot.CURRENT_STATE_ID);
   }
@@ -712,7 +711,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
 
   /**
    * @return the storage policy directly specified on the INode. Return
-   * {@link HdfsConstantsClient#BLOCK_STORAGE_POLICY_ID_UNSPECIFIED} if no policy has
+   * {@link HdfsConstants#BLOCK_STORAGE_POLICY_ID_UNSPECIFIED} if no policy has
    * been specified.
    */
   public abstract byte getLocalStoragePolicyID();
@@ -721,13 +720,13 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    * Get the storage policy ID while computing quota usage
    * @param parentStoragePolicyId the storage policy ID of the parent directory
    * @return the storage policy ID of this INode. Note that for an
-   * {@link INodeSymlink} we return {@link HdfsConstantsClient#BLOCK_STORAGE_POLICY_ID_UNSPECIFIED}
+   * {@link INodeSymlink} we return {@link HdfsConstants#BLOCK_STORAGE_POLICY_ID_UNSPECIFIED}
    * instead of throwing Exception
    */
   public byte getStoragePolicyIDForQuota(byte parentStoragePolicyId) {
     byte localId = isSymlink() ?
-        HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED : getLocalStoragePolicyID();
-    return localId != HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED ?
+        HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED : getLocalStoragePolicyID();
+    return localId != HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED ?
         localId : parentStoragePolicyId;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
index 12fa7aa..098594d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
@@ -45,7 +45,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
-import static org.apache.hadoop.hdfs.protocol.HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
 
 /**
  * Directory INode class.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index e9d3644..110bd71 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.apache.hadoop.hdfs.protocol.HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.CURRENT_STATE_ID;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.NO_SNAPSHOT_ID;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeId.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeId.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeId.java
index 5344ca7..00b33cd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeId.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeId.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.FileNotFoundException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.util.SequentialNumber;
 
 /**
@@ -44,7 +44,7 @@ public class INodeId extends SequentialNumber {
    */
   public static void checkId(long requestId, INode inode)
       throws FileNotFoundException {
-    if (requestId != HdfsConstantsClient.GRANDFATHER_INODE_ID && requestId != inode.getId()) {
+    if (requestId != HdfsConstants.GRANDFATHER_INODE_ID && requestId != inode.getId()) {
       throw new FileNotFoundException(
           "ID mismatch. Request id and saved id: " + requestId + " , "
               + inode.getId() + " for file " + inode.getFullPathName());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
index 9a1e1f4..7b1332b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
@@ -22,7 +22,7 @@ import java.util.List;
 
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.util.GSet;
 import org.apache.hadoop.util.LightWeightGSet;
@@ -124,12 +124,12 @@ public class INodeMap {
 
       @Override
       public byte getStoragePolicyID(){
-        return HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
+        return HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
       }
 
       @Override
       public byte getLocalStoragePolicyID() {
-        return HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
+        return HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
       }
     };
       

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
index f1892c5..72ca6ff 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
@@ -49,7 +50,7 @@ public class INodesInPath {
    */
   private static boolean isDotSnapshotDir(byte[] pathComponent) {
     return pathComponent != null &&
-        Arrays.equals(HdfsConstants.DOT_SNAPSHOT_DIR_BYTES, pathComponent);
+        Arrays.equals(HdfsServerConstants.DOT_SNAPSHOT_DIR_BYTES, pathComponent);
   }
 
   static INodesInPath fromINode(INode inode) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
index 9ce8ebc..c6a92be 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
@@ -36,7 +36,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnresolvedLinkException;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.util.Daemon;
@@ -72,8 +71,8 @@ public class LeaseManager {
 
   private final FSNamesystem fsnamesystem;
 
-  private long softLimit = HdfsConstants.LEASE_SOFTLIMIT_PERIOD;
-  private long hardLimit = HdfsConstants.LEASE_HARDLIMIT_PERIOD;
+  private long softLimit = HdfsServerConstants.LEASE_SOFTLIMIT_PERIOD;
+  private long hardLimit = HdfsServerConstants.LEASE_HARDLIMIT_PERIOD;
 
   //
   // Used for handling lock-leases

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
index dbb2c50..26a13bd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
@@ -37,7 +37,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
@@ -129,7 +128,7 @@ public class NNStorage extends Storage implements Closeable,
    * recent fsimage file. This does not include any transactions
    * that have since been written to the edit log.
    */
-  protected volatile long mostRecentCheckpointTxId = HdfsConstants.INVALID_TXID;
+  protected volatile long mostRecentCheckpointTxId = HdfsServerConstants.INVALID_TXID;
   
   /**
    * Time of the last checkpoint, in milliseconds since the epoch.
@@ -558,7 +557,7 @@ public class NNStorage extends Storage implements Closeable,
    */
   public void format(NamespaceInfo nsInfo) throws IOException {
     Preconditions.checkArgument(nsInfo.getLayoutVersion() == 0 ||
-        nsInfo.getLayoutVersion() == HdfsConstants.NAMENODE_LAYOUT_VERSION,
+        nsInfo.getLayoutVersion() == HdfsServerConstants.NAMENODE_LAYOUT_VERSION,
         "Bad layout version: %s", nsInfo.getLayoutVersion());
     
     this.setStorageInfo(nsInfo);
@@ -577,7 +576,7 @@ public class NNStorage extends Storage implements Closeable,
   }
   
   public void format() throws IOException {
-    this.layoutVersion = HdfsConstants.NAMENODE_LAYOUT_VERSION;
+    this.layoutVersion = HdfsServerConstants.NAMENODE_LAYOUT_VERSION;
     for (Iterator<StorageDirectory> it =
                            dirIterator(); it.hasNext();) {
       StorageDirectory sd = it.next();
@@ -634,7 +633,7 @@ public class NNStorage extends Storage implements Closeable,
             "storage directory " + sd.getRoot().getAbsolutePath());
       }
       props.setProperty("layoutVersion",
-          Integer.toString(HdfsConstants.NAMENODE_LAYOUT_VERSION));
+          Integer.toString(HdfsServerConstants.NAMENODE_LAYOUT_VERSION));
     }
     setFieldsFromProperties(props, sd);
   }
@@ -657,7 +656,7 @@ public class NNStorage extends Storage implements Closeable,
    * This should only be used during upgrades.
    */
   String getDeprecatedProperty(String prop) {
-    assert getLayoutVersion() > HdfsConstants.NAMENODE_LAYOUT_VERSION :
+    assert getLayoutVersion() > HdfsServerConstants.NAMENODE_LAYOUT_VERSION :
       "getDeprecatedProperty should only be done when loading " +
       "storage from past versions during upgrade.";
     return deprecatedProperties.get(prop);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
index 132b93e..979378a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
@@ -486,7 +486,7 @@ public class NameNode implements NameNodeStatusMXBean {
   public static URI getUri(InetSocketAddress namenode) {
     int port = namenode.getPort();
     String portString = port == DEFAULT_PORT ? "" : (":"+port);
-    return URI.create(HdfsConstants.HDFS_URI_SCHEME + "://" 
+    return URI.create(HdfsConstants.HDFS_URI_SCHEME + "://"
         + namenode.getHostName()+portString);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/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 83686e0..3311609 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
@@ -21,8 +21,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HANDLER_COUNT_DE
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HANDLER_COUNT_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_HANDLER_COUNT_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_HANDLER_COUNT_KEY;
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.MAX_PATH_DEPTH;
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.MAX_PATH_LENGTH;
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.MAX_PATH_DEPTH;
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.MAX_PATH_LENGTH;
 import static org.apache.hadoop.util.Time.now;
 
 import java.io.FileNotFoundException;
@@ -87,7 +87,6 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FSLimitException;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
@@ -117,6 +116,7 @@ import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
@@ -1500,9 +1500,9 @@ class NameNodeRpcServer implements NamenodeProtocols {
    * @throws IOException on layout version mismatch
    */
   void verifyLayoutVersion(int version) throws IOException {
-    if (version != HdfsConstants.NAMENODE_LAYOUT_VERSION)
+    if (version != HdfsServerConstants.NAMENODE_LAYOUT_VERSION)
       throw new IncorrectVersionException(
-          HdfsConstants.NAMENODE_LAYOUT_VERSION, version, "data node");
+          HdfsServerConstants.NAMENODE_LAYOUT_VERSION, version, "data node");
   }
   
   private void verifySoftwareVersion(DatanodeRegistration dnReg)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RedundantEditLogInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RedundantEditLogInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RedundantEditLogInputStream.java
index 674a957..33be8b0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RedundantEditLogInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RedundantEditLogInputStream.java
@@ -23,7 +23,7 @@ import java.util.Collection;
 import java.util.Comparator;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.io.IOUtils;
 
 import com.google.common.base.Preconditions;
@@ -88,8 +88,8 @@ class RedundantEditLogInputStream extends EditLogInputStream {
   RedundantEditLogInputStream(Collection<EditLogInputStream> streams,
       long startTxId) {
     this.curIdx = 0;
-    this.prevTxId = (startTxId == HdfsConstants.INVALID_TXID) ?
-      HdfsConstants.INVALID_TXID : (startTxId - 1);
+    this.prevTxId = (startTxId == HdfsServerConstants.INVALID_TXID) ?
+      HdfsServerConstants.INVALID_TXID : (startTxId - 1);
     this.state = (streams.isEmpty()) ? State.EOF : State.SKIP_UNTIL;
     this.prevException = null;
     // EditLogInputStreams in a RedundantEditLogInputStream must be finalized,
@@ -97,9 +97,9 @@ class RedundantEditLogInputStream extends EditLogInputStream {
     EditLogInputStream first = null;
     for (EditLogInputStream s : streams) {
       Preconditions.checkArgument(s.getFirstTxId() !=
-          HdfsConstants.INVALID_TXID, "invalid first txid in stream: %s", s);
+          HdfsServerConstants.INVALID_TXID, "invalid first txid in stream: %s", s);
       Preconditions.checkArgument(s.getLastTxId() !=
-          HdfsConstants.INVALID_TXID, "invalid last txid in stream: %s", s);
+          HdfsServerConstants.INVALID_TXID, "invalid last txid in stream: %s", s);
       if (first == null) {
         first = s;
       } else {
@@ -172,7 +172,7 @@ class RedundantEditLogInputStream extends EditLogInputStream {
       switch (state) {
       case SKIP_UNTIL:
        try {
-          if (prevTxId != HdfsConstants.INVALID_TXID) {
+          if (prevTxId != HdfsServerConstants.INVALID_TXID) {
             LOG.info("Fast-forwarding stream '" + streams[curIdx].getName() +
                 "' to transaction ID " + (prevTxId + 1));
             streams[curIdx].skipUntil(prevTxId + 1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
index 0d32758..041c3cb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
@@ -44,7 +44,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.StorageErrorReporter;
@@ -336,7 +336,7 @@ public class TransferFsImage {
   private static void copyFileToStream(OutputStream out, File localfile,
       FileInputStream infile, DataTransferThrottler throttler,
       Canceler canceler) throws IOException {
-    byte buf[] = new byte[HdfsConstants.IO_FILE_BUFFER_SIZE];
+    byte buf[] = new byte[HdfsServerConstants.IO_FILE_BUFFER_SIZE];
     try {
       CheckpointFaultInjector.getInstance()
           .aboutToSendFile(localfile);
@@ -345,7 +345,7 @@ public class TransferFsImage {
             shouldSendShortFile(localfile)) {
           // Test sending image shorter than localfile
           long len = localfile.length();
-          buf = new byte[(int)Math.min(len/2, HdfsConstants.IO_FILE_BUFFER_SIZE)];
+          buf = new byte[(int)Math.min(len/2, HdfsServerConstants.IO_FILE_BUFFER_SIZE)];
           // This will read at most half of the image
           // and the rest of the image will be sent over the wire
           infile.read(buf);
@@ -510,7 +510,7 @@ public class TransferFsImage {
       }
       
       int num = 1;
-      byte[] buf = new byte[HdfsConstants.IO_FILE_BUFFER_SIZE];
+      byte[] buf = new byte[HdfsServerConstants.IO_FILE_BUFFER_SIZE];
       while (num > 0) {
         num = stream.read(buf);
         if (num > 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
index 9d8f2f8..0accf53 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
@@ -40,7 +40,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.NameNodeProxies;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@@ -167,7 +167,7 @@ public class BootstrapStandby implements Tool, Configurable {
     if (!checkLayoutVersion(nsInfo)) {
       LOG.fatal("Layout version on remote node (" + nsInfo.getLayoutVersion()
           + ") does not match " + "this node's layout version ("
-          + HdfsConstants.NAMENODE_LAYOUT_VERSION + ")");
+          + HdfsServerConstants.NAMENODE_LAYOUT_VERSION + ")");
       return ERR_CODE_INVALID_VERSION;
     }
 
@@ -366,7 +366,7 @@ public class BootstrapStandby implements Tool, Configurable {
   }
 
   private boolean checkLayoutVersion(NamespaceInfo nsInfo) throws IOException {
-    return (nsInfo.getLayoutVersion() == HdfsConstants.NAMENODE_LAYOUT_VERSION);
+    return (nsInfo.getLayoutVersion() == HdfsServerConstants.NAMENODE_LAYOUT_VERSION);
   }
   
   private void parseConfAndFindOtherNN() throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
index 1897d8d..38aa358 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
@@ -31,9 +31,9 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HAUtil;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.EditLogInputException;
 import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
@@ -75,12 +75,12 @@ public class EditLogTailer {
   /**
    * The last transaction ID at which an edit log roll was initiated.
    */
-  private long lastRollTriggerTxId = HdfsConstants.INVALID_TXID;
+  private long lastRollTriggerTxId = HdfsServerConstants.INVALID_TXID;
   
   /**
    * The highest transaction ID loaded by the Standby.
    */
-  private long lastLoadedTxnId = HdfsConstants.INVALID_TXID;
+  private long lastLoadedTxnId = HdfsServerConstants.INVALID_TXID;
 
   /**
    * The last time we successfully loaded a non-zero number of edits from the

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
index 7baf7dc..c4406a0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
@@ -21,7 +21,7 @@ import java.util.List;
 
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.AclFeature;
@@ -151,7 +151,7 @@ public class FileWithSnapshotFeature implements INode.Feature {
     BlockStoragePolicy bsp = null;
     EnumCounters<StorageType> typeSpaces =
         new EnumCounters<StorageType>(StorageType.class);
-    if (storagePolicyID != HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED) {
+    if (storagePolicyID != HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED) {
       bsp = bsps.getPolicy(file.getStoragePolicyID());
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NNHAStatusHeartbeat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NNHAStatusHeartbeat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NNHAStatusHeartbeat.java
index faaf8f4..d06d0db 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NNHAStatusHeartbeat.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NNHAStatusHeartbeat.java
@@ -20,14 +20,14 @@ package org.apache.hadoop.hdfs.server.protocol;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class NNHAStatusHeartbeat {
 
   private final HAServiceState state;
-  private long txid = HdfsConstants.INVALID_TXID;
+  private long txid = HdfsServerConstants.INVALID_TXID;
   
   public NNHAStatusHeartbeat(HAServiceState state, long txid) {
     this.state = state;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
index a7439a0..dfdf449 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
@@ -22,7 +22,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
@@ -83,7 +83,7 @@ public class NamespaceInfo extends StorageInfo {
   public NamespaceInfo(int nsID, String clusterID, String bpID,
       long cT, String buildVersion, String softwareVersion,
       long capabilities) {
-    super(HdfsConstants.NAMENODE_LAYOUT_VERSION, nsID, clusterID, cT,
+    super(HdfsServerConstants.NAMENODE_LAYOUT_VERSION, nsID, clusterID, cT,
         NodeType.NAME_NODE);
     blockPoolID = bpID;
     this.buildVersion = buildVersion;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RemoteEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RemoteEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RemoteEditLog.java
index 4b191f2..1d26bc4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RemoteEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RemoteEditLog.java
@@ -17,14 +17,13 @@
  */
 package org.apache.hadoop.hdfs.server.protocol;
 
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-
 import com.google.common.base.Function;
 import com.google.common.collect.ComparisonChain;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 
 public class RemoteEditLog implements Comparable<RemoteEditLog> {
-  private long startTxId = HdfsConstants.INVALID_TXID;
-  private long endTxId = HdfsConstants.INVALID_TXID;
+  private long startTxId = HdfsServerConstants.INVALID_TXID;
+  private long endTxId = HdfsServerConstants.INVALID_TXID;
   private boolean isInProgress = false;
   
   public RemoteEditLog() {
@@ -33,7 +32,7 @@ public class RemoteEditLog implements Comparable<RemoteEditLog> {
   public RemoteEditLog(long startTxId, long endTxId) {
     this.startTxId = startTxId;
     this.endTxId = endTxId;
-    this.isInProgress = (endTxId == HdfsConstants.INVALID_TXID);
+    this.isInProgress = (endTxId == HdfsServerConstants.INVALID_TXID);
   }
   
   public RemoteEditLog(long startTxId, long endTxId, boolean inProgress) {
@@ -90,7 +89,7 @@ public class RemoteEditLog implements Comparable<RemoteEditLog> {
       @Override
       public Long apply(RemoteEditLog log) {
         if (null == log) {
-          return HdfsConstants.INVALID_TXID;
+          return HdfsServerConstants.INVALID_TXID;
         }
         return log.getStartTxId();
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java
index ba1fd0f..e3bdffa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java
@@ -22,7 +22,7 @@ import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.tools.TableListing;
 import org.apache.hadoop.util.StringUtils;
@@ -151,7 +151,7 @@ public class StoragePolicyAdmin extends Configured implements Tool {
           return 2;
         }
         byte storagePolicyId = status.getStoragePolicy();
-        if (storagePolicyId == HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED) {
+        if (storagePolicyId == HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED) {
           System.out.println("The storage policy of " + path + " is unspecified");
           return 0;
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsLoader.java
index 0ce1e78..73d1798 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsLoader.java
@@ -22,8 +22,7 @@ import java.io.IOException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
-import org.apache.hadoop.hdfs.tools.offlineEditsViewer.OfflineEditsViewer;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream;
 
 import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
@@ -49,8 +48,8 @@ interface OfflineEditsLoader {
         OfflineEditsLoader loader = null;
         try {
           file = new File(inputFileName);
-          elis = new EditLogFileInputStream(file, HdfsConstants.INVALID_TXID,
-              HdfsConstants.INVALID_TXID, false);
+          elis = new EditLogFileInputStream(file, HdfsServerConstants.INVALID_TXID,
+              HdfsServerConstants.INVALID_TXID, false);
           loader = new OfflineEditsBinaryLoader(visitor, elis, flags);
         } finally {
           if ((loader == null) && (elis != null)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
index 3693239..f2c7427 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
@@ -28,7 +28,7 @@ import java.util.Map;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutFlags;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
@@ -687,7 +687,7 @@ class ImageLoaderCurrent implements ImageLoader {
     final String pathName = readINodePath(in, parentName);
     v.visit(ImageElement.INODE_PATH, pathName);
 
-    long inodeId = HdfsConstantsClient.GRANDFATHER_INODE_ID;
+    long inodeId = HdfsConstants.GRANDFATHER_INODE_ID;
     if (supportInodeId) {
       inodeId = in.readLong();
       v.visit(ImageElement.INODE_ID, inodeId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfs.java
index c98ba66..6d7ef55 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfs.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.web.WebHdfsConstants;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
@@ -259,7 +260,7 @@ abstract public class TestSymlinkHdfs extends SymlinkBaseTest {
   public void testCreateLinkMaxPathLink() throws IOException {
     Path dir  = new Path(testBaseDir1());
     Path file = new Path(testBaseDir1(), "file");
-    final int maxPathLen = HdfsConstants.MAX_PATH_LENGTH;
+    final int maxPathLen = HdfsServerConstants.MAX_PATH_LENGTH;
     final int dirLen     = dir.toString().length() + 1;
     int   len            = maxPathLen - dirLen;
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index 681e9bd..a8df991 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -113,6 +113,7 @@ 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.DatanodeStorageInfo;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
@@ -950,7 +951,7 @@ public class DFSTestUtil {
     final long writeTimeout = dfsClient.getDatanodeWriteTimeout(datanodes.length);
     final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
         NetUtils.getOutputStream(s, writeTimeout),
-        HdfsConstants.SMALL_BUFFER_SIZE));
+        HdfsServerConstants.SMALL_BUFFER_SIZE));
     final DataInputStream in = new DataInputStream(NetUtils.getInputStream(s));
 
     // send the request
@@ -1222,7 +1223,7 @@ public class DFSTestUtil {
     s2.close();
     // OP_SET_STORAGE_POLICY 45
     filesystem.setStoragePolicy(pathFileCreate,
-        HdfsConstants.HOT_STORAGE_POLICY_NAME);
+        HdfsServerConstants.HOT_STORAGE_POLICY_NAME);
     // OP_RENAME_OLD 1
     final Path pathFileMoved = new Path("/file_moved");
     filesystem.rename(pathFileCreate, pathFileMoved);
@@ -1689,8 +1690,7 @@ public class DFSTestUtil {
     modifiersField.setInt(field, field.getModifiers() & ~Modifier.FINAL);
     field.setInt(null, lv);
 
-    // Override {@link HdfsConstants#DATANODE_LAYOUT_VERSION}
-    field = HdfsConstants.class.getField("DATANODE_LAYOUT_VERSION");
+    field = HdfsServerConstants.class.getField("DATANODE_LAYOUT_VERSION");
     field.setAccessible(true);
     modifiersField.setInt(field, field.getModifiers() & ~Modifier.FINAL);
     field.setInt(null, lv);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
index 89c8e11..9621dc8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.apache.hadoop.hdfs.protocol.HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
 
 import java.io.File;
 import java.io.FileNotFoundException;
@@ -32,6 +32,7 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.*;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.blockmanagement.*;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -67,12 +68,12 @@ public class TestBlockStoragePolicy {
   static final long FILE_LEN = 1024;
   static final short REPLICATION = 3;
 
-  static final byte COLD = HdfsConstants.COLD_STORAGE_POLICY_ID;
-  static final byte WARM = HdfsConstants.WARM_STORAGE_POLICY_ID;
-  static final byte HOT  = HdfsConstants.HOT_STORAGE_POLICY_ID;
-  static final byte ONESSD  = HdfsConstants.ONESSD_STORAGE_POLICY_ID;
-  static final byte ALLSSD  = HdfsConstants.ALLSSD_STORAGE_POLICY_ID;
-  static final byte LAZY_PERSIST  = HdfsConstants.MEMORY_STORAGE_POLICY_ID;
+  static final byte COLD = HdfsServerConstants.COLD_STORAGE_POLICY_ID;
+  static final byte WARM = HdfsServerConstants.WARM_STORAGE_POLICY_ID;
+  static final byte HOT  = HdfsServerConstants.HOT_STORAGE_POLICY_ID;
+  static final byte ONESSD  = HdfsServerConstants.ONESSD_STORAGE_POLICY_ID;
+  static final byte ALLSSD  = HdfsServerConstants.ALLSSD_STORAGE_POLICY_ID;
+  static final byte LAZY_PERSIST  = HdfsServerConstants.MEMORY_STORAGE_POLICY_ID;
 
   @Test (timeout=300000)
   public void testConfigKeyEnabled() throws IOException {
@@ -83,7 +84,7 @@ public class TestBlockStoragePolicy {
     try {
       cluster.waitActive();
       cluster.getFileSystem().setStoragePolicy(new Path("/"),
-          HdfsConstants.COLD_STORAGE_POLICY_NAME);
+          HdfsServerConstants.COLD_STORAGE_POLICY_NAME);
     } finally {
       cluster.shutdown();
     }
@@ -103,7 +104,7 @@ public class TestBlockStoragePolicy {
     try {
       cluster.waitActive();
       cluster.getFileSystem().setStoragePolicy(new Path("/"),
-          HdfsConstants.COLD_STORAGE_POLICY_NAME);
+          HdfsServerConstants.COLD_STORAGE_POLICY_NAME);
     } finally {
       cluster.shutdown();
     }
@@ -860,15 +861,15 @@ public class TestBlockStoragePolicy {
 
       final Path invalidPath = new Path("/invalidPath");
       try {
-        fs.setStoragePolicy(invalidPath, HdfsConstants.WARM_STORAGE_POLICY_NAME);
+        fs.setStoragePolicy(invalidPath, HdfsServerConstants.WARM_STORAGE_POLICY_NAME);
         Assert.fail("Should throw a FileNotFoundException");
       } catch (FileNotFoundException e) {
         GenericTestUtils.assertExceptionContains(invalidPath.toString(), e);
       }
 
-      fs.setStoragePolicy(fooFile, HdfsConstants.COLD_STORAGE_POLICY_NAME);
-      fs.setStoragePolicy(barDir, HdfsConstants.WARM_STORAGE_POLICY_NAME);
-      fs.setStoragePolicy(barFile2, HdfsConstants.HOT_STORAGE_POLICY_NAME);
+      fs.setStoragePolicy(fooFile, HdfsServerConstants.COLD_STORAGE_POLICY_NAME);
+      fs.setStoragePolicy(barDir, HdfsServerConstants.WARM_STORAGE_POLICY_NAME);
+      fs.setStoragePolicy(barFile2, HdfsServerConstants.HOT_STORAGE_POLICY_NAME);
 
       dirList = fs.getClient().listPaths(dir.toString(),
           HdfsFileStatus.EMPTY_NAME).getPartialListing();
@@ -916,7 +917,7 @@ public class TestBlockStoragePolicy {
       DFSTestUtil.createFile(fs, fooFile1, FILE_LEN, REPLICATION, 0L);
       DFSTestUtil.createFile(fs, fooFile2, FILE_LEN, REPLICATION, 0L);
 
-      fs.setStoragePolicy(fooDir, HdfsConstants.WARM_STORAGE_POLICY_NAME);
+      fs.setStoragePolicy(fooDir, HdfsServerConstants.WARM_STORAGE_POLICY_NAME);
 
       HdfsFileStatus[] dirList = fs.getClient().listPaths(dir.toString(),
           HdfsFileStatus.EMPTY_NAME, true).getPartialListing();
@@ -928,7 +929,7 @@ public class TestBlockStoragePolicy {
       // take snapshot
       SnapshotTestHelper.createSnapshot(fs, dir, "s1");
       // change the storage policy of fooFile1
-      fs.setStoragePolicy(fooFile1, HdfsConstants.COLD_STORAGE_POLICY_NAME);
+      fs.setStoragePolicy(fooFile1, HdfsServerConstants.COLD_STORAGE_POLICY_NAME);
 
       fooList = fs.getClient().listPaths(fooDir.toString(),
           HdfsFileStatus.EMPTY_NAME).getPartialListing();
@@ -951,7 +952,7 @@ public class TestBlockStoragePolicy {
           HdfsFileStatus.EMPTY_NAME).getPartialListing(), COLD);
 
       // change the storage policy of foo dir
-      fs.setStoragePolicy(fooDir, HdfsConstants.HOT_STORAGE_POLICY_NAME);
+      fs.setStoragePolicy(fooDir, HdfsServerConstants.HOT_STORAGE_POLICY_NAME);
       // /dir/foo is now hot
       dirList = fs.getClient().listPaths(dir.toString(),
           HdfsFileStatus.EMPTY_NAME, true).getPartialListing();
@@ -1068,7 +1069,7 @@ public class TestBlockStoragePolicy {
    */
   @Test
   public void testChangeHotFileRep() throws Exception {
-    testChangeFileRep(HdfsConstants.HOT_STORAGE_POLICY_NAME, HOT,
+    testChangeFileRep(HdfsServerConstants.HOT_STORAGE_POLICY_NAME, HOT,
         new StorageType[]{StorageType.DISK, StorageType.DISK,
             StorageType.DISK},
         new StorageType[]{StorageType.DISK, StorageType.DISK, StorageType.DISK,
@@ -1082,7 +1083,7 @@ public class TestBlockStoragePolicy {
    */
   @Test
   public void testChangeWarmRep() throws Exception {
-    testChangeFileRep(HdfsConstants.WARM_STORAGE_POLICY_NAME, WARM,
+    testChangeFileRep(HdfsServerConstants.WARM_STORAGE_POLICY_NAME, WARM,
         new StorageType[]{StorageType.DISK, StorageType.ARCHIVE,
             StorageType.ARCHIVE},
         new StorageType[]{StorageType.DISK, StorageType.ARCHIVE,
@@ -1095,7 +1096,7 @@ public class TestBlockStoragePolicy {
    */
   @Test
   public void testChangeColdRep() throws Exception {
-    testChangeFileRep(HdfsConstants.COLD_STORAGE_POLICY_NAME, COLD,
+    testChangeFileRep(HdfsServerConstants.COLD_STORAGE_POLICY_NAME, COLD,
         new StorageType[]{StorageType.ARCHIVE, StorageType.ARCHIVE,
             StorageType.ARCHIVE},
         new StorageType[]{StorageType.ARCHIVE, StorageType.ARCHIVE,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java
index 68687ed..3fe5626 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java
@@ -30,7 +30,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
@@ -191,7 +191,7 @@ public class TestDFSRollback {
           UpgradeUtilities.getCurrentBlockPoolID(cluster));
       // Put newer layout version in current.
       storageInfo = new StorageInfo(
-          HdfsConstants.DATANODE_LAYOUT_VERSION - 1,
+          HdfsServerConstants.DATANODE_LAYOUT_VERSION - 1,
           UpgradeUtilities.getCurrentNamespaceID(cluster),
           UpgradeUtilities.getCurrentClusterID(cluster),
           UpgradeUtilities.getCurrentFsscTime(cluster),
@@ -277,7 +277,7 @@ public class TestDFSRollback {
       
       UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "current");
       baseDirs = UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "previous");
-      storageInfo = new StorageInfo(HdfsConstants.DATANODE_LAYOUT_VERSION,
+      storageInfo = new StorageInfo(HdfsServerConstants.DATANODE_LAYOUT_VERSION,
           UpgradeUtilities.getCurrentNamespaceID(cluster),
           UpgradeUtilities.getCurrentClusterID(cluster), Long.MAX_VALUE,
           NodeType.DATA_NODE);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStartupVersions.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStartupVersions.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStartupVersions.java
index 889e579..3a27be6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStartupVersions.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStartupVersions.java
@@ -27,7 +27,7 @@ import java.io.File;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
@@ -89,7 +89,7 @@ public class TestDFSStartupVersions {
    */
   private StorageData[] initializeVersions() throws Exception {
     int layoutVersionOld = Storage.LAST_UPGRADABLE_LAYOUT_VERSION;
-    int layoutVersionCur = HdfsConstants.DATANODE_LAYOUT_VERSION;
+    int layoutVersionCur = HdfsServerConstants.DATANODE_LAYOUT_VERSION;
     int layoutVersionNew = Integer.MIN_VALUE;
     int namespaceIdCur = UpgradeUtilities.getCurrentNamespaceID(null);
     int namespaceIdOld = Integer.MIN_VALUE;
@@ -200,7 +200,7 @@ public class TestDFSStartupVersions {
       return false;
     }
     // check #3
-    int softwareLV = HdfsConstants.DATANODE_LAYOUT_VERSION;
+    int softwareLV = HdfsServerConstants.DATANODE_LAYOUT_VERSION;
     int storedLV = datanodeVer.getLayoutVersion();
     if (softwareLV == storedLV &&  
         datanodeVer.getCTime() == namenodeVer.getCTime()) 
@@ -252,7 +252,7 @@ public class TestDFSStartupVersions {
                                               .startupOption(StartupOption.REGULAR)
                                               .build();
     StorageData nameNodeVersion = new StorageData(
-        HdfsConstants.NAMENODE_LAYOUT_VERSION,
+        HdfsServerConstants.NAMENODE_LAYOUT_VERSION,
         UpgradeUtilities.getCurrentNamespaceID(cluster),
         UpgradeUtilities.getCurrentClusterID(cluster),
         UpgradeUtilities.getCurrentFsscTime(cluster),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java
index f0a094e..544537c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java
@@ -37,9 +37,9 @@ import java.util.regex.Pattern;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
@@ -307,7 +307,7 @@ public class TestDFSUpgrade {
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       cluster = createCluster();
       baseDirs = UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "current");
-      storageInfo = new StorageInfo(HdfsConstants.DATANODE_LAYOUT_VERSION,
+      storageInfo = new StorageInfo(HdfsServerConstants.DATANODE_LAYOUT_VERSION,
           UpgradeUtilities.getCurrentNamespaceID(cluster),
           UpgradeUtilities.getCurrentClusterID(cluster), Long.MAX_VALUE,
           NodeType.DATA_NODE);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeRegistration.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeRegistration.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeRegistration.java
index 15b3fb1..211e6aa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeRegistration.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeRegistration.java
@@ -22,9 +22,9 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -33,7 +33,6 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.VersionInfo;
 import org.junit.Test;
 
-import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.security.Permission;
 
@@ -180,7 +179,7 @@ public class TestDatanodeRegistration {
           .getCTime();
       StorageInfo mockStorageInfo = mock(StorageInfo.class);
       doReturn(nnCTime).when(mockStorageInfo).getCTime();
-      doReturn(HdfsConstants.DATANODE_LAYOUT_VERSION).when(mockStorageInfo)
+      doReturn(HdfsServerConstants.DATANODE_LAYOUT_VERSION).when(mockStorageInfo)
           .getLayoutVersion();
       DatanodeRegistration dnReg = new DatanodeRegistration(dnId,
           mockStorageInfo, null, VersionInfo.getVersion());
@@ -225,7 +224,7 @@ public class TestDatanodeRegistration {
       doReturn(nnCTime).when(mockStorageInfo).getCTime();
       
       DatanodeRegistration mockDnReg = mock(DatanodeRegistration.class);
-      doReturn(HdfsConstants.DATANODE_LAYOUT_VERSION).when(mockDnReg).getVersion();
+      doReturn(HdfsServerConstants.DATANODE_LAYOUT_VERSION).when(mockDnReg).getVersion();
       doReturn("127.0.0.1").when(mockDnReg).getIpAddr();
       doReturn(123).when(mockDnReg).getXferPort();
       doReturn("fake-storage-id").when(mockDnReg).getDatanodeUuid();
@@ -274,7 +273,7 @@ public class TestDatanodeRegistration {
       doReturn(nnCTime).when(mockStorageInfo).getCTime();
       
       DatanodeRegistration mockDnReg = mock(DatanodeRegistration.class);
-      doReturn(HdfsConstants.DATANODE_LAYOUT_VERSION).when(mockDnReg).getVersion();
+      doReturn(HdfsServerConstants.DATANODE_LAYOUT_VERSION).when(mockDnReg).getVersion();
       doReturn("fake-storage-id").when(mockDnReg).getDatanodeUuid();
       doReturn(mockStorageInfo).when(mockDnReg).getStorageInfo();
       

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
index ca25018..969f2b5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
@@ -33,15 +33,14 @@ import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
@@ -111,7 +110,7 @@ public class TestFileAppend4 {
 
     // set the soft limit to be 1 second so that the
     // namenode triggers lease recovery upon append request
-    cluster.setLeasePeriod(1000, HdfsConstants.LEASE_HARDLIMIT_PERIOD);
+    cluster.setLeasePeriod(1000, HdfsServerConstants.LEASE_HARDLIMIT_PERIOD);
 
     // Trying recovery
     int tries = 60;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
index 0a7b712..f56ff9e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
@@ -70,7 +70,6 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@@ -537,7 +536,7 @@ public class TestFileCreation {
 
       // add one block to the file
       LocatedBlock location = client.getNamenode().addBlock(file1.toString(),
-          client.clientName, null, null, HdfsConstantsClient.GRANDFATHER_INODE_ID, null);
+          client.clientName, null, null, HdfsConstants.GRANDFATHER_INODE_ID, null);
       System.out.println("testFileCreationError2: "
           + "Added block " + location.getBlock());
 
@@ -588,7 +587,7 @@ public class TestFileCreation {
       createFile(dfs, f, 3);
       try {
         cluster.getNameNodeRpc().addBlock(f.toString(), client.clientName,
-            null, null, HdfsConstantsClient.GRANDFATHER_INODE_ID, null);
+            null, null, HdfsConstants.GRANDFATHER_INODE_ID, null);
         fail();
       } catch(IOException ioe) {
         FileSystem.LOG.info("GOOD!", ioe);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
index 8ff8655..103151e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
@@ -34,8 +34,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
@@ -278,7 +278,7 @@ public class TestGetBlocks {
 
     for (int i = 0; i < blkids.length; i++) {
       Block b = new Block(blkids[i], 0,
-          HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP);
+          HdfsConstants.GRANDFATHER_GENERATION_STAMP);
       Long v = map.get(b);
       System.out.println(b + " => " + v);
       assertEquals(blkids[i], v.longValue());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/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 9b5a7c0..88dbd5e 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
@@ -18,13 +18,10 @@
 package org.apache.hadoop.hdfs;
 
 import static org.mockito.Matchers.anyBoolean;
-import static org.mockito.Matchers.anyList;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Matchers.anyShort;
 import static org.mockito.Matchers.anyLong;
 import static org.mockito.Matchers.anyObject;
-import static org.mockito.Matchers.anyShort;
-import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.spy;
@@ -32,12 +29,10 @@ import static org.mockito.Mockito.spy;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
-import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.crypto.CipherSuite;
 import org.apache.hadoop.crypto.CryptoProtocolVersion;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -47,8 +42,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.client.impl.LeaseRenewer;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.io.EnumSetWritable;
@@ -103,7 +98,7 @@ public class TestLease {
       // make it look like the soft limit has been exceeded.
       LeaseRenewer originalRenewer = dfs.getLeaseRenewer();
       dfs.lastLeaseRenewal = Time.monotonicNow()
-      - HdfsConstants.LEASE_SOFTLIMIT_PERIOD - 1000;
+      - HdfsServerConstants.LEASE_SOFTLIMIT_PERIOD - 1000;
       try {
         dfs.renewLease();
       } catch (IOException e) {}
@@ -119,7 +114,7 @@ public class TestLease {
 
       // make it look like the hard limit has been exceeded.
       dfs.lastLeaseRenewal = Time.monotonicNow()
-      - HdfsConstants.LEASE_HARDLIMIT_PERIOD - 1000;
+      - HdfsServerConstants.LEASE_HARDLIMIT_PERIOD - 1000;
       dfs.renewLease();
 
       // this should not work.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery2.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery2.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery2.java
index 6d981fb..b8aac28 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery2.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery2.java
@@ -38,7 +38,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -333,8 +332,8 @@ public class TestLeaseRecovery2 {
     DFSTestUtil.updateConfWithFakeGroupMapping(conf, u2g_map);
 
     // Reset default lease periods
-    cluster.setLeasePeriod(HdfsConstants.LEASE_SOFTLIMIT_PERIOD,
-                           HdfsConstants.LEASE_HARDLIMIT_PERIOD);
+    cluster.setLeasePeriod(HdfsServerConstants.LEASE_SOFTLIMIT_PERIOD,
+                           HdfsServerConstants.LEASE_HARDLIMIT_PERIOD);
     //create a file
     // create a random file name
     String filestr = "/foo" + AppendTestUtil.nextInt();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java
index e9891bf..9ac58ba 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java
@@ -37,9 +37,9 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
@@ -533,7 +533,7 @@ public class UpgradeUtilities {
    * of the Namenode, whether it is running or not.
    */
   public static int getCurrentNameNodeLayoutVersion() {
-    return HdfsConstants.NAMENODE_LAYOUT_VERSION;
+    return HdfsServerConstants.NAMENODE_LAYOUT_VERSION;
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/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 2e84499..2115671 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
@@ -34,11 +34,11 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.qjournal.QJMTestUtil;
 import org.apache.hadoop.hdfs.qjournal.client.IPCLoggerChannel;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
@@ -185,7 +185,7 @@ public class TestJournalNode {
     byte[] retrievedViaHttp = DFSTestUtil.urlGetBytes(new URL(urlRoot +
         "/getJournal?segmentTxId=1&jid=" + journalId));
     byte[] expected = Bytes.concat(
-            Ints.toByteArray(HdfsConstants.NAMENODE_LAYOUT_VERSION),
+            Ints.toByteArray(HdfsServerConstants.NAMENODE_LAYOUT_VERSION),
             (new byte[] { 0, 0, 0, 0 }), // layout flags section
             EDITS_DATA);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/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 91abb2a..9ce16f2 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
@@ -44,8 +44,8 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
@@ -88,7 +88,8 @@ public class TestBlockManager {
   @Before
   public void setupMockCluster() throws IOException {
     Configuration conf = new HdfsConfiguration();
-    conf.set(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY, "need to set a dummy value here so it assumes a multi-rack cluster");
+    conf.set(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY,
+             "need to set a dummy value here so it assumes a multi-rack cluster");
     fsn = Mockito.mock(FSNamesystem.class);
     Mockito.doReturn(true).when(fsn).hasWriteLock();
     bm = new BlockManager(fsn, conf);
@@ -111,8 +112,8 @@ public class TestBlockManager {
     for (DatanodeDescriptor dn : nodesToAdd) {
       cluster.add(dn);
       dn.getStorageInfos()[0].setUtilizationForTesting(
-          2 * HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
-          2 * HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L);
+          2 * HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+          2 * HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L);
       dn.updateHeartbeat(
           BlockManagerTestUtil.getStorageReportsForDatanode(dn), 0L, 0L, 0, 0,
           null);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
index 296003f..1d6dad8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
@@ -50,8 +50,8 @@ import org.apache.hadoop.hdfs.LogVerificationAppender;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.TestBlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager.StatefulBlockInfo;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -138,8 +138,8 @@ public class TestReplicationPolicy {
     }
     for (int i=0; i < NUM_OF_DATANODES; i++) {
       updateHeartbeatWithUsage(dataNodes[i],
-          2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
-          2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
+          2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+          2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
     }    
   }
 
@@ -162,8 +162,8 @@ public class TestReplicationPolicy {
   @Test
   public void testChooseTarget1() throws Exception {
     updateHeartbeatWithUsage(dataNodes[0],
-        2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 
-        HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+        2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+        HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
         0L, 0L, 4, 0); // overloaded
 
     DatanodeStorageInfo[] targets;
@@ -193,8 +193,8 @@ public class TestReplicationPolicy {
     assertFalse(isOnSameRack(targets[0], targets[2]));
     
     updateHeartbeatWithUsage(dataNodes[0],
-        2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
-        HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
+        2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+        HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
   }
 
   private static DatanodeStorageInfo[] chooseTarget(int numOfReplicas) {
@@ -316,8 +316,8 @@ public class TestReplicationPolicy {
   public void testChooseTarget3() throws Exception {
     // make data node 0 to be not qualified to choose
     updateHeartbeatWithUsage(dataNodes[0],
-        2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
-        (HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L,
+        2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+        (HdfsServerConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L,
         0L, 0L, 0, 0); // no space
         
     DatanodeStorageInfo[] targets;
@@ -350,8 +350,8 @@ public class TestReplicationPolicy {
     assertFalse(isOnSameRack(targets[1], targets[3]));
 
     updateHeartbeatWithUsage(dataNodes[0],
-        2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
-        HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
+        2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+        HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
   }
   
   /**
@@ -367,8 +367,8 @@ public class TestReplicationPolicy {
     // make data node 0 & 1 to be not qualified to choose: not enough disk space
     for(int i=0; i<2; i++) {
       updateHeartbeatWithUsage(dataNodes[i],
-          2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
-          (HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
+          2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+          (HdfsServerConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
     }
       
     DatanodeStorageInfo[] targets;
@@ -395,8 +395,8 @@ public class TestReplicationPolicy {
     
     for(int i=0; i<2; i++) {
       updateHeartbeatWithUsage(dataNodes[i],
-          2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
-          HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
+          2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+          HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
     }
   }
 
@@ -459,8 +459,8 @@ public class TestReplicationPolicy {
     bm.getDatanodeManager().getNetworkTopology().add(newDn);
     bm.getDatanodeManager().getHeartbeatManager().addDatanode(newDn);
     updateHeartbeatWithUsage(newDn,
-        2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
-        2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
+        2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+        2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
 
     // Try picking three nodes. Only two should return.
     excludedNodes.clear();
@@ -506,8 +506,8 @@ public class TestReplicationPolicy {
     // make data node 0 & 1 to be not qualified to choose: not enough disk space
     for(int i=0; i<2; i++) {
       updateHeartbeatWithUsage(dataNodes[i],
-          2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
-          (HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
+          2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+          (HdfsServerConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
     }
     
     final LogVerificationAppender appender = new LogVerificationAppender();
@@ -531,8 +531,8 @@ public class TestReplicationPolicy {
     
     for(int i=0; i<2; i++) {
       updateHeartbeatWithUsage(dataNodes[i],
-          2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
-          HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
+          2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+          HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/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 d514768..7ff2930 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
@@ -34,8 +34,8 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.TestBlockStoragePolicy;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -93,8 +93,8 @@ public class TestReplicationPolicyConsiderLoad {
       dnrList.add(dnr);
       dnManager.registerDatanode(dnr);
       dataNodes[i].getStorageInfos()[0].setUtilizationForTesting(
-          2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*blockSize, 0L,
-          2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*blockSize, 0L);
+          2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*blockSize, 0L,
+          2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*blockSize, 0L);
       dataNodes[i].updateHeartbeat(
           BlockManagerTestUtil.getStorageReportsForDatanode(dataNodes[i]),
           0L, 0L, 0, 0, null);


[07/50] [abbrv] hadoop git commit: HDFS-8280. Code Cleanup in DFSInputStream. Contributed by Jing Zhao.

Posted by zj...@apache.org.
HDFS-8280. Code Cleanup in DFSInputStream. Contributed by Jing Zhao.


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

Branch: refs/heads/YARN-2928
Commit: a742c10e77a1a4ac8a272d722de491ad9c5e7738
Parents: be804ef
Author: Haohui Mai <wh...@apache.org>
Authored: Tue Apr 28 18:11:59 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:53 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 .../org/apache/hadoop/hdfs/DFSInputStream.java  | 141 ++++++++-----------
 2 files changed, 61 insertions(+), 82 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a742c10e/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 1c4cfb4..e7fa8fd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -478,6 +478,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-8176. Record from/to snapshots in audit log for snapshot diff report.
     (J. Andreina via jing9)
 
+    HDFS-8280. Code Cleanup in DFSInputStream. (Jing Zhao via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a742c10e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index 3f90397..3290223 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@ -601,7 +601,6 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
             targetBlock.getBlockSize() - 1;
       this.currentLocatedBlock = targetBlock;
 
-      assert (target==pos) : "Wrong postion " + pos + " expect " + target;
       long offsetIntoBlock = target - targetBlock.getStartOffset();
 
       DNAddrPair retval = chooseDataNode(targetBlock, null);
@@ -610,35 +609,12 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       StorageType storageType = retval.storageType;
 
       try {
-        ExtendedBlock blk = targetBlock.getBlock();
-        Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken();
-        CachingStrategy curCachingStrategy;
-        boolean shortCircuitForbidden;
-        synchronized(infoLock) {
-          curCachingStrategy = cachingStrategy;
-          shortCircuitForbidden = shortCircuitForbidden();
-        }
-        blockReader = new BlockReaderFactory(dfsClient.getConf()).
-            setInetSocketAddress(targetAddr).
-            setRemotePeerFactory(dfsClient).
-            setDatanodeInfo(chosenNode).
-            setStorageType(storageType).
-            setFileName(src).
-            setBlock(blk).
-            setBlockToken(accessToken).
-            setStartOffset(offsetIntoBlock).
-            setVerifyChecksum(verifyChecksum).
-            setClientName(dfsClient.clientName).
-            setLength(blk.getNumBytes() - offsetIntoBlock).
-            setCachingStrategy(curCachingStrategy).
-            setAllowShortCircuitLocalReads(!shortCircuitForbidden).
-            setClientCacheContext(dfsClient.getClientContext()).
-            setUserGroupInformation(dfsClient.ugi).
-            setConfiguration(dfsClient.getConfiguration()).
-            build();
+        blockReader = getBlockReader(targetBlock, offsetIntoBlock,
+            targetBlock.getBlockSize() - offsetIntoBlock, targetAddr,
+            storageType, chosenNode);
         if(connectFailedOnce) {
           DFSClient.LOG.info("Successfully connected to " + targetAddr +
-                             " for " + blk);
+                             " for " + targetBlock.getBlock());
         }
         return chosenNode;
       } catch (IOException ex) {
@@ -663,6 +639,37 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     }
   }
 
+  protected BlockReader getBlockReader(LocatedBlock targetBlock,
+      long offsetInBlock, long length, InetSocketAddress targetAddr,
+      StorageType storageType, DatanodeInfo datanode) throws IOException {
+    ExtendedBlock blk = targetBlock.getBlock();
+    Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken();
+    CachingStrategy curCachingStrategy;
+    boolean shortCircuitForbidden;
+    synchronized (infoLock) {
+      curCachingStrategy = cachingStrategy;
+      shortCircuitForbidden = shortCircuitForbidden();
+    }
+    return new BlockReaderFactory(dfsClient.getConf()).
+        setInetSocketAddress(targetAddr).
+        setRemotePeerFactory(dfsClient).
+        setDatanodeInfo(datanode).
+        setStorageType(storageType).
+        setFileName(src).
+        setBlock(blk).
+        setBlockToken(accessToken).
+        setStartOffset(offsetInBlock).
+        setVerifyChecksum(verifyChecksum).
+        setClientName(dfsClient.clientName).
+        setLength(length).
+        setCachingStrategy(curCachingStrategy).
+        setAllowShortCircuitLocalReads(!shortCircuitForbidden).
+        setClientCacheContext(dfsClient.getClientContext()).
+        setUserGroupInformation(dfsClient.ugi).
+        setConfiguration(dfsClient.getConfiguration()).
+        build();
+  }
+
   /**
    * Close it down!
    */
@@ -935,9 +942,10 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   private DNAddrPair chooseDataNode(LocatedBlock block,
       Collection<DatanodeInfo> ignoredNodes) throws IOException {
     while (true) {
-      try {
-        return getBestNodeDNAddrPair(block, ignoredNodes);
-      } catch (IOException ie) {
+      DNAddrPair result = getBestNodeDNAddrPair(block, ignoredNodes);
+      if (result != null) {
+        return result;
+      } else {
         String errMsg = getBestNodeDNAddrPairErrorString(block.getLocations(),
           deadNodes, ignoredNodes);
         String blockInfo = block.getBlock() + " file=" + src;
@@ -954,7 +962,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           DFSClient.LOG.info("No node available for " + blockInfo);
         }
         DFSClient.LOG.info("Could not obtain " + block.getBlock()
-            + " from any node: " + ie + errMsg
+            + " from any node: " + errMsg
             + ". Will get new block locations from namenode and retry...");
         try {
           // Introducing a random factor to the wait time before another retry.
@@ -977,7 +985,6 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         openInfo();
         block = getBlockAt(block.getStartOffset());
         failures++;
-        continue;
       }
     }
   }
@@ -986,11 +993,10 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * Get the best node from which to stream the data.
    * @param block LocatedBlock, containing nodes in priority order.
    * @param ignoredNodes Do not choose nodes in this array (may be null)
-   * @return The DNAddrPair of the best node.
-   * @throws IOException
+   * @return The DNAddrPair of the best node. Null if no node can be chosen.
    */
   private DNAddrPair getBestNodeDNAddrPair(LocatedBlock block,
-      Collection<DatanodeInfo> ignoredNodes) throws IOException {
+      Collection<DatanodeInfo> ignoredNodes) {
     DatanodeInfo[] nodes = block.getLocations();
     StorageType[] storageTypes = block.getStorageTypes();
     DatanodeInfo chosenNode = null;
@@ -1010,9 +1016,10 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       }
     }
     if (chosenNode == null) {
-      throw new IOException("No live nodes contain block " + block.getBlock() +
+      DFSClient.LOG.warn("No live nodes contain block " + block.getBlock() +
           " after checking nodes = " + Arrays.toString(nodes) +
           ", ignoredNodes = " + ignoredNodes);
+      return null;
     }
     final String dnAddr =
         chosenNode.getXferAddr(dfsClient.getConf().isConnectToDnViaHostname());
@@ -1102,40 +1109,13 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       // cached block locations may have been updated by chooseDataNode()
       // or fetchBlockAt(). Always get the latest list of locations at the
       // start of the loop.
-      CachingStrategy curCachingStrategy;
-      boolean allowShortCircuitLocalReads;
       LocatedBlock block = getBlockAt(blockStartOffset);
-      synchronized(infoLock) {
-        curCachingStrategy = cachingStrategy;
-        allowShortCircuitLocalReads = !shortCircuitForbidden();
-      }
-      DatanodeInfo chosenNode = datanode.info;
-      InetSocketAddress targetAddr = datanode.addr;
-      StorageType storageType = datanode.storageType;
       BlockReader reader = null;
-
       try {
         DFSClientFaultInjector.get().fetchFromDatanodeException();
-        Token<BlockTokenIdentifier> blockToken = block.getBlockToken();
         int len = (int) (end - start + 1);
-        reader = new BlockReaderFactory(dfsClient.getConf()).
-            setInetSocketAddress(targetAddr).
-            setRemotePeerFactory(dfsClient).
-            setDatanodeInfo(chosenNode).
-            setStorageType(storageType).
-            setFileName(src).
-            setBlock(block.getBlock()).
-            setBlockToken(blockToken).
-            setStartOffset(start).
-            setVerifyChecksum(verifyChecksum).
-            setClientName(dfsClient.clientName).
-            setLength(len).
-            setCachingStrategy(curCachingStrategy).
-            setAllowShortCircuitLocalReads(allowShortCircuitLocalReads).
-            setClientCacheContext(dfsClient.getClientContext()).
-            setUserGroupInformation(dfsClient.ugi).
-            setConfiguration(dfsClient.getConfiguration()).
-            build();
+        reader = getBlockReader(block, start, len, datanode.addr,
+            datanode.storageType, datanode.info);
         int nread = reader.readAll(buf, offset, len);
         updateReadStatistics(readStatistics, nread, reader);
 
@@ -1148,34 +1128,33 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       } catch (ChecksumException e) {
         String msg = "fetchBlockByteRange(). Got a checksum exception for "
             + src + " at " + block.getBlock() + ":" + e.getPos() + " from "
-            + chosenNode;
+            + datanode.info;
         DFSClient.LOG.warn(msg);
         // we want to remember what we have tried
-        addIntoCorruptedBlockMap(block.getBlock(), chosenNode, corruptedBlockMap);
-        addToDeadNodes(chosenNode);
+        addIntoCorruptedBlockMap(block.getBlock(), datanode.info,
+            corruptedBlockMap);
+        addToDeadNodes(datanode.info);
         throw new IOException(msg);
       } catch (IOException e) {
         if (e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
           DFSClient.LOG.info("Will fetch a new encryption key and retry, " 
-              + "encryption key was invalid when connecting to " + targetAddr
+              + "encryption key was invalid when connecting to " + datanode.addr
               + " : " + e);
           // The encryption key used is invalid.
           refetchEncryptionKey--;
           dfsClient.clearDataEncryptionKey();
-          continue;
-        } else if (refetchToken > 0 && tokenRefetchNeeded(e, targetAddr)) {
+        } else if (refetchToken > 0 && tokenRefetchNeeded(e, datanode.addr)) {
           refetchToken--;
           try {
             fetchBlockAt(block.getStartOffset());
           } catch (IOException fbae) {
             // ignore IOE, since we can retry it later in a loop
           }
-          continue;
         } else {
-          String msg = "Failed to connect to " + targetAddr + " for file "
+          String msg = "Failed to connect to " + datanode.addr + " for file "
               + src + " for block " + block.getBlock() + ":" + e;
           DFSClient.LOG.warn("Connection failure: " + msg, e);
-          addToDeadNodes(chosenNode);
+          addToDeadNodes(datanode.info);
           throw new IOException(msg);
         }
       } finally {
@@ -1187,10 +1166,9 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   }
 
   /**
-   * Like {@link #fetchBlockByteRange(LocatedBlock, long, long, byte[],
-   * int, Map)} except we start up a second, parallel, 'hedged' read
-   * if the first read is taking longer than configured amount of
-   * time.  We then wait on which ever read returns first.
+   * Like {@link #fetchBlockByteRange} except we start up a second, parallel,
+   * 'hedged' read if the first read is taking longer than configured amount of
+   * time. We then wait on which ever read returns first.
    */
   private void hedgedFetchBlockByteRange(long blockStartOffset, long start,
       long end, byte[] buf, int offset,
@@ -1248,9 +1226,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         // ongoing. Call getBestNodeDNAddrPair instead of chooseDataNode.
         // If no nodes to do hedged reads against, pass.
         try {
-          try {
-            chosenNode = getBestNodeDNAddrPair(block, ignored);
-          } catch (IOException ioe) {
+          chosenNode = getBestNodeDNAddrPair(block, ignored);
+          if (chosenNode == null) {
             chosenNode = chooseDataNode(block, ignored);
           }
           bb = ByteBuffer.allocate(len);


[27/50] [abbrv] hadoop git commit: Updating CHANGES.txt - Pulling in HDFS-8091 to branch-2.7 (for 2.7.1)

Posted by zj...@apache.org.
Updating CHANGES.txt - Pulling in HDFS-8091 to branch-2.7 (for 2.7.1)


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

Branch: refs/heads/YARN-2928
Commit: 884d677d3eedbcc1254a25676d64a7598f96ad8e
Parents: fe95065
Author: Arun Suresh <as...@apache.org>
Authored: Fri May 1 11:48:54 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:57 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/884d677d/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 16094a2..5f6b283 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -539,9 +539,6 @@ Release 2.8.0 - UNRELEASED
     HDFS-8096. DatanodeMetrics#blocksReplicated will get incremented early and
     even for failed transfers (vinayakumarb)
 
-    HDFS-8091: ACLStatus and XAttributes should be presented to
-    INodeAttributesProvider before returning to client (asuresh)
-
     HDFS-7939. Two fsimage_rollback_* files are created which are not deleted
     after rollback. (J.Andreina via vinayakumarb)
 
@@ -660,6 +657,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-8269. getBlockLocations() does not resolve the .reserved path and
     generates incorrect edit logs when updating the atime. (wheat9)
 
+    HDFS-8091: ACLStatus and XAttributes should be presented to
+    INodeAttributesProvider before returning to client (asuresh)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES


[10/50] [abbrv] hadoop git commit: HDFS-5574. Remove buffer copy in BlockReader.skip. Contributed by Binglin Chang.

Posted by zj...@apache.org.
HDFS-5574. Remove buffer copy in BlockReader.skip. Contributed by Binglin Chang.


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

Branch: refs/heads/YARN-2928
Commit: 69f6468e5a9db3620e933207e18e3c8f37b234ad
Parents: 26fee33
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Apr 30 19:09:57 2015 +0900
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:54 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/FSInputChecker.java    |  25 +++-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../apache/hadoop/hdfs/RemoteBlockReader.java   |  18 +--
 .../apache/hadoop/hdfs/RemoteBlockReader2.java  |  37 +++---
 .../apache/hadoop/hdfs/TestBlockReaderBase.java |  94 +++++++++++++++
 .../apache/hadoop/hdfs/TestDFSInputStream.java  | 114 +++++++++++++++++++
 .../hadoop/hdfs/TestRemoteBlockReader.java      |  27 +++++
 .../hadoop/hdfs/TestRemoteBlockReader2.java     |  25 ++++
 8 files changed, 309 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/69f6468e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSInputChecker.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSInputChecker.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSInputChecker.java
index 889ccc1..9b66c95 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSInputChecker.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSInputChecker.java
@@ -214,7 +214,30 @@ abstract public class FSInputChecker extends FSInputStream {
     count = readChecksumChunk(buf, 0, maxChunkSize);
     if (count < 0) count = 0;
   }
-  
+
+  /**
+   * Like read(byte[], int, int), but does not provide a dest buffer,
+   * so the read data is discarded.
+   * @param      len maximum number of bytes to read.
+   * @return     the number of bytes read.
+   * @throws     IOException  if an I/O error occurs.
+   */
+  final protected synchronized int readAndDiscard(int len) throws IOException {
+    int total = 0;
+    while (total < len) {
+      if (pos >= count) {
+        count = readChecksumChunk(buf, 0, maxChunkSize);
+        if (count <= 0) {
+          break;
+        }
+      }
+      int rd = Math.min(count - pos, len - total);
+      pos += rd;
+      total += rd;
+    }
+    return total;
+  }
+
   /*
    * Read characters into a portion of an array, reading from the underlying
    * stream at most once if necessary.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69f6468e/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index fbeb45d..c538b78 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -483,6 +483,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8283. DataStreamer cleanup and some minor improvement. (szetszwo via
     jing9)
 
+    HDFS-5574. Remove buffer copy in BlockReader.skip.
+    (Binglin Chang via aajisaka)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69f6468e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
index ce96ac9..d70f419 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
@@ -97,7 +97,6 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
   private boolean eos = false;
   private boolean sentStatusCode = false;
   
-  byte[] skipBuf = null;
   ByteBuffer checksumBytes = null;
   /** Amount of unread data in the current received packet */
   int dataLeft = 0;
@@ -126,10 +125,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
     if (lastChunkLen < 0 && startOffset > firstChunkOffset && len > 0) {
       // Skip these bytes. But don't call this.skip()!
       int toSkip = (int)(startOffset - firstChunkOffset);
-      if ( skipBuf == null ) {
-        skipBuf = new byte[bytesPerChecksum];
-      }
-      if ( super.read(skipBuf, 0, toSkip) != toSkip ) {
+      if ( super.readAndDiscard(toSkip) != toSkip ) {
         // should never happen
         throw new IOException("Could not skip required number of bytes");
       }
@@ -152,15 +148,11 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
   public synchronized long skip(long n) throws IOException {
     /* How can we make sure we don't throw a ChecksumException, at least
      * in majority of the cases?. This one throws. */  
-    if ( skipBuf == null ) {
-      skipBuf = new byte[bytesPerChecksum]; 
-    }
-
     long nSkipped = 0;
-    while ( nSkipped < n ) {
-      int toSkip = (int)Math.min(n-nSkipped, skipBuf.length);
-      int ret = read(skipBuf, 0, toSkip);
-      if ( ret <= 0 ) {
+    while (nSkipped < n) {
+      int toSkip = (int)Math.min(n-nSkipped, Integer.MAX_VALUE);
+      int ret = readAndDiscard(toSkip);
+      if (ret <= 0) {
         return nSkipped;
       }
       nSkipped += ret;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69f6468e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
index 9245a84..c368d65 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
@@ -122,12 +122,7 @@ public class RemoteBlockReader2  implements BlockReader {
   private final boolean verifyChecksum;
 
   private boolean sentStatusCode = false;
-  
-  byte[] skipBuf = null;
-  ByteBuffer checksumBytes = null;
-  /** Amount of unread data in the current received packet */
-  int dataLeft = 0;
-  
+
   @VisibleForTesting
   public Peer getPeer() {
     return peer;
@@ -172,7 +167,7 @@ public class RemoteBlockReader2  implements BlockReader {
 
 
   @Override
-  public int read(ByteBuffer buf) throws IOException {
+  public synchronized int read(ByteBuffer buf) throws IOException {
     if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
       TraceScope scope = Trace.startSpan(
           "RemoteBlockReader2#readNextPacket(" + blockId + ")", Sampler.NEVER);
@@ -257,21 +252,23 @@ public class RemoteBlockReader2  implements BlockReader {
   @Override
   public synchronized long skip(long n) throws IOException {
     /* How can we make sure we don't throw a ChecksumException, at least
-     * in majority of the cases?. This one throws. */  
-    if ( skipBuf == null ) {
-      skipBuf = new byte[bytesPerChecksum]; 
-    }
-
-    long nSkipped = 0;
-    while ( nSkipped < n ) {
-      int toSkip = (int)Math.min(n-nSkipped, skipBuf.length);
-      int ret = read(skipBuf, 0, toSkip);
-      if ( ret <= 0 ) {
-        return nSkipped;
+     * in majority of the cases?. This one throws. */
+    long skipped = 0;
+    while (skipped < n) {
+      long needToSkip = n - skipped;
+      if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
+        readNextPacket();
       }
-      nSkipped += ret;
+      if (curDataSlice.remaining() == 0) {
+        // we're at EOF now
+        break;
+      }
+
+      int skip = (int)Math.min(curDataSlice.remaining(), needToSkip);
+      curDataSlice.position(curDataSlice.position() + skip);
+      skipped += skip;
     }
-    return nSkipped;
+    return skipped;
   }
 
   private void readTrailingEmptyPacket() throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69f6468e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderBase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderBase.java
new file mode 100644
index 0000000..3d916a7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderBase.java
@@ -0,0 +1,94 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+abstract public class TestBlockReaderBase {
+  private BlockReaderTestUtil util;
+  private byte[] blockData;
+  private BlockReader reader;
+
+  /**
+   * if override this, make sure return array length is less than
+   * block size.
+   */
+  byte [] getBlockData() {
+    int length = 1 << 22;
+    byte[] data = new byte[length];
+    for (int i = 0; i < length; i++) {
+      data[i] = (byte) (i % 133);
+    }
+    return data;
+  }
+
+  private BlockReader getBlockReader(LocatedBlock block) throws Exception {
+    return util.getBlockReader(block, 0, blockData.length);
+  }
+
+  abstract HdfsConfiguration createConf();
+
+  @Before
+  public void setup() throws Exception {
+    util = new BlockReaderTestUtil(1, createConf());
+    blockData = getBlockData();
+    DistributedFileSystem fs = util.getCluster().getFileSystem();
+    Path testfile = new Path("/testfile");
+    FSDataOutputStream fout = fs.create(testfile);
+    fout.write(blockData);
+    fout.close();
+    LocatedBlock blk = util.getFileBlocks(testfile, blockData.length).get(0);
+    reader = getBlockReader(blk);
+  }
+
+  @After
+  public void shutdown() throws Exception {
+    util.shutdown();
+  }
+
+  @Test(timeout=60000)
+  public void testSkip() throws IOException {
+    Random random = new Random();
+    byte [] buf = new byte[1];
+    for (int pos = 0; pos < blockData.length;) {
+      long skip = random.nextInt(100) + 1;
+      long skipped = reader.skip(skip);
+      if (pos + skip >= blockData.length) {
+        assertEquals(blockData.length, pos + skipped);
+        break;
+      } else {
+        assertEquals(skip, skipped);
+        pos += skipped;
+        assertEquals(1, reader.read(buf, 0, 1));
+
+        assertEquals(blockData[pos], buf[0]);
+        pos += 1;
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69f6468e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInputStream.java
new file mode 100644
index 0000000..b9ec2ce
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInputStream.java
@@ -0,0 +1,114 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.hamcrest.CoreMatchers.equalTo;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.net.unix.DomainSocket;
+import org.apache.hadoop.net.unix.TemporarySocketDirectory;
+import org.junit.Assume;
+import org.junit.Test;
+
+public class TestDFSInputStream {
+  private void testSkipInner(MiniDFSCluster cluster) throws IOException {
+    DistributedFileSystem fs = cluster.getFileSystem();
+    DFSClient client = fs.dfs;
+    Path file = new Path("/testfile");
+    int fileLength = 1 << 22;
+    byte[] fileContent = new byte[fileLength];
+    for (int i = 0; i < fileLength; i++) {
+      fileContent[i] = (byte) (i % 133);
+    }
+    FSDataOutputStream fout = fs.create(file);
+    fout.write(fileContent);
+    fout.close();
+    Random random = new Random();
+    for (int i = 3; i < 18; i++) {
+      DFSInputStream fin = client.open("/testfile");
+      for (long pos = 0; pos < fileLength;) {
+        long skip = random.nextInt(1 << i) + 1;
+        long skipped = fin.skip(skip);
+        if (pos + skip >= fileLength) {
+          assertEquals(fileLength, pos + skipped);
+          break;
+        } else {
+          assertEquals(skip, skipped);
+          pos += skipped;
+          int data = fin.read();
+          assertEquals(pos % 133, data);
+          pos += 1;
+        }
+      }
+      fin.close();
+    }
+  }
+
+  @Test(timeout=60000)
+  public void testSkipWithRemoteBlockReader() throws IOException {
+    Configuration conf = new Configuration();
+    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADER, true);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
+    try {
+      testSkipInner(cluster);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  @Test(timeout=60000)
+  public void testSkipWithRemoteBlockReader2() throws IOException {
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
+    try {
+      testSkipInner(cluster);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  @Test(timeout=60000)
+  public void testSkipWithLocalBlockReader() throws IOException {
+    Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
+    TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
+    DomainSocket.disableBindPathValidation();
+    Configuration conf = new Configuration();
+    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
+    conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
+        new File(sockDir.getDir(),
+          "TestShortCircuitLocalRead._PORT.sock").getAbsolutePath());
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
+    try {
+      DFSInputStream.tcpReadsDisabledForTesting = true;
+      testSkipInner(cluster);
+    } finally {
+      DFSInputStream.tcpReadsDisabledForTesting = false;
+      cluster.shutdown();
+      sockDir.close();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69f6468e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRemoteBlockReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRemoteBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRemoteBlockReader.java
new file mode 100644
index 0000000..8ab110d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRemoteBlockReader.java
@@ -0,0 +1,27 @@
+/**
+ * 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;
+
+public class TestRemoteBlockReader extends TestBlockReaderBase {
+
+  HdfsConfiguration createConf() {
+    HdfsConfiguration conf = new HdfsConfiguration();
+    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADER, true);
+    return conf;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69f6468e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRemoteBlockReader2.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRemoteBlockReader2.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRemoteBlockReader2.java
new file mode 100644
index 0000000..c23b4b7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRemoteBlockReader2.java
@@ -0,0 +1,25 @@
+/**
+ * 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;
+
+public class TestRemoteBlockReader2 extends TestBlockReaderBase {
+  HdfsConfiguration createConf() {
+    HdfsConfiguration conf = new HdfsConfiguration();
+    return conf;
+  }
+}


[46/50] [abbrv] hadoop git commit: HDFS-8290. WebHDFS calls before namesystem initialization can cause NullPointerException. Contributed by Chris Nauroth.

Posted by zj...@apache.org.
HDFS-8290. WebHDFS calls before namesystem initialization can cause NullPointerException. Contributed by Chris Nauroth.


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

Branch: refs/heads/YARN-2928
Commit: f58d6cf79ba0cc834c5800eb4195795f2ba911c1
Parents: 0e9b9a9
Author: cnauroth <cn...@apache.org>
Authored: Mon May 4 11:35:04 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:59:01 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +++
 .../web/resources/NamenodeWebHdfsMethods.java   |  7 ++++++-
 .../web/resources/TestWebHdfsDataLocality.java  | 20 +++++++++++++++++++-
 3 files changed, 28 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f58d6cf7/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index e525800..a365b86 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -610,6 +610,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8309. Skip unit test using DataNodeTestUtils#injectDataDirFailure() on Windows.
     (xyao)
 
+    HDFS-8290. WebHDFS calls before namesystem initialization can cause
+    NullPointerException. (cnauroth)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f58d6cf7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
index 2c1148e..d33721c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
@@ -69,6 +69,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.web.JsonUtil;
@@ -164,7 +165,11 @@ public class NamenodeWebHdfsMethods {
   static DatanodeInfo chooseDatanode(final NameNode namenode,
       final String path, final HttpOpParam.Op op, final long openOffset,
       final long blocksize, final String excludeDatanodes) throws IOException {
-    final BlockManager bm = namenode.getNamesystem().getBlockManager();
+    FSNamesystem fsn = namenode.getNamesystem();
+    if (fsn == null) {
+      throw new IOException("Namesystem has not been intialized yet.");
+    }
+    final BlockManager bm = fsn.getBlockManager();
     
     HashSet<Node> excludes = new HashSet<Node>();
     if (excludeDatanodes != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f58d6cf7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/web/resources/TestWebHdfsDataLocality.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/web/resources/TestWebHdfsDataLocality.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/web/resources/TestWebHdfsDataLocality.java
index 077361c..15e1c04 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/web/resources/TestWebHdfsDataLocality.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/web/resources/TestWebHdfsDataLocality.java
@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.web.resources;
 
+import static org.mockito.Mockito.*;
+
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.List;
 
@@ -42,7 +45,9 @@ import org.apache.hadoop.hdfs.web.resources.PostOpParam;
 import org.apache.hadoop.hdfs.web.resources.PutOpParam;
 import org.apache.log4j.Level;
 import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
 
 /**
  * Test WebHDFS which provides data locality using HTTP redirection.
@@ -57,6 +62,9 @@ public class TestWebHdfsDataLocality {
   private static final String RACK1 = "/rack1";
   private static final String RACK2 = "/rack2";
 
+  @Rule
+  public final ExpectedException exception = ExpectedException.none();
+
   @Test
   public void testDataLocality() throws Exception {
     final Configuration conf = WebHdfsTestUtil.createConf();
@@ -213,4 +221,14 @@ public class TestWebHdfsDataLocality {
       cluster.shutdown();
     }
   }
-}
\ No newline at end of file
+
+  @Test
+  public void testChooseDatanodeBeforeNamesystemInit() throws Exception {
+    NameNode nn = mock(NameNode.class);
+    when(nn.getNamesystem()).thenReturn(null);
+    exception.expect(IOException.class);
+    exception.expectMessage("Namesystem has not been intialized yet.");
+    NamenodeWebHdfsMethods.chooseDatanode(nn, "/path", PutOpParam.Op.CREATE, 0,
+        DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT, null);
+  }
+}


[44/50] [abbrv] hadoop git commit: MAPREDUCE-6349. Fix typo in property org.apache.hadoop.mapreduce.lib.chain.Chain.REDUCER_INPUT_VALUE_CLASS. Contributed by Ray Chiang.

Posted by zj...@apache.org.
MAPREDUCE-6349. Fix typo in property org.apache.hadoop.mapreduce.lib.chain.Chain.REDUCER_INPUT_VALUE_CLASS. Contributed by Ray Chiang.


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

Branch: refs/heads/YARN-2928
Commit: 341c617d6be61e4264cfa99e89905d22646c45b1
Parents: 62d72e2
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Mon May 4 15:36:33 2015 +0900
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:59:00 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                              | 3 +++
 .../main/java/org/apache/hadoop/mapreduce/lib/chain/Chain.java    | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/341c617d/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 86477c3..117aafa 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -380,6 +380,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-5905. CountersStrings.toEscapedCompactStrings outputs
     unnecessary "null" strings. (Akira AJISAKA via ozawa)
 
+    MAPREDUCE-6349. Fix typo in property org.apache.hadoop.mapreduce.
+    lib.chain.Chain.REDUCER_INPUT_VALUE_CLASS. (Ray Chiang via ozawa)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/341c617d/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/chain/Chain.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/chain/Chain.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/chain/Chain.java
index 1dad13e..803ece7 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/chain/Chain.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/chain/Chain.java
@@ -68,7 +68,7 @@ public class Chain {
   protected static final String REDUCER_INPUT_KEY_CLASS = 
     "mapreduce.chain.reducer.input.key.class";
   protected static final String REDUCER_INPUT_VALUE_CLASS = 
-    "maperduce.chain.reducer.input.value.class";
+    "mapreduce.chain.reducer.input.value.class";
   protected static final String REDUCER_OUTPUT_KEY_CLASS = 
     "mapreduce.chain.reducer.output.key.class";
   protected static final String REDUCER_OUTPUT_VALUE_CLASS = 


[47/50] [abbrv] hadoop git commit: MAPREDUCE-5649. Reduce cannot use more than 2G memory for the final merge. Contributed by Gera Shegalov

Posted by zj...@apache.org.
MAPREDUCE-5649. Reduce cannot use more than 2G memory for the final merge. Contributed by Gera Shegalov


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

Branch: refs/heads/YARN-2928
Commit: aa5c6f409510d26deed943c8ce0863c3056a13cf
Parents: f58d6cf
Author: Jason Lowe <jl...@apache.org>
Authored: Mon May 4 19:02:39 2015 +0000
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:59:01 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |  3 ++
 .../mapreduce/task/reduce/MergeManagerImpl.java | 47 +++++++++++---------
 .../mapreduce/task/reduce/TestMergeManager.java | 29 ++++++++++++
 3 files changed, 57 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa5c6f40/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 117aafa..8a62f92 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -383,6 +383,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6349. Fix typo in property org.apache.hadoop.mapreduce.
     lib.chain.Chain.REDUCER_INPUT_VALUE_CLASS. (Ray Chiang via ozawa)
 
+    MAPREDUCE-5649. Reduce cannot use more than 2G memory for the final merge
+    (Gera Shegalov via jlowe)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa5c6f40/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MergeManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MergeManagerImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MergeManagerImpl.java
index 8bf17ef..f788707 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MergeManagerImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MergeManagerImpl.java
@@ -93,8 +93,10 @@ public class MergeManagerImpl<K, V> implements MergeManager<K, V> {
   
   Set<CompressAwarePath> onDiskMapOutputs = new TreeSet<CompressAwarePath>();
   private final OnDiskMerger onDiskMerger;
-  
-  private final long memoryLimit;
+
+  @VisibleForTesting
+  final long memoryLimit;
+
   private long usedMemory;
   private long commitMemory;
   private final long maxSingleShuffleLimit;
@@ -167,11 +169,10 @@ public class MergeManagerImpl<K, V> implements MergeManager<K, V> {
     }
 
     // Allow unit tests to fix Runtime memory
-    this.memoryLimit = 
-      (long)(jobConf.getLong(MRJobConfig.REDUCE_MEMORY_TOTAL_BYTES,
-          Math.min(Runtime.getRuntime().maxMemory(), Integer.MAX_VALUE))
-        * maxInMemCopyUse);
- 
+    this.memoryLimit = (long)(jobConf.getLong(
+        MRJobConfig.REDUCE_MEMORY_TOTAL_BYTES,
+        Runtime.getRuntime().maxMemory()) * maxInMemCopyUse);
+
     this.ioSortFactor = jobConf.getInt(MRJobConfig.IO_SORT_FACTOR, 100);
 
     final float singleShuffleMemoryLimitPercent =
@@ -202,7 +203,7 @@ public class MergeManagerImpl<K, V> implements MergeManager<K, V> {
 
     if (this.maxSingleShuffleLimit >= this.mergeThreshold) {
       throw new RuntimeException("Invalid configuration: "
-          + "maxSingleShuffleLimit should be less than mergeThreshold"
+          + "maxSingleShuffleLimit should be less than mergeThreshold "
           + "maxSingleShuffleLimit: " + this.maxSingleShuffleLimit
           + "mergeThreshold: " + this.mergeThreshold);
     }
@@ -668,24 +669,26 @@ public class MergeManagerImpl<K, V> implements MergeManager<K, V> {
     }
   }
 
-  private RawKeyValueIterator finalMerge(JobConf job, FileSystem fs,
-                                       List<InMemoryMapOutput<K,V>> inMemoryMapOutputs,
-                                       List<CompressAwarePath> onDiskMapOutputs
-                                       ) throws IOException {
-    LOG.info("finalMerge called with " + 
-             inMemoryMapOutputs.size() + " in-memory map-outputs and " + 
-             onDiskMapOutputs.size() + " on-disk map-outputs");
-    
+  @VisibleForTesting
+  final long getMaxInMemReduceLimit() {
     final float maxRedPer =
-      job.getFloat(MRJobConfig.REDUCE_INPUT_BUFFER_PERCENT, 0f);
+        jobConf.getFloat(MRJobConfig.REDUCE_INPUT_BUFFER_PERCENT, 0f);
     if (maxRedPer > 1.0 || maxRedPer < 0.0) {
-      throw new IOException(MRJobConfig.REDUCE_INPUT_BUFFER_PERCENT +
-                            maxRedPer);
+      throw new RuntimeException(maxRedPer + ": "
+          + MRJobConfig.REDUCE_INPUT_BUFFER_PERCENT
+          + " must be a float between 0 and 1.0");
     }
-    int maxInMemReduce = (int)Math.min(
-        Runtime.getRuntime().maxMemory() * maxRedPer, Integer.MAX_VALUE);
-    
+    return (long)(memoryLimit * maxRedPer);
+  }
 
+  private RawKeyValueIterator finalMerge(JobConf job, FileSystem fs,
+                                       List<InMemoryMapOutput<K,V>> inMemoryMapOutputs,
+                                       List<CompressAwarePath> onDiskMapOutputs
+                                       ) throws IOException {
+    LOG.info("finalMerge called with " +
+        inMemoryMapOutputs.size() + " in-memory map-outputs and " +
+        onDiskMapOutputs.size() + " on-disk map-outputs");
+    final long maxInMemReduce = getMaxInMemReduceLimit();
     // merge config params
     Class<K> keyClass = (Class<K>)job.getMapOutputKeyClass();
     Class<V> valueClass = (Class<V>)job.getMapOutputValueClass();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa5c6f40/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestMergeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestMergeManager.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestMergeManager.java
index 8d6bab9..ef860af 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestMergeManager.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestMergeManager.java
@@ -260,4 +260,33 @@ public class TestMergeManager {
     }
 
   }
+
+  @Test
+  public void testLargeMemoryLimits() throws Exception {
+    final JobConf conf = new JobConf();
+    // Xmx in production
+    conf.setLong(MRJobConfig.REDUCE_MEMORY_TOTAL_BYTES,
+        8L * 1024 * 1024 * 1024);
+
+    // M1 = Xmx fraction for map outputs
+    conf.setFloat(MRJobConfig.SHUFFLE_INPUT_BUFFER_PERCENT, 1.0f);
+
+    // M2 = max M1 fraction for a single maple output
+    conf.setFloat(MRJobConfig.SHUFFLE_MEMORY_LIMIT_PERCENT, 0.95f);
+
+    // M3 = M1 fraction at which in memory merge is triggered
+    conf.setFloat(MRJobConfig.SHUFFLE_MERGE_PERCENT, 1.0f);
+
+    // M4 = M1 fraction of map outputs remaining in memory for a reduce
+    conf.setFloat(MRJobConfig.REDUCE_INPUT_BUFFER_PERCENT, 1.0f);
+
+    final MergeManagerImpl<Text, Text> mgr = new MergeManagerImpl<Text, Text>(
+        null, conf, mock(LocalFileSystem.class), null, null, null, null, null,
+        null, null, null, null, null, new MROutputFiles());
+    assertTrue("Large shuffle area unusable: " + mgr.memoryLimit,
+        mgr.memoryLimit > Integer.MAX_VALUE);
+    final long maxInMemReduce = mgr.getMaxInMemReduceLimit();
+    assertTrue("Large in-memory reduce area unusable: " + maxInMemReduce,
+        maxInMemReduce > Integer.MAX_VALUE);
+  }
 }


[21/50] [abbrv] hadoop git commit: HDFS-8292. Move conditional in fmt_time from dfs-dust.js to status.html. Contributed by Charles Lamb.

Posted by zj...@apache.org.
HDFS-8292. Move conditional in fmt_time from dfs-dust.js to status.html. Contributed by Charles Lamb.


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

Branch: refs/heads/YARN-2928
Commit: 17a96fafe7b7d37133a34aece18c63796a657a50
Parents: f07f356
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Apr 30 18:23:13 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:56 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                       | 3 +++
 .../hadoop-hdfs/src/main/webapps/secondary/status.html            | 2 +-
 .../hadoop-hdfs/src/main/webapps/static/dfs-dust.js               | 3 ---
 3 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/17a96faf/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 29e3f85..7e4a6f2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -488,6 +488,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8200. Refactor FSDirStatAndListingOp. (wheat9)
 
+    HDFS-8292. Move conditional in fmt_time from dfs-dust.js to status.html.
+    (Charles Lamb via wang)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/17a96faf/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.html
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.html
index 2ef58ad..72f3ff7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.html
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.html
@@ -65,7 +65,7 @@
   <tr><th>Compiled</th><td>{CompileInfo}</td></tr>
   <tr><th>NameNode Address</th><td>{HostAndPort}</td></tr>
   <tr><th>Started</th><td>{StartTime|date_tostring}</td></tr>
-  <tr><th>Last Checkpoint</th><td>{@if cond="{LastCheckpointTime} === 0"}Never{:else}{LastCheckpointTime|date_tostring} ({LastCheckpointDeltaMs|fmt_time} ago){/if}</td></tr>
+  <tr><th>Last Checkpoint</th><td>{@if cond="{LastCheckpointTime} === 0"}Never{:else}{LastCheckpointTime|date_tostring}{/if} {@if cond="{LastCheckpointDeltaMs} >= 0"} ({LastCheckpointDeltaMs|fmt_time} ago){/if}</td></tr>
   <tr><th>Checkpoint Period</th><td>{CheckpointPeriod} seconds</td></tr>
   <tr><th>Checkpoint Transactions</th><td>{TxnCount}</td></tr>
 </table>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/17a96faf/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dfs-dust.js
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dfs-dust.js b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dfs-dust.js
index f483d23..3c8efd9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dfs-dust.js
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dfs-dust.js
@@ -40,9 +40,6 @@
     },
 
     'fmt_time': function (v) {
-      if (v < 0) {
-        return "unknown";
-      }
       var s = Math.floor(v / 1000), h = Math.floor(s / 3600);
       s -= h * 3600;
       var m = Math.floor(s / 60);


[18/50] [abbrv] hadoop git commit: HDFS-7770. Need document for storage type label of data node storage locations under dfs.data.dir. Contributed by Xiaoyu Yao.

Posted by zj...@apache.org.
HDFS-7770. Need document for storage type label of data node storage locations under dfs.data.dir. Contributed by Xiaoyu Yao.


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

Branch: refs/heads/YARN-2928
Commit: e5bb1e51cecfece97cc31ba9f386cd992f3d3b50
Parents: 5e30ec9
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Apr 30 20:12:32 2015 +0900
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:55 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                  | 3 +++
 .../hadoop-hdfs/src/main/resources/hdfs-default.xml          | 7 +++++--
 .../hadoop-hdfs/src/site/markdown/ArchivalStorage.md         | 8 ++++++++
 3 files changed, 16 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e5bb1e51/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index c538b78..e092dc8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -603,6 +603,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-8204. Mover/Balancer should not schedule two replicas to the same
     datanode.  (Walter Su via szetszwo)
 
+    HDFS-7770. Need document for storage type label of data node storage
+    locations under dfs.data.dir. (Xiaoyu Yao via aajisaka)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e5bb1e51/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 092d5aa..f03c0fb 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
@@ -430,8 +430,11 @@
   <description>Determines where on the local filesystem an DFS data node
   should store its blocks.  If this is a comma-delimited
   list of directories, then data will be stored in all named
-  directories, typically on different devices.
-  Directories that do not exist are ignored.
+  directories, typically on different devices. The directories should be tagged
+  with corresponding storage types ([SSD]/[DISK]/[ARCHIVE]/[RAM_DISK]) for HDFS
+  storage policies. The default storage type will be DISK if the directory does
+  not have a storage type tagged explicitly. Directories that do not exist will
+  be created if local filesystem permission allows.
   </description>
 </property>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e5bb1e51/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
index 2038401..d924f80 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ArchivalStorage.md
@@ -95,6 +95,14 @@ The effective storage policy can be retrieved by the "[`dfsadmin -getStoragePoli
 ### Configuration
 
 * **dfs.storage.policy.enabled** - for enabling/disabling the storage policy feature. The default value is `true`.
+* **dfs.datanode.data.dir** - on each data node, the comma-separated storage locations should be tagged with their storage types. This allows storage policies to place the blocks on different storage types according to policy. For example:
+
+    1.  A datanode storage location /grid/dn/disk0 on DISK should be configured with `[DISK]file:///grid/dn/disk0`
+    2.  A datanode storage location /grid/dn/ssd0 on SSD can should configured with `[SSD]file:///grid/dn/ssd0`
+    3.  A datanode storage location /grid/dn/archive0 on ARCHIVE should be configured with `[ARCHIVE]file:///grid/dn/archive0`
+    4.  A datanode storage location /grid/dn/ram0 on RAM_DISK should be configured with `[RAM_DISK]file:///grid/dn/ram0`
+
+    The default storage type of a datanode storage location will be DISK if it does not have a storage type tagged explicitly.
 
 Mover - A New Data Migration Tool
 ---------------------------------


[34/50] [abbrv] hadoop git commit: HADOOP-11900. Add failIfNoTests=false to hadoop-build-tools pom. (gera)

Posted by zj...@apache.org.
HADOOP-11900. Add failIfNoTests=false to hadoop-build-tools pom. (gera)


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

Branch: refs/heads/YARN-2928
Commit: 1fde5bea611c9d4b98db228c4ba8213321f1b9c9
Parents: ad8c70a
Author: Gera Shegalov <ge...@apache.org>
Authored: Fri May 1 15:57:42 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:59 2015 -0700

----------------------------------------------------------------------
 hadoop-build-tools/pom.xml                      | 19 +++++++++++--------
 hadoop-common-project/hadoop-common/CHANGES.txt |  5 +++++
 2 files changed, 16 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1fde5bea/hadoop-build-tools/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-build-tools/pom.xml b/hadoop-build-tools/pom.xml
index 1931072..bb902b8 100644
--- a/hadoop-build-tools/pom.xml
+++ b/hadoop-build-tools/pom.xml
@@ -15,14 +15,17 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0"
          xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
          xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-<parent>
-  <artifactId>hadoop-main</artifactId>
-  <groupId>org.apache.hadoop</groupId>
-  <version>3.0.0-SNAPSHOT</version>
-</parent>
-<modelVersion>4.0.0</modelVersion>
-
-<artifactId>hadoop-build-tools</artifactId>
+  <parent>
+    <artifactId>hadoop-main</artifactId>
+    <groupId>org.apache.hadoop</groupId>
+    <version>3.0.0-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+  <artifactId>hadoop-build-tools</artifactId>
   <description>Apache Hadoop Build Tools Project</description>
   <name>Apache Hadoop Build Tools</name>
+
+  <properties>
+    <failIfNoTests>false</failIfNoTests>
+  </properties>
 </project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1fde5bea/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index d00e3ef..ea3316a 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -588,6 +588,11 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11491. HarFs incorrectly declared as requiring an authority.
     (Brahma Reddy Battula via gera)
 
+    HADOOP-11889. Make checkstyle runnable from root project
+    (Gera Shegalov via jeagles)
+
+    HADOOP-11900. Add failIfNoTests=false to hadoop-build-tools pom. (gera)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[45/50] [abbrv] hadoop git commit: HADOOP-9658. SnappyCodec#checkNativeCodeLoaded may unexpectedly fail when native code is not loaded. Contributed by Zhijie Shen.

Posted by zj...@apache.org.
HADOOP-9658. SnappyCodec#checkNativeCodeLoaded may unexpectedly fail when native code is not loaded. Contributed by Zhijie Shen.


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

Branch: refs/heads/YARN-2928
Commit: 922924775579a0a90228b558873a4da35360d9bc
Parents: 341c617
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Mon May 4 17:05:00 2015 +0900
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:59:01 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                   | 3 +++
 .../src/main/java/org/apache/hadoop/io/compress/SnappyCodec.java  | 3 ++-
 2 files changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/92292477/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index ea3316a..bb8f900 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -616,6 +616,9 @@ Release 2.7.1 - UNRELEASED
 
     HADOOP-11891. OsSecureRandom should lazily fill its reservoir (asuresh)
 
+    HADOOP-9658. SnappyCodec#checkNativeCodeLoaded may unexpectedly fail when
+    native code is not loaded. (Zhijie Shen via ozawa)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92292477/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/SnappyCodec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/SnappyCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/SnappyCodec.java
index 8d2fa1a..2a9c5d0 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/SnappyCodec.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/SnappyCodec.java
@@ -60,7 +60,8 @@ public class SnappyCodec implements Configurable, CompressionCodec, DirectDecomp
    * Are the native snappy libraries loaded & initialized?
    */
   public static void checkNativeCodeLoaded() {
-      if (!NativeCodeLoader.buildSupportsSnappy()) {
+      if (!NativeCodeLoader.isNativeCodeLoaded() ||
+          !NativeCodeLoader.buildSupportsSnappy()) {
         throw new RuntimeException("native snappy library not available: " +
             "this version of libhadoop was built without " +
             "snappy support.");


[30/50] [abbrv] hadoop git commit: YARN-2893. AMLaucher: sporadic job failures due to EOFException in readTokenStorageStream. (Zhihai Xu via gera)

Posted by zj...@apache.org.
YARN-2893. AMLaucher: sporadic job failures due to EOFException in readTokenStorageStream. (Zhihai Xu via gera)


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

Branch: refs/heads/YARN-2928
Commit: 9b14ff8da473d9ff3bd1750bcf348a4af2d719d4
Parents: 41ef214
Author: Gera Shegalov <ge...@apache.org>
Authored: Fri May 1 14:49:09 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:58 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../server/resourcemanager/RMAppManager.java    | 36 +++++------
 .../resourcemanager/amlauncher/AMLauncher.java  | 11 +++-
 .../server/resourcemanager/TestAppManager.java  | 60 ++++++++++++++++++
 .../TestApplicationMasterLauncher.java          | 64 ++++++++++++++++++++
 5 files changed, 153 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9b14ff8d/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 6c1a2e1..55bf9e4 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -351,6 +351,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3564. Fix TestContainerAllocation.testAMContainerAllocationWhenDNSUnavailable 
     fails randomly. (Jian He via wangda)
 
+    YARN-2893. AMLaucher: sporadic job failures due to EOFException in
+    readTokenStorageStream. (Zhihai Xu via gera)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9b14ff8d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.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/RMAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
index dcc2a64..d946a2f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
@@ -281,29 +281,29 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
     RMAppImpl application =
         createAndPopulateNewRMApp(submissionContext, submitTime, user, false);
     ApplicationId appId = submissionContext.getApplicationId();
-
-    if (UserGroupInformation.isSecurityEnabled()) {
-      try {
+    Credentials credentials = null;
+    try {
+      credentials = parseCredentials(submissionContext);
+      if (UserGroupInformation.isSecurityEnabled()) {
         this.rmContext.getDelegationTokenRenewer().addApplicationAsync(appId,
-            parseCredentials(submissionContext),
-            submissionContext.getCancelTokensWhenComplete(),
+            credentials, submissionContext.getCancelTokensWhenComplete(),
             application.getUser());
-      } catch (Exception e) {
-        LOG.warn("Unable to parse credentials.", e);
-        // Sending APP_REJECTED is fine, since we assume that the
-        // RMApp is in NEW state and thus we haven't yet informed the
-        // scheduler about the existence of the application
-        assert application.getState() == RMAppState.NEW;
+      } else {
+        // Dispatcher is not yet started at this time, so these START events
+        // enqueued should be guaranteed to be first processed when dispatcher
+        // gets started.
         this.rmContext.getDispatcher().getEventHandler()
-          .handle(new RMAppRejectedEvent(applicationId, e.getMessage()));
-        throw RPCUtil.getRemoteException(e);
+            .handle(new RMAppEvent(applicationId, RMAppEventType.START));
       }
-    } else {
-      // Dispatcher is not yet started at this time, so these START events
-      // enqueued should be guaranteed to be first processed when dispatcher
-      // gets started.
+    } catch (Exception e) {
+      LOG.warn("Unable to parse credentials.", e);
+      // Sending APP_REJECTED is fine, since we assume that the
+      // RMApp is in NEW state and thus we haven't yet informed the
+      // scheduler about the existence of the application
+      assert application.getState() == RMAppState.NEW;
       this.rmContext.getDispatcher().getEventHandler()
-        .handle(new RMAppEvent(applicationId, RMAppEventType.START));
+          .handle(new RMAppRejectedEvent(applicationId, e.getMessage()));
+      throw RPCUtil.getRemoteException(e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9b14ff8d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.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/amlauncher/AMLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java
index b5021ca..f62ee50 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java
@@ -28,6 +28,7 @@ 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.conf.Configuration;
 import org.apache.hadoop.io.DataInputByteBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
@@ -201,7 +202,9 @@ public class AMLauncher implements Runnable {
     return container;
   }
 
-  private void setupTokens(
+  @Private
+  @VisibleForTesting
+  protected void setupTokens(
       ContainerLaunchContext container, ContainerId containerID)
       throws IOException {
     Map<String, String> environment = container.getEnvironment();
@@ -227,10 +230,12 @@ public class AMLauncher implements Runnable {
     }
     Credentials credentials = new Credentials();
     DataInputByteBuffer dibb = new DataInputByteBuffer();
-    if (container.getTokens() != null) {
+    ByteBuffer tokens = container.getTokens();
+    if (tokens != null) {
       // TODO: Don't do this kind of checks everywhere.
-      dibb.reset(container.getTokens());
+      dibb.reset(tokens);
       credentials.readTokenStorageStream(dibb);
+      tokens.rewind();
     }
 
     // Add AMRMToken

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9b14ff8d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.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/TestAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java
index 5ebc68c..3db8b7c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java
@@ -21,6 +21,8 @@ package org.apache.hadoop.yarn.server.resourcemanager;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 
@@ -33,6 +35,7 @@ import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.List;
 import java.util.concurrent.ConcurrentMap;
@@ -479,6 +482,63 @@ public class TestAppManager{
         getAppEventType());
   }
 
+  @Test
+  public void testRMAppSubmitWithInvalidTokens() throws Exception {
+    // Setup invalid security tokens
+    DataOutputBuffer dob = new DataOutputBuffer();
+    ByteBuffer securityTokens = ByteBuffer.wrap(dob.getData(), 0,
+        dob.getLength());
+    asContext.getAMContainerSpec().setTokens(securityTokens);
+    try {
+      appMonitor.submitApplication(asContext, "test");
+      Assert.fail("Application submission should fail because" +
+          " Tokens are invalid.");
+    } catch (YarnException e) {
+      // Exception is expected
+      Assert.assertTrue("The thrown exception is not" +
+          " java.io.EOFException",
+          e.getMessage().contains("java.io.EOFException"));
+    }
+    int timeoutSecs = 0;
+    while ((getAppEventType() == RMAppEventType.KILL) &&
+        timeoutSecs++ < 20) {
+      Thread.sleep(1000);
+    }
+    Assert.assertEquals("app event type sent is wrong",
+        RMAppEventType.APP_REJECTED, getAppEventType());
+    asContext.getAMContainerSpec().setTokens(null);
+  }
+
+  @Test
+  public void testRMAppSubmitWithValidTokens() throws Exception {
+    // Setup valid security tokens
+    DataOutputBuffer dob = new DataOutputBuffer();
+    Credentials credentials = new Credentials();
+    credentials.writeTokenStorageToStream(dob);
+    ByteBuffer securityTokens = ByteBuffer.wrap(dob.getData(), 0,
+        dob.getLength());
+    asContext.getAMContainerSpec().setTokens(securityTokens);
+    appMonitor.submitApplication(asContext, "test");
+    RMApp app = rmContext.getRMApps().get(appId);
+    Assert.assertNotNull("app is null", app);
+    Assert.assertEquals("app id doesn't match", appId,
+        app.getApplicationId());
+    Assert.assertEquals("app state doesn't match", RMAppState.NEW,
+        app.getState());
+    verify(metricsPublisher).appACLsUpdated(
+        any(RMApp.class), any(String.class), anyLong());
+
+    // wait for event to be processed
+    int timeoutSecs = 0;
+    while ((getAppEventType() == RMAppEventType.KILL) &&
+        timeoutSecs++ < 20) {
+      Thread.sleep(1000);
+    }
+    Assert.assertEquals("app event type sent is wrong", RMAppEventType.START,
+        getAppEventType());
+    asContext.getAMContainerSpec().setTokens(null);
+  }
+
   @Test (timeout = 30000)
   public void testRMAppSubmitMaxAppAttempts() throws Exception {
     int[] globalMaxAppAttempts = new int[] { 10, 1 };

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9b14ff8d/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 11cd1fd..9a4395e 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
@@ -26,6 +26,9 @@ import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
@@ -38,6 +41,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.SerializedException;
@@ -47,7 +51,10 @@ import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException;
 import org.apache.hadoop.yarn.exceptions.ApplicationMasterNotRegisteredException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
+import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
+import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncher;
+import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
@@ -238,4 +245,61 @@ public class TestApplicationMasterLauncher {
     } catch (ApplicationAttemptNotFoundException e) {
     }
   }
+
+  @Test
+  public void testSetupTokens() throws Exception {
+    MockRM rm = new MockRM();
+    rm.start();
+    MockNM nm1 = rm.registerNode("h1:1234", 5000);
+    RMApp app = rm.submitApp(2000);
+    /// kick the scheduling
+    nm1.nodeHeartbeat(true);
+    RMAppAttempt attempt = app.getCurrentAppAttempt();
+    MyAMLauncher launcher = new MyAMLauncher(rm.getRMContext(),
+        attempt, AMLauncherEventType.LAUNCH, rm.getConfig());
+    DataOutputBuffer dob = new DataOutputBuffer();
+    Credentials ts = new Credentials();
+    ts.writeTokenStorageToStream(dob);
+    ByteBuffer securityTokens = ByteBuffer.wrap(dob.getData(),
+        0, dob.getLength());
+    ContainerLaunchContext amContainer =
+        ContainerLaunchContext.newInstance(null, null,
+            null, null, securityTokens, null);
+    ContainerId containerId = ContainerId.newContainerId(
+        attempt.getAppAttemptId(), 0L);
+
+    try {
+      launcher.setupTokens(amContainer, containerId);
+    } catch (Exception e) {
+      // ignore the first fake exception
+    }
+    try {
+      launcher.setupTokens(amContainer, containerId);
+    } catch (java.io.EOFException e) {
+      Assert.fail("EOFException should not happen.");
+    }
+  }
+
+  static class MyAMLauncher extends AMLauncher {
+    int count;
+    public MyAMLauncher(RMContext rmContext, RMAppAttempt application,
+        AMLauncherEventType eventType, Configuration conf) {
+      super(rmContext, application, eventType, conf);
+      count = 0;
+    }
+
+    protected org.apache.hadoop.security.token.Token<AMRMTokenIdentifier>
+        createAndSetAMRMToken() {
+      count++;
+      if (count == 1) {
+        throw new RuntimeException("createAndSetAMRMToken failure");
+      }
+      return null;
+    }
+
+    protected void setupTokens(ContainerLaunchContext container,
+        ContainerId containerID) throws IOException {
+      super.setupTokens(container, containerID);
+    }
+  }
 }


[28/50] [abbrv] hadoop git commit: HDFS-8276. LazyPersistFileScrubber should be disabled if scrubber interval configured zero. (Contributed by Surendra Singh Lilhore)

Posted by zj...@apache.org.
HDFS-8276. LazyPersistFileScrubber should be disabled if scrubber interval configured zero. (Contributed by Surendra Singh Lilhore)


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

Branch: refs/heads/YARN-2928
Commit: 49aee364e78a59bf83b7cc5c19913822c50990da
Parents: 884d677
Author: Arpit Agarwal <ar...@apache.org>
Authored: Fri May 1 11:11:48 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:57 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +++
 .../hdfs/server/namenode/FSNamesystem.java      |  8 +++++--
 .../fsdataset/impl/LazyPersistTestCase.java     | 20 ++++++++++++----
 .../fsdataset/impl/TestLazyPersistFiles.java    | 25 ++++++++++++++++++++
 4 files changed, 50 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/49aee364/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 5f6b283..b5c5e6b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -595,6 +595,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8300. Fix unit test failures and findbugs warning caused by HDFS-8283.
     (jing9)
 
+    HDFS-8276. LazyPersistFileScrubber should be disabled if scrubber interval
+    configured zero. (Surendra Singh Lilhore via Arpit Agarwal)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49aee364/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 b938263..d5ff80e 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
@@ -825,9 +825,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC,
           DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC_DEFAULT);
 
-      if (this.lazyPersistFileScrubIntervalSec == 0) {
+      if (this.lazyPersistFileScrubIntervalSec < 0) {
         throw new IllegalArgumentException(
-            DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC + " must be non-zero.");
+            DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC
+                + " must be zero (for disable) or greater than zero.");
       }
 
       // For testing purposes, allow the DT secret manager to be started regardless
@@ -1173,6 +1174,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         lazyPersistFileScrubber = new Daemon(new LazyPersistFileScrubber(
             lazyPersistFileScrubIntervalSec));
         lazyPersistFileScrubber.start();
+      } else {
+        LOG.warn("Lazy persist file scrubber is disabled,"
+            + " configured scrub interval is zero.");
       }
 
       cacheManager.startMonitorThread();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49aee364/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
index d46964b..93cd23a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
@@ -58,6 +58,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -239,12 +240,17 @@ public abstract class LazyPersistTestCase {
       long ramDiskStorageLimit,
       long evictionLowWatermarkReplicas,
       boolean useSCR,
-      boolean useLegacyBlockReaderLocal) throws IOException {
+      boolean useLegacyBlockReaderLocal,
+      boolean disableScrubber) throws IOException {
 
     Configuration conf = new Configuration();
     conf.setLong(DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
-    conf.setInt(DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC,
-                LAZY_WRITE_FILE_SCRUBBER_INTERVAL_SEC);
+    if (disableScrubber) {
+      conf.setInt(DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC, 0);
+    } else {
+      conf.setInt(DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC,
+          LAZY_WRITE_FILE_SCRUBBER_INTERVAL_SEC);
+    }
     conf.setLong(DFS_HEARTBEAT_INTERVAL_KEY, HEARTBEAT_INTERVAL_SEC);
     conf.setInt(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY,
                 HEARTBEAT_RECHECK_INTERVAL_MSEC);
@@ -357,11 +363,16 @@ public abstract class LazyPersistTestCase {
       return this;
     }
 
+    public ClusterWithRamDiskBuilder disableScrubber() {
+      this.disableScrubber = true;
+      return this;
+    }
+
     public void build() throws IOException {
       LazyPersistTestCase.this.startUpCluster(
           numDatanodes, hasTransientStorage, storageTypes, ramDiskReplicaCapacity,
           ramDiskStorageLimit, evictionLowWatermarkReplicas,
-          useScr, useLegacyBlockReaderLocal);
+          useScr, useLegacyBlockReaderLocal,disableScrubber);
     }
 
     private int numDatanodes = REPL_FACTOR;
@@ -372,6 +383,7 @@ public abstract class LazyPersistTestCase {
     private boolean useScr = false;
     private boolean useLegacyBlockReaderLocal = false;
     private long evictionLowWatermarkReplicas = EVICTION_LOW_WATERMARK;
+    private boolean disableScrubber=false;
   }
 
   protected final void triggerBlockReport()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49aee364/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java
index 41398c8..60cc8fe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java
@@ -120,6 +120,31 @@ public class TestLazyPersistFiles extends LazyPersistTestCase {
                is(0L));
   }
 
+  @Test
+  public void testDisableLazyPersistFileScrubber()
+      throws IOException, InterruptedException {
+    getClusterBuilder().setRamDiskReplicaCapacity(2).disableScrubber().build();
+    final String METHOD_NAME = GenericTestUtils.getMethodName();
+    Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
+
+    makeTestFile(path1, BLOCK_SIZE, true);
+    ensureFileReplicasOnStorageType(path1, RAM_DISK);
+
+    // Stop the DataNode and sleep for the time it takes the NN to
+    // detect the DN as being dead.
+    cluster.shutdownDataNodes();
+    Thread.sleep(30000L);
+
+    // Next, wait for the replication monitor to mark the file as corrupt
+    Thread.sleep(2 * DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT * 1000);
+
+    // Wait for the LazyPersistFileScrubber to run
+    Thread.sleep(2 * LAZY_WRITE_FILE_SCRUBBER_INTERVAL_SEC * 1000);
+
+    // Ensure that path1 exist.
+    Assert.assertTrue(fs.exists(path1));
+
+  }
   /**
    * Concurrent read from the same node and verify the contents.
    */


[14/50] [abbrv] hadoop git commit: HDFS-8200. Refactor FSDirStatAndListingOp. Contributed by Haohui Mai.

Posted by zj...@apache.org.
HDFS-8200. Refactor FSDirStatAndListingOp. Contributed by Haohui Mai.


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

Branch: refs/heads/YARN-2928
Commit: 98401412b87f48788ff2e18376b4117f93895fd8
Parents: c2390c9
Author: Haohui Mai <wh...@apache.org>
Authored: Thu Apr 30 13:41:46 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:55 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 .../server/namenode/FSDirStatAndListingOp.java  | 155 ++++++++++---------
 .../hdfs/server/namenode/FSEditLogLoader.java   |   6 +-
 3 files changed, 91 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/98401412/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index e092dc8..29e3f85 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -486,6 +486,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-5574. Remove buffer copy in BlockReader.skip.
     (Binglin Chang via aajisaka)
 
+    HDFS-8200. Refactor FSDirStatAndListingOp. (wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98401412/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 cea2b82..e77ddae 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
@@ -24,7 +24,6 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.DirectoryListingStartAfterNotFoundException;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.InvalidPathException;
-import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtil;
@@ -180,10 +179,14 @@ class FSDirStatAndListingOp {
           .BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
 
       if (!targetNode.isDirectory()) {
+        INodeAttributes nodeAttrs = getINodeAttributes(
+            fsd, src, HdfsFileStatus.EMPTY_NAME, targetNode,
+            snapshot);
         return new DirectoryListing(
-            new HdfsFileStatus[]{createFileStatus(fsd, src,
-                HdfsFileStatus.EMPTY_NAME, targetNode, needLocation,
-                parentStoragePolicy, snapshot, isRawPath, iip)}, 0);
+            new HdfsFileStatus[]{ createFileStatus(
+                fsd, HdfsFileStatus.EMPTY_NAME, targetNode, nodeAttrs,
+                needLocation, parentStoragePolicy, snapshot, isRawPath, iip)
+            }, 0);
       }
 
       final INodeDirectory dirInode = targetNode.asDirectory();
@@ -200,8 +203,11 @@ class FSDirStatAndListingOp {
         byte curPolicy = isSuperUser && !cur.isSymlink()?
             cur.getLocalStoragePolicyID():
             HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
-        listing[i] = createFileStatus(fsd, src, cur.getLocalNameBytes(), cur,
-            needLocation, getStoragePolicyID(curPolicy,
+        INodeAttributes nodeAttrs = getINodeAttributes(
+            fsd, src, cur.getLocalNameBytes(), cur,
+            snapshot);
+        listing[i] = createFileStatus(fsd, cur.getLocalNameBytes(),
+            cur, nodeAttrs, needLocation, getStoragePolicyID(curPolicy,
                 parentStoragePolicy), snapshot, isRawPath, iip);
         listingCnt++;
         if (needLocation) {
@@ -253,9 +259,15 @@ class FSDirStatAndListingOp {
     final HdfsFileStatus listing[] = new HdfsFileStatus[numOfListing];
     for (int i = 0; i < numOfListing; i++) {
       Snapshot.Root sRoot = snapshots.get(i + skipSize).getRoot();
-      listing[i] = createFileStatus(fsd, src, sRoot.getLocalNameBytes(), sRoot,
-          HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID,
-          false, INodesInPath.fromINode(sRoot));
+      INodeAttributes nodeAttrs = getINodeAttributes(
+          fsd, src, sRoot.getLocalNameBytes(),
+          node, Snapshot.CURRENT_STATE_ID);
+      listing[i] = createFileStatus(
+          fsd, sRoot.getLocalNameBytes(),
+          sRoot, nodeAttrs,
+          HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
+          Snapshot.CURRENT_STATE_ID, false,
+          INodesInPath.fromINode(sRoot));
     }
     return new DirectoryListing(
         listing, snapshots.size() - skipSize - numOfListing);
@@ -276,11 +288,20 @@ class FSDirStatAndListingOp {
     fsd.readLock();
     try {
       final INode i = src.getLastINode();
-      byte policyId = includeStoragePolicy && i != null && !i.isSymlink() ?
-          i.getStoragePolicyID() : HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
-      return i == null ? null : createFileStatus(
-          fsd, path, HdfsFileStatus.EMPTY_NAME, i, policyId,
-          src.getPathSnapshotId(), isRawPath, src);
+      if (i == null) {
+        return null;
+      }
+
+      byte policyId = includeStoragePolicy && !i.isSymlink() ?
+          i.getStoragePolicyID() : HdfsConstantsClient
+          .BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
+      INodeAttributes nodeAttrs = getINodeAttributes(
+          fsd, path, HdfsFileStatus.EMPTY_NAME, i, src.getPathSnapshotId());
+      return createFileStatus(
+          fsd, HdfsFileStatus.EMPTY_NAME,
+          i, nodeAttrs, policyId,
+          src.getPathSnapshotId(),
+          isRawPath, src);
     } finally {
       fsd.readUnlock();
     }
@@ -309,23 +330,6 @@ class FSDirStatAndListingOp {
     }
   }
 
-  /**
-   * Currently we only support "ls /xxx/.snapshot" which will return all the
-   * snapshots of a directory. The FSCommand Ls will first call getFileInfo to
-   * make sure the file/directory exists (before the real getListing call).
-   * Since we do not have a real INode for ".snapshot", we return an empty
-   * non-null HdfsFileStatus here.
-   */
-  private static HdfsFileStatus getFileInfo4DotSnapshot(
-      FSDirectory fsd, String src)
-      throws UnresolvedLinkException {
-    if (fsd.getINode4DotSnapshot(src) != null) {
-      return new HdfsFileStatus(0, true, 0, 0, 0, 0, null, null, null, null,
-          HdfsFileStatus.EMPTY_NAME, -1L, 0, null,
-          HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
-    }
-    return null;
-  }
 
   /**
    * create an hdfs file status from an inode
@@ -339,52 +343,63 @@ class FSDirStatAndListingOp {
    * @return a file status
    * @throws java.io.IOException if any error occurs
    */
-  static HdfsFileStatus createFileStatus(
-      FSDirectory fsd, String fullPath, byte[] path, INode node,
+  private static HdfsFileStatus createFileStatus(
+      FSDirectory fsd, byte[] path, INode node, INodeAttributes nodeAttrs,
       boolean needLocation, byte storagePolicy, int snapshot, boolean isRawPath,
       INodesInPath iip)
       throws IOException {
     if (needLocation) {
-      return createLocatedFileStatus(fsd, fullPath, path, node, storagePolicy,
-          snapshot, isRawPath, iip);
+      return createLocatedFileStatus(fsd, path, node, nodeAttrs, storagePolicy,
+                                     snapshot, isRawPath, iip);
     } else {
-      return createFileStatus(fsd, fullPath, path, node, storagePolicy, snapshot,
-          isRawPath, iip);
+      return createFileStatus(fsd, path, node, nodeAttrs, storagePolicy,
+                              snapshot, isRawPath, iip);
     }
   }
 
   /**
    * Create FileStatus by file INode
    */
-  static HdfsFileStatus createFileStatus(
+  static HdfsFileStatus createFileStatusForEditLog(
       FSDirectory fsd, String fullPath, byte[] path, INode node,
       byte storagePolicy, int snapshot, boolean isRawPath,
       INodesInPath iip) throws IOException {
-     long size = 0;     // length is zero for directories
-     short replication = 0;
-     long blocksize = 0;
-     final boolean isEncrypted;
-
-     final FileEncryptionInfo feInfo = isRawPath ? null :
-         fsd.getFileEncryptionInfo(node, snapshot, iip);
-
-     if (node.isFile()) {
-       final INodeFile fileNode = node.asFile();
-       size = fileNode.computeFileSize(snapshot);
-       replication = fileNode.getFileReplication(snapshot);
-       blocksize = fileNode.getPreferredBlockSize();
-       isEncrypted = (feInfo != null) ||
-           (isRawPath && fsd.isInAnEZ(INodesInPath.fromINode(node)));
-     } else {
-       isEncrypted = fsd.isInAnEZ(INodesInPath.fromINode(node));
-     }
-
-     int childrenNum = node.isDirectory() ?
-         node.asDirectory().getChildrenNum(snapshot) : 0;
-
-     INodeAttributes nodeAttrs =
-         fsd.getAttributes(fullPath, path, node, snapshot);
-     return new HdfsFileStatus(
+    INodeAttributes nodeAttrs = getINodeAttributes(
+        fsd, fullPath, path, node, snapshot);
+    return createFileStatus(fsd, path, node, nodeAttrs,
+                            storagePolicy, snapshot, isRawPath, iip);
+  }
+
+  /**
+   * Create FileStatus by file INode
+   */
+  static HdfsFileStatus createFileStatus(
+      FSDirectory fsd, byte[] path, INode node,
+      INodeAttributes nodeAttrs, byte storagePolicy, int snapshot,
+      boolean isRawPath, INodesInPath iip) throws IOException {
+    long size = 0;     // length is zero for directories
+    short replication = 0;
+    long blocksize = 0;
+    final boolean isEncrypted;
+
+    final FileEncryptionInfo feInfo = isRawPath ? null :
+        fsd.getFileEncryptionInfo(node, snapshot, iip);
+
+    if (node.isFile()) {
+      final INodeFile fileNode = node.asFile();
+      size = fileNode.computeFileSize(snapshot);
+      replication = fileNode.getFileReplication(snapshot);
+      blocksize = fileNode.getPreferredBlockSize();
+      isEncrypted = (feInfo != null) ||
+          (isRawPath && fsd.isInAnEZ(INodesInPath.fromINode(node)));
+    } else {
+      isEncrypted = fsd.isInAnEZ(INodesInPath.fromINode(node));
+    }
+
+    int childrenNum = node.isDirectory() ?
+        node.asDirectory().getChildrenNum(snapshot) : 0;
+
+    return new HdfsFileStatus(
         size,
         node.isDirectory(),
         replication,
@@ -402,13 +417,18 @@ class FSDirStatAndListingOp {
         storagePolicy);
   }
 
+  private static INodeAttributes getINodeAttributes(
+      FSDirectory fsd, String fullPath, byte[] path, INode node, int snapshot) {
+    return fsd.getAttributes(fullPath, path, node, snapshot);
+  }
+
   /**
    * Create FileStatus with location info by file INode
    */
   private static HdfsLocatedFileStatus createLocatedFileStatus(
-      FSDirectory fsd, String fullPath, byte[] path, INode node,
-      byte storagePolicy, int snapshot, boolean isRawPath,
-      INodesInPath iip) throws IOException {
+      FSDirectory fsd, byte[] path, INode node, INodeAttributes nodeAttrs,
+      byte storagePolicy, int snapshot,
+      boolean isRawPath, INodesInPath iip) throws IOException {
     assert fsd.hasReadLock();
     long size = 0; // length is zero for directories
     short replication = 0;
@@ -442,8 +462,6 @@ class FSDirStatAndListingOp {
     int childrenNum = node.isDirectory() ?
         node.asDirectory().getChildrenNum(snapshot) : 0;
 
-    INodeAttributes nodeAttrs =
-        fsd.getAttributes(fullPath, path, node, snapshot);
     HdfsLocatedFileStatus status =
         new HdfsLocatedFileStatus(size, node.isDirectory(), replication,
           blocksize, node.getModificationTime(snapshot),
@@ -468,7 +486,6 @@ class FSDirStatAndListingOp {
    * return an FsPermissionExtension.
    *
    * @param node INode to check
-   * @param snapshot int snapshot ID
    * @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.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98401412/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index 43171de..d234cf2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -378,7 +378,7 @@ public class FSEditLogLoader {
 
         // add the op into retry cache if necessary
         if (toAddRetryCache) {
-          HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatus(
+          HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatusForEditLog(
               fsNamesys.dir, path, HdfsFileStatus.EMPTY_NAME, newFile,
               HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID,
               false, iip);
@@ -397,7 +397,7 @@ public class FSEditLogLoader {
               false);
           // add the op into retry cache if necessary
           if (toAddRetryCache) {
-            HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatus(
+            HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatusForEditLog(
                 fsNamesys.dir, path,
                 HdfsFileStatus.EMPTY_NAME, newFile,
                 HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
@@ -471,7 +471,7 @@ public class FSEditLogLoader {
             false, false);
         // add the op into retry cache if necessary
         if (toAddRetryCache) {
-          HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatus(
+          HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatusForEditLog(
               fsNamesys.dir, path, HdfsFileStatus.EMPTY_NAME, file,
               HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
               Snapshot.CURRENT_STATE_ID, false, iip);


[24/50] [abbrv] hadoop git commit: HDFS-8086. Move LeaseRenewer to the hdfs.client.impl package. Contributed by Takanobu

Posted by zj...@apache.org.
HDFS-8086. Move LeaseRenewer to the hdfs.client.impl package.  Contributed by Takanobu


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

Branch: refs/heads/YARN-2928
Commit: cb5ae0845c7a6339963fda215109284b80a8660e
Parents: 49aee36
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Fri May 1 15:11:09 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:57 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  13 +-
 .../org/apache/hadoop/hdfs/LeaseRenewer.java    | 512 ------------------
 .../hadoop/hdfs/client/impl/LeaseRenewer.java   | 514 +++++++++++++++++++
 .../hadoop/hdfs/TestDistributedFileSystem.java  |  59 ++-
 .../java/org/apache/hadoop/hdfs/TestLease.java  |   1 +
 .../apache/hadoop/hdfs/TestLeaseRenewer.java    | 207 --------
 .../hdfs/client/impl/TestLeaseRenewer.java      | 209 ++++++++
 8 files changed, 769 insertions(+), 749 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb5ae084/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index b5c5e6b..179fe7e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -494,6 +494,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8292. Move conditional in fmt_time from dfs-dust.js to status.html.
     (Charles Lamb via wang)
 
+    HDFS-8086. Move LeaseRenewer to the hdfs.client.impl package.  (Takanobu
+    Asanuma via szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb5ae084/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index d47992b..aaba543 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -101,6 +101,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
+import org.apache.hadoop.hdfs.client.impl.LeaseRenewer;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.net.TcpPeerServer;
 import org.apache.hadoop.hdfs.protocol.AclException;
@@ -481,7 +482,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    *  enforced to consistently update its local dfsclients array and 
    *  client's filesBeingWritten map.
    */
-  void putFileBeingWritten(final long inodeId, final DFSOutputStream out) {
+  public void putFileBeingWritten(final long inodeId, final DFSOutputStream out) {
     synchronized(filesBeingWritten) {
       filesBeingWritten.put(inodeId, out);
       // update the last lease renewal time only when there was no
@@ -494,7 +495,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   }
 
   /** Remove a file. Only called from LeaseRenewer. */
-  void removeFileBeingWritten(final long inodeId) {
+  public void removeFileBeingWritten(final long inodeId) {
     synchronized(filesBeingWritten) {
       filesBeingWritten.remove(inodeId);
       if (filesBeingWritten.isEmpty()) {
@@ -504,14 +505,14 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   }
 
   /** Is file-being-written map empty? */
-  boolean isFilesBeingWrittenEmpty() {
+  public boolean isFilesBeingWrittenEmpty() {
     synchronized(filesBeingWritten) {
       return filesBeingWritten.isEmpty();
     }
   }
   
   /** @return true if the client is running */
-  boolean isClientRunning() {
+  public boolean isClientRunning() {
     return clientRunning;
   }
 
@@ -533,7 +534,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @return true if lease was renewed. May return false if this
    * client has been closed or has no files open.
    **/
-  boolean renewLease() throws IOException {
+  public boolean renewLease() throws IOException {
     if (clientRunning && !isFilesBeingWrittenEmpty()) {
       try {
         namenode.renewLease(clientName);
@@ -565,7 +566,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   }
   
   /** Abort and release resources held.  Ignore all errors. */
-  void abort() {
+  public void abort() {
     clientRunning = false;
     closeAllFilesBeingWritten(true);
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb5ae084/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java
deleted file mode 100644
index 511bddb..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java
+++ /dev/null
@@ -1,512 +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;
-
-import java.io.IOException;
-import java.net.SocketTimeoutException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.HadoopIllegalArgumentException;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.Daemon;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.util.Time;
-import com.google.common.annotations.VisibleForTesting;
-
-/**
- * <p>
- * Used by {@link DFSClient} for renewing file-being-written leases
- * on the namenode.
- * When a file is opened for write (create or append),
- * namenode stores a file lease for recording the identity of the writer.
- * The writer (i.e. the DFSClient) is required to renew the lease periodically.
- * When the lease is not renewed before it expires,
- * the namenode considers the writer as failed and then it may either let
- * another writer to obtain the lease or close the file.
- * </p>
- * <p>
- * This class also provides the following functionality:
- * <ul>
- * <li>
- * It maintains a map from (namenode, user) pairs to lease renewers. 
- * The same {@link LeaseRenewer} instance is used for renewing lease
- * for all the {@link DFSClient} to the same namenode and the same user.
- * </li>
- * <li>
- * Each renewer maintains a list of {@link DFSClient}.
- * Periodically the leases for all the clients are renewed.
- * A client is removed from the list when the client is closed.
- * </li>
- * <li>
- * A thread per namenode per user is used by the {@link LeaseRenewer}
- * to renew the leases.
- * </li>
- * </ul>
- * </p>
- */
-@InterfaceAudience.Private
-class LeaseRenewer {
-  static final Log LOG = LogFactory.getLog(LeaseRenewer.class);
-
-  static final long LEASE_RENEWER_GRACE_DEFAULT = 60*1000L;
-  static final long LEASE_RENEWER_SLEEP_DEFAULT = 1000L;
-
-  /** Get a {@link LeaseRenewer} instance */
-  static LeaseRenewer getInstance(final String authority,
-      final UserGroupInformation ugi, final DFSClient dfsc) throws IOException {
-    final LeaseRenewer r = Factory.INSTANCE.get(authority, ugi);
-    r.addClient(dfsc);
-    return r;
-  }
-
-  /** 
-   * A factory for sharing {@link LeaseRenewer} objects
-   * among {@link DFSClient} instances
-   * so that there is only one renewer per authority per user.
-   */
-  private static class Factory {
-    private static final Factory INSTANCE = new Factory();
-
-    private static class Key {
-      /** Namenode info */
-      final String authority;
-      /** User info */
-      final UserGroupInformation ugi;
-
-      private Key(final String authority, final UserGroupInformation ugi) {
-        if (authority == null) {
-          throw new HadoopIllegalArgumentException("authority == null");
-        } else if (ugi == null) {
-          throw new HadoopIllegalArgumentException("ugi == null");
-        }
-
-        this.authority = authority;
-        this.ugi = ugi;
-      }
-
-      @Override
-      public int hashCode() {
-        return authority.hashCode() ^ ugi.hashCode();
-      }
-
-      @Override
-      public boolean equals(Object obj) {
-        if (obj == this) {
-          return true;
-        }
-        if (obj != null && obj instanceof Key) {
-          final Key that = (Key)obj;
-          return this.authority.equals(that.authority)
-                 && this.ugi.equals(that.ugi);
-        }
-        return false;        
-      }
-
-      @Override
-      public String toString() {
-        return ugi.getShortUserName() + "@" + authority;
-      }
-    }
-
-    /** A map for per user per namenode renewers. */
-    private final Map<Key, LeaseRenewer> renewers = new HashMap<Key, LeaseRenewer>();
-
-    /** Get a renewer. */
-    private synchronized LeaseRenewer get(final String authority,
-        final UserGroupInformation ugi) {
-      final Key k = new Key(authority, ugi);
-      LeaseRenewer r = renewers.get(k);
-      if (r == null) {
-        r = new LeaseRenewer(k);
-        renewers.put(k, r);
-      }
-      return r;
-    }
-
-    /** Remove the given renewer. */
-    private synchronized void remove(final LeaseRenewer r) {
-      final LeaseRenewer stored = renewers.get(r.factorykey);
-      //Since a renewer may expire, the stored renewer can be different.
-      if (r == stored) {
-        if (!r.clientsRunning()) {
-          renewers.remove(r.factorykey);
-        }
-      }
-    }
-  }
-
-  /** The time in milliseconds that the map became empty. */
-  private long emptyTime = Long.MAX_VALUE;
-  /** A fixed lease renewal time period in milliseconds */
-  private long renewal = HdfsConstants.LEASE_SOFTLIMIT_PERIOD/2;
-
-  /** A daemon for renewing lease */
-  private Daemon daemon = null;
-  /** Only the daemon with currentId should run. */
-  private int currentId = 0;
-
-  /** 
-   * A period in milliseconds that the lease renewer thread should run
-   * after the map became empty.
-   * In other words,
-   * if the map is empty for a time period longer than the grace period,
-   * the renewer should terminate.  
-   */
-  private long gracePeriod;
-  /**
-   * The time period in milliseconds
-   * that the renewer sleeps for each iteration. 
-   */
-  private long sleepPeriod;
-
-  private final Factory.Key factorykey;
-
-  /** A list of clients corresponding to this renewer. */
-  private final List<DFSClient> dfsclients = new ArrayList<DFSClient>();
-
-  /**
-   * A stringified stack trace of the call stack when the Lease Renewer
-   * was instantiated. This is only generated if trace-level logging is
-   * enabled on this class.
-   */
-  private final String instantiationTrace;
-
-  private LeaseRenewer(Factory.Key factorykey) {
-    this.factorykey = factorykey;
-    unsyncSetGraceSleepPeriod(LEASE_RENEWER_GRACE_DEFAULT);
-    
-    if (LOG.isTraceEnabled()) {
-      instantiationTrace = StringUtils.stringifyException(
-        new Throwable("TRACE"));
-    } else {
-      instantiationTrace = null;
-    }
-  }
-
-  /** @return the renewal time in milliseconds. */
-  private synchronized long getRenewalTime() {
-    return renewal;
-  }
-
-  /** Add a client. */
-  private synchronized void addClient(final DFSClient dfsc) {
-    for(DFSClient c : dfsclients) {
-      if (c == dfsc) {
-        //client already exists, nothing to do.
-        return;
-      }
-    }
-    //client not found, add it
-    dfsclients.add(dfsc);
-
-    //update renewal time
-    final int hdfsTimeout = dfsc.getConf().getHdfsTimeout();
-    if (hdfsTimeout > 0) {
-      final long half = hdfsTimeout/2;
-      if (half < renewal) {
-        this.renewal = half;
-      }
-    }
-  }
-
-  private synchronized boolean clientsRunning() {
-    for(Iterator<DFSClient> i = dfsclients.iterator(); i.hasNext(); ) {
-      if (!i.next().isClientRunning()) {
-        i.remove();
-      }
-    }
-    return !dfsclients.isEmpty();
-  }
-
-  private synchronized long getSleepPeriod() {
-    return sleepPeriod;    
-  }
-
-  /** Set the grace period and adjust the sleep period accordingly. */
-  synchronized void setGraceSleepPeriod(final long gracePeriod) {
-    unsyncSetGraceSleepPeriod(gracePeriod);
-  }
-
-  private void unsyncSetGraceSleepPeriod(final long gracePeriod) {
-    if (gracePeriod < 100L) {
-      throw new HadoopIllegalArgumentException(gracePeriod
-          + " = gracePeriod < 100ms is too small.");
-    }
-    this.gracePeriod = gracePeriod;
-    final long half = gracePeriod/2;
-    this.sleepPeriod = half < LEASE_RENEWER_SLEEP_DEFAULT?
-        half: LEASE_RENEWER_SLEEP_DEFAULT;
-  }
-
-  /** Is the daemon running? */
-  synchronized boolean isRunning() {
-    return daemon != null && daemon.isAlive();
-  }
-
-  /** Does this renewer have nothing to renew? */
-  public boolean isEmpty() {
-    return dfsclients.isEmpty();
-  }
-  
-  /** Used only by tests */
-  synchronized String getDaemonName() {
-    return daemon.getName();
-  }
-
-  /** Is the empty period longer than the grace period? */  
-  private synchronized boolean isRenewerExpired() {
-    return emptyTime != Long.MAX_VALUE
-        && Time.monotonicNow() - emptyTime > gracePeriod;
-  }
-
-  synchronized void put(final long inodeId, final DFSOutputStream out,
-      final DFSClient dfsc) {
-    if (dfsc.isClientRunning()) {
-      if (!isRunning() || isRenewerExpired()) {
-        //start a new deamon with a new id.
-        final int id = ++currentId;
-        daemon = new Daemon(new Runnable() {
-          @Override
-          public void run() {
-            try {
-              if (LOG.isDebugEnabled()) {
-                LOG.debug("Lease renewer daemon for " + clientsString()
-                    + " with renew id " + id + " started");
-              }
-              LeaseRenewer.this.run(id);
-            } catch(InterruptedException e) {
-              if (LOG.isDebugEnabled()) {
-                LOG.debug(LeaseRenewer.this.getClass().getSimpleName()
-                    + " is interrupted.", e);
-              }
-            } finally {
-              synchronized(LeaseRenewer.this) {
-                Factory.INSTANCE.remove(LeaseRenewer.this);
-              }
-              if (LOG.isDebugEnabled()) {
-                LOG.debug("Lease renewer daemon for " + clientsString()
-                    + " with renew id " + id + " exited");
-              }
-            }
-          }
-          
-          @Override
-          public String toString() {
-            return String.valueOf(LeaseRenewer.this);
-          }
-        });
-        daemon.start();
-      }
-      dfsc.putFileBeingWritten(inodeId, out);
-      emptyTime = Long.MAX_VALUE;
-    }
-  }
-
-  @VisibleForTesting
-  synchronized void setEmptyTime(long time) {
-    emptyTime = time;
-  }
-
-  /** Close a file. */
-  void closeFile(final long inodeId, final DFSClient dfsc) {
-    dfsc.removeFileBeingWritten(inodeId);
-
-    synchronized(this) {
-      if (dfsc.isFilesBeingWrittenEmpty()) {
-        dfsclients.remove(dfsc);
-      }
-      //update emptyTime if necessary
-      if (emptyTime == Long.MAX_VALUE) {
-        for(DFSClient c : dfsclients) {
-          if (!c.isFilesBeingWrittenEmpty()) {
-            //found a non-empty file-being-written map
-            return;
-          }
-        }
-        //discover the first time that all file-being-written maps are empty.
-        emptyTime = Time.monotonicNow();
-      }
-    }
-  }
-
-  /** Close the given client. */
-  synchronized void closeClient(final DFSClient dfsc) {
-    dfsclients.remove(dfsc);
-    if (dfsclients.isEmpty()) {
-      if (!isRunning() || isRenewerExpired()) {
-        Factory.INSTANCE.remove(LeaseRenewer.this);
-        return;
-      }
-      if (emptyTime == Long.MAX_VALUE) {
-        //discover the first time that the client list is empty.
-        emptyTime = Time.monotonicNow();
-      }
-    }
-
-    //update renewal time
-    if (renewal == dfsc.getConf().getHdfsTimeout()/2) {
-      long min = HdfsConstants.LEASE_SOFTLIMIT_PERIOD;
-      for(DFSClient c : dfsclients) {
-        final int timeout = c.getConf().getHdfsTimeout();
-        if (timeout > 0 && timeout < min) {
-          min = timeout;
-        }
-      }
-      renewal = min/2;
-    }
-  }
-
-  void interruptAndJoin() throws InterruptedException {
-    Daemon daemonCopy = null;
-    synchronized (this) {
-      if (isRunning()) {
-        daemon.interrupt();
-        daemonCopy = daemon;
-      }
-    }
-   
-    if (daemonCopy != null) {
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("Wait for lease checker to terminate");
-      }
-      daemonCopy.join();
-    }
-  }
-
-  private void renew() throws IOException {
-    final List<DFSClient> copies;
-    synchronized(this) {
-      copies = new ArrayList<DFSClient>(dfsclients);
-    }
-    //sort the client names for finding out repeated names.
-    Collections.sort(copies, new Comparator<DFSClient>() {
-      @Override
-      public int compare(final DFSClient left, final DFSClient right) {
-        return left.getClientName().compareTo(right.getClientName());
-      }
-    });
-    String previousName = "";
-    for(int i = 0; i < copies.size(); i++) {
-      final DFSClient c = copies.get(i);
-      //skip if current client name is the same as the previous name.
-      if (!c.getClientName().equals(previousName)) {
-        if (!c.renewLease()) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Did not renew lease for client " +
-                c);
-          }
-          continue;
-        }
-        previousName = c.getClientName();
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Lease renewed for client " + previousName);
-        }
-      }
-    }
-  }
-
-  /**
-   * Periodically check in with the namenode and renew all the leases
-   * when the lease period is half over.
-   */
-  private void run(final int id) throws InterruptedException {
-    for(long lastRenewed = Time.monotonicNow(); !Thread.interrupted();
-        Thread.sleep(getSleepPeriod())) {
-      final long elapsed = Time.monotonicNow() - lastRenewed;
-      if (elapsed >= getRenewalTime()) {
-        try {
-          renew();
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Lease renewer daemon for " + clientsString()
-                + " with renew id " + id + " executed");
-          }
-          lastRenewed = Time.monotonicNow();
-        } catch (SocketTimeoutException ie) {
-          LOG.warn("Failed to renew lease for " + clientsString() + " for "
-              + (elapsed/1000) + " seconds.  Aborting ...", ie);
-          synchronized (this) {
-            while (!dfsclients.isEmpty()) {
-              dfsclients.get(0).abort();
-            }
-          }
-          break;
-        } catch (IOException ie) {
-          LOG.warn("Failed to renew lease for " + clientsString() + " for "
-              + (elapsed/1000) + " seconds.  Will retry shortly ...", ie);
-        }
-      }
-
-      synchronized(this) {
-        if (id != currentId || isRenewerExpired()) {
-          if (LOG.isDebugEnabled()) {
-            if (id != currentId) {
-              LOG.debug("Lease renewer daemon for " + clientsString()
-                  + " with renew id " + id + " is not current");
-            } else {
-               LOG.debug("Lease renewer daemon for " + clientsString()
-                  + " with renew id " + id + " expired");
-            }
-          }
-          //no longer the current daemon or expired
-          return;
-        }
-
-        // if no clients are in running state or there is no more clients
-        // registered with this renewer, stop the daemon after the grace
-        // period.
-        if (!clientsRunning() && emptyTime == Long.MAX_VALUE) {
-          emptyTime = Time.monotonicNow();
-        }
-      }
-    }
-  }
-
-  @Override
-  public String toString() {
-    String s = getClass().getSimpleName() + ":" + factorykey;
-    if (LOG.isTraceEnabled()) {
-      return s + ", clients=" +  clientsString()
-        + ", created at " + instantiationTrace;
-    }
-    return s;
-  }
-
-  /** Get the names of all clients */
-  private synchronized String clientsString() {
-    if (dfsclients.isEmpty()) {
-      return "[]";
-    } else {
-      final StringBuilder b = new StringBuilder("[").append(
-          dfsclients.get(0).getClientName());
-      for(int i = 1; i < dfsclients.size(); i++) {
-        b.append(", ").append(dfsclients.get(i).getClientName());
-      }
-      return b.append("]").toString();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb5ae084/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
new file mode 100644
index 0000000..4cdf168
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
@@ -0,0 +1,514 @@
+/**
+ * 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.client.impl;
+
+import java.io.IOException;
+import java.net.SocketTimeoutException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSOutputStream;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Time;
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * <p>
+ * Used by {@link org.apache.hadoop.hdfs.DFSClient} for renewing file-being-written leases
+ * on the namenode.
+ * When a file is opened for write (create or append),
+ * namenode stores a file lease for recording the identity of the writer.
+ * The writer (i.e. the DFSClient) is required to renew the lease periodically.
+ * When the lease is not renewed before it expires,
+ * the namenode considers the writer as failed and then it may either let
+ * another writer to obtain the lease or close the file.
+ * </p>
+ * <p>
+ * This class also provides the following functionality:
+ * <ul>
+ * <li>
+ * It maintains a map from (namenode, user) pairs to lease renewers.
+ * The same {@link LeaseRenewer} instance is used for renewing lease
+ * for all the {@link org.apache.hadoop.hdfs.DFSClient} to the same namenode and the same user.
+ * </li>
+ * <li>
+ * Each renewer maintains a list of {@link org.apache.hadoop.hdfs.DFSClient}.
+ * Periodically the leases for all the clients are renewed.
+ * A client is removed from the list when the client is closed.
+ * </li>
+ * <li>
+ * A thread per namenode per user is used by the {@link LeaseRenewer}
+ * to renew the leases.
+ * </li>
+ * </ul>
+ * </p>
+ */
+@InterfaceAudience.Private
+public class LeaseRenewer {
+  static final Log LOG = LogFactory.getLog(LeaseRenewer.class);
+
+  static final long LEASE_RENEWER_GRACE_DEFAULT = 60*1000L;
+  static final long LEASE_RENEWER_SLEEP_DEFAULT = 1000L;
+
+  /** Get a {@link LeaseRenewer} instance */
+  public static LeaseRenewer getInstance(final String authority,
+      final UserGroupInformation ugi, final DFSClient dfsc) throws IOException {
+    final LeaseRenewer r = Factory.INSTANCE.get(authority, ugi);
+    r.addClient(dfsc);
+    return r;
+  }
+
+  /**
+   * A factory for sharing {@link LeaseRenewer} objects
+   * among {@link DFSClient} instances
+   * so that there is only one renewer per authority per user.
+   */
+  private static class Factory {
+    private static final Factory INSTANCE = new Factory();
+
+    private static class Key {
+      /** Namenode info */
+      final String authority;
+      /** User info */
+      final UserGroupInformation ugi;
+
+      private Key(final String authority, final UserGroupInformation ugi) {
+        if (authority == null) {
+          throw new HadoopIllegalArgumentException("authority == null");
+        } else if (ugi == null) {
+          throw new HadoopIllegalArgumentException("ugi == null");
+        }
+
+        this.authority = authority;
+        this.ugi = ugi;
+      }
+
+      @Override
+      public int hashCode() {
+        return authority.hashCode() ^ ugi.hashCode();
+      }
+
+      @Override
+      public boolean equals(Object obj) {
+        if (obj == this) {
+          return true;
+        }
+        if (obj != null && obj instanceof Key) {
+          final Key that = (Key)obj;
+          return this.authority.equals(that.authority)
+                 && this.ugi.equals(that.ugi);
+        }
+        return false;
+      }
+
+      @Override
+      public String toString() {
+        return ugi.getShortUserName() + "@" + authority;
+      }
+    }
+
+    /** A map for per user per namenode renewers. */
+    private final Map<Key, LeaseRenewer> renewers = new HashMap<Key, LeaseRenewer>();
+
+    /** Get a renewer. */
+    private synchronized LeaseRenewer get(final String authority,
+        final UserGroupInformation ugi) {
+      final Key k = new Key(authority, ugi);
+      LeaseRenewer r = renewers.get(k);
+      if (r == null) {
+        r = new LeaseRenewer(k);
+        renewers.put(k, r);
+      }
+      return r;
+    }
+
+    /** Remove the given renewer. */
+    private synchronized void remove(final LeaseRenewer r) {
+      final LeaseRenewer stored = renewers.get(r.factorykey);
+      //Since a renewer may expire, the stored renewer can be different.
+      if (r == stored) {
+        if (!r.clientsRunning()) {
+          renewers.remove(r.factorykey);
+        }
+      }
+    }
+  }
+
+  /** The time in milliseconds that the map became empty. */
+  private long emptyTime = Long.MAX_VALUE;
+  /** A fixed lease renewal time period in milliseconds */
+  private long renewal = HdfsConstants.LEASE_SOFTLIMIT_PERIOD/2;
+
+  /** A daemon for renewing lease */
+  private Daemon daemon = null;
+  /** Only the daemon with currentId should run. */
+  private int currentId = 0;
+
+  /**
+   * A period in milliseconds that the lease renewer thread should run
+   * after the map became empty.
+   * In other words,
+   * if the map is empty for a time period longer than the grace period,
+   * the renewer should terminate.
+   */
+  private long gracePeriod;
+  /**
+   * The time period in milliseconds
+   * that the renewer sleeps for each iteration.
+   */
+  private long sleepPeriod;
+
+  private final Factory.Key factorykey;
+
+  /** A list of clients corresponding to this renewer. */
+  private final List<DFSClient> dfsclients = new ArrayList<DFSClient>();
+
+  /**
+   * A stringified stack trace of the call stack when the Lease Renewer
+   * was instantiated. This is only generated if trace-level logging is
+   * enabled on this class.
+   */
+  private final String instantiationTrace;
+
+  private LeaseRenewer(Factory.Key factorykey) {
+    this.factorykey = factorykey;
+    unsyncSetGraceSleepPeriod(LEASE_RENEWER_GRACE_DEFAULT);
+
+    if (LOG.isTraceEnabled()) {
+      instantiationTrace = StringUtils.stringifyException(
+        new Throwable("TRACE"));
+    } else {
+      instantiationTrace = null;
+    }
+  }
+
+  /** @return the renewal time in milliseconds. */
+  private synchronized long getRenewalTime() {
+    return renewal;
+  }
+
+  /** Add a client. */
+  private synchronized void addClient(final DFSClient dfsc) {
+    for(DFSClient c : dfsclients) {
+      if (c == dfsc) {
+        //client already exists, nothing to do.
+        return;
+      }
+    }
+    //client not found, add it
+    dfsclients.add(dfsc);
+
+    //update renewal time
+    final int hdfsTimeout = dfsc.getConf().getHdfsTimeout();
+    if (hdfsTimeout > 0) {
+      final long half = hdfsTimeout/2;
+      if (half < renewal) {
+        this.renewal = half;
+      }
+    }
+  }
+
+  private synchronized boolean clientsRunning() {
+    for(Iterator<DFSClient> i = dfsclients.iterator(); i.hasNext(); ) {
+      if (!i.next().isClientRunning()) {
+        i.remove();
+      }
+    }
+    return !dfsclients.isEmpty();
+  }
+
+  private synchronized long getSleepPeriod() {
+    return sleepPeriod;
+  }
+
+  /** Set the grace period and adjust the sleep period accordingly. */
+  synchronized void setGraceSleepPeriod(final long gracePeriod) {
+    unsyncSetGraceSleepPeriod(gracePeriod);
+  }
+
+  private void unsyncSetGraceSleepPeriod(final long gracePeriod) {
+    if (gracePeriod < 100L) {
+      throw new HadoopIllegalArgumentException(gracePeriod
+          + " = gracePeriod < 100ms is too small.");
+    }
+    this.gracePeriod = gracePeriod;
+    final long half = gracePeriod/2;
+    this.sleepPeriod = half < LEASE_RENEWER_SLEEP_DEFAULT?
+        half: LEASE_RENEWER_SLEEP_DEFAULT;
+  }
+
+  /** Is the daemon running? */
+  synchronized boolean isRunning() {
+    return daemon != null && daemon.isAlive();
+  }
+
+  /** Does this renewer have nothing to renew? */
+  public boolean isEmpty() {
+    return dfsclients.isEmpty();
+  }
+
+  /** Used only by tests */
+  synchronized String getDaemonName() {
+    return daemon.getName();
+  }
+
+  /** Is the empty period longer than the grace period? */
+  private synchronized boolean isRenewerExpired() {
+    return emptyTime != Long.MAX_VALUE
+        && Time.monotonicNow() - emptyTime > gracePeriod;
+  }
+
+  public synchronized void put(final long inodeId, final DFSOutputStream out,
+      final DFSClient dfsc) {
+    if (dfsc.isClientRunning()) {
+      if (!isRunning() || isRenewerExpired()) {
+        //start a new deamon with a new id.
+        final int id = ++currentId;
+        daemon = new Daemon(new Runnable() {
+          @Override
+          public void run() {
+            try {
+              if (LOG.isDebugEnabled()) {
+                LOG.debug("Lease renewer daemon for " + clientsString()
+                    + " with renew id " + id + " started");
+              }
+              LeaseRenewer.this.run(id);
+            } catch(InterruptedException e) {
+              if (LOG.isDebugEnabled()) {
+                LOG.debug(LeaseRenewer.this.getClass().getSimpleName()
+                    + " is interrupted.", e);
+              }
+            } finally {
+              synchronized(LeaseRenewer.this) {
+                Factory.INSTANCE.remove(LeaseRenewer.this);
+              }
+              if (LOG.isDebugEnabled()) {
+                LOG.debug("Lease renewer daemon for " + clientsString()
+                    + " with renew id " + id + " exited");
+              }
+            }
+          }
+
+          @Override
+          public String toString() {
+            return String.valueOf(LeaseRenewer.this);
+          }
+        });
+        daemon.start();
+      }
+      dfsc.putFileBeingWritten(inodeId, out);
+      emptyTime = Long.MAX_VALUE;
+    }
+  }
+
+  @VisibleForTesting
+  synchronized void setEmptyTime(long time) {
+    emptyTime = time;
+  }
+
+  /** Close a file. */
+  public void closeFile(final long inodeId, final DFSClient dfsc) {
+    dfsc.removeFileBeingWritten(inodeId);
+
+    synchronized(this) {
+      if (dfsc.isFilesBeingWrittenEmpty()) {
+        dfsclients.remove(dfsc);
+      }
+      //update emptyTime if necessary
+      if (emptyTime == Long.MAX_VALUE) {
+        for(DFSClient c : dfsclients) {
+          if (!c.isFilesBeingWrittenEmpty()) {
+            //found a non-empty file-being-written map
+            return;
+          }
+        }
+        //discover the first time that all file-being-written maps are empty.
+        emptyTime = Time.monotonicNow();
+      }
+    }
+  }
+
+  /** Close the given client. */
+  public synchronized void closeClient(final DFSClient dfsc) {
+    dfsclients.remove(dfsc);
+    if (dfsclients.isEmpty()) {
+      if (!isRunning() || isRenewerExpired()) {
+        Factory.INSTANCE.remove(LeaseRenewer.this);
+        return;
+      }
+      if (emptyTime == Long.MAX_VALUE) {
+        //discover the first time that the client list is empty.
+        emptyTime = Time.monotonicNow();
+      }
+    }
+
+    //update renewal time
+    if (renewal == dfsc.getConf().getHdfsTimeout()/2) {
+      long min = HdfsConstants.LEASE_SOFTLIMIT_PERIOD;
+      for(DFSClient c : dfsclients) {
+        final int timeout = c.getConf().getHdfsTimeout();
+        if (timeout > 0 && timeout < min) {
+          min = timeout;
+        }
+      }
+      renewal = min/2;
+    }
+  }
+
+  public void interruptAndJoin() throws InterruptedException {
+    Daemon daemonCopy = null;
+    synchronized (this) {
+      if (isRunning()) {
+        daemon.interrupt();
+        daemonCopy = daemon;
+      }
+    }
+
+    if (daemonCopy != null) {
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Wait for lease checker to terminate");
+      }
+      daemonCopy.join();
+    }
+  }
+
+  private void renew() throws IOException {
+    final List<DFSClient> copies;
+    synchronized(this) {
+      copies = new ArrayList<DFSClient>(dfsclients);
+    }
+    //sort the client names for finding out repeated names.
+    Collections.sort(copies, new Comparator<DFSClient>() {
+      @Override
+      public int compare(final DFSClient left, final DFSClient right) {
+        return left.getClientName().compareTo(right.getClientName());
+      }
+    });
+    String previousName = "";
+    for(int i = 0; i < copies.size(); i++) {
+      final DFSClient c = copies.get(i);
+      //skip if current client name is the same as the previous name.
+      if (!c.getClientName().equals(previousName)) {
+        if (!c.renewLease()) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Did not renew lease for client " +
+                c);
+          }
+          continue;
+        }
+        previousName = c.getClientName();
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Lease renewed for client " + previousName);
+        }
+      }
+    }
+  }
+
+  /**
+   * Periodically check in with the namenode and renew all the leases
+   * when the lease period is half over.
+   */
+  private void run(final int id) throws InterruptedException {
+    for(long lastRenewed = Time.monotonicNow(); !Thread.interrupted();
+        Thread.sleep(getSleepPeriod())) {
+      final long elapsed = Time.monotonicNow() - lastRenewed;
+      if (elapsed >= getRenewalTime()) {
+        try {
+          renew();
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Lease renewer daemon for " + clientsString()
+                + " with renew id " + id + " executed");
+          }
+          lastRenewed = Time.monotonicNow();
+        } catch (SocketTimeoutException ie) {
+          LOG.warn("Failed to renew lease for " + clientsString() + " for "
+              + (elapsed/1000) + " seconds.  Aborting ...", ie);
+          synchronized (this) {
+            while (!dfsclients.isEmpty()) {
+              dfsclients.get(0).abort();
+            }
+          }
+          break;
+        } catch (IOException ie) {
+          LOG.warn("Failed to renew lease for " + clientsString() + " for "
+              + (elapsed/1000) + " seconds.  Will retry shortly ...", ie);
+        }
+      }
+
+      synchronized(this) {
+        if (id != currentId || isRenewerExpired()) {
+          if (LOG.isDebugEnabled()) {
+            if (id != currentId) {
+              LOG.debug("Lease renewer daemon for " + clientsString()
+                  + " with renew id " + id + " is not current");
+            } else {
+               LOG.debug("Lease renewer daemon for " + clientsString()
+                  + " with renew id " + id + " expired");
+            }
+          }
+          //no longer the current daemon or expired
+          return;
+        }
+
+        // if no clients are in running state or there is no more clients
+        // registered with this renewer, stop the daemon after the grace
+        // period.
+        if (!clientsRunning() && emptyTime == Long.MAX_VALUE) {
+          emptyTime = Time.monotonicNow();
+        }
+      }
+    }
+  }
+
+  @Override
+  public String toString() {
+    String s = getClass().getSimpleName() + ":" + factorykey;
+    if (LOG.isTraceEnabled()) {
+      return s + ", clients=" +  clientsString()
+        + ", created at " + instantiationTrace;
+    }
+    return s;
+  }
+
+  /** Get the names of all clients */
+  private synchronized String clientsString() {
+    if (dfsclients.isEmpty()) {
+      return "[]";
+    } else {
+      final StringBuilder b = new StringBuilder("[").append(
+          dfsclients.get(0).getClientName());
+      for(int i = 1; i < dfsclients.size(); i++) {
+        b.append(", ").append(dfsclients.get(i).getClientName());
+      }
+      return b.append("]").toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb5ae084/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
index 0689a53..837665e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
@@ -31,6 +31,7 @@ import static org.mockito.Mockito.mock;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.lang.reflect.Method;
 import java.net.InetSocketAddress;
 import java.net.ServerSocket;
 import java.net.SocketTimeoutException;
@@ -64,6 +65,7 @@ import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.VolumeId;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
+import org.apache.hadoop.hdfs.client.impl.LeaseRenewer;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeFaultInjector;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
@@ -264,78 +266,84 @@ public class TestDistributedFileSystem {
 
       {
         final DistributedFileSystem dfs = cluster.getFileSystem();
-        dfs.dfs.getLeaseRenewer().setGraceSleepPeriod(grace);
-        assertFalse(dfs.dfs.getLeaseRenewer().isRunning());
+        Method setMethod = dfs.dfs.getLeaseRenewer().getClass()
+            .getDeclaredMethod("setGraceSleepPeriod", long.class);
+        setMethod.setAccessible(true);
+        setMethod.invoke(dfs.dfs.getLeaseRenewer(), grace);
+        Method checkMethod = dfs.dfs.getLeaseRenewer().getClass()
+            .getDeclaredMethod("isRunning");
+        checkMethod.setAccessible(true);
+        assertFalse((boolean) checkMethod.invoke(dfs.dfs.getLeaseRenewer()));
   
         {
           //create a file
           final FSDataOutputStream out = dfs.create(filepaths[0]);
-          assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
+          assertTrue((boolean)checkMethod.invoke(dfs.dfs.getLeaseRenewer()));
           //write something
           out.writeLong(millis);
-          assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
+          assertTrue((boolean)checkMethod.invoke(dfs.dfs.getLeaseRenewer()));
           //close
           out.close();
           Thread.sleep(grace/4*3);
           //within grace period
-          assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
+          assertTrue((boolean)checkMethod.invoke(dfs.dfs.getLeaseRenewer()));
           for(int i = 0; i < 3; i++) {
-            if (dfs.dfs.getLeaseRenewer().isRunning()) {
+            if ((boolean)checkMethod.invoke(dfs.dfs.getLeaseRenewer())) {
               Thread.sleep(grace/2);
             }
           }
           //passed grace period
-          assertFalse(dfs.dfs.getLeaseRenewer().isRunning());
+          assertFalse((boolean)checkMethod.invoke(dfs.dfs.getLeaseRenewer()));
         }
 
         {
           //create file1
           final FSDataOutputStream out1 = dfs.create(filepaths[1]);
-          assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
+          assertTrue((boolean)checkMethod.invoke(dfs.dfs.getLeaseRenewer()));
           //create file2
           final FSDataOutputStream out2 = dfs.create(filepaths[2]);
-          assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
+          assertTrue((boolean)checkMethod.invoke(dfs.dfs.getLeaseRenewer()));
 
           //write something to file1
           out1.writeLong(millis);
-          assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
+          assertTrue((boolean)checkMethod.invoke(dfs.dfs.getLeaseRenewer()));
           //close file1
           out1.close();
-          assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
+          assertTrue((boolean)checkMethod.invoke(dfs.dfs.getLeaseRenewer()));
 
           //write something to file2
           out2.writeLong(millis);
-          assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
+          assertTrue((boolean)checkMethod.invoke(dfs.dfs.getLeaseRenewer()));
           //close file2
           out2.close();
           Thread.sleep(grace/4*3);
           //within grace period
-          assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
+          assertTrue((boolean)checkMethod.invoke(dfs.dfs.getLeaseRenewer()));
         }
 
         {
           //create file3
           final FSDataOutputStream out3 = dfs.create(filepaths[3]);
-          assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
+          assertTrue((boolean)checkMethod.invoke(dfs.dfs.getLeaseRenewer()));
           Thread.sleep(grace/4*3);
           //passed previous grace period, should still running
-          assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
+          assertTrue((boolean)checkMethod.invoke(dfs.dfs.getLeaseRenewer()));
           //write something to file3
           out3.writeLong(millis);
-          assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
+          assertTrue((boolean)checkMethod.invoke(dfs.dfs.getLeaseRenewer()));
           //close file3
           out3.close();
-          assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
+          assertTrue((boolean)checkMethod.invoke(dfs.dfs.getLeaseRenewer()));
           Thread.sleep(grace/4*3);
           //within grace period
-          assertTrue(dfs.dfs.getLeaseRenewer().isRunning());
+          assertTrue((boolean)checkMethod.invoke(dfs.dfs.getLeaseRenewer()));
           for(int i = 0; i < 3; i++) {
-            if (dfs.dfs.getLeaseRenewer().isRunning()) {
+            if ((boolean)checkMethod.invoke(dfs.dfs.getLeaseRenewer())) {
               Thread.sleep(grace/2);
             }
           }
           //passed grace period
-          assertFalse(dfs.dfs.getLeaseRenewer().isRunning());
+          assertFalse((boolean)checkMethod.invoke(dfs.dfs.getLeaseRenewer()));
         }
 
         dfs.close();
@@ -364,15 +372,18 @@ public class TestDistributedFileSystem {
 
       {
         final DistributedFileSystem dfs = cluster.getFileSystem();
-        assertFalse(dfs.dfs.getLeaseRenewer().isRunning());
+        Method checkMethod = dfs.dfs.getLeaseRenewer().getClass()
+            .getDeclaredMethod("isRunning");
+        checkMethod.setAccessible(true);
+        assertFalse((boolean)checkMethod.invoke(dfs.dfs.getLeaseRenewer()));
 
         //open and check the file
         FSDataInputStream in = dfs.open(filepaths[0]);
-        assertFalse(dfs.dfs.getLeaseRenewer().isRunning());
+        assertFalse((boolean)checkMethod.invoke(dfs.dfs.getLeaseRenewer()));
         assertEquals(millis, in.readLong());
-        assertFalse(dfs.dfs.getLeaseRenewer().isRunning());
+        assertFalse((boolean)checkMethod.invoke(dfs.dfs.getLeaseRenewer()));
         in.close();
-        assertFalse(dfs.dfs.getLeaseRenewer().isRunning());
+        assertFalse((boolean)checkMethod.invoke(dfs.dfs.getLeaseRenewer()));
         dfs.close();
       }
       

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb5ae084/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 1cf7add..9b5a7c0 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
@@ -45,6 +45,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.client.impl.LeaseRenewer;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb5ae084/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRenewer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRenewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRenewer.java
deleted file mode 100644
index f091db7..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRenewer.java
+++ /dev/null
@@ -1,207 +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;
-
-import static org.junit.Assert.assertSame;
-
-import java.io.IOException;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.util.Time;
-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 com.google.common.base.Supplier;
-
-public class TestLeaseRenewer {
-  private final String FAKE_AUTHORITY="hdfs://nn1/";
-  private final UserGroupInformation FAKE_UGI_A =
-    UserGroupInformation.createUserForTesting(
-      "myuser", new String[]{"group1"});
-  private final UserGroupInformation FAKE_UGI_B =
-    UserGroupInformation.createUserForTesting(
-      "myuser", new String[]{"group1"});
-
-  private DFSClient MOCK_DFSCLIENT;
-  private LeaseRenewer renewer;
-  
-  /** Cause renewals often so test runs quickly. */
-  private static final long FAST_GRACE_PERIOD = 100L;
-  
-  @Before
-  public void setupMocksAndRenewer() throws IOException {
-    MOCK_DFSCLIENT = createMockClient();
-    
-    renewer = LeaseRenewer.getInstance(
-        FAKE_AUTHORITY, FAKE_UGI_A, MOCK_DFSCLIENT);
-    renewer.setGraceSleepPeriod(FAST_GRACE_PERIOD);
-}
- 
-  private DFSClient createMockClient() {
-    final DfsClientConf mockConf = Mockito.mock(DfsClientConf.class);
-    Mockito.doReturn((int)FAST_GRACE_PERIOD).when(mockConf).getHdfsTimeout();
-
-    DFSClient mock = Mockito.mock(DFSClient.class);
-    Mockito.doReturn(true).when(mock).isClientRunning();
-    Mockito.doReturn(mockConf).when(mock).getConf();
-    Mockito.doReturn("myclient").when(mock).getClientName();
-    return mock;
-  }
-
-  @Test
-  public void testInstanceSharing() throws IOException {
-    // Two lease renewers with the same UGI should return
-    // the same instance
-    LeaseRenewer lr = LeaseRenewer.getInstance(
-        FAKE_AUTHORITY, FAKE_UGI_A, MOCK_DFSCLIENT);
-    LeaseRenewer lr2 = LeaseRenewer.getInstance(
-        FAKE_AUTHORITY, FAKE_UGI_A, MOCK_DFSCLIENT);
-    Assert.assertSame(lr, lr2);
-    
-    // But a different UGI should return a different instance
-    LeaseRenewer lr3 = LeaseRenewer.getInstance(
-        FAKE_AUTHORITY, FAKE_UGI_B, MOCK_DFSCLIENT);
-    Assert.assertNotSame(lr, lr3);
-    
-    // A different authority with same UGI should also be a different
-    // instance.
-    LeaseRenewer lr4 = LeaseRenewer.getInstance(
-        "someOtherAuthority", FAKE_UGI_B, MOCK_DFSCLIENT);
-    Assert.assertNotSame(lr, lr4);
-    Assert.assertNotSame(lr3, lr4);
-  }
-  
-  @Test
-  public void testRenewal() throws Exception {
-    // Keep track of how many times the lease gets renewed
-    final AtomicInteger leaseRenewalCount = new AtomicInteger();
-    Mockito.doAnswer(new Answer<Boolean>() {
-      @Override
-      public Boolean answer(InvocationOnMock invocation) throws Throwable {
-        leaseRenewalCount.incrementAndGet();
-        return true;
-      }
-    }).when(MOCK_DFSCLIENT).renewLease();
-
-    
-    // Set up a file so that we start renewing our lease.
-    DFSOutputStream mockStream = Mockito.mock(DFSOutputStream.class);
-    long fileId = 123L;
-    renewer.put(fileId, mockStream, MOCK_DFSCLIENT);
-
-    // Wait for lease to get renewed
-    long failTime = Time.monotonicNow() + 5000;
-    while (Time.monotonicNow() < failTime &&
-        leaseRenewalCount.get() == 0) {
-      Thread.sleep(50);
-    }
-    if (leaseRenewalCount.get() == 0) {
-      Assert.fail("Did not renew lease at all!");
-    }
-
-    renewer.closeFile(fileId, MOCK_DFSCLIENT);
-  }
-  
-  /**
-   * Regression test for HDFS-2810. In this bug, the LeaseRenewer has handles
-   * to several DFSClients with the same name, the first of which has no files
-   * open. Previously, this was causing the lease to not get renewed.
-   */
-  @Test
-  public void testManyDfsClientsWhereSomeNotOpen() throws Exception {
-    // First DFSClient has no files open so doesn't renew leases.
-    final DFSClient mockClient1 = createMockClient();
-    Mockito.doReturn(false).when(mockClient1).renewLease();
-    assertSame(renewer, LeaseRenewer.getInstance(
-        FAKE_AUTHORITY, FAKE_UGI_A, mockClient1));
-    
-    // Set up a file so that we start renewing our lease.
-    DFSOutputStream mockStream1 = Mockito.mock(DFSOutputStream.class);
-    long fileId = 456L;
-    renewer.put(fileId, mockStream1, mockClient1);
-
-    // Second DFSClient does renew lease
-    final DFSClient mockClient2 = createMockClient();
-    Mockito.doReturn(true).when(mockClient2).renewLease();
-    assertSame(renewer, LeaseRenewer.getInstance(
-        FAKE_AUTHORITY, FAKE_UGI_A, mockClient2));
-
-    // Set up a file so that we start renewing our lease.
-    DFSOutputStream mockStream2 = Mockito.mock(DFSOutputStream.class);
-    renewer.put(fileId, mockStream2, mockClient2);
-
-    
-    // Wait for lease to get renewed
-    GenericTestUtils.waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        try {
-          Mockito.verify(mockClient1, Mockito.atLeastOnce()).renewLease();
-          Mockito.verify(mockClient2, Mockito.atLeastOnce()).renewLease();
-          return true;
-        } catch (AssertionError err) {
-          LeaseRenewer.LOG.warn("Not yet satisfied", err);
-          return false;
-        } catch (IOException e) {
-          // should not throw!
-          throw new RuntimeException(e);
-        }
-      }
-    }, 100, 10000);
-
-    renewer.closeFile(fileId, mockClient1);
-    renewer.closeFile(fileId, mockClient2);
-  }
-  
-  @Test
-  public void testThreadName() throws Exception {
-    DFSOutputStream mockStream = Mockito.mock(DFSOutputStream.class);
-    long fileId = 789L;
-    Assert.assertFalse("Renewer not initially running",
-        renewer.isRunning());
-    
-    // Pretend to open a file
-    renewer.put(fileId, mockStream, MOCK_DFSCLIENT);
-    
-    Assert.assertTrue("Renewer should have started running",
-        renewer.isRunning());
-    
-    // Check the thread name is reasonable
-    String threadName = renewer.getDaemonName();
-    Assert.assertEquals("LeaseRenewer:myuser@hdfs://nn1/", threadName);
-    
-    // Pretend to close the file
-    renewer.closeFile(fileId, MOCK_DFSCLIENT);
-    renewer.setEmptyTime(Time.monotonicNow());
-    
-    // Should stop the renewer running within a few seconds
-    long failTime = Time.monotonicNow() + 5000;
-    while (renewer.isRunning() && Time.monotonicNow() < failTime) {
-      Thread.sleep(50);
-    }
-    Assert.assertFalse(renewer.isRunning());
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb5ae084/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestLeaseRenewer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestLeaseRenewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestLeaseRenewer.java
new file mode 100644
index 0000000..a4e00d9
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestLeaseRenewer.java
@@ -0,0 +1,209 @@
+/**
+ * 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.client.impl;
+
+import static org.junit.Assert.assertSame;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSOutputStream;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.Time;
+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 com.google.common.base.Supplier;
+
+public class TestLeaseRenewer {
+  private final String FAKE_AUTHORITY="hdfs://nn1/";
+  private final UserGroupInformation FAKE_UGI_A =
+    UserGroupInformation.createUserForTesting(
+      "myuser", new String[]{"group1"});
+  private final UserGroupInformation FAKE_UGI_B =
+    UserGroupInformation.createUserForTesting(
+      "myuser", new String[]{"group1"});
+
+  private DFSClient MOCK_DFSCLIENT;
+  private LeaseRenewer renewer;
+
+  /** Cause renewals often so test runs quickly. */
+  private static final long FAST_GRACE_PERIOD = 100L;
+
+  @Before
+  public void setupMocksAndRenewer() throws IOException {
+    MOCK_DFSCLIENT = createMockClient();
+
+    renewer = LeaseRenewer.getInstance(
+        FAKE_AUTHORITY, FAKE_UGI_A, MOCK_DFSCLIENT);
+    renewer.setGraceSleepPeriod(FAST_GRACE_PERIOD);
+}
+
+  private DFSClient createMockClient() {
+    final DfsClientConf mockConf = Mockito.mock(DfsClientConf.class);
+    Mockito.doReturn((int)FAST_GRACE_PERIOD).when(mockConf).getHdfsTimeout();
+
+    DFSClient mock = Mockito.mock(DFSClient.class);
+    Mockito.doReturn(true).when(mock).isClientRunning();
+    Mockito.doReturn(mockConf).when(mock).getConf();
+    Mockito.doReturn("myclient").when(mock).getClientName();
+    return mock;
+  }
+
+  @Test
+  public void testInstanceSharing() throws IOException {
+    // Two lease renewers with the same UGI should return
+    // the same instance
+    LeaseRenewer lr = LeaseRenewer.getInstance(
+        FAKE_AUTHORITY, FAKE_UGI_A, MOCK_DFSCLIENT);
+    LeaseRenewer lr2 = LeaseRenewer.getInstance(
+        FAKE_AUTHORITY, FAKE_UGI_A, MOCK_DFSCLIENT);
+    Assert.assertSame(lr, lr2);
+
+    // But a different UGI should return a different instance
+    LeaseRenewer lr3 = LeaseRenewer.getInstance(
+        FAKE_AUTHORITY, FAKE_UGI_B, MOCK_DFSCLIENT);
+    Assert.assertNotSame(lr, lr3);
+
+    // A different authority with same UGI should also be a different
+    // instance.
+    LeaseRenewer lr4 = LeaseRenewer.getInstance(
+        "someOtherAuthority", FAKE_UGI_B, MOCK_DFSCLIENT);
+    Assert.assertNotSame(lr, lr4);
+    Assert.assertNotSame(lr3, lr4);
+  }
+
+  @Test
+  public void testRenewal() throws Exception {
+    // Keep track of how many times the lease gets renewed
+    final AtomicInteger leaseRenewalCount = new AtomicInteger();
+    Mockito.doAnswer(new Answer<Boolean>() {
+      @Override
+      public Boolean answer(InvocationOnMock invocation) throws Throwable {
+        leaseRenewalCount.incrementAndGet();
+        return true;
+      }
+    }).when(MOCK_DFSCLIENT).renewLease();
+
+
+    // Set up a file so that we start renewing our lease.
+    DFSOutputStream mockStream = Mockito.mock(DFSOutputStream.class);
+    long fileId = 123L;
+    renewer.put(fileId, mockStream, MOCK_DFSCLIENT);
+
+    // Wait for lease to get renewed
+    long failTime = Time.monotonicNow() + 5000;
+    while (Time.monotonicNow() < failTime &&
+        leaseRenewalCount.get() == 0) {
+      Thread.sleep(50);
+    }
+    if (leaseRenewalCount.get() == 0) {
+      Assert.fail("Did not renew lease at all!");
+    }
+
+    renewer.closeFile(fileId, MOCK_DFSCLIENT);
+  }
+
+  /**
+   * Regression test for HDFS-2810. In this bug, the LeaseRenewer has handles
+   * to several DFSClients with the same name, the first of which has no files
+   * open. Previously, this was causing the lease to not get renewed.
+   */
+  @Test
+  public void testManyDfsClientsWhereSomeNotOpen() throws Exception {
+    // First DFSClient has no files open so doesn't renew leases.
+    final DFSClient mockClient1 = createMockClient();
+    Mockito.doReturn(false).when(mockClient1).renewLease();
+    assertSame(renewer, LeaseRenewer.getInstance(
+        FAKE_AUTHORITY, FAKE_UGI_A, mockClient1));
+
+    // Set up a file so that we start renewing our lease.
+    DFSOutputStream mockStream1 = Mockito.mock(DFSOutputStream.class);
+    long fileId = 456L;
+    renewer.put(fileId, mockStream1, mockClient1);
+
+    // Second DFSClient does renew lease
+    final DFSClient mockClient2 = createMockClient();
+    Mockito.doReturn(true).when(mockClient2).renewLease();
+    assertSame(renewer, LeaseRenewer.getInstance(
+        FAKE_AUTHORITY, FAKE_UGI_A, mockClient2));
+
+    // Set up a file so that we start renewing our lease.
+    DFSOutputStream mockStream2 = Mockito.mock(DFSOutputStream.class);
+    renewer.put(fileId, mockStream2, mockClient2);
+
+
+    // Wait for lease to get renewed
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        try {
+          Mockito.verify(mockClient1, Mockito.atLeastOnce()).renewLease();
+          Mockito.verify(mockClient2, Mockito.atLeastOnce()).renewLease();
+          return true;
+        } catch (AssertionError err) {
+          LeaseRenewer.LOG.warn("Not yet satisfied", err);
+          return false;
+        } catch (IOException e) {
+          // should not throw!
+          throw new RuntimeException(e);
+        }
+      }
+    }, 100, 10000);
+
+    renewer.closeFile(fileId, mockClient1);
+    renewer.closeFile(fileId, mockClient2);
+  }
+
+  @Test
+  public void testThreadName() throws Exception {
+    DFSOutputStream mockStream = Mockito.mock(DFSOutputStream.class);
+    long fileId = 789L;
+    Assert.assertFalse("Renewer not initially running",
+        renewer.isRunning());
+
+    // Pretend to open a file
+    renewer.put(fileId, mockStream, MOCK_DFSCLIENT);
+
+    Assert.assertTrue("Renewer should have started running",
+        renewer.isRunning());
+
+    // Check the thread name is reasonable
+    String threadName = renewer.getDaemonName();
+    Assert.assertEquals("LeaseRenewer:myuser@hdfs://nn1/", threadName);
+
+    // Pretend to close the file
+    renewer.closeFile(fileId, MOCK_DFSCLIENT);
+    renewer.setEmptyTime(Time.monotonicNow());
+
+    // Should stop the renewer running within a few seconds
+    long failTime = Time.monotonicNow() + 5000;
+    while (renewer.isRunning() && Time.monotonicNow() < failTime) {
+      Thread.sleep(50);
+    }
+    Assert.assertFalse(renewer.isRunning());
+  }
+
+}


[12/50] [abbrv] hadoop git commit: HDFS-8214. Secondary NN Web UI shows wrong date for Last Checkpoint. Contributed by Charles Lamb.

Posted by zj...@apache.org.
HDFS-8214. Secondary NN Web UI shows wrong date for Last Checkpoint. Contributed by Charles Lamb.


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

Branch: refs/heads/YARN-2928
Commit: 0953b5e7e5a7ce1b59fcb8c519ecff9763dad4ac
Parents: a49bcaa
Author: Andrew Wang <wa...@apache.org>
Authored: Wed Apr 29 17:37:56 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:54 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  2 ++
 .../hdfs/server/namenode/SecondaryNameNode.java | 35 ++++++++++++++------
 .../namenode/SecondaryNameNodeInfoMXBean.java   |  6 ++++
 .../src/main/webapps/secondary/status.html      |  2 +-
 .../src/main/webapps/static/dfs-dust.js         |  3 ++
 5 files changed, 36 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0953b5e7/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index a3f219b..fbeb45d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -582,6 +582,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-8232. Missing datanode counters when using Metrics2 sink interface.
     (Anu Engineer via cnauroth)
 
+    HDFS-8214. Secondary NN Web UI shows wrong date for Last Checkpoint. (clamb via wang)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0953b5e7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
index ec7e0c9..b499e74 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
@@ -108,6 +108,7 @@ public class SecondaryNameNode implements Runnable,
 
   private final long starttime = Time.now();
   private volatile long lastCheckpointTime = 0;
+  private volatile long lastCheckpointWallclockTime = 0;
 
   private URL fsName;
   private CheckpointStorage checkpointImage;
@@ -134,8 +135,9 @@ public class SecondaryNameNode implements Runnable,
       + "\nName Node Address      : " + nameNodeAddr
       + "\nStart Time             : " + new Date(starttime)
       + "\nLast Checkpoint        : " + (lastCheckpointTime == 0? "--":
-				       ((Time.monotonicNow() - lastCheckpointTime) / 1000))
-	                            + " seconds ago"
+        new Date(lastCheckpointWallclockTime))
+      + " (" + ((Time.monotonicNow() - lastCheckpointTime) / 1000)
+      + " seconds ago)"
       + "\nCheckpoint Period      : " + checkpointConf.getPeriod() + " seconds"
       + "\nCheckpoint Transactions: " + checkpointConf.getTxnCount()
       + "\nCheckpoint Dirs        : " + checkpointDirs
@@ -388,12 +390,14 @@ public class SecondaryNameNode implements Runnable,
         if(UserGroupInformation.isSecurityEnabled())
           UserGroupInformation.getCurrentUser().checkTGTAndReloginFromKeytab();
         
-        final long now = Time.monotonicNow();
+        final long monotonicNow = Time.monotonicNow();
+        final long now = Time.now();
 
         if (shouldCheckpointBasedOnCount() ||
-            now >= lastCheckpointTime + 1000 * checkpointConf.getPeriod()) {
+            monotonicNow >= lastCheckpointTime + 1000 * checkpointConf.getPeriod()) {
           doCheckpoint();
-          lastCheckpointTime = now;
+          lastCheckpointTime = monotonicNow;
+          lastCheckpointWallclockTime = now;
         }
       } catch (IOException e) {
         LOG.error("Exception in doCheckpoint", e);
@@ -695,22 +699,31 @@ public class SecondaryNameNode implements Runnable,
     checkpointThread.start();
   }
 
-  @Override // SecondaryNameNodeInfoMXXBean
+  @Override // SecondaryNameNodeInfoMXBean
   public String getHostAndPort() {
     return NetUtils.getHostPortString(nameNodeAddr);
   }
 
-  @Override // SecondaryNameNodeInfoMXXBean
+  @Override // SecondaryNameNodeInfoMXBean
   public long getStartTime() {
     return starttime;
   }
 
-  @Override // SecondaryNameNodeInfoMXXBean
+  @Override // SecondaryNameNodeInfoMXBean
   public long getLastCheckpointTime() {
-    return lastCheckpointTime;
+    return lastCheckpointWallclockTime;
   }
 
-  @Override // SecondaryNameNodeInfoMXXBean
+  @Override // SecondaryNameNodeInfoMXBean
+  public long getLastCheckpointDeltaMs() {
+    if (lastCheckpointTime == 0) {
+      return -1;
+    } else {
+      return (Time.monotonicNow() - lastCheckpointTime);
+    }
+  }
+
+  @Override // SecondaryNameNodeInfoMXBean
   public String[] getCheckpointDirectories() {
     ArrayList<String> r = Lists.newArrayListWithCapacity(checkpointDirs.size());
     for (URI d : checkpointDirs) {
@@ -719,7 +732,7 @@ public class SecondaryNameNode implements Runnable,
     return r.toArray(new String[r.size()]);
   }
 
-  @Override // SecondaryNameNodeInfoMXXBean
+  @Override // SecondaryNameNodeInfoMXBean
   public String[] getCheckpointEditlogDirectories() {
     ArrayList<String> r = Lists.newArrayListWithCapacity(checkpointEditsDirs.size());
     for (URI d : checkpointEditsDirs) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0953b5e7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNodeInfoMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNodeInfoMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNodeInfoMXBean.java
index 01f6fac..785c5ee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNodeInfoMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNodeInfoMXBean.java
@@ -42,6 +42,12 @@ public interface SecondaryNameNodeInfoMXBean extends VersionInfoMXBean {
   public long getLastCheckpointTime();
 
   /**
+   * @return the number of msec since the last checkpoint, or -1 if no
+   * checkpoint has been done yet.
+   */
+  public long getLastCheckpointDeltaMs();
+
+  /**
    * @return the directories that store the checkpoint images
    */
   public String[] getCheckpointDirectories();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0953b5e7/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.html
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.html
index 7893a17..2ef58ad 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.html
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/status.html
@@ -65,7 +65,7 @@
   <tr><th>Compiled</th><td>{CompileInfo}</td></tr>
   <tr><th>NameNode Address</th><td>{HostAndPort}</td></tr>
   <tr><th>Started</th><td>{StartTime|date_tostring}</td></tr>
-  <tr><th>Last Checkpoint</th><td>{@if cond="{LastCheckpointTime} === 0"}Never{:else}{LastCheckpointTime|date_tostring}{/if}</td></tr>
+  <tr><th>Last Checkpoint</th><td>{@if cond="{LastCheckpointTime} === 0"}Never{:else}{LastCheckpointTime|date_tostring} ({LastCheckpointDeltaMs|fmt_time} ago){/if}</td></tr>
   <tr><th>Checkpoint Period</th><td>{CheckpointPeriod} seconds</td></tr>
   <tr><th>Checkpoint Transactions</th><td>{TxnCount}</td></tr>
 </table>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0953b5e7/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dfs-dust.js
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dfs-dust.js b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dfs-dust.js
index 3c8efd9..f483d23 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dfs-dust.js
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dfs-dust.js
@@ -40,6 +40,9 @@
     },
 
     'fmt_time': function (v) {
+      if (v < 0) {
+        return "unknown";
+      }
       var s = Math.floor(v / 1000), h = Math.floor(s / 3600);
       s -= h * 3600;
       var m = Math.floor(s / 60);


[37/50] [abbrv] hadoop git commit: HDFS-8249. Separate HdfsConstants into the client and the server side class. Contributed by Haohui Mai.

Posted by zj...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java
index 7708ddc..ed54aeb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java
@@ -38,7 +38,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.TestBlockStoragePolicy;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NetworkTopologyWithNodeGroup;
@@ -191,8 +191,8 @@ public class TestReplicationPolicyWithNodeGroup {
   private static void setupDataNodeCapacity() {
     for(int i=0; i<NUM_OF_DATANODES; i++) {
       updateHeartbeatWithUsage(dataNodes[i],
-          2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
-          2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
+          2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+          2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
     }
   }
   
@@ -274,8 +274,8 @@ public class TestReplicationPolicyWithNodeGroup {
   @Test
   public void testChooseTarget1() throws Exception {
     updateHeartbeatWithUsage(dataNodes[0],
-        2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 
-        HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+        2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+        HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
         0L, 0L, 4, 0); // overloaded
 
     DatanodeStorageInfo[] targets;
@@ -312,8 +312,8 @@ public class TestReplicationPolicyWithNodeGroup {
     verifyNoTwoTargetsOnSameNodeGroup(targets);
 
     updateHeartbeatWithUsage(dataNodes[0],
-        2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
-        HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
+        2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+        HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
   }
 
   private void verifyNoTwoTargetsOnSameNodeGroup(DatanodeStorageInfo[] targets) {
@@ -380,8 +380,8 @@ public class TestReplicationPolicyWithNodeGroup {
   public void testChooseTarget3() throws Exception {
     // make data node 0 to be not qualified to choose
     updateHeartbeatWithUsage(dataNodes[0],
-        2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
-        (HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L,
+        2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+        (HdfsServerConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L,
         0L, 0L, 0, 0); // no space
 
     DatanodeStorageInfo[] targets;
@@ -412,8 +412,8 @@ public class TestReplicationPolicyWithNodeGroup {
                isOnSameRack(targets[2], targets[3]));
 
     updateHeartbeatWithUsage(dataNodes[0],
-        2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
-        HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
+        2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+        HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
   }
 
   /**
@@ -430,8 +430,8 @@ public class TestReplicationPolicyWithNodeGroup {
     // make data node 0-2 to be not qualified to choose: not enough disk space
     for(int i=0; i<3; i++) {
       updateHeartbeatWithUsage(dataNodes[i],
-          2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
-          (HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
+          2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+          (HdfsServerConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
     }
 
     DatanodeStorageInfo[] targets;
@@ -661,13 +661,13 @@ public class TestReplicationPolicyWithNodeGroup {
     }
     for(int i=0; i<NUM_OF_DATANODES_BOUNDARY; i++) {
       updateHeartbeatWithUsage(dataNodes[0],
-                2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
-                (HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE,
+                2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+                (HdfsServerConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE,
                 0L, 0L, 0L, 0, 0);
 
       updateHeartbeatWithUsage(dataNodesInBoundaryCase[i],
-          2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
-          2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
+          2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+          2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
     }
 
     DatanodeStorageInfo[] targets;
@@ -697,8 +697,8 @@ public class TestReplicationPolicyWithNodeGroup {
   public void testRereplicateOnBoundaryTopology() throws Exception {
     for(int i=0; i<NUM_OF_DATANODES_BOUNDARY; i++) {
       updateHeartbeatWithUsage(dataNodesInBoundaryCase[i],
-          2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
-          2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
+          2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+          2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
     }
     List<DatanodeStorageInfo> chosenNodes = new ArrayList<DatanodeStorageInfo>();
     chosenNodes.add(storagesInBoundaryCase[0]);
@@ -735,8 +735,8 @@ public class TestReplicationPolicyWithNodeGroup {
 
     for(int i=0; i<NUM_OF_DATANODES_MORE_TARGETS; i++) {
       updateHeartbeatWithUsage(dataNodesInMoreTargetsCase[i],
-          2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
-          2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
+          2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+          2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
     }
 
     DatanodeStorageInfo[] targets;
@@ -786,8 +786,8 @@ public class TestReplicationPolicyWithNodeGroup {
     //Update heartbeat
     for(int i=0; i<NUM_OF_DATANODES_FOR_DEPENDENCIES; i++) {
       updateHeartbeatWithUsage(dataNodesForDependencies[i],
-          2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
-          2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
+          2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+          2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
     }
     
     List<DatanodeStorageInfo> chosenNodes = new ArrayList<DatanodeStorageInfo>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java
index 3c16743..216ff3d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java
@@ -29,8 +29,8 @@ import java.net.InetSocketAddress;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -67,7 +67,7 @@ public class TestDatanodeRegister {
     // Return a a good software version.
     doReturn(VersionInfo.getVersion()).when(fakeNsInfo).getSoftwareVersion();
     // Return a good layout version for now.
-    doReturn(HdfsConstants.NAMENODE_LAYOUT_VERSION).when(fakeNsInfo)
+    doReturn(HdfsServerConstants.NAMENODE_LAYOUT_VERSION).when(fakeNsInfo)
         .getLayoutVersion();
     
     DatanodeProtocolClientSideTranslatorPB fakeDnProt = 
@@ -104,12 +104,12 @@ public class TestDatanodeRegister {
   @Test
   public void testDifferentLayoutVersions() throws Exception {
     // We expect no exceptions to be thrown when the layout versions match.
-    assertEquals(HdfsConstants.NAMENODE_LAYOUT_VERSION,
+    assertEquals(HdfsServerConstants.NAMENODE_LAYOUT_VERSION,
         actor.retrieveNamespaceInfo().getLayoutVersion());
     
     // We expect an exception to be thrown when the NN reports a layout version
     // different from that of the DN.
-    doReturn(HdfsConstants.NAMENODE_LAYOUT_VERSION * 1000).when(fakeNsInfo)
+    doReturn(HdfsServerConstants.NAMENODE_LAYOUT_VERSION * 1000).when(fakeNsInfo)
         .getLayoutVersion();
     try {
       actor.retrieveNamespaceInfo();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
index fa7a838..150b5e4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
@@ -47,7 +47,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
@@ -405,7 +405,7 @@ public class TestDirectoryScanner {
       // Test2: block metafile is missing
       long blockId = deleteMetaFile();
       scan(totalBlocks, 1, 1, 0, 0, 1);
-      verifyGenStamp(blockId, HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP);
+      verifyGenStamp(blockId, HdfsConstants.GRANDFATHER_GENERATION_STAMP);
       scan(totalBlocks, 0, 0, 0, 0, 0);
 
       // Test3: block file is missing
@@ -420,7 +420,7 @@ public class TestDirectoryScanner {
       blockId = createBlockFile();
       totalBlocks++;
       scan(totalBlocks, 1, 1, 0, 1, 0);
-      verifyAddition(blockId, HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP, 0);
+      verifyAddition(blockId, HdfsConstants.GRANDFATHER_GENERATION_STAMP, 0);
       scan(totalBlocks, 0, 0, 0, 0, 0);
 
       // Test5: A metafile exists for which there is no block file and

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
index 7eaf5c7..a5d5848 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
@@ -43,7 +43,6 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -54,6 +53,7 @@ import org.apache.hadoop.hdfs.server.balancer.ExitStatus;
 import org.apache.hadoop.hdfs.server.balancer.TestBalancer;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
@@ -98,9 +98,9 @@ public class TestStorageMover {
     DEFAULT_CONF.setLong(DFSConfigKeys.DFS_MOVER_MOVEDWINWIDTH_KEY, 2000L);
 
     DEFAULT_POLICIES = BlockStoragePolicySuite.createDefaultSuite();
-    HOT = DEFAULT_POLICIES.getPolicy(HdfsConstants.HOT_STORAGE_POLICY_NAME);
-    WARM = DEFAULT_POLICIES.getPolicy(HdfsConstants.WARM_STORAGE_POLICY_NAME);
-    COLD = DEFAULT_POLICIES.getPolicy(HdfsConstants.COLD_STORAGE_POLICY_NAME);
+    HOT = DEFAULT_POLICIES.getPolicy(HdfsServerConstants.HOT_STORAGE_POLICY_NAME);
+    WARM = DEFAULT_POLICIES.getPolicy(HdfsServerConstants.WARM_STORAGE_POLICY_NAME);
+    COLD = DEFAULT_POLICIES.getPolicy(HdfsServerConstants.COLD_STORAGE_POLICY_NAME);
     TestBalancer.initTestSetup();
     Dispatcher.setDelayAfterErrors(1000L);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
index 360261d..db0185d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
@@ -42,7 +42,6 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
@@ -596,7 +595,7 @@ public class NNThroughputBenchmark implements Tool {
       long end = Time.now();
       for(boolean written = !closeUponCreate; !written; 
         written = nameNodeProto.complete(fileNames[daemonId][inputIdx],
-                                    clientName, null, HdfsConstantsClient.GRANDFATHER_INODE_ID));
+                                    clientName, null, HdfsConstants.GRANDFATHER_INODE_ID));
       return end-start;
     }
 
@@ -1142,7 +1141,7 @@ public class NNThroughputBenchmark implements Tool {
             new EnumSetWritable<CreateFlag>(EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)), true, replication,
             BLOCK_SIZE, null);
         ExtendedBlock lastBlock = addBlocks(fileName, clientName);
-        nameNodeProto.complete(fileName, clientName, lastBlock, HdfsConstantsClient.GRANDFATHER_INODE_ID);
+        nameNodeProto.complete(fileName, clientName, lastBlock, HdfsConstants.GRANDFATHER_INODE_ID);
       }
       // prepare block reports
       for(int idx=0; idx < nrDatanodes; idx++) {
@@ -1155,7 +1154,7 @@ public class NNThroughputBenchmark implements Tool {
       ExtendedBlock prevBlock = null;
       for(int jdx = 0; jdx < blocksPerFile; jdx++) {
         LocatedBlock loc = nameNodeProto.addBlock(fileName, clientName,
-            prevBlock, null, HdfsConstantsClient.GRANDFATHER_INODE_ID, null);
+            prevBlock, null, HdfsConstants.GRANDFATHER_INODE_ID, null);
         prevBlock = loc.getBlock();
         for(DatanodeInfo dnInfo : loc.getLocations()) {
           int dnIdx = Arrays.binarySearch(datanodes, dnInfo.getXferAddr());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java
index 2d29a68..5a4134c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
@@ -89,14 +89,14 @@ public class TestAddBlockRetry {
     LOG.info("Starting first addBlock for " + src);
     LocatedBlock[] onRetryBlock = new LocatedBlock[1];
     DatanodeStorageInfo targets[] = ns.getNewBlockTargets(
-        src, HdfsConstantsClient.GRANDFATHER_INODE_ID, "clientName",
+        src, HdfsConstants.GRANDFATHER_INODE_ID, "clientName",
         null, null, null, onRetryBlock);
     assertNotNull("Targets must be generated", targets);
 
     // run second addBlock()
     LOG.info("Starting second addBlock for " + src);
     nn.addBlock(src, "clientName", null, null,
-        HdfsConstantsClient.GRANDFATHER_INODE_ID, null);
+        HdfsConstants.GRANDFATHER_INODE_ID, null);
     assertTrue("Penultimate block must be complete",
         checkFileProgress(src, false));
     LocatedBlocks lbs = nn.getBlockLocations(src, 0, Long.MAX_VALUE);
@@ -106,7 +106,7 @@ public class TestAddBlockRetry {
 
     // continue first addBlock()
     LocatedBlock newBlock = ns.storeAllocatedBlock(
-        src, HdfsConstantsClient.GRANDFATHER_INODE_ID, "clientName", null, targets);
+        src, HdfsConstants.GRANDFATHER_INODE_ID, "clientName", null, targets);
     assertEquals("Blocks are not equal", lb2.getBlock(), newBlock.getBlock());
 
     // check locations
@@ -144,14 +144,14 @@ public class TestAddBlockRetry {
     // start first addBlock()
     LOG.info("Starting first addBlock for " + src);
     LocatedBlock lb1 = nameNodeRpc.addBlock(src, "clientName", null, null,
-        HdfsConstantsClient.GRANDFATHER_INODE_ID, null);
+        HdfsConstants.GRANDFATHER_INODE_ID, null);
     assertTrue("Block locations should be present",
         lb1.getLocations().length > 0);
 
     cluster.restartNameNode();
     nameNodeRpc = cluster.getNameNodeRpc();
     LocatedBlock lb2 = nameNodeRpc.addBlock(src, "clientName", null, null,
-        HdfsConstantsClient.GRANDFATHER_INODE_ID, null);
+        HdfsConstants.GRANDFATHER_INODE_ID, null);
     assertEquals("Blocks are not equal", lb1.getBlock(), lb2.getBlock());
     assertTrue("Wrong locations with retry", lb2.getLocations().length > 0);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
index 6d8d205..1e42e34 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
@@ -68,9 +68,9 @@ import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
@@ -880,12 +880,12 @@ public class TestEditLog {
   
     @Override
     public long getFirstTxId() {
-      return HdfsConstants.INVALID_TXID;
+      return HdfsServerConstants.INVALID_TXID;
     }
     
     @Override
     public long getLastTxId() {
-      return HdfsConstants.INVALID_TXID;
+      return HdfsServerConstants.INVALID_TXID;
     }
   
     @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java
index c3d2997..c0eb890 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java
@@ -29,7 +29,7 @@ import java.net.HttpURLConnection;
 import java.net.URL;
 import java.util.EnumMap;
 
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.util.Holder;
 import org.apache.hadoop.hdfs.web.URLConnectionFactory;
 import org.junit.Test;
@@ -51,7 +51,7 @@ public class TestEditLogFileInputStream {
 
     URL url = new URL("http://localhost/fakeLog");
     EditLogInputStream elis = EditLogFileInputStream.fromUrl(factory, url,
-        HdfsConstants.INVALID_TXID, HdfsConstants.INVALID_TXID, false);
+        HdfsServerConstants.INVALID_TXID, HdfsServerConstants.INVALID_TXID, false);
     // Read the edit log and verify that we got all of the data.
     EnumMap<FSEditLogOpCodes, Holder<Integer>> counts = FSImageTestUtil
         .countEditLogOpTypes(elis);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
index 833ef95..bc55d12 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
@@ -43,7 +43,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
@@ -365,7 +365,7 @@ public class TestFSEditLogLoader {
       truncateFile(logFile, txOffset);
       validation = EditLogFileInputStream.validateEditLog(logFile);
       long expectedEndTxId = (txId == 0) ?
-          HdfsConstants.INVALID_TXID : (txId - 1);
+          HdfsServerConstants.INVALID_TXID : (txId - 1);
       assertEquals("Failed when corrupting txid " + txId + " txn opcode " +
         "at " + txOffset, expectedEndTxId, validation.getEndTxId());
       assertTrue(!validation.hasCorruptHeader());
@@ -383,7 +383,7 @@ public class TestFSEditLogLoader {
     EditLogValidation validation =
         EditLogFileInputStream.validateEditLog(logFile);
     assertTrue(!validation.hasCorruptHeader());
-    assertEquals(HdfsConstants.INVALID_TXID, validation.getEndTxId());
+    assertEquals(HdfsServerConstants.INVALID_TXID, validation.getEndTxId());
   }
 
   private static final Map<Byte, FSEditLogOpCodes> byteToEnum =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java
index d11b783..9903906 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java
@@ -47,7 +47,7 @@ import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -430,7 +430,7 @@ public class TestFSPermissionChecker {
     PermissionStatus permStatus = PermissionStatus.createImmutable(owner, group,
       FsPermission.createImmutable(perm));
     INodeDirectory inodeDirectory = new INodeDirectory(
-      HdfsConstantsClient.GRANDFATHER_INODE_ID, name.getBytes("UTF-8"), permStatus, 0L);
+      HdfsConstants.GRANDFATHER_INODE_ID, name.getBytes("UTF-8"), permStatus, 0L);
     parent.addChild(inodeDirectory);
     return inodeDirectory;
   }
@@ -439,7 +439,7 @@ public class TestFSPermissionChecker {
       String owner, String group, short perm) throws IOException {
     PermissionStatus permStatus = PermissionStatus.createImmutable(owner, group,
       FsPermission.createImmutable(perm));
-    INodeFile inodeFile = new INodeFile(HdfsConstantsClient.GRANDFATHER_INODE_ID,
+    INodeFile inodeFile = new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID,
       name.getBytes("UTF-8"), permStatus, 0L, 0L, null, REPLICATION,
       PREFERRED_BLOCK_SIZE);
     parent.addChild(inodeFile);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
index fbcc73f..bd19a5b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
@@ -51,7 +51,6 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -650,8 +649,8 @@ public class TestFileTruncate {
     checkBlockRecovery(p);
 
     NameNodeAdapter.getLeaseManager(cluster.getNamesystem())
-        .setLeasePeriod(HdfsConstants.LEASE_SOFTLIMIT_PERIOD,
-            HdfsConstants.LEASE_HARDLIMIT_PERIOD);
+        .setLeasePeriod(HdfsServerConstants.LEASE_SOFTLIMIT_PERIOD,
+            HdfsServerConstants.LEASE_HARDLIMIT_PERIOD);
 
     checkFullFile(p, newLength, contents);
     fs.delete(p, false);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
index 4e6c59a..b45d2f6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
@@ -55,7 +55,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -86,12 +86,12 @@ public class TestINodeFile {
   private long preferredBlockSize = 1024;
 
   INodeFile createINodeFile(short replication, long preferredBlockSize) {
-    return new INodeFile(HdfsConstantsClient.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
+    return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
         null, replication, preferredBlockSize);
   }
 
   private static INodeFile createINodeFile(byte storagePolicyID) {
-    return new INodeFile(HdfsConstantsClient.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
+    return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
         null, (short)3, 1024L, storagePolicyID);
   }
 
@@ -194,9 +194,9 @@ public class TestINodeFile {
     INodeFile inf = createINodeFile(replication, preferredBlockSize);
     inf.setLocalName(DFSUtil.string2Bytes("f"));
 
-    INodeDirectory root = new INodeDirectory(HdfsConstantsClient.GRANDFATHER_INODE_ID,
+    INodeDirectory root = new INodeDirectory(HdfsConstants.GRANDFATHER_INODE_ID,
         INodeDirectory.ROOT_NAME, perm, 0L);
-    INodeDirectory dir = new INodeDirectory(HdfsConstantsClient.GRANDFATHER_INODE_ID,
+    INodeDirectory dir = new INodeDirectory(HdfsConstants.GRANDFATHER_INODE_ID,
         DFSUtil.string2Bytes("d"), perm, 0L);
 
     assertEquals("f", inf.getFullPathName());
@@ -345,7 +345,7 @@ public class TestINodeFile {
 
     {//cast from INodeFileUnderConstruction
       final INode from = new INodeFile(
-          HdfsConstantsClient.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, null, replication,
+          HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, null, replication,
           1024L);
       from.asFile().toUnderConstruction("client", "machine");
     
@@ -363,7 +363,7 @@ public class TestINodeFile {
     }
 
     {//cast from INodeDirectory
-      final INode from = new INodeDirectory(HdfsConstantsClient.GRANDFATHER_INODE_ID, null,
+      final INode from = new INodeDirectory(HdfsConstants.GRANDFATHER_INODE_ID, null,
           perm, 0L);
 
       //cast to INodeFile, should fail
@@ -1108,7 +1108,7 @@ public class TestINodeFile {
   @Test
   public void testFileUnderConstruction() {
     replication = 3;
-    final INodeFile file = new INodeFile(HdfsConstantsClient.GRANDFATHER_INODE_ID, null,
+    final INodeFile file = new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null,
         perm, 0L, 0L, null, replication, 1024L);
     assertFalse(file.isUnderConstruction());
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetadataVersionOutput.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetadataVersionOutput.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetadataVersionOutput.java
index 03c7557..4e4ed0c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetadataVersionOutput.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetadataVersionOutput.java
@@ -22,8 +22,8 @@ import static org.junit.Assert.assertTrue;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.junit.After;
 import org.junit.Test;
 
@@ -79,7 +79,7 @@ public class TestMetadataVersionOutput {
       assertExceptionContains("ExitException", e);
     }
     /* Check if meta data version is printed correctly. */
-    final String verNumStr = HdfsConstants.NAMENODE_LAYOUT_VERSION + "";
+    final String verNumStr = HdfsServerConstants.NAMENODE_LAYOUT_VERSION + "";
     assertTrue(baos.toString("UTF-8").
       contains("HDFS Image Version: " + verNumStr));
     assertTrue(baos.toString("UTF-8").

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeOptionParsing.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeOptionParsing.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeOptionParsing.java
index a3582ce..7ee49a9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeOptionParsing.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeOptionParsing.java
@@ -22,7 +22,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.junit.Assert;
@@ -70,11 +70,11 @@ public class TestNameNodeOptionParsing {
     opt = NameNode.parseArguments(new String[] { "-upgrade", "-renameReserved"});
     assertEquals(StartupOption.UPGRADE, opt);
     assertEquals(
-        ".snapshot." + HdfsConstants.NAMENODE_LAYOUT_VERSION
+        ".snapshot." + HdfsServerConstants.NAMENODE_LAYOUT_VERSION
             + ".UPGRADE_RENAMED",
         FSImageFormat.renameReservedMap.get(".snapshot"));
     assertEquals(
-        ".reserved." + HdfsConstants.NAMENODE_LAYOUT_VERSION
+        ".reserved." + HdfsServerConstants.NAMENODE_LAYOUT_VERSION
             + ".UPGRADE_RENAMED",
         FSImageFormat.renameReservedMap.get(".reserved"));
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java
index a4cb97f..49d01c1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
 import org.junit.After;
 import org.junit.Assert;
@@ -68,7 +68,7 @@ public class TestTruncateQuotaUpdate {
     dfs.mkdirs(dir);
     dfs.setQuota(dir, Long.MAX_VALUE - 1, DISKQUOTA);
     dfs.setQuotaByStorageType(dir, StorageType.DISK, DISKQUOTA);
-    dfs.setStoragePolicy(dir, HdfsConstants.HOT_STORAGE_POLICY_NAME);
+    dfs.setStoragePolicy(dir, HdfsServerConstants.HOT_STORAGE_POLICY_NAME);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSUpgradeWithHA.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSUpgradeWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSUpgradeWithHA.java
index a77b435..cb7cb35 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSUpgradeWithHA.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSUpgradeWithHA.java
@@ -38,10 +38,10 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
 import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster.Builder;
 import org.apache.hadoop.hdfs.qjournal.server.Journal;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -304,7 +304,7 @@ public class TestDFSUpgradeWithHA {
     BestEffortLongFile committedTxnId = (BestEffortLongFile) Whitebox
         .getInternalState(journal1, "committedTxnId");
     return committedTxnId != null ? committedTxnId.get() :
-        HdfsConstants.INVALID_TXID;
+        HdfsServerConstants.INVALID_TXID;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java
index 29c6b10..f0c5482 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java
@@ -28,7 +28,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
@@ -196,7 +196,7 @@ public class TestOpenFilesWithSnapshot {
     String clientName = fs.getClient().getClientName();
     // create one empty block
     nameNodeRpc.addBlock(fileWithEmptyBlock.toString(), clientName, null, null,
-        HdfsConstantsClient.GRANDFATHER_INODE_ID, null);
+        HdfsConstants.GRANDFATHER_INODE_ID, null);
     fs.createSnapshot(path, "s2");
 
     fs.rename(new Path("/test/test"), new Path("/test/test-renamed"));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java
index b20e2ad..abceea4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java
@@ -57,7 +57,6 @@ import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper.TestDirectoryTree;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper.TestDirectoryTree.Node;
 import org.apache.hadoop.hdfs.tools.offlineImageViewer.PBImageXmlWriter;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/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 a5d5087..391f190 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
@@ -39,7 +39,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.util.Time;
 import org.codehaus.jackson.map.ObjectMapper;
@@ -65,7 +65,7 @@ public class TestJsonUtil {
     final HdfsFileStatus status = new HdfsFileStatus(1001L, false, 3, 1L << 26,
         now, now + 10, new FsPermission((short) 0644), "user", "group",
         DFSUtil.string2Bytes("bar"), DFSUtil.string2Bytes("foo"),
-        HdfsConstantsClient.GRANDFATHER_INODE_ID, 0, null, (byte) 0);
+        HdfsConstants.GRANDFATHER_INODE_ID, 0, null, (byte) 0);
     final FileStatus fstatus = toFileStatus(status, parent);
     System.out.println("status  = " + status);
     System.out.println("fstatus = " + fstatus);


[22/50] [abbrv] hadoop git commit: HADOOP-11866. increase readability and reliability of checkstyle, shellcheck, and whitespace reports (aw)

Posted by zj...@apache.org.
HADOOP-11866. increase readability and reliability of checkstyle, shellcheck, and whitespace reports (aw)


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

Branch: refs/heads/YARN-2928
Commit: f07f35675c1d6f75f0a680abfc6f291327d3c43a
Parents: 6f6b313
Author: Allen Wittenauer <aw...@apache.org>
Authored: Thu Apr 30 15:15:32 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:56 2015 -0700

----------------------------------------------------------------------
 dev-support/test-patch.d/checkstyle.sh          | 216 ++++++++++++-------
 dev-support/test-patch.d/shellcheck.sh          |  52 ++++-
 dev-support/test-patch.d/whitespace.sh          |  12 +-
 dev-support/test-patch.sh                       |  73 ++++++-
 hadoop-common-project/hadoop-common/CHANGES.txt |   3 +
 5 files changed, 255 insertions(+), 101 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f07f3567/dev-support/test-patch.d/checkstyle.sh
----------------------------------------------------------------------
diff --git a/dev-support/test-patch.d/checkstyle.sh b/dev-support/test-patch.d/checkstyle.sh
index 460709e..6311584 100755
--- a/dev-support/test-patch.d/checkstyle.sh
+++ b/dev-support/test-patch.d/checkstyle.sh
@@ -29,8 +29,39 @@ function checkstyle_filefilter
   fi
 }
 
+function checkstyle_mvnrunner
+{
+  local logfile=$1
+  local output=$2
+  local tmp=${PATCH_DIR}/$$.${RANDOM}
+  local j
+
+  "${MVN}" clean test checkstyle:checkstyle -DskipTests \
+    -Dcheckstyle.consoleOutput=true \
+    "-D${PROJECT_NAME}PatchProcess" 2>&1 \
+      | tee "${logfile}" \
+      | ${GREP} ^/ \
+      | ${SED} -e "s,${BASEDIR},.,g" \
+          > "${tmp}"
+
+  # the checkstyle output files are massive, so
+  # let's reduce the work by filtering out files
+  # that weren't changed.  Some modules are
+  # MASSIVE and this can cut the output down to
+  # by orders of magnitude!!
+  for j in ${CHANGED_FILES}; do
+    ${GREP} "${j}" "${tmp}" >> "${output}"
+  done
+
+  rm "${tmp}" 2>/dev/null
+}
+
 function checkstyle_preapply
 {
+  local module_suffix
+  local modules=${CHANGED_MODULES}
+  local module
+
   verify_needed_test checkstyle
 
   if [[ $? == 0 ]]; then
@@ -40,23 +71,78 @@ function checkstyle_preapply
   big_console_header "checkstyle plugin: prepatch"
 
   start_clock
-  echo_and_redirect "${PATCH_DIR}/${PATCH_BRANCH}checkstyle.txt" "${MVN}" test checkstyle:checkstyle-aggregate -DskipTests "-D${PROJECT_NAME}PatchProcess"
-  if [[ $? != 0 ]] ; then
-    echo "Pre-patch ${PATCH_BRANCH} checkstyle compilation is broken?"
-    add_jira_table -1 checkstyle "Pre-patch ${PATCH_BRANCH} checkstyle compilation may be broken."
-    return 1
-  fi
 
-  cp -p "${BASEDIR}/target/checkstyle-result.xml" \
-    "${PATCH_DIR}/checkstyle-result-${PATCH_BRANCH}.xml"
+  for module in ${modules}
+  do
+    pushd "${module}" >/dev/null
+    echo "  Running checkstyle in ${module}"
+    module_suffix=$(basename "${module}")
+
+    checkstyle_mvnrunner \
+      "${PATCH_DIR}/maven-${PATCH_BRANCH}checkstyle-${module_suffix}.txt" \
+      "${PATCH_DIR}/${PATCH_BRANCH}checkstyle${module_suffix}.txt"
+
+    if [[ $? != 0 ]] ; then
+      echo "Pre-patch ${PATCH_BRANCH} checkstyle compilation is broken?"
+      add_jira_table -1 checkstyle "Pre-patch ${PATCH_BRANCH} ${module} checkstyle compilation may be broken."
+    fi
+    popd >/dev/null
+  done
 
   # keep track of how much as elapsed for us already
   CHECKSTYLE_TIMER=$(stop_clock)
   return 0
 }
 
+function checkstyle_calcdiffs
+{
+  local orig=$1
+  local new=$2
+  local diffout=$3
+  local tmp=${PATCH_DIR}/cs.$$.${RANDOM}
+  local count=0
+  local j
+
+  # first, pull out just the errors
+  # shellcheck disable=SC2016
+  ${AWK} -F: '{print $NF}' "${orig}" >> "${tmp}.branch"
+
+  # shellcheck disable=SC2016
+  ${AWK} -F: '{print $NF}' "${new}" >> "${tmp}.patch"
+
+  # compare the errors, generating a string of line
+  # numbers.  Sorry portability: GNU diff makes this too easy
+  ${DIFF} --unchanged-line-format="" \
+     --old-line-format="" \
+     --new-line-format="%dn " \
+     "${tmp}.branch" \
+     "${tmp}.patch" > "${tmp}.lined"
+
+  # now, pull out those lines of the raw output
+  # shellcheck disable=SC2013
+  for j in $(cat "${tmp}.lined"); do
+    # shellcheck disable=SC2086
+    head -${j} "${new}" | tail -1 >> "${diffout}"
+  done
+
+  if [[ -f "${diffout}" ]]; then
+    # shellcheck disable=SC2016
+    count=$(wc -l "${diffout}" | ${AWK} '{print $1}' )
+  fi
+  rm "${tmp}.branch" "${tmp}.patch" "${tmp}.lined" 2>/dev/null
+  echo "${count}"
+}
+
 function checkstyle_postapply
 {
+  local rc=0
+  local module
+  local modules=${CHANGED_MODULES}
+  local module_suffix
+  local numprepatch=0
+  local numpostpatch=0
+  local diffpostpatch=0
+
   verify_needed_test checkstyle
 
   if [[ $? == 0 ]]; then
@@ -71,79 +157,49 @@ function checkstyle_postapply
   # by setting the clock back
   offset_clock "${CHECKSTYLE_TIMER}"
 
-  echo_and_redirect "${PATCH_DIR}/patchcheckstyle.txt" "${MVN}" test checkstyle:checkstyle-aggregate -DskipTests "-D${PROJECT_NAME}PatchProcess"
-  if [[ $? != 0 ]] ; then
-    echo "Post-patch checkstyle compilation is broken."
-    add_jira_table -1 checkstyle "Post-patch checkstyle compilation is broken."
-    return 1
-  fi
-
-  cp -p "${BASEDIR}/target/checkstyle-result.xml" \
-    "${PATCH_DIR}/checkstyle-result-patch.xml"
-
-  checkstyle_runcomparison
-
-  # shellcheck disable=SC2016
-  CHECKSTYLE_POSTPATCH=$(wc -l "${PATCH_DIR}/checkstyle-result-diff.txt" | ${AWK} '{print $1}')
-
-  if [[ ${CHECKSTYLE_POSTPATCH} -gt 0 ]] ; then
-
-    add_jira_table -1 checkstyle "The applied patch generated "\
-      "${CHECKSTYLE_POSTPATCH}" \
-      " additional checkstyle issues."
-    add_jira_footer checkstyle "@@BASE@@/checkstyle-result-diff.txt"
-
+  for module in ${modules}
+  do
+    pushd "${module}" >/dev/null
+    echo "  Running checkstyle in ${module}"
+    module_suffix=$(basename "${module}")
+
+    checkstyle_mvnrunner \
+      "${PATCH_DIR}/maven-patchcheckstyle-${module_suffix}.txt" \
+      "${PATCH_DIR}/patchcheckstyle${module_suffix}.txt"
+
+    if [[ $? != 0 ]] ; then
+      ((rc = rc +1))
+      echo "Post-patch checkstyle compilation is broken."
+      add_jira_table -1 checkstyle "Post-patch checkstyle ${module} compilation is broken."
+      continue
+    fi
+
+    #shellcheck disable=SC2016
+    diffpostpatch=$(checkstyle_calcdiffs \
+      "${PATCH_DIR}/${PATCH_BRANCH}checkstyle${module_suffix}.txt" \
+      "${PATCH_DIR}/patchcheckstyle${module_suffix}.txt" \
+      "${PATCH_DIR}/diffcheckstyle${module_suffix}.txt" )
+
+    if [[ ${diffpostpatch} -gt 0 ]] ; then
+      ((rc = rc + 1))
+
+      # shellcheck disable=SC2016
+      numprepatch=$(wc -l "${PATCH_DIR}/${PATCH_BRANCH}checkstyle${module_suffix}.txt" | ${AWK} '{print $1}')
+      # shellcheck disable=SC2016
+      numpostpatch=$(wc -l "${PATCH_DIR}/patchcheckstyle${module_suffix}.txt" | ${AWK} '{print $1}')
+
+      add_jira_table -1 checkstyle "The applied patch generated "\
+        "${diffpostpatch} new checkstyle issues (total was ${numprepatch}, now ${numpostpatch})."
+      footer="${footer} @@BASE@@/diffcheckstyle${module_suffix}.txt"
+    fi
+
+    popd >/dev/null
+  done
+
+  if [[ ${rc} -gt 0 ]] ; then
+    add_jira_footer checkstyle "${footer}"
     return 1
   fi
   add_jira_table +1 checkstyle "There were no new checkstyle issues."
   return 0
-}
-
-
-function checkstyle_runcomparison
-{
-
-  python <(cat <<EOF
-import os
-import sys
-import xml.etree.ElementTree as etree
-from collections import defaultdict
-
-if len(sys.argv) != 3 :
-  print "usage: %s checkstyle-result-master.xml checkstyle-result-patch.xml" % sys.argv[0]
-  exit(1)
-
-def path_key(x):
-  path = x.attrib['name']
-  return path[path.find('${PROJECT_NAME}-'):]
-
-def print_row(path, master_errors, patch_errors):
-    print '%s\t%s\t%s' % (k,master_dict[k],child_errors)
-
-master = etree.parse(sys.argv[1])
-patch = etree.parse(sys.argv[2])
-
-master_dict = defaultdict(int)
-
-for child in master.getroot().getchildren():
-    if child.tag != 'file':
-        continue
-    child_errors = len(child.getchildren())
-    if child_errors == 0:
-        continue
-    master_dict[path_key(child)] = child_errors
-
-for child in patch.getroot().getchildren():
-    if child.tag != 'file':
-        continue
-    child_errors = len(child.getchildren())
-    if child_errors == 0:
-        continue
-    k = path_key(child)
-    if child_errors > master_dict[k]:
-        print_row(k, master_dict[k], child_errors)
-
-EOF
-) "${PATCH_DIR}/checkstyle-result-${PATCH_BRANCH}.xml" "${PATCH_DIR}/checkstyle-result-patch.xml" > "${PATCH_DIR}/checkstyle-result-diff.txt"
-
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f07f3567/dev-support/test-patch.d/shellcheck.sh
----------------------------------------------------------------------
diff --git a/dev-support/test-patch.d/shellcheck.sh b/dev-support/test-patch.d/shellcheck.sh
index c1084a2..5f38b6a 100755
--- a/dev-support/test-patch.d/shellcheck.sh
+++ b/dev-support/test-patch.d/shellcheck.sh
@@ -87,6 +87,45 @@ function shellcheck_preapply
   return 0
 }
 
+function shellcheck_calcdiffs
+{
+  local orig=$1
+  local new=$2
+  local diffout=$3
+  local tmp=${PATCH_DIR}/sc.$$.${RANDOM}
+  local count=0
+  local j
+
+  # first, pull out just the errors
+  # shellcheck disable=SC2016
+  ${AWK} -F: '{print $NF}' "${orig}" >> "${tmp}.branch"
+
+  # shellcheck disable=SC2016
+  ${AWK} -F: '{print $NF}' "${new}" >> "${tmp}.patch"
+
+  # compare the errors, generating a string of line
+  # numbers.  Sorry portability: GNU diff makes this too easy
+  ${DIFF} --unchanged-line-format="" \
+     --old-line-format="" \
+     --new-line-format="%dn " \
+     "${tmp}.branch" \
+     "${tmp}.patch" > "${tmp}.lined"
+
+  # now, pull out those lines of the raw output
+  # shellcheck disable=SC2013
+  for j in $(cat "${tmp}.lined"); do
+    # shellcheck disable=SC2086
+    head -${j} "${new}" | tail -1 >> "${diffout}"
+  done
+
+  if [[ -f "${diffout}" ]]; then
+    # shellcheck disable=SC2016
+    count=$(wc -l "${diffout}" | ${AWK} '{print $1}' )
+  fi
+  rm "${tmp}.branch" "${tmp}.patch" "${tmp}.lined" 2>/dev/null
+  echo "${count}"
+}
+
 function shellcheck_postapply
 {
   local i
@@ -121,16 +160,13 @@ function shellcheck_postapply
   # shellcheck disable=SC2016
   numPostpatch=$(wc -l "${PATCH_DIR}/patchshellcheck-result.txt" | ${AWK} '{print $1}')
 
-  ${DIFF} -u "${PATCH_DIR}/${PATCH_BRANCH}shellcheck-result.txt" \
+  diffPostpatch=$(shellcheck_calcdiffs \
+    "${PATCH_DIR}/${PATCH_BRANCH}shellcheck-result.txt" \
     "${PATCH_DIR}/patchshellcheck-result.txt" \
-      | ${GREP} '^+\.' \
-      > "${PATCH_DIR}/diffpatchshellcheck.txt"
-
-  # shellcheck disable=SC2016
-  diffPostpatch=$(wc -l "${PATCH_DIR}/diffpatchshellcheck.txt" | ${AWK} '{print $1}')
+      "${PATCH_DIR}/diffpatchshellcheck.txt"
+    )
 
-  if [[ ${diffPostpatch} -gt 0
-    && ${numPostpatch} -gt ${numPrepatch} ]] ; then
+  if [[ ${diffPostpatch} -gt 0 ]] ; then
     add_jira_table -1 shellcheck "The applied patch generated "\
       "${diffPostpatch} new shellcheck (v${SHELLCHECK_VERSION}) issues (total was ${numPrepatch}, now ${numPostpatch})."
     add_jira_footer shellcheck "@@BASE@@/diffpatchshellcheck.txt"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f07f3567/dev-support/test-patch.d/whitespace.sh
----------------------------------------------------------------------
diff --git a/dev-support/test-patch.d/whitespace.sh b/dev-support/test-patch.d/whitespace.sh
index deac654..324481c 100755
--- a/dev-support/test-patch.d/whitespace.sh
+++ b/dev-support/test-patch.d/whitespace.sh
@@ -16,25 +16,31 @@
 
 add_plugin whitespace
 
-function whitespace_preapply
+function whitespace_postapply
 {
   local count
+  local j
 
   big_console_header "Checking for whitespace at the end of lines"
   start_clock
 
-  ${GREP} '^+' "${PATCH_DIR}/patch" | ${GREP} '[[:blank:]]$' > "${PATCH_DIR}/whitespace.txt"
+  pushd "${BASEDIR}" >/dev/null
+  for j in ${CHANGED_FILES}; do
+    ${GREP} -nHE '[[:blank:]]$' "./${j}" | ${GREP} -f "${GITDIFFLINES}" >> "${PATCH_DIR}/whitespace.txt"
+  done
 
   # shellcheck disable=SC2016
   count=$(wc -l "${PATCH_DIR}/whitespace.txt" | ${AWK} '{print $1}')
 
   if [[ ${count} -gt 0 ]]; then
     add_jira_table -1 whitespace "The patch has ${count}"\
-      " line(s) that end in whitespace."
+      " line(s) that end in whitespace. Use git apply --whitespace=fix."
     add_jira_footer whitespace "@@BASE@@/whitespace.txt"
+    popd >/dev/null
     return 1
   fi
 
+  popd >/dev/null
   add_jira_table +1 whitespace "The patch has no lines that end in whitespace."
   return 0
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f07f3567/dev-support/test-patch.sh
----------------------------------------------------------------------
diff --git a/dev-support/test-patch.sh b/dev-support/test-patch.sh
index ae21837..b6e1b03 100755
--- a/dev-support/test-patch.sh
+++ b/dev-support/test-patch.sh
@@ -67,7 +67,7 @@ function setup_defaults
       EGREP=${EGREP:-/usr/xpg4/bin/egrep}
       GREP=${GREP:-/usr/xpg4/bin/grep}
       PATCH=${PATCH:-patch}
-      DIFF=${DIFF:-diff}
+      DIFF=${DIFF:-/usr/gnu/bin/diff}
       JIRACLI=${JIRA:-jira}
     ;;
     *)
@@ -449,7 +449,7 @@ function verify_patchdir_still_exists
   if [[ ! -d ${PATCH_DIR} ]]; then
       rm "${commentfile}" 2>/dev/null
 
-      echo "(!) The patch artifact directory on has been removed! " > "${commentfile}"
+      echo "(!) The patch artifact directory has been removed! " > "${commentfile}"
       echo "This is a fatal error for test-patch.sh.  Aborting. " >> "${commentfile}"
       echo
       cat ${commentfile}
@@ -468,6 +468,49 @@ function verify_patchdir_still_exists
     fi
 }
 
+## @description generate a list of all files and line numbers that
+## @description that were added/changed in the source repo
+## @audience    private
+## @stability   stable
+## @params      filename
+## @replaceable no
+function compute_gitdiff
+{
+  local outfile=$1
+  local file
+  local line
+  local startline
+  local counter
+  local numlines
+  local actual
+
+  pushd "${BASEDIR}" >/dev/null
+  while read line; do
+    if [[ ${line} =~ ^\+\+\+ ]]; then
+      file="./"$(echo "${line}" | cut -f2- -d/)
+      continue
+    elif [[ ${line} =~ ^@@ ]]; then
+      startline=$(echo "${line}" | cut -f3 -d' ' | cut -f1 -d, | tr -d + )
+      numlines=$(echo "${line}" | cut -f3 -d' ' | cut -s -f2 -d, )
+      # if this is empty, then just this line
+      # if it is 0, then no lines were added and this part of the patch
+      # is strictly a delete
+      if [[ ${numlines} == 0 ]]; then
+        continue
+      elif [[ -z ${numlines} ]]; then
+        numlines=1
+      fi
+      counter=0
+      until [[ ${counter} -gt ${numlines} ]]; do
+          ((actual=counter+startline))
+          echo "${file}:${actual}:" >> "${outfile}"
+          ((counter=counter+1))
+      done
+    fi
+  done < <("${GIT}" diff --unified=0 --no-color)
+  popd >/dev/null
+}
+
 ## @description  Print the command to be executing to the screen. Then
 ## @description  run the command, sending stdout and stderr to the given filename
 ## @description  This will also ensure that any directories in ${BASEDIR} have
@@ -481,7 +524,7 @@ function verify_patchdir_still_exists
 ## @returns      $?
 function echo_and_redirect
 {
-  logfile=$1
+  local logfile=$1
   shift
 
   verify_patchdir_still_exists
@@ -522,7 +565,7 @@ function hadoop_usage
 
   echo "Shell binary overrides:"
   echo "--awk-cmd=<cmd>        The 'awk' command to use (default 'awk')"
-  echo "--diff-cmd=<cmd>       The 'diff' command to use (default 'diff')"
+  echo "--diff-cmd=<cmd>       The GNU-compatible 'diff' command to use (default 'diff')"
   echo "--git-cmd=<cmd>        The 'git' command to use (default 'git')"
   echo "--grep-cmd=<cmd>       The 'grep' command to use (default 'grep')"
   echo "--mvn-cmd=<cmd>        The 'mvn' command to use (default \${MAVEN_HOME}/bin/mvn, or 'mvn')"
@@ -585,6 +628,10 @@ function parse_args
       --grep-cmd=*)
         GREP=${i#*=}
       ;;
+      --help|-help|-h|help|--h|--\?|-\?|\?)
+        hadoop_usage
+        exit 0
+      ;;
       --java-home)
         JAVA_HOME=${i#*=}
       ;;
@@ -680,6 +727,8 @@ function parse_args
       cleanup_and_exit 1
     fi
   fi
+
+  GITDIFFLINES=${PATCH_DIR}/gitdifflines.txt
 }
 
 ## @description  Locate the pom.xml file for a given directory
@@ -716,12 +765,14 @@ function find_changed_files
   # get a list of all of the files that have been changed,
   # except for /dev/null (which would be present for new files).
   # Additionally, remove any a/ b/ patterns at the front
-  # of the patch filenames
+  # of the patch filenames and any revision info at the end
+  # shellcheck disable=SC2016
   CHANGED_FILES=$(${GREP} -E '^(\+\+\+|---) ' "${PATCH_DIR}/patch" \
     | ${SED} \
       -e 's,^....,,' \
       -e 's,^[ab]/,,' \
     | ${GREP} -v /dev/null \
+    | ${AWK} '{print $1}' \
     | sort -u)
 }
 
@@ -1552,7 +1603,7 @@ function check_javac
         > "${PATCH_DIR}/diffJavacWarnings.txt"
 
         add_jira_table -1 javac "The applied patch generated "\
-        "$((patchJavacWarnings-branchJavacWarnings))" \
+        "$((patchJavacWarnings-${PATCH_BRANCH}JavacWarnings))" \
         " additional warning messages."
 
         add_jira_footer javac "@@BASE@@/diffJavacWarnings.txt"
@@ -1712,6 +1763,7 @@ function check_findbugs
       "${PATCH_DIR}/patchFindbugsWarnings${module_suffix}.xml" \
       "${PATCH_DIR}/patchFindbugsWarnings${module_suffix}.xml"
 
+    #shellcheck disable=SC2016
     newFindbugsWarnings=$("${FINDBUGS_HOME}/bin/filterBugs" \
       -first "01/01/2000" "${PATCH_DIR}/patchFindbugsWarnings${module_suffix}.xml" \
       "${PATCH_DIR}/newPatchFindbugsWarnings${module_suffix}.xml" \
@@ -1887,10 +1939,12 @@ function check_unittests
       test_timeouts="${test_timeouts} ${module_test_timeouts}"
       result=1
     fi
-    #shellcheck disable=SC2026,SC2038
+
+    #shellcheck disable=SC2026,SC2038,SC2016
     module_failed_tests=$(find . -name 'TEST*.xml'\
       | xargs "${GREP}" -l -E "<failure|<error"\
       | ${AWK} -F/ '{sub("TEST-org.apache.",""); sub(".xml",""); print $NF}')
+
     if [[ -n "${module_failed_tests}" ]] ; then
       failed_tests="${failed_tests} ${module_failed_tests}"
       result=1
@@ -2054,8 +2108,6 @@ function output_to_console
     printf "%s\n" "${comment}"
     ((i=i+1))
   done
-
-
 }
 
 ## @description  Print out the finished details to the JIRA issue
@@ -2189,7 +2241,6 @@ function postcheckout
       #shellcheck disable=SC2086
       ${plugin}_postcheckout
 
-
       (( RESULT = RESULT + $? ))
       if [[ ${RESULT} != 0 ]] ; then
         output_to_console 1
@@ -2244,6 +2295,8 @@ function postapply
   local plugin
   local retval
 
+  compute_gitdiff "${GITDIFFLINES}"
+
   check_javac
   retval=$?
   if [[ ${retval} -gt 1 ]] ; then

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f07f3567/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index d2e1d4a..0a53396 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -582,6 +582,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11821. Fix findbugs warnings in hadoop-sls.
     (Brahma Reddy Battula via aajisaka)
 
+    HADOOP-11866. increase readability and reliability of checkstyle,
+    shellcheck, and whitespace reports (aw)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[25/50] [abbrv] hadoop git commit: HADOOP-11889. Make checkstyle runnable from root project (Gera Shegalov via jeagles)

Posted by zj...@apache.org.
HADOOP-11889. Make checkstyle runnable from root project (Gera Shegalov via jeagles)


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

Branch: refs/heads/YARN-2928
Commit: 45f012055e8db2ab43d4848f8ca471df63d9bfe9
Parents: 78bfacc
Author: Jonathan Eagles <je...@gmail.com>
Authored: Fri May 1 13:11:50 2015 -0500
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:57 2015 -0700

----------------------------------------------------------------------
 hadoop-build-tools/pom.xml                      |  28 +++
 .../checkstyle/checkstyle-noframes-sorted.xsl   | 194 +++++++++++++++++++
 .../main/resources/checkstyle/checkstyle.xml    | 186 ++++++++++++++++++
 .../hadoop-common/dev-support/checkstyle.xml    | 185 ------------------
 .../src/test/checkstyle-noframes-sorted.xsl     | 194 -------------------
 .../dev-support/checkstyle-noframes-sorted.xsl  | 178 -----------------
 .../hadoop-hdfs/dev-support/checkstyle.xml      | 169 ----------------
 hadoop-project-dist/pom.xml                     |  11 --
 hadoop-project/pom.xml                          |   6 -
 .../hadoop-azure/src/config/checkstyle.xml      |   1 -
 hadoop-tools/hadoop-distcp/pom.xml              |   7 -
 pom.xml                                         |  32 +++
 12 files changed, 440 insertions(+), 751 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/45f01205/hadoop-build-tools/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-build-tools/pom.xml b/hadoop-build-tools/pom.xml
new file mode 100644
index 0000000..1931072
--- /dev/null
+++ b/hadoop-build-tools/pom.xml
@@ -0,0 +1,28 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<parent>
+  <artifactId>hadoop-main</artifactId>
+  <groupId>org.apache.hadoop</groupId>
+  <version>3.0.0-SNAPSHOT</version>
+</parent>
+<modelVersion>4.0.0</modelVersion>
+
+<artifactId>hadoop-build-tools</artifactId>
+  <description>Apache Hadoop Build Tools Project</description>
+  <name>Apache Hadoop Build Tools</name>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45f01205/hadoop-build-tools/src/main/resources/checkstyle/checkstyle-noframes-sorted.xsl
----------------------------------------------------------------------
diff --git a/hadoop-build-tools/src/main/resources/checkstyle/checkstyle-noframes-sorted.xsl b/hadoop-build-tools/src/main/resources/checkstyle/checkstyle-noframes-sorted.xsl
new file mode 100644
index 0000000..b7826e3
--- /dev/null
+++ b/hadoop-build-tools/src/main/resources/checkstyle/checkstyle-noframes-sorted.xsl
@@ -0,0 +1,194 @@
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+<xsl:stylesheet	xmlns:xsl="http://www.w3.org/1999/XSL/Transform" version="1.0">
+<xsl:output method="html" indent="yes"/>
+<xsl:decimal-format decimal-separator="." grouping-separator="," />
+
+<xsl:key name="files" match="file" use="@name" />
+
+<!-- Checkstyle XML Style Sheet by Stephane Bailliez <sb...@apache.org>         -->
+<!-- Part of the Checkstyle distribution found at http://checkstyle.sourceforge.net -->
+<!-- Usage (generates checkstyle_report.html):                                      -->
+<!--    <checkstyle failonviolation="false" config="${check.config}">               -->
+<!--      <fileset dir="${src.dir}" includes="**/*.java"/>                          -->
+<!--      <formatter type="xml" toFile="${doc.dir}/checkstyle_report.xml"/>         -->
+<!--    </checkstyle>                                                               -->
+<!--    <style basedir="${doc.dir}" destdir="${doc.dir}"                            -->
+<!--            includes="checkstyle_report.xml"                                    -->
+<!--            style="${doc.dir}/checkstyle-noframes-sorted.xsl"/>                 -->
+
+<xsl:template match="checkstyle">
+	<html>
+		<head>
+		<style type="text/css">
+    .bannercell {
+      border: 0px;
+      padding: 0px;
+    }
+    body {
+      margin-left: 10;
+      margin-right: 10;
+      font:normal 80% arial,helvetica,sanserif;
+      background-color:#FFFFFF;
+      color:#000000;
+    }
+    .a td {
+      background: #efefef;
+    }
+    .b td {
+      background: #fff;
+    }
+    th, td {
+      text-align: left;
+      vertical-align: top;
+    }
+    th {
+      font-weight:bold;
+      background: #ccc;
+      color: black;
+    }
+    table, th, td {
+      font-size:100%;
+      border: none
+    }
+    table.log tr td, tr th {
+
+    }
+    h2 {
+      font-weight:bold;
+      font-size:140%;
+      margin-bottom: 5;
+    }
+    h3 {
+      font-size:100%;
+      font-weight:bold;
+      background: #525D76;
+      color: white;
+      text-decoration: none;
+      padding: 5px;
+      margin-right: 2px;
+      margin-left: 2px;
+      margin-bottom: 0;
+    }
+		</style>
+		</head>
+		<body>
+			<a name="top"></a>
+      <!-- jakarta logo -->
+      <table border="0" cellpadding="0" cellspacing="0" width="100%">
+      <tr>
+        <td class="bannercell" rowspan="2">
+          <!--a href="http://jakarta.apache.org/">
+          <img src="http://jakarta.apache.org/images/jakarta-logo.gif" alt="http://jakarta.apache.org" align="left" border="0"/>
+          </a-->
+        </td>
+    		<td class="text-align:right"><h2>CheckStyle Audit</h2></td>
+    		</tr>
+    		<tr>
+    		<td class="text-align:right">Designed for use with <a href='http://checkstyle.sourceforge.net/'>CheckStyle</a> and <a href='http://jakarta.apache.org'>Ant</a>.</td>
+    		</tr>
+      </table>
+    	<hr size="1"/>
+
+			<!-- Summary part -->
+			<xsl:apply-templates select="." mode="summary"/>
+			<hr size="1" width="100%" align="left"/>
+
+			<!-- Package List part -->
+			<xsl:apply-templates select="." mode="filelist"/>
+			<hr size="1" width="100%" align="left"/>
+
+			<!-- For each package create its part -->
+            <xsl:apply-templates select="file[@name and generate-id(.) = generate-id(key('files', @name))]" />
+
+			<hr size="1" width="100%" align="left"/>
+
+
+		</body>
+	</html>
+</xsl:template>
+
+
+
+	<xsl:template match="checkstyle" mode="filelist">
+		<h3>Files</h3>
+		<table class="log" border="0" cellpadding="5" cellspacing="2" width="100%">
+      <tr>
+        <th>Name</th>
+        <th>Errors</th>
+      </tr>
+          <xsl:for-each select="file[@name and generate-id(.) = generate-id(key('files', @name))]">
+                <xsl:sort data-type="number" order="descending" select="count(key('files', @name)/error)"/>
+				<xsl:variable name="errorCount" select="count(error)"/>
+				<tr>
+          <xsl:call-template name="alternated-row"/>
+					<td><a href="#f-{@name}"><xsl:value-of select="@name"/></a></td>
+					<td><xsl:value-of select="$errorCount"/></td>
+				</tr>
+			</xsl:for-each>
+		</table>
+	</xsl:template>
+
+
+	<xsl:template match="file">
+    <a name="f-{@name}"></a>
+    <h3>File <xsl:value-of select="@name"/></h3>
+
+    <table class="log" border="0" cellpadding="5" cellspacing="2" width="100%">
+    	<tr>
+    	  <th>Error Description</th>
+    	  <th>Line</th>
+      </tr>
+        <xsl:for-each select="key('files', @name)/error">
+          <xsl:sort data-type="number" order="ascending" select="@line"/>
+    	<tr>
+        <xsl:call-template name="alternated-row"/>
+    	  <td><xsl:value-of select="@message"/></td>
+    	  <td><xsl:value-of select="@line"/></td>
+    	</tr>
+    	</xsl:for-each>
+    </table>
+    <a href="#top">Back to top</a>
+	</xsl:template>
+
+
+	<xsl:template match="checkstyle" mode="summary">
+		<h3>Summary</h3>
+        <xsl:variable name="fileCount" select="count(file[@name and generate-id(.) = generate-id(key('files', @name))])"/>
+		<xsl:variable name="errorCount" select="count(file/error)"/>
+		<table class="log" border="0" cellpadding="5" cellspacing="2" width="100%">
+		<tr>
+			<th>Files</th>
+			<th>Errors</th>
+		</tr>
+		<tr>
+		  <xsl:call-template name="alternated-row"/>
+			<td><xsl:value-of select="$fileCount"/></td>
+			<td><xsl:value-of select="$errorCount"/></td>
+		</tr>
+		</table>
+	</xsl:template>
+
+  <xsl:template name="alternated-row">
+    <xsl:attribute name="class">
+      <xsl:if test="position() mod 2 = 1">a</xsl:if>
+      <xsl:if test="position() mod 2 = 0">b</xsl:if>
+    </xsl:attribute>
+  </xsl:template>
+</xsl:stylesheet>
+
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45f01205/hadoop-build-tools/src/main/resources/checkstyle/checkstyle.xml
----------------------------------------------------------------------
diff --git a/hadoop-build-tools/src/main/resources/checkstyle/checkstyle.xml b/hadoop-build-tools/src/main/resources/checkstyle/checkstyle.xml
new file mode 100644
index 0000000..de7319c
--- /dev/null
+++ b/hadoop-build-tools/src/main/resources/checkstyle/checkstyle.xml
@@ -0,0 +1,186 @@
+<?xml version="1.0"?>
+<!DOCTYPE module PUBLIC
+    "-//Puppy Crawl//DTD Check Configuration 1.2//EN"
+    "http://www.puppycrawl.com/dtds/configuration_1_2.dtd">
+
+<!--
+   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.
+-->
+<!--
+
+  Checkstyle configuration for Hadoop that is based on the sun_checks.xml file
+  that is bundled with Checkstyle and includes checks for:
+
+    - the Java Language Specification at
+      http://java.sun.com/docs/books/jls/second_edition/html/index.html
+
+    - the Sun Code Conventions at http://java.sun.com/docs/codeconv/
+
+    - the Javadoc guidelines at
+      http://java.sun.com/j2se/javadoc/writingdoccomments/index.html
+
+    - the JDK Api documentation http://java.sun.com/j2se/docs/api/index.html
+
+    - some best practices
+
+  Checkstyle is very configurable. Be sure to read the documentation at
+  http://checkstyle.sf.net (or in your downloaded distribution).
+
+  Most Checks are configurable, be sure to consult the documentation.
+
+  To completely disable a check, just comment it out or delete it from the file.
+
+  Finally, it is worth reading the documentation.
+
+-->
+
+<module name="Checker">
+
+    <!-- Checks that a package.html file exists for each package.     -->
+    <!-- See http://checkstyle.sf.net/config_javadoc.html#PackageHtml -->
+    <module name="JavadocPackage"/>
+
+    <!-- Checks whether files end with a new line.                        -->
+    <!-- See http://checkstyle.sf.net/config_misc.html#NewlineAtEndOfFile -->
+    <!-- module name="NewlineAtEndOfFile"/-->
+
+    <!-- Checks that property files contain the same keys.         -->
+    <!-- See http://checkstyle.sf.net/config_misc.html#Translation -->
+    <module name="Translation"/>
+
+    <module name="FileLength"/>
+    <module name="FileTabCharacter"/>
+
+    <module name="TreeWalker">
+
+        <!-- Checks for Javadoc comments.                     -->
+        <!-- See http://checkstyle.sf.net/config_javadoc.html -->
+        <module name="JavadocType">
+          <property name="scope" value="public"/>
+          <property name="allowMissingParamTags" value="true"/>
+        </module>
+        <module name="JavadocStyle"/>
+
+        <!-- Checks for Naming Conventions.                  -->
+        <!-- See http://checkstyle.sf.net/config_naming.html -->
+        <module name="ConstantName"/>
+        <module name="LocalFinalVariableName"/>
+        <module name="LocalVariableName"/>
+        <module name="MemberName"/>
+        <module name="MethodName"/>
+        <module name="PackageName"/>
+        <module name="ParameterName"/>
+        <module name="StaticVariableName"/>
+        <module name="TypeName"/>
+
+
+        <!-- Checks for Headers                                -->
+        <!-- See http://checkstyle.sf.net/config_header.html   -->
+        <!-- <module name="Header">                            -->
+            <!-- The follow property value demonstrates the ability     -->
+            <!-- to have access to ANT properties. In this case it uses -->
+            <!-- the ${basedir} property to allow Checkstyle to be run  -->
+            <!-- from any directory within a project. See property      -->
+            <!-- expansion,                                             -->
+            <!-- http://checkstyle.sf.net/config.html#properties        -->
+            <!-- <property                                              -->
+            <!--     name="headerFile"                                  -->
+            <!--     value="${basedir}/java.header"/>                   -->
+        <!-- </module> -->
+
+        <!-- Following interprets the header file as regular expressions. -->
+        <!-- <module name="RegexpHeader"/>                                -->
+
+
+        <!-- Checks for imports                              -->
+        <!-- See http://checkstyle.sf.net/config_import.html -->
+        <module name="IllegalImport"/> <!-- defaults to sun.* packages -->
+        <module name="RedundantImport"/>
+        <module name="UnusedImports"/>
+
+
+        <!-- Checks for Size Violations.                    -->
+        <!-- See http://checkstyle.sf.net/config_sizes.html -->
+        <module name="LineLength">
+          <property name="ignorePattern" value="^import"/>
+        </module>
+        <module name="MethodLength"/>
+        <module name="ParameterNumber"/>
+
+
+        <!-- Checks for whitespace                               -->
+        <!-- See http://checkstyle.sf.net/config_whitespace.html -->
+        <module name="EmptyForIteratorPad"/>
+        <module name="MethodParamPad"/>
+        <module name="NoWhitespaceAfter"/>
+        <module name="NoWhitespaceBefore"/>
+        <module name="ParenPad"/>
+        <module name="TypecastParenPad"/>
+        <module name="WhitespaceAfter">
+	    	<property name="tokens" value="COMMA, SEMI"/>
+		</module>
+
+
+        <!-- Modifier Checks                                    -->
+        <!-- See http://checkstyle.sf.net/config_modifiers.html -->
+        <module name="ModifierOrder"/>
+        <module name="RedundantModifier"/>
+
+
+        <!-- Checks for blocks. You know, those {}'s         -->
+        <!-- See http://checkstyle.sf.net/config_blocks.html -->
+        <module name="AvoidNestedBlocks"/>
+        <module name="EmptyBlock"/>
+        <module name="LeftCurly"/>
+        <module name="NeedBraces"/>
+        <module name="RightCurly"/>
+
+
+        <!-- Checks for common coding problems               -->
+        <!-- See http://checkstyle.sf.net/config_coding.html -->
+        <!-- module name="AvoidInlineConditionals"/-->
+        <module name="EmptyStatement"/>
+        <module name="EqualsHashCode"/>
+        <module name="HiddenField">
+          <property name="ignoreConstructorParameter" value="true"/>
+        </module>
+        <module name="IllegalInstantiation"/>
+        <module name="InnerAssignment"/>
+        <module name="MissingSwitchDefault"/>
+        <module name="SimplifyBooleanExpression"/>
+        <module name="SimplifyBooleanReturn"/>
+
+        <!-- Checks for class design                         -->
+        <!-- See http://checkstyle.sf.net/config_design.html -->
+        <module name="FinalClass"/>
+        <module name="HideUtilityClassConstructor"/>
+        <module name="InterfaceIsType"/>
+        <module name="VisibilityModifier"/>
+
+
+        <!-- Miscellaneous other checks.                   -->
+        <!-- See http://checkstyle.sf.net/config_misc.html -->
+        <module name="ArrayTypeStyle"/>
+        <module name="Indentation">
+            <property name="basicOffset" value="2" />
+            <property name="caseIndent" value="0" />
+        </module> 
+        <module name="TodoComment"/>
+        <module name="UpperEll"/>
+
+    </module>
+
+</module>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45f01205/hadoop-common-project/hadoop-common/dev-support/checkstyle.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/dev-support/checkstyle.xml b/hadoop-common-project/hadoop-common/dev-support/checkstyle.xml
deleted file mode 100644
index 4caa03f..0000000
--- a/hadoop-common-project/hadoop-common/dev-support/checkstyle.xml
+++ /dev/null
@@ -1,185 +0,0 @@
-<?xml version="1.0"?>
-<!DOCTYPE module PUBLIC
-    "-//Puppy Crawl//DTD Check Configuration 1.2//EN"
-    "http://www.puppycrawl.com/dtds/configuration_1_2.dtd">
-
-<!--
-   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.
--->
-<!--
-
-  Checkstyle configuration for Hadoop that is based on the sun_checks.xml file
-  that is bundled with Checkstyle and includes checks for:
-
-    - the Java Language Specification at
-      http://java.sun.com/docs/books/jls/second_edition/html/index.html
-
-    - the Sun Code Conventions at http://java.sun.com/docs/codeconv/
-
-    - the Javadoc guidelines at
-      http://java.sun.com/j2se/javadoc/writingdoccomments/index.html
-
-    - the JDK Api documentation http://java.sun.com/j2se/docs/api/index.html
-
-    - some best practices
-
-  Checkstyle is very configurable. Be sure to read the documentation at
-  http://checkstyle.sf.net (or in your downloaded distribution).
-
-  Most Checks are configurable, be sure to consult the documentation.
-
-  To completely disable a check, just comment it out or delete it from the file.
-
-  Finally, it is worth reading the documentation.
-
--->
-
-<module name="Checker">
-
-    <!-- Checks that a package.html file exists for each package.     -->
-    <!-- See http://checkstyle.sf.net/config_javadoc.html#PackageHtml -->
-    <module name="JavadocPackage"/>
-
-    <!-- Checks whether files end with a new line.                        -->
-    <!-- See http://checkstyle.sf.net/config_misc.html#NewlineAtEndOfFile -->
-    <!-- module name="NewlineAtEndOfFile"/-->
-
-    <!-- Checks that property files contain the same keys.         -->
-    <!-- See http://checkstyle.sf.net/config_misc.html#Translation -->
-    <module name="Translation"/>
-
-    <module name="FileLength"/>
-    <module name="FileTabCharacter"/>
-
-    <module name="TreeWalker">
-
-        <!-- Checks for Javadoc comments.                     -->
-        <!-- See http://checkstyle.sf.net/config_javadoc.html -->
-        <module name="JavadocType">
-          <property name="scope" value="public"/>
-          <property name="allowMissingParamTags" value="true"/>
-        </module>
-        <module name="JavadocStyle"/>
-
-        <!-- Checks for Naming Conventions.                  -->
-        <!-- See http://checkstyle.sf.net/config_naming.html -->
-        <module name="ConstantName"/>
-        <module name="LocalFinalVariableName"/>
-        <module name="LocalVariableName"/>
-        <module name="MemberName"/>
-        <module name="MethodName"/>
-        <module name="PackageName"/>
-        <module name="ParameterName"/>
-        <module name="StaticVariableName"/>
-        <module name="TypeName"/>
-
-
-        <!-- Checks for Headers                                -->
-        <!-- See http://checkstyle.sf.net/config_header.html   -->
-        <!-- <module name="Header">                            -->
-            <!-- The follow property value demonstrates the ability     -->
-            <!-- to have access to ANT properties. In this case it uses -->
-            <!-- the ${basedir} property to allow Checkstyle to be run  -->
-            <!-- from any directory within a project. See property      -->
-            <!-- expansion,                                             -->
-            <!-- http://checkstyle.sf.net/config.html#properties        -->
-            <!-- <property                                              -->
-            <!--     name="headerFile"                                  -->
-            <!--     value="${basedir}/java.header"/>                   -->
-        <!-- </module> -->
-
-        <!-- Following interprets the header file as regular expressions. -->
-        <!-- <module name="RegexpHeader"/>                                -->
-
-
-        <!-- Checks for imports                              -->
-        <!-- See http://checkstyle.sf.net/config_import.html -->
-        <module name="IllegalImport"/> <!-- defaults to sun.* packages -->
-        <module name="RedundantImport"/>
-        <module name="UnusedImports"/>
-
-
-        <!-- Checks for Size Violations.                    -->
-        <!-- See http://checkstyle.sf.net/config_sizes.html -->
-        <module name="LineLength"/>
-        <module name="MethodLength"/>
-        <module name="ParameterNumber"/>
-
-
-        <!-- Checks for whitespace                               -->
-        <!-- See http://checkstyle.sf.net/config_whitespace.html -->
-        <module name="EmptyForIteratorPad"/>
-        <module name="MethodParamPad"/>
-        <module name="NoWhitespaceAfter"/>
-        <module name="NoWhitespaceBefore"/>
-        <module name="ParenPad"/>
-        <module name="TypecastParenPad"/>
-        <module name="WhitespaceAfter">
-	    	<property name="tokens" value="COMMA, SEMI"/>
-		</module>
-
-
-        <!-- Modifier Checks                                    -->
-        <!-- See http://checkstyle.sf.net/config_modifiers.html -->
-        <module name="ModifierOrder"/>
-        <module name="RedundantModifier"/>
-
-
-        <!-- Checks for blocks. You know, those {}'s         -->
-        <!-- See http://checkstyle.sf.net/config_blocks.html -->
-        <module name="AvoidNestedBlocks"/>
-        <module name="EmptyBlock"/>
-        <module name="LeftCurly"/>
-        <module name="NeedBraces"/>
-        <module name="RightCurly"/>
-
-
-        <!-- Checks for common coding problems               -->
-        <!-- See http://checkstyle.sf.net/config_coding.html -->
-        <!-- module name="AvoidInlineConditionals"/-->
-        <module name="EmptyStatement"/>
-        <module name="EqualsHashCode"/>
-        <module name="HiddenField">
-          <property name="ignoreConstructorParameter" value="true"/>
-        </module>
-        <module name="IllegalInstantiation"/>
-        <module name="InnerAssignment"/>
-        <module name="MissingSwitchDefault"/>
-        <module name="RedundantThrows"/>
-        <module name="SimplifyBooleanExpression"/>
-        <module name="SimplifyBooleanReturn"/>
-
-        <!-- Checks for class design                         -->
-        <!-- See http://checkstyle.sf.net/config_design.html -->
-        <module name="FinalClass"/>
-        <module name="HideUtilityClassConstructor"/>
-        <module name="InterfaceIsType"/>
-        <module name="VisibilityModifier"/>
-
-
-        <!-- Miscellaneous other checks.                   -->
-        <!-- See http://checkstyle.sf.net/config_misc.html -->
-        <module name="ArrayTypeStyle"/>
-        <module name="Indentation">
-            <property name="basicOffset" value="2" />
-            <property name="caseIndent" value="0" />
-        </module> 
-        <module name="TodoComment"/>
-        <module name="UpperEll"/>
-
-    </module>
-
-</module>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45f01205/hadoop-common-project/hadoop-common/src/test/checkstyle-noframes-sorted.xsl
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/checkstyle-noframes-sorted.xsl b/hadoop-common-project/hadoop-common/src/test/checkstyle-noframes-sorted.xsl
deleted file mode 100644
index b7826e3..0000000
--- a/hadoop-common-project/hadoop-common/src/test/checkstyle-noframes-sorted.xsl
+++ /dev/null
@@ -1,194 +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.
--->
-<xsl:stylesheet	xmlns:xsl="http://www.w3.org/1999/XSL/Transform" version="1.0">
-<xsl:output method="html" indent="yes"/>
-<xsl:decimal-format decimal-separator="." grouping-separator="," />
-
-<xsl:key name="files" match="file" use="@name" />
-
-<!-- Checkstyle XML Style Sheet by Stephane Bailliez <sb...@apache.org>         -->
-<!-- Part of the Checkstyle distribution found at http://checkstyle.sourceforge.net -->
-<!-- Usage (generates checkstyle_report.html):                                      -->
-<!--    <checkstyle failonviolation="false" config="${check.config}">               -->
-<!--      <fileset dir="${src.dir}" includes="**/*.java"/>                          -->
-<!--      <formatter type="xml" toFile="${doc.dir}/checkstyle_report.xml"/>         -->
-<!--    </checkstyle>                                                               -->
-<!--    <style basedir="${doc.dir}" destdir="${doc.dir}"                            -->
-<!--            includes="checkstyle_report.xml"                                    -->
-<!--            style="${doc.dir}/checkstyle-noframes-sorted.xsl"/>                 -->
-
-<xsl:template match="checkstyle">
-	<html>
-		<head>
-		<style type="text/css">
-    .bannercell {
-      border: 0px;
-      padding: 0px;
-    }
-    body {
-      margin-left: 10;
-      margin-right: 10;
-      font:normal 80% arial,helvetica,sanserif;
-      background-color:#FFFFFF;
-      color:#000000;
-    }
-    .a td {
-      background: #efefef;
-    }
-    .b td {
-      background: #fff;
-    }
-    th, td {
-      text-align: left;
-      vertical-align: top;
-    }
-    th {
-      font-weight:bold;
-      background: #ccc;
-      color: black;
-    }
-    table, th, td {
-      font-size:100%;
-      border: none
-    }
-    table.log tr td, tr th {
-
-    }
-    h2 {
-      font-weight:bold;
-      font-size:140%;
-      margin-bottom: 5;
-    }
-    h3 {
-      font-size:100%;
-      font-weight:bold;
-      background: #525D76;
-      color: white;
-      text-decoration: none;
-      padding: 5px;
-      margin-right: 2px;
-      margin-left: 2px;
-      margin-bottom: 0;
-    }
-		</style>
-		</head>
-		<body>
-			<a name="top"></a>
-      <!-- jakarta logo -->
-      <table border="0" cellpadding="0" cellspacing="0" width="100%">
-      <tr>
-        <td class="bannercell" rowspan="2">
-          <!--a href="http://jakarta.apache.org/">
-          <img src="http://jakarta.apache.org/images/jakarta-logo.gif" alt="http://jakarta.apache.org" align="left" border="0"/>
-          </a-->
-        </td>
-    		<td class="text-align:right"><h2>CheckStyle Audit</h2></td>
-    		</tr>
-    		<tr>
-    		<td class="text-align:right">Designed for use with <a href='http://checkstyle.sourceforge.net/'>CheckStyle</a> and <a href='http://jakarta.apache.org'>Ant</a>.</td>
-    		</tr>
-      </table>
-    	<hr size="1"/>
-
-			<!-- Summary part -->
-			<xsl:apply-templates select="." mode="summary"/>
-			<hr size="1" width="100%" align="left"/>
-
-			<!-- Package List part -->
-			<xsl:apply-templates select="." mode="filelist"/>
-			<hr size="1" width="100%" align="left"/>
-
-			<!-- For each package create its part -->
-            <xsl:apply-templates select="file[@name and generate-id(.) = generate-id(key('files', @name))]" />
-
-			<hr size="1" width="100%" align="left"/>
-
-
-		</body>
-	</html>
-</xsl:template>
-
-
-
-	<xsl:template match="checkstyle" mode="filelist">
-		<h3>Files</h3>
-		<table class="log" border="0" cellpadding="5" cellspacing="2" width="100%">
-      <tr>
-        <th>Name</th>
-        <th>Errors</th>
-      </tr>
-          <xsl:for-each select="file[@name and generate-id(.) = generate-id(key('files', @name))]">
-                <xsl:sort data-type="number" order="descending" select="count(key('files', @name)/error)"/>
-				<xsl:variable name="errorCount" select="count(error)"/>
-				<tr>
-          <xsl:call-template name="alternated-row"/>
-					<td><a href="#f-{@name}"><xsl:value-of select="@name"/></a></td>
-					<td><xsl:value-of select="$errorCount"/></td>
-				</tr>
-			</xsl:for-each>
-		</table>
-	</xsl:template>
-
-
-	<xsl:template match="file">
-    <a name="f-{@name}"></a>
-    <h3>File <xsl:value-of select="@name"/></h3>
-
-    <table class="log" border="0" cellpadding="5" cellspacing="2" width="100%">
-    	<tr>
-    	  <th>Error Description</th>
-    	  <th>Line</th>
-      </tr>
-        <xsl:for-each select="key('files', @name)/error">
-          <xsl:sort data-type="number" order="ascending" select="@line"/>
-    	<tr>
-        <xsl:call-template name="alternated-row"/>
-    	  <td><xsl:value-of select="@message"/></td>
-    	  <td><xsl:value-of select="@line"/></td>
-    	</tr>
-    	</xsl:for-each>
-    </table>
-    <a href="#top">Back to top</a>
-	</xsl:template>
-
-
-	<xsl:template match="checkstyle" mode="summary">
-		<h3>Summary</h3>
-        <xsl:variable name="fileCount" select="count(file[@name and generate-id(.) = generate-id(key('files', @name))])"/>
-		<xsl:variable name="errorCount" select="count(file/error)"/>
-		<table class="log" border="0" cellpadding="5" cellspacing="2" width="100%">
-		<tr>
-			<th>Files</th>
-			<th>Errors</th>
-		</tr>
-		<tr>
-		  <xsl:call-template name="alternated-row"/>
-			<td><xsl:value-of select="$fileCount"/></td>
-			<td><xsl:value-of select="$errorCount"/></td>
-		</tr>
-		</table>
-	</xsl:template>
-
-  <xsl:template name="alternated-row">
-    <xsl:attribute name="class">
-      <xsl:if test="position() mod 2 = 1">a</xsl:if>
-      <xsl:if test="position() mod 2 = 0">b</xsl:if>
-    </xsl:attribute>
-  </xsl:template>
-</xsl:stylesheet>
-
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45f01205/hadoop-hdfs-project/hadoop-hdfs/dev-support/checkstyle-noframes-sorted.xsl
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/dev-support/checkstyle-noframes-sorted.xsl b/hadoop-hdfs-project/hadoop-hdfs/dev-support/checkstyle-noframes-sorted.xsl
deleted file mode 100644
index 5f9e93b..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/dev-support/checkstyle-noframes-sorted.xsl
+++ /dev/null
@@ -1,178 +0,0 @@
-<xsl:stylesheet	xmlns:xsl="http://www.w3.org/1999/XSL/Transform" version="1.0">
-<xsl:output method="html" indent="yes"/>
-<xsl:decimal-format decimal-separator="." grouping-separator="," />
-
-<xsl:key name="files" match="file" use="@name" />
-
-<!-- Checkstyle XML Style Sheet by Stephane Bailliez <sb...@apache.org>         -->
-<!-- Part of the Checkstyle distribution found at http://checkstyle.sourceforge.net -->
-<!-- Usage (generates checkstyle_report.html):                                      -->
-<!--    <checkstyle failonviolation="false" config="${check.config}">               -->
-<!--      <fileset dir="${src.dir}" includes="**/*.java"/>                          -->
-<!--      <formatter type="xml" toFile="${doc.dir}/checkstyle_report.xml"/>         -->
-<!--    </checkstyle>                                                               -->
-<!--    <style basedir="${doc.dir}" destdir="${doc.dir}"                            -->
-<!--            includes="checkstyle_report.xml"                                    -->
-<!--            style="${doc.dir}/checkstyle-noframes-sorted.xsl"/>                 -->
-
-<xsl:template match="checkstyle">
-	<html>
-		<head>
-		<style type="text/css">
-    .bannercell {
-      border: 0px;
-      padding: 0px;
-    }
-    body {
-      margin-left: 10;
-      margin-right: 10;
-      font:normal 80% arial,helvetica,sanserif;
-      background-color:#FFFFFF;
-      color:#000000;
-    }
-    .a td {
-      background: #efefef;
-    }
-    .b td {
-      background: #fff;
-    }
-    th, td {
-      text-align: left;
-      vertical-align: top;
-    }
-    th {
-      font-weight:bold;
-      background: #ccc;
-      color: black;
-    }
-    table, th, td {
-      font-size:100%;
-      border: none
-    }
-    table.log tr td, tr th {
-
-    }
-    h2 {
-      font-weight:bold;
-      font-size:140%;
-      margin-bottom: 5;
-    }
-    h3 {
-      font-size:100%;
-      font-weight:bold;
-      background: #525D76;
-      color: white;
-      text-decoration: none;
-      padding: 5px;
-      margin-right: 2px;
-      margin-left: 2px;
-      margin-bottom: 0;
-    }
-		</style>
-		</head>
-		<body>
-			<a name="top"></a>
-      <!-- jakarta logo -->
-      <table border="0" cellpadding="0" cellspacing="0" width="100%">
-      <tr>
-        <td class="bannercell" rowspan="2">
-          <!--a href="http://jakarta.apache.org/">
-          <img src="http://jakarta.apache.org/images/jakarta-logo.gif" alt="http://jakarta.apache.org" align="left" border="0"/>
-          </a-->
-        </td>
-    		<td class="text-align:right"><h2>CheckStyle Audit</h2></td>
-    		</tr>
-    		<tr>
-    		<td class="text-align:right">Designed for use with <a href='http://checkstyle.sourceforge.net/'>CheckStyle</a> and <a href='http://jakarta.apache.org'>Ant</a>.</td>
-    		</tr>
-      </table>
-    	<hr size="1"/>
-
-			<!-- Summary part -->
-			<xsl:apply-templates select="." mode="summary"/>
-			<hr size="1" width="100%" align="left"/>
-
-			<!-- Package List part -->
-			<xsl:apply-templates select="." mode="filelist"/>
-			<hr size="1" width="100%" align="left"/>
-
-			<!-- For each package create its part -->
-            <xsl:apply-templates select="file[@name and generate-id(.) = generate-id(key('files', @name))]" />
-
-			<hr size="1" width="100%" align="left"/>
-
-
-		</body>
-	</html>
-</xsl:template>
-
-
-
-	<xsl:template match="checkstyle" mode="filelist">
-		<h3>Files</h3>
-		<table class="log" border="0" cellpadding="5" cellspacing="2" width="100%">
-      <tr>
-        <th>Name</th>
-        <th>Errors</th>
-      </tr>
-          <xsl:for-each select="file[@name and generate-id(.) = generate-id(key('files', @name))]">
-                <xsl:sort data-type="number" order="descending" select="count(key('files', @name)/error)"/>
-				<xsl:variable name="errorCount" select="count(error)"/>
-				<tr>
-          <xsl:call-template name="alternated-row"/>
-					<td><a href="#f-{@name}"><xsl:value-of select="@name"/></a></td>
-					<td><xsl:value-of select="$errorCount"/></td>
-				</tr>
-			</xsl:for-each>
-		</table>
-	</xsl:template>
-
-
-	<xsl:template match="file">
-    <a name="f-{@name}"></a>
-    <h3>File <xsl:value-of select="@name"/></h3>
-
-    <table class="log" border="0" cellpadding="5" cellspacing="2" width="100%">
-    	<tr>
-    	  <th>Error Description</th>
-    	  <th>Line</th>
-      </tr>
-        <xsl:for-each select="key('files', @name)/error">
-          <xsl:sort data-type="number" order="ascending" select="@line"/>
-    	<tr>
-        <xsl:call-template name="alternated-row"/>
-    	  <td><xsl:value-of select="@message"/></td>
-    	  <td><xsl:value-of select="@line"/></td>
-    	</tr>
-    	</xsl:for-each>
-    </table>
-    <a href="#top">Back to top</a>
-	</xsl:template>
-
-
-	<xsl:template match="checkstyle" mode="summary">
-		<h3>Summary</h3>
-        <xsl:variable name="fileCount" select="count(file[@name and generate-id(.) = generate-id(key('files', @name))])"/>
-		<xsl:variable name="errorCount" select="count(file/error)"/>
-		<table class="log" border="0" cellpadding="5" cellspacing="2" width="100%">
-		<tr>
-			<th>Files</th>
-			<th>Errors</th>
-		</tr>
-		<tr>
-		  <xsl:call-template name="alternated-row"/>
-			<td><xsl:value-of select="$fileCount"/></td>
-			<td><xsl:value-of select="$errorCount"/></td>
-		</tr>
-		</table>
-	</xsl:template>
-
-  <xsl:template name="alternated-row">
-    <xsl:attribute name="class">
-      <xsl:if test="position() mod 2 = 1">a</xsl:if>
-      <xsl:if test="position() mod 2 = 0">b</xsl:if>
-    </xsl:attribute>
-  </xsl:template>
-</xsl:stylesheet>
-
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45f01205/hadoop-hdfs-project/hadoop-hdfs/dev-support/checkstyle.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/dev-support/checkstyle.xml b/hadoop-hdfs-project/hadoop-hdfs/dev-support/checkstyle.xml
deleted file mode 100644
index eda4a47..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/dev-support/checkstyle.xml
+++ /dev/null
@@ -1,169 +0,0 @@
-<?xml version="1.0"?>
-<!DOCTYPE module PUBLIC
-    "-//Puppy Crawl//DTD Check Configuration 1.2//EN"
-    "http://www.puppycrawl.com/dtds/configuration_1_2.dtd">
-
-<!--
-
-  Checkstyle configuration for Hadoop that is based on the sun_checks.xml file
-  that is bundled with Checkstyle and includes checks for:
-
-    - the Java Language Specification at
-      http://java.sun.com/docs/books/jls/second_edition/html/index.html
-
-    - the Sun Code Conventions at http://java.sun.com/docs/codeconv/
-
-    - the Javadoc guidelines at
-      http://java.sun.com/j2se/javadoc/writingdoccomments/index.html
-
-    - the JDK Api documentation http://java.sun.com/j2se/docs/api/index.html
-
-    - some best practices
-
-  Checkstyle is very configurable. Be sure to read the documentation at
-  http://checkstyle.sf.net (or in your downloaded distribution).
-
-  Most Checks are configurable, be sure to consult the documentation.
-
-  To completely disable a check, just comment it out or delete it from the file.
-
-  Finally, it is worth reading the documentation.
-
--->
-
-<module name="Checker">
-
-    <!-- Checks that a package.html file exists for each package.     -->
-    <!-- See http://checkstyle.sf.net/config_javadoc.html#PackageHtml -->
-    <module name="JavadocPackage"/>
-
-    <!-- Checks whether files end with a new line.                        -->
-    <!-- See http://checkstyle.sf.net/config_misc.html#NewlineAtEndOfFile -->
-    <!-- module name="NewlineAtEndOfFile"/-->
-
-    <!-- Checks that property files contain the same keys.         -->
-    <!-- See http://checkstyle.sf.net/config_misc.html#Translation -->
-    <module name="Translation"/>
-
-    <module name="FileLength"/>
-    <module name="FileTabCharacter"/>
-
-    <module name="TreeWalker">
-
-        <!-- Checks for Javadoc comments.                     -->
-        <!-- See http://checkstyle.sf.net/config_javadoc.html -->
-        <module name="JavadocType">
-          <property name="scope" value="public"/>
-          <property name="allowMissingParamTags" value="true"/>
-        </module>
-        <module name="JavadocStyle"/>
-
-        <!-- Checks for Naming Conventions.                  -->
-        <!-- See http://checkstyle.sf.net/config_naming.html -->
-        <module name="ConstantName"/>
-        <module name="LocalFinalVariableName"/>
-        <module name="LocalVariableName"/>
-        <module name="MemberName"/>
-        <module name="MethodName"/>
-        <module name="PackageName"/>
-        <module name="ParameterName"/>
-        <module name="StaticVariableName"/>
-        <module name="TypeName"/>
-
-
-        <!-- Checks for Headers                                -->
-        <!-- See http://checkstyle.sf.net/config_header.html   -->
-        <!-- <module name="Header">                            -->
-            <!-- The follow property value demonstrates the ability     -->
-            <!-- to have access to ANT properties. In this case it uses -->
-            <!-- the ${basedir} property to allow Checkstyle to be run  -->
-            <!-- from any directory within a project. See property      -->
-            <!-- expansion,                                             -->
-            <!-- http://checkstyle.sf.net/config.html#properties        -->
-            <!-- <property                                              -->
-            <!--     name="headerFile"                                  -->
-            <!--     value="${basedir}/java.header"/>                   -->
-        <!-- </module> -->
-
-        <!-- Following interprets the header file as regular expressions. -->
-        <!-- <module name="RegexpHeader"/>                                -->
-
-
-        <!-- Checks for imports                              -->
-        <!-- See http://checkstyle.sf.net/config_import.html -->
-        <module name="IllegalImport"/> <!-- defaults to sun.* packages -->
-        <module name="RedundantImport"/>
-        <module name="UnusedImports"/>
-
-
-        <!-- Checks for Size Violations.                    -->
-        <!-- See http://checkstyle.sf.net/config_sizes.html -->
-        <module name="LineLength"/>
-        <module name="MethodLength"/>
-        <module name="ParameterNumber"/>
-
-
-        <!-- Checks for whitespace                               -->
-        <!-- See http://checkstyle.sf.net/config_whitespace.html -->
-        <module name="EmptyForIteratorPad"/>
-        <module name="MethodParamPad"/>
-        <module name="NoWhitespaceAfter"/>
-        <module name="NoWhitespaceBefore"/>
-        <module name="ParenPad"/>
-        <module name="TypecastParenPad"/>
-        <module name="WhitespaceAfter">
-	    	<property name="tokens" value="COMMA, SEMI"/>
-		</module>
-
-
-        <!-- Modifier Checks                                    -->
-        <!-- See http://checkstyle.sf.net/config_modifiers.html -->
-        <module name="ModifierOrder"/>
-        <module name="RedundantModifier"/>
-
-
-        <!-- Checks for blocks. You know, those {}'s         -->
-        <!-- See http://checkstyle.sf.net/config_blocks.html -->
-        <module name="AvoidNestedBlocks"/>
-        <module name="EmptyBlock"/>
-        <module name="LeftCurly"/>
-        <module name="NeedBraces"/>
-        <module name="RightCurly"/>
-
-
-        <!-- Checks for common coding problems               -->
-        <!-- See http://checkstyle.sf.net/config_coding.html -->
-        <!-- module name="AvoidInlineConditionals"/-->
-        <module name="EmptyStatement"/>
-        <module name="EqualsHashCode"/>
-        <module name="HiddenField">
-          <property name="ignoreConstructorParameter" value="true"/>
-        </module>
-        <module name="IllegalInstantiation"/>
-        <module name="InnerAssignment"/>
-        <module name="MissingSwitchDefault"/>
-        <module name="RedundantThrows"/>
-        <module name="SimplifyBooleanExpression"/>
-        <module name="SimplifyBooleanReturn"/>
-
-        <!-- Checks for class design                         -->
-        <!-- See http://checkstyle.sf.net/config_design.html -->
-        <module name="FinalClass"/>
-        <module name="HideUtilityClassConstructor"/>
-        <module name="InterfaceIsType"/>
-        <module name="VisibilityModifier"/>
-
-
-        <!-- Miscellaneous other checks.                   -->
-        <!-- See http://checkstyle.sf.net/config_misc.html -->
-        <module name="ArrayTypeStyle"/>
-        <module name="Indentation">
-            <property name="basicOffset" value="2" />
-            <property name="caseIndent" value="0" />
-        </module> 
-        <module name="TodoComment"/>
-        <module name="UpperEll"/>
-
-    </module>
-
-</module>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45f01205/hadoop-project-dist/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project-dist/pom.xml b/hadoop-project-dist/pom.xml
index d206593..c26aa9a 100644
--- a/hadoop-project-dist/pom.xml
+++ b/hadoop-project-dist/pom.xml
@@ -108,17 +108,6 @@
       </plugin>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-checkstyle-plugin</artifactId>
-        <configuration>
-          <configLocation>file://${basedir}/dev-support/checkstyle.xml</configLocation>
-          <failOnViolation>false</failOnViolation>
-          <format>xml</format>
-          <format>html</format>
-          <outputFile>${project.build.directory}/test/checkstyle-errors.xml</outputFile>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-javadoc-plugin</artifactId>
         <configuration>
           <linksource>true</linksource>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45f01205/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index d205749..8160438 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -102,7 +102,6 @@
     <maven-jar-plugin.version>2.5</maven-jar-plugin.version>
     <maven-war-plugin.version>2.4</maven-war-plugin.version>
     <maven-source-plugin.version>2.3</maven-source-plugin.version>
-    <maven-checkstyle-plugin.version>2.12.1</maven-checkstyle-plugin.version>
     <maven-project-info-reports-plugin.version>2.7</maven-project-info-reports-plugin.version>
     <maven-pdf-plugin.version>1.2</maven-pdf-plugin.version>
     <build-helper-maven-plugin.version>1.9</build-helper-maven-plugin.version>
@@ -1043,11 +1042,6 @@
           <version>${findbugs.version}</version>
         </plugin>
         <plugin>
-          <groupId>org.apache.maven.plugins</groupId>
-          <artifactId>maven-checkstyle-plugin</artifactId>
-          <version>${maven-checkstyle-plugin.version}</version>
-        </plugin>
-        <plugin>
           <groupId>org.codehaus.mojo</groupId>
           <artifactId>native-maven-plugin</artifactId>
           <version>${native-maven-plugin.version}</version>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45f01205/hadoop-tools/hadoop-azure/src/config/checkstyle.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/config/checkstyle.xml b/hadoop-tools/hadoop-azure/src/config/checkstyle.xml
index f68f6c8..acf6f7f 100644
--- a/hadoop-tools/hadoop-azure/src/config/checkstyle.xml
+++ b/hadoop-tools/hadoop-azure/src/config/checkstyle.xml
@@ -157,7 +157,6 @@
             <property name="ignoreNumbers" value="-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 10, 255, 1000, 1024"/>
         </module>
         <module name="MissingSwitchDefault"/>
-        <module name="RedundantThrows"/>
         <module name="SimplifyBooleanExpression"/>
         <module name="SimplifyBooleanReturn"/>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45f01205/hadoop-tools/hadoop-distcp/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/pom.xml b/hadoop-tools/hadoop-distcp/pom.xml
index 2dace6a..374155d 100644
--- a/hadoop-tools/hadoop-distcp/pom.xml
+++ b/hadoop-tools/hadoop-distcp/pom.xml
@@ -162,13 +162,6 @@
       </plugin>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-checkstyle-plugin</artifactId>
-        <configuration>
-          <enableRulesSummary>true</enableRulesSummary>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-jar-plugin</artifactId>
         <configuration>
           <archive>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45f01205/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 3bad969..1ea52eb 100644
--- a/pom.xml
+++ b/pom.xml
@@ -103,6 +103,8 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
     <maven-clover2-plugin.version>3.3.0</maven-clover2-plugin.version>
     <maven-bundle-plugin.version>2.5.0</maven-bundle-plugin.version>
     <lifecycle-mapping.version>1.0.0</lifecycle-mapping.version>
+    <maven-checkstyle-plugin.version>2.15</maven-checkstyle-plugin.version>
+    <checkstyle.version>6.6</checkstyle.version>
   </properties>
 
   <modules>
@@ -118,6 +120,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
     <module>hadoop-dist</module>
     <module>hadoop-client</module>
     <module>hadoop-minicluster</module>
+    <module>hadoop-build-tools</module>
   </modules>
 
   <build>
@@ -292,6 +295,30 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
           <artifactId>maven-bundle-plugin</artifactId>
           <version>${maven-bundle-plugin.version}</version>
         </plugin>
+        <plugin>
+          <groupId>org.apache.maven.plugins</groupId>
+          <artifactId>maven-checkstyle-plugin</artifactId>
+          <version>${maven-checkstyle-plugin.version}</version>
+          <dependencies>
+            <dependency>
+              <groupId>org.apache.hadoop</groupId>
+              <artifactId>hadoop-build-tools</artifactId>
+              <version>${project.version}</version>
+            </dependency>
+            <dependency>
+              <groupId>com.puppycrawl.tools</groupId>
+              <artifactId>checkstyle</artifactId>
+              <version>${checkstyle.version}</version>
+            </dependency>
+          </dependencies>
+          <configuration>
+            <configLocation>checkstyle/checkstyle.xml</configLocation>
+            <failOnViolation>false</failOnViolation>
+            <format>xml</format>
+            <format>html</format>
+            <outputFile>${project.build.directory}/test/checkstyle-errors.xml</outputFile>
+          </configuration>
+        </plugin>
       </plugins>
     </pluginManagement>
 
@@ -357,6 +384,11 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
         <inherited>true</inherited>
         <extensions>true</extensions>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-checkstyle-plugin</artifactId>
+        <version>${maven-checkstyle-plugin.version}</version>
+      </plugin>
     </plugins>
   </build>
   


[43/50] [abbrv] hadoop git commit: HDFS-8309. Skip unit test using DataNodeTestUtils#injectDataDirFailure() on Windows. (xyao)

Posted by zj...@apache.org.
HDFS-8309. Skip unit test using DataNodeTestUtils#injectDataDirFailure() on Windows. (xyao)


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

Branch: refs/heads/YARN-2928
Commit: 195d746ae0ef3cecdb45b7aad864de0b26c96766
Parents: 4b999c7
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Sat May 2 22:15:24 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:59:00 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt              |  5 ++++-
 .../hdfs/server/datanode/TestDataNodeHotSwapVolumes.java |  5 +++++
 .../hdfs/server/datanode/TestDataNodeVolumeFailure.java  | 11 +++++++----
 .../datanode/TestDataNodeVolumeFailureReporting.java     |  5 ++---
 4 files changed, 18 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/195d746a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 71873a4..e525800 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -605,7 +605,10 @@ Release 2.8.0 - UNRELEASED
     configured zero. (Surendra Singh Lilhore via Arpit Agarwal)
 
     HDFS-8229. LAZY_PERSIST file gets deleted after NameNode restart.
-    (Surendra Singh Lilhore via Arpit Agarwal) 
+    (Surendra Singh Lilhore via Arpit Agarwal)
+
+    HDFS-8309. Skip unit test using DataNodeTestUtils#injectDataDirFailure() on Windows.
+    (xyao)
 
 Release 2.7.1 - UNRELEASED
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/195d746a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
index 668084b..315529c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
@@ -78,6 +78,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.doAnswer;
@@ -705,6 +706,10 @@ public class TestDataNodeHotSwapVolumes {
   public void testDirectlyReloadAfterCheckDiskError()
       throws IOException, TimeoutException, InterruptedException,
       ReconfigurationException {
+    // The test uses DataNodeTestUtils#injectDataDirFailure() to simulate
+    // volume failures which is currently not supported on Windows.
+    assumeTrue(!Path.WINDOWS);
+
     startDFSCluster(1, 2);
     createFile(new Path("/test"), 32, (short)2);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/195d746a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
index 0a90947..0d158c9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
@@ -207,8 +207,12 @@ public class TestDataNodeVolumeFailure {
    * after failure.
    */
   @Test(timeout=150000)
-  public void testFailedVolumeBeingRemovedFromDataNode()
+    public void testFailedVolumeBeingRemovedFromDataNode()
       throws InterruptedException, IOException, TimeoutException {
+    // The test uses DataNodeTestUtils#injectDataDirFailure() to simulate
+    // volume failures which is currently not supported on Windows.
+    assumeTrue(!Path.WINDOWS);
+
     Path file1 = new Path("/test1");
     DFSTestUtil.createFile(fs, file1, 1024, (short) 2, 1L);
     DFSTestUtil.waitReplication(fs, file1, (short) 2);
@@ -270,9 +274,8 @@ public class TestDataNodeVolumeFailure {
    */
   @Test
   public void testUnderReplicationAfterVolFailure() throws Exception {
-    // This test relies on denying access to data volumes to simulate data volume
-    // failure.  This doesn't work on Windows, because an owner of an object
-    // always has the ability to read and change permissions on the object.
+    // The test uses DataNodeTestUtils#injectDataDirFailure() to simulate
+    // volume failures which is currently not supported on Windows.
     assumeTrue(!Path.WINDOWS);
 
     // Bring up one more datanode

http://git-wip-us.apache.org/repos/asf/hadoop/blob/195d746a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java
index aac288a..e0728dc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java
@@ -76,9 +76,8 @@ public class TestDataNodeVolumeFailureReporting {
 
   @Before
   public void setUp() throws Exception {
-    // These tests simulate volume failures by denying execute permission on the
-    // volume's path.  On Windows, the owner of an object is always allowed
-    // access, so we can't run these tests on Windows.
+    // These tests use DataNodeTestUtils#injectDataDirFailure() to simulate
+    // volume failures which is currently not supported on Windows.
     assumeTrue(!Path.WINDOWS);
     // Allow a single volume failure (there are two volumes)
     initCluster(1, 2, 1);


[42/50] [abbrv] hadoop git commit: MAPREDUCE-5905. CountersStrings.toEscapedCompactStrings outputs unnecessary null strings. Contributed by Akira AJISAKA.

Posted by zj...@apache.org.
MAPREDUCE-5905. CountersStrings.toEscapedCompactStrings outputs unnecessary null strings. Contributed by Akira AJISAKA.


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

Branch: refs/heads/YARN-2928
Commit: 62d72e26fd55ab8e2152ae17f0e99509bd76250a
Parents: 195d746
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Mon May 4 15:02:21 2015 +0900
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:59:00 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt               |  3 +++
 .../hadoop/mapreduce/util/CountersStrings.java     | 17 ++---------------
 .../org/apache/hadoop/mapred/TestCounters.java     |  3 +++
 3 files changed, 8 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/62d72e26/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 1b23ee2..86477c3 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -377,6 +377,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6345. Documentation fix for when CRLA is enabled for MRAppMaster
     logs. (Rohit Agarwal via gera)
 
+    MAPREDUCE-5905. CountersStrings.toEscapedCompactStrings outputs
+    unnecessary "null" strings. (Akira AJISAKA via ozawa)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62d72e26/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/CountersStrings.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/CountersStrings.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/CountersStrings.java
index ce799f5..ac16c12 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/CountersStrings.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/CountersStrings.java
@@ -151,25 +151,12 @@ public class CountersStrings {
   public static <C extends Counter, G extends CounterGroupBase<C>,
                  T extends AbstractCounters<C, G>>
   String toEscapedCompactString(T counters) {
-    String[] groupsArray;
-    int length = 0;
+    StringBuilder builder = new StringBuilder();
     synchronized(counters) {
-      groupsArray = new String[counters.countCounters()];
-      int i = 0;
-      // First up, obtain the escaped string for each group so that we can
-      // determine the buffer length apriori.
       for (G group : counters) {
-        String escapedString = toEscapedCompactString(group);
-        groupsArray[i++] = escapedString;
-        length += escapedString.length();
+        builder.append(toEscapedCompactString(group));
       }
     }
-
-    // Now construct the buffer
-    StringBuilder builder = new StringBuilder(length);
-    for (String group : groupsArray) {
-      builder.append(group);
-    }
     return builder.toString();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/62d72e26/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestCounters.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestCounters.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestCounters.java
index 46e7221..5e2763e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestCounters.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestCounters.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.mapred;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
@@ -84,6 +85,8 @@ public class TestCounters {
    */
   private void testCounter(Counters counter) throws ParseException {
     String compactEscapedString = counter.makeEscapedCompactString();
+    assertFalse("compactEscapedString should not contain null",
+                compactEscapedString.contains("null"));
     
     Counters recoveredCounter = 
       Counters.fromEscapedCompactString(compactEscapedString);


[49/50] [abbrv] hadoop git commit: YARN-3097. Logging of resource recovery on NM restart has redundancies. Contributed by Eric Payne

Posted by zj...@apache.org.
YARN-3097. Logging of resource recovery on NM restart has redundancies. Contributed by Eric Payne


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

Branch: refs/heads/YARN-2928
Commit: 0e9b9a989a61418f2a34e8b80e66dfe98a48a1fa
Parents: 54e2887
Author: Jason Lowe <jl...@apache.org>
Authored: Mon May 4 15:31:15 2015 +0000
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:59:01 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                | 2 ++
 .../localizer/ResourceLocalizationService.java                 | 6 ++++--
 2 files changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e9b9a98/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index fde0168..3957b24 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -360,6 +360,8 @@ Release 2.8.0 - UNRELEASED
     YARN-1993. Cross-site scripting vulnerability in TextView.java. (Kenji Kikushima
     via ozawa)
 
+    YARN-3097. Logging of resource recovery on NM restart has redundancies
+    (Eric Payne via jlowe)
 
 Release 2.7.1 - UNRELEASED
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e9b9a98/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.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/ResourceLocalizationService.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/ResourceLocalizationService.java
index cdd252c..e9c45f3 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/ResourceLocalizationService.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/ResourceLocalizationService.java
@@ -303,8 +303,10 @@ public class ResourceLocalizationService extends CompositeService
     for (LocalizedResourceProto proto : state.getLocalizedResources()) {
       LocalResource rsrc = new LocalResourcePBImpl(proto.getResource());
       LocalResourceRequest req = new LocalResourceRequest(rsrc);
-      LOG.info("Recovering localized resource " + req + " at "
-          + proto.getLocalPath());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Recovering localized resource " + req + " at "
+            + proto.getLocalPath());
+      }
       tracker.handle(new ResourceRecoveredEvent(req,
           new Path(proto.getLocalPath()), proto.getSize()));
     }


[17/50] [abbrv] hadoop git commit: YARN-3564. Fix TestContainerAllocation.testAMContainerAllocationWhenDNSUnavailable fails randomly. (Jian He via wangda)

Posted by zj...@apache.org.
YARN-3564. Fix TestContainerAllocation.testAMContainerAllocationWhenDNSUnavailable fails randomly. (Jian He via wangda)


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

Branch: refs/heads/YARN-2928
Commit: 71580ae6a33289c23788f3a179e8d15e4ecc237d
Parents: e5bb1e5
Author: Wangda Tan <wa...@apache.org>
Authored: Thu Apr 30 11:03:13 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:55 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                   | 3 +++
 .../scheduler/capacity/TestContainerAllocation.java               | 1 -
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/71580ae6/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 10eba67..bb00ee6 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -338,6 +338,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3533. Test: Fix launchAM in MockRM to wait for attempt to be scheduled.
     (Anubhav Dhoot via jianhe)
 
+    YARN-3564. Fix TestContainerAllocation.testAMContainerAllocationWhenDNSUnavailable 
+    fails randomly. (Jian He via wangda)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/71580ae6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
----------------------------------------------------------------------
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/TestContainerAllocation.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/TestContainerAllocation.java
index 6aceda5..0ea993f 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/TestContainerAllocation.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/TestContainerAllocation.java
@@ -325,7 +325,6 @@ public class TestContainerAllocation {
     }
 
     SecurityUtilTestHelper.setTokenServiceUseIp(false);
-    rm1.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.ALLOCATED);
     MockRM.launchAndRegisterAM(app1, rm1, nm1);
   }
 }


[35/50] [abbrv] hadoop git commit: YARN-2454. Fix compareTo of variable UNBOUNDED in o.a.h.y.util.resource.Resources. Contributed by Xu Yang.

Posted by zj...@apache.org.
YARN-2454. Fix compareTo of variable UNBOUNDED in o.a.h.y.util.resource.Resources. Contributed by Xu Yang.


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

Branch: refs/heads/YARN-2928
Commit: f5f6a68e275fb51b0ed2d9225aaed5e74c9dc279
Parents: a0e3a4c
Author: Junping Du <ju...@apache.org>
Authored: Sat May 2 10:18:44 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:59 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../hadoop/yarn/util/resource/Resources.java    |  8 ++--
 .../yarn/util/resource/TestResources.java       | 50 ++++++++++++++++++++
 3 files changed, 57 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5f6a68e/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 55bf9e4..511ddb7 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -354,6 +354,9 @@ Release 2.8.0 - UNRELEASED
     YARN-2893. AMLaucher: sporadic job failures due to EOFException in
     readTokenStorageStream. (Zhihai Xu via gera)
 
+    YARN-2454. Fix compareTo of variable UNBOUNDED in o.a.h.y.util.resource.Resources.
+    (Xu Yang via junping_du)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5f6a68e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
index bcb0421..472811a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
@@ -70,7 +70,7 @@ public class Resources {
 
     @Override
     public void setMemory(int memory) {
-      throw new RuntimeException("NONE cannot be modified!");
+      throw new RuntimeException("UNBOUNDED cannot be modified!");
     }
 
     @Override
@@ -80,14 +80,14 @@ public class Resources {
 
     @Override
     public void setVirtualCores(int cores) {
-      throw new RuntimeException("NONE cannot be modified!");
+      throw new RuntimeException("UNBOUNDED cannot be modified!");
     }
 
     @Override
     public int compareTo(Resource o) {
-      int diff = 0 - o.getMemory();
+      int diff = Integer.MAX_VALUE - o.getMemory();
       if (diff == 0) {
-        diff = 0 - o.getVirtualCores();
+        diff = Integer.MAX_VALUE - o.getVirtualCores();
       }
       return diff;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f5f6a68e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResources.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResources.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResources.java
new file mode 100644
index 0000000..d38ffdc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResources.java
@@ -0,0 +1,50 @@
+/**
+ * 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.util.resource;
+
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+
+public class TestResources {
+  
+  public Resource createResource(int memory, int vCores) {
+    return Resource.newInstance(memory, vCores);
+  }
+
+  @Test(timeout=1000)
+  public void testCompareToWithUnboundedResource() {
+    assertTrue(Resources.unbounded().compareTo(
+            createResource(Integer.MAX_VALUE, Integer.MAX_VALUE)) == 0);
+    assertTrue(Resources.unbounded().compareTo(
+        createResource(Integer.MAX_VALUE, 0)) > 0);
+    assertTrue(Resources.unbounded().compareTo(
+        createResource(0, Integer.MAX_VALUE)) > 0);
+  }
+
+  @Test(timeout=1000)
+  public void testCompareToWithNoneResource() {
+    assertTrue(Resources.none().compareTo(createResource(0, 0)) == 0);
+    assertTrue(Resources.none().compareTo(
+        createResource(1, 0)) < 0);
+    assertTrue(Resources.none().compareTo(
+        createResource(0, 1)) < 0);
+  }
+  
+}


[04/50] [abbrv] hadoop git commit: YARN-3517. RM web ui for dumping scheduler logs should be for admins only (Varun Vasudev via tgraves)

Posted by zj...@apache.org.
YARN-3517. RM web ui for dumping scheduler logs should be for admins only (Varun Vasudev via tgraves)


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

Branch: refs/heads/YARN-2928
Commit: 6d0351c8cea3405d357678740e482907513e3fad
Parents: c1e8813
Author: tgraves <tg...@apache.org>
Authored: Wed Apr 29 21:25:42 2015 +0000
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:53 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../server/security/ApplicationACLsManager.java | 11 +++
 .../webapp/CapacitySchedulerPage.java           | 51 +++++++++----
 .../resourcemanager/webapp/RMWebServices.java   | 13 +++-
 .../webapp/TestRMWebServices.java               | 77 ++++++++++++++++++++
 5 files changed, 139 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d0351c8/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 7d90477..5ea419c 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -332,6 +332,9 @@ Release 2.8.0 - UNRELEASED
     YARN-2740. Fix NodeLabelsManager to properly handle node label modifications 
     when distributed node label configuration enabled. (Naganarasimha G R via wangda)
 
+    YARN-3517. RM web ui for dumping scheduler logs should be for admins only
+    (Varun Vasudev via tgraves)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d0351c8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java
index 4daaa68..97b4163 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java
@@ -138,4 +138,15 @@ public class ApplicationACLsManager {
     }
     return false;
   }
+
+  /**
+   * Check if the given user in an admin.
+   *
+   * @param calledUGI
+   *          UserGroupInformation for the user
+   * @return true if the user is an admin, false otherwise
+   */
+  public final boolean isAdmin(final UserGroupInformation calledUGI) {
+    return this.adminAclsManager.isAdmin(calledUGI);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d0351c8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.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/webapp/CapacitySchedulerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
index 2eeda66..fa22a0d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
@@ -24,6 +24,7 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerHealth;
@@ -33,6 +34,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.UserInfo
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerLeafQueueInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerQueueInfo;
+import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.webapp.ResponseInfo;
 import org.apache.hadoop.yarn.webapp.SubView;
@@ -190,28 +192,46 @@ class CapacitySchedulerPage extends RmView {
   static class QueuesBlock extends HtmlBlock {
     final CapacityScheduler cs;
     final CSQInfo csqinfo;
+    private final ResourceManager rm;
 
     @Inject QueuesBlock(ResourceManager rm, CSQInfo info) {
       cs = (CapacityScheduler) rm.getResourceScheduler();
       csqinfo = info;
+      this.rm = rm;
     }
 
     @Override
     public void render(Block html) {
       html._(MetricsOverviewTable.class);
-      // Dump CapacityScheduler debug logs
-      html.div()
+
+      UserGroupInformation callerUGI = this.getCallerUGI();
+      boolean isAdmin = false;
+      ApplicationACLsManager aclsManager = rm.getApplicationACLsManager();
+      if (aclsManager.areACLsEnabled()) {
+        if (callerUGI != null && aclsManager.isAdmin(callerUGI)) {
+          isAdmin = true;
+        }
+      } else {
+        isAdmin = true;
+      }
+
+      // only show button to dump CapacityScheduler debug logs to admins
+      if (isAdmin) {
+        html.div()
           .button()
-          .$onclick("confirmAction()").b("Dump scheduler logs")._()
-          .select().$id("time")
-            .option().$value("60")._("1 min")._()
-            .option().$value("300")._("5 min")._()
-            .option().$value("600")._("10 min")._()
+          .$style(
+              "border-style: solid; border-color: #000000; border-width: 1px;"
+                  + " cursor: hand; cursor: pointer; border-radius: 4px")
+          .$onclick("confirmAction()").b("Dump scheduler logs")._().select()
+          .$id("time").option().$value("60")._("1 min")._().option()
+          .$value("300")._("5 min")._().option().$value("600")._("10 min")._()
           ._()._();
 
-      StringBuilder script = new StringBuilder();
-      script.append("function confirmAction() {")
-          .append(" b = confirm(\"Are you sure you wish to generate scheduler logs?\");")
+        StringBuilder script = new StringBuilder();
+        script
+          .append("function confirmAction() {")
+          .append(" b = confirm(\"Are you sure you wish to generate"
+              + " scheduler logs?\");")
           .append(" if (b == true) {")
           .append(" var timePeriod = $(\"#time\").val();")
           .append(" $.ajax({")
@@ -225,13 +245,14 @@ class CapacitySchedulerPage extends RmView {
           .append(" alert(\"Scheduler log is being generated.\");")
           .append(" }, 1000);")
           .append(" }).fail(function(data){")
-          .append(" alert(\"Scheduler log generation failed. Please check the ResourceManager log for more informtion.\");")
-          .append(" console.log(data);")
-          .append(" });")
-          .append(" }")
+          .append(
+              " alert(\"Scheduler log generation failed. Please check the"
+                  + " ResourceManager log for more informtion.\");")
+          .append(" console.log(data);").append(" });").append(" }")
           .append("}");
 
-      html.script().$type("text/javascript")._(script.toString())._();
+        html.script().$type("text/javascript")._(script.toString())._();
+      }
 
       UL<DIV<DIV<Hamlet>>> ul = html.
         div("#cs-wrapper.ui-widget").

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d0351c8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.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/webapp/RMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
index 9aea62d..4ce2b54 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
@@ -142,10 +142,12 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ResourceInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerTypeInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.StatisticsItemInfo;
+import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.AdHocLogDumper;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.webapp.BadRequestException;
+import org.apache.hadoop.yarn.webapp.ForbiddenException;
 import org.apache.hadoop.yarn.webapp.NotFoundException;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 
@@ -263,8 +265,17 @@ public class RMWebServices {
   @POST
   @Path("/scheduler/logs")
   @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
-  public String dumpSchedulerLogs(@FormParam("time") String time) throws IOException {
+  public String dumpSchedulerLogs(@FormParam("time") String time,
+      @Context HttpServletRequest hsr) throws IOException {
     init();
+    UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true);
+    ApplicationACLsManager aclsManager = rm.getApplicationACLsManager();
+    if (aclsManager.areACLsEnabled()) {
+      if (callerUGI == null || !aclsManager.isAdmin(callerUGI)) {
+        String msg = "Only admins can carry out this operation.";
+        throw new ForbiddenException(msg);
+      }
+    }
     ResourceScheduler rs = rm.getResourceScheduler();
     int period = Integer.parseInt(time);
     if (period <= 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d0351c8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.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/webapp/TestRMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java
index e4614f8..cd1d771 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java
@@ -26,7 +26,9 @@ import static org.mockito.Matchers.isA;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
+import java.io.File;
 import java.io.StringReader;
+import java.security.Principal;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Set;
@@ -37,6 +39,7 @@ import javax.ws.rs.core.MediaType;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.Service.STATE;
 import org.apache.hadoop.util.VersionInfo;
@@ -54,9 +57,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppsInfo;
+import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.util.YarnVersionInfo;
+import org.apache.hadoop.yarn.webapp.ForbiddenException;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
@@ -643,4 +650,74 @@ public class TestRMWebServices extends JerseyTestBase {
         null, null, null, null, null, null, null, emptySet, emptySet);
     assertTrue(appsInfo.getApps().isEmpty());
   }
+
+  @Test
+  public void testDumpingSchedulerLogs() throws Exception {
+
+    ResourceManager mockRM = mock(ResourceManager.class);
+    Configuration conf = new YarnConfiguration();
+    HttpServletRequest mockHsr = mock(HttpServletRequest.class);
+    ApplicationACLsManager aclsManager = new ApplicationACLsManager(conf);
+    when(mockRM.getApplicationACLsManager()).thenReturn(aclsManager);
+    RMWebServices webSvc =
+        new RMWebServices(mockRM, conf, mock(HttpServletResponse.class));
+
+    // nothing should happen
+    webSvc.dumpSchedulerLogs("1", mockHsr);
+    Thread.sleep(1000);
+    checkSchedulerLogFileAndCleanup();
+
+    conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
+    conf.setStrings(YarnConfiguration.YARN_ADMIN_ACL, "admin");
+    aclsManager = new ApplicationACLsManager(conf);
+    when(mockRM.getApplicationACLsManager()).thenReturn(aclsManager);
+    webSvc = new RMWebServices(mockRM, conf, mock(HttpServletResponse.class));
+    boolean exceptionThrown = false;
+    try {
+      webSvc.dumpSchedulerLogs("1", mockHsr);
+      fail("Dumping logs should fail");
+    } catch (ForbiddenException ae) {
+      exceptionThrown = true;
+    }
+    assertTrue("ForbiddenException expected", exceptionThrown);
+    exceptionThrown = false;
+    when(mockHsr.getUserPrincipal()).thenReturn(new Principal() {
+      @Override
+      public String getName() {
+        return "testuser";
+      }
+    });
+    try {
+      webSvc.dumpSchedulerLogs("1", mockHsr);
+      fail("Dumping logs should fail");
+    } catch (ForbiddenException ae) {
+      exceptionThrown = true;
+    }
+    assertTrue("ForbiddenException expected", exceptionThrown);
+
+    when(mockHsr.getUserPrincipal()).thenReturn(new Principal() {
+      @Override
+      public String getName() {
+        return "admin";
+      }
+    });
+    webSvc.dumpSchedulerLogs("1", mockHsr);
+    Thread.sleep(1000);
+    checkSchedulerLogFileAndCleanup();
+  }
+
+  private void checkSchedulerLogFileAndCleanup() {
+    String targetFile;
+    ResourceScheduler scheduler = rm.getResourceScheduler();
+    if (scheduler instanceof FairScheduler) {
+      targetFile = "yarn-fair-scheduler-debug.log";
+    } else if (scheduler instanceof CapacityScheduler) {
+      targetFile = "yarn-capacity-scheduler-debug.log";
+    } else {
+      targetFile = "yarn-scheduler-debug.log";
+    }
+    File logFile = new File(System.getProperty("yarn.log.dir"), targetFile);
+    assertTrue("scheduler log file doesn't exist", logFile.exists());
+    FileUtils.deleteQuietly(logFile);
+  }
 }


[20/50] [abbrv] hadoop git commit: YARN-2619. Added NodeManager support for disk io isolation through cgroups. Contributed by Varun Vasudev and Wei Yan.

Posted by zj...@apache.org.
YARN-2619. Added NodeManager support for disk io isolation through cgroups. Contributed by Varun Vasudev and Wei Yan.


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

Branch: refs/heads/YARN-2928
Commit: f2a548a54f0b75acc2c8883a09b054484c09a35d
Parents: 960b4e9
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Thu Apr 30 21:41:07 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:56 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |  54 ++++--
 .../CGroupsBlkioResourceHandlerImpl.java        | 170 +++++++++++++++++++
 .../linux/resources/CGroupsHandler.java         |   4 +-
 .../linux/resources/CGroupsHandlerImpl.java     |  67 ++++----
 .../linux/resources/DiskResourceHandler.java    |  30 ++++
 .../linux/resources/ResourceHandlerModule.java  |  45 ++++-
 .../util/CgroupsLCEResourcesHandler.java        |   6 +
 .../TestCGroupsBlkioResourceHandlerImpl.java    | 116 +++++++++++++
 .../linux/resources/TestCGroupsHandlerImpl.java | 101 +++++++++--
 .../resources/TestResourceHandlerModule.java    |  26 ++-
 .../util/TestCgroupsLCEResourcesHandler.java    | 112 +++++-------
 12 files changed, 600 insertions(+), 134 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2a548a5/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 27b4fe8..8dff48d 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -169,6 +169,9 @@ Release 2.8.0 - UNRELEASED
     YARN-2498. Respect labels in preemption policy of capacity scheduler for
     inter-queue preemption. (Wangda Tan via jianhe)
 
+    YARN-2619. Added NodeManager support for disk io isolation through cgroups.
+    (Varun Vasudev and Wei Yan via vinodkv)
+
   IMPROVEMENTS
 
     YARN-1880. Cleanup TestApplicationClientProtocolOnHA

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2a548a5/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 7303e44..3833ad4 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
@@ -844,38 +844,68 @@ public class YarnConfiguration extends Configuration {
   public static final int DEFAULT_NM_RESOURCE_PERCENTAGE_PHYSICAL_CPU_LIMIT =
       100;
 
+  /**
+   * Prefix for disk configurations. Work in progress: This configuration
+   * parameter may be changed/removed in the future.
+   */
+  @Private
+  public static final String NM_DISK_RESOURCE_PREFIX = NM_PREFIX
+      + "resource.disk.";
+  /**
+   * This setting controls if resource handling for disk operations is enabled.
+   * Work in progress: This configuration parameter may be changed/removed in
+   * the future
+   */
+  @Private
+  public static final String NM_DISK_RESOURCE_ENABLED = NM_DISK_RESOURCE_PREFIX
+      + "enabled";
+  /** Disk as a resource is disabled by default. **/
+  @Private
+  public static final boolean DEFAULT_NM_DISK_RESOURCE_ENABLED = false;
 
-  public static final String NM_NETWORK_RESOURCE_PREFIX = NM_PREFIX + "resource.network.";
+  public static final String NM_NETWORK_RESOURCE_PREFIX = NM_PREFIX
+      + "resource.network.";
 
-  /** This setting controls if resource handling for network bandwidth is enabled **/
-  /* Work in progress: This configuration parameter may be changed/removed in the future */
+  /**
+   * This setting controls if resource handling for network bandwidth is
+   * enabled. Work in progress: This configuration parameter may be
+   * changed/removed in the future
+   */
   @Private
   public static final String NM_NETWORK_RESOURCE_ENABLED =
       NM_NETWORK_RESOURCE_PREFIX + "enabled";
-  /** Network as a resource is disabled by default **/
+  /** Network as a resource is disabled by default. **/
   @Private
   public static final boolean DEFAULT_NM_NETWORK_RESOURCE_ENABLED = false;
 
-  /** Specifies the interface to be used for applying network throttling rules **/
-  /* Work in progress: This configuration parameter may be changed/removed in the future */
+  /**
+   * Specifies the interface to be used for applying network throttling rules.
+   * Work in progress: This configuration parameter may be changed/removed in
+   * the future
+   */
   @Private
   public static final String NM_NETWORK_RESOURCE_INTERFACE =
       NM_NETWORK_RESOURCE_PREFIX + "interface";
   @Private
   public static final String DEFAULT_NM_NETWORK_RESOURCE_INTERFACE = "eth0";
 
-  /** Specifies the total available outbound bandwidth on the node **/
-  /* Work in progress: This configuration parameter may be changed/removed in the future */
+  /**
+   * Specifies the total available outbound bandwidth on the node. Work in
+   * progress: This configuration parameter may be changed/removed in the future
+   */
   @Private
   public static final String NM_NETWORK_RESOURCE_OUTBOUND_BANDWIDTH_MBIT =
       NM_NETWORK_RESOURCE_PREFIX + "outbound-bandwidth-mbit";
   @Private
-  public static final int DEFAULT_NM_NETWORK_RESOURCE_OUTBOUND_BANDWIDTH_MBIT = 1000;
+  public static final int DEFAULT_NM_NETWORK_RESOURCE_OUTBOUND_BANDWIDTH_MBIT =
+      1000;
 
-  /** Specifies the total outbound bandwidth available to YARN containers. defaults to
-   * NM_NETWORK_RESOURCE_OUTBOUND_BANDWIDTH_MBIT if not specified.
+  /**
+   * Specifies the total outbound bandwidth available to YARN containers.
+   * defaults to NM_NETWORK_RESOURCE_OUTBOUND_BANDWIDTH_MBIT if not specified.
+   * Work in progress: This configuration parameter may be changed/removed in
+   * the future
    */
-  /* Work in progress: This configuration parameter may be changed/removed in the future */
   @Private
   public static final String NM_NETWORK_RESOURCE_OUTBOUND_BANDWIDTH_YARN_MBIT =
       NM_NETWORK_RESOURCE_PREFIX + "outbound-bandwidth-yarn-mbit";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2a548a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsBlkioResourceHandlerImpl.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/CGroupsBlkioResourceHandlerImpl.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/CGroupsBlkioResourceHandlerImpl.java
new file mode 100644
index 0000000..e7eea1f
--- /dev/null
+++ 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/CGroupsBlkioResourceHandlerImpl.java
@@ -0,0 +1,170 @@
+/**
+ * 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.server.nodemanager.containermanager.linux.resources;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Handler class to handle the blkio controller. Currently it splits resources
+ * evenly across all containers. Once we have scheduling sorted out, we can
+ * modify the function to represent the disk resources allocated.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class CGroupsBlkioResourceHandlerImpl implements DiskResourceHandler {
+
+  static final Log LOG = LogFactory
+      .getLog(CGroupsBlkioResourceHandlerImpl.class);
+
+  private CGroupsHandler cGroupsHandler;
+  // Arbitrarily choose a weight - all that matters is that all containers
+  // get the same weight assigned to them. Once we have scheduling support
+  // this number will be determined dynamically for each container.
+  @VisibleForTesting
+  static final String DEFAULT_WEIGHT = "500";
+  private static final String PARTITIONS_FILE = "/proc/partitions";
+
+  CGroupsBlkioResourceHandlerImpl(CGroupsHandler cGroupsHandler) {
+    this.cGroupsHandler = cGroupsHandler;
+    // check for linux so that we don't print messages for tests running on
+    // other platforms
+    if(Shell.LINUX) {
+      checkDiskScheduler();
+    }
+  }
+
+
+  private void checkDiskScheduler() {
+    String data;
+
+    // read /proc/partitions and check to make sure that sd* and hd*
+    // are using the CFQ scheduler. If they aren't print a warning
+    try {
+      byte[] contents = Files.readAllBytes(Paths.get(PARTITIONS_FILE));
+      data = new String(contents, "UTF-8").trim();
+    } catch (IOException e) {
+      String msg = "Couldn't read " + PARTITIONS_FILE +
+          "; can't determine disk scheduler type";
+      LOG.warn(msg, e);
+      return;
+    }
+    String[] lines = data.split(System.lineSeparator());
+    if (lines.length > 0) {
+      for (String line : lines) {
+        String[] columns = line.split("\\s+");
+        if (columns.length > 4) {
+          String partition = columns[4];
+          // check some known partitions to make sure  the disk scheduler
+          // is cfq - not meant to be comprehensive, more a sanity check
+          if (partition.startsWith("sd") || partition.startsWith("hd")
+              || partition.startsWith("vd") || partition.startsWith("xvd")) {
+            String schedulerPath =
+                "/sys/block/" + partition + "/queue/scheduler";
+            File schedulerFile = new File(schedulerPath);
+            if (schedulerFile.exists()) {
+              try {
+                byte[] contents = Files.readAllBytes(Paths.get(schedulerPath));
+                String schedulerString = new String(contents, "UTF-8").trim();
+                if (!schedulerString.contains("[cfq]")) {
+                  LOG.warn("Device " + partition + " does not use the CFQ"
+                      + " scheduler; disk isolation using "
+                      + "CGroups will not work on this partition.");
+                }
+              } catch (IOException ie) {
+                LOG.warn(
+                    "Unable to determine disk scheduler type for partition "
+                      + partition, ie);
+              }
+            }
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public List<PrivilegedOperation> bootstrap(Configuration configuration)
+      throws ResourceHandlerException {
+    // if bootstrap is called on this class, disk is already enabled
+    // so no need to check again
+    this.cGroupsHandler
+      .mountCGroupController(CGroupsHandler.CGroupController.BLKIO);
+    return null;
+  }
+
+  @Override
+  public List<PrivilegedOperation> preStart(Container container)
+      throws ResourceHandlerException {
+
+    String cgroupId = container.getContainerId().toString();
+    cGroupsHandler
+      .createCGroup(CGroupsHandler.CGroupController.BLKIO, cgroupId);
+    try {
+      cGroupsHandler.updateCGroupParam(CGroupsHandler.CGroupController.BLKIO,
+          cgroupId, CGroupsHandler.CGROUP_PARAM_BLKIO_WEIGHT, DEFAULT_WEIGHT);
+    } catch (ResourceHandlerException re) {
+      cGroupsHandler.deleteCGroup(CGroupsHandler.CGroupController.BLKIO,
+          cgroupId);
+      LOG.warn("Could not update cgroup for container", re);
+      throw re;
+    }
+    List<PrivilegedOperation> ret = new ArrayList<>();
+    ret.add(new PrivilegedOperation(
+      PrivilegedOperation.OperationType.ADD_PID_TO_CGROUP,
+      PrivilegedOperation.CGROUP_ARG_PREFIX
+          + cGroupsHandler.getPathForCGroupTasks(
+            CGroupsHandler.CGroupController.BLKIO, cgroupId)));
+    return ret;
+  }
+
+  @Override
+  public List<PrivilegedOperation> reacquireContainer(ContainerId containerId)
+      throws ResourceHandlerException {
+    return null;
+  }
+
+  @Override
+  public List<PrivilegedOperation> postComplete(ContainerId containerId)
+      throws ResourceHandlerException {
+    cGroupsHandler.deleteCGroup(CGroupsHandler.CGroupController.BLKIO,
+        containerId.toString());
+    return null;
+  }
+
+  @Override
+  public List<PrivilegedOperation> teardown() throws ResourceHandlerException {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2a548a5/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 34429d3..70dc818 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
@@ -33,7 +33,8 @@ import org.apache.hadoop.classification.InterfaceStability;
 public interface CGroupsHandler {
   public enum CGroupController {
     CPU("cpu"),
-    NET_CLS("net_cls");
+    NET_CLS("net_cls"),
+    BLKIO("blkio");
 
     private final String name;
 
@@ -48,6 +49,7 @@ public interface CGroupsHandler {
 
   public static final String CGROUP_FILE_TASKS = "tasks";
   public static final String CGROUP_PARAM_CLASSID = "classid";
+  public static final String CGROUP_PARAM_BLKIO_WEIGHT = "weight";
 
   /**
    * Mounts a cgroup controller

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2a548a5/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 9a4230f..ff56121 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
@@ -20,6 +20,7 @@
 
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -38,6 +39,7 @@ import java.io.*;
 import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -63,7 +65,7 @@ class CGroupsHandlerImpl implements CGroupsHandler {
   private final String cGroupMountPath;
   private final long deleteCGroupTimeout;
   private final long deleteCGroupDelay;
-  private final Map<CGroupController, String> controllerPaths;
+  private Map<CGroupController, String> controllerPaths;
   private final ReadWriteLock rwLock;
   private final PrivilegedOperationExecutor privilegedOperationExecutor;
   private final Clock clock;
@@ -106,55 +108,61 @@ class CGroupsHandlerImpl implements CGroupsHandler {
 
   private void initializeControllerPaths() throws ResourceHandlerException {
     if (enableCGroupMount) {
-      //nothing to do here - we support 'deferred' mounting of specific
-      //controllers - we'll populate the path for a given controller when an
-      //explicit mountCGroupController request is issued.
+      // nothing to do here - we support 'deferred' mounting of specific
+      // controllers - we'll populate the path for a given controller when an
+      // explicit mountCGroupController request is issued.
       LOG.info("CGroup controller mounting enabled.");
     } else {
-      //cluster admins are expected to have mounted controllers in specific
-      //locations - we'll attempt to figure out mount points
-      initializeControllerPathsFromMtab();
+      // cluster admins are expected to have mounted controllers in specific
+      // locations - we'll attempt to figure out mount points
+
+      Map<CGroupController, String> cPaths =
+          initializeControllerPathsFromMtab(MTAB_FILE, this.cGroupPrefix);
+      // we want to do a bulk update without the paths changing concurrently
+      try {
+        rwLock.writeLock().lock();
+        controllerPaths = cPaths;
+      } finally {
+        rwLock.writeLock().unlock();
+      }
     }
   }
 
-  private void initializeControllerPathsFromMtab()
-      throws ResourceHandlerException {
+  @VisibleForTesting
+  static Map<CGroupController, String> initializeControllerPathsFromMtab(
+      String mtab, String cGroupPrefix) throws ResourceHandlerException {
     try {
-      Map<String, List<String>> parsedMtab = parseMtab();
-
-      //we want to do a bulk update without the paths changing concurrently
-      rwLock.writeLock().lock();
+      Map<String, List<String>> parsedMtab = parseMtab(mtab);
+      Map<CGroupController, String> ret = new HashMap<>();
 
       for (CGroupController controller : CGroupController.values()) {
         String name = controller.getName();
         String controllerPath = findControllerInMtab(name, parsedMtab);
 
         if (controllerPath != null) {
-          File f = new File(controllerPath + "/" + this.cGroupPrefix);
+          File f = new File(controllerPath + "/" + cGroupPrefix);
 
           if (FileUtil.canWrite(f)) {
-            controllerPaths.put(controller, controllerPath);
+            ret.put(controller, controllerPath);
           } else {
             String error =
                 new StringBuffer("Mount point Based on mtab file: ")
-                    .append(MTAB_FILE).append(
-                    ". Controller mount point not writable for: ")
-                    .append(name).toString();
+                  .append(mtab)
+                  .append(". Controller mount point not writable for: ")
+                  .append(name).toString();
 
             LOG.error(error);
             throw new ResourceHandlerException(error);
           }
         } else {
-
-            LOG.warn("Controller not mounted but automount disabled: " + name);
+          LOG.warn("Controller not mounted but automount disabled: " + name);
         }
       }
+      return ret;
     } catch (IOException e) {
       LOG.warn("Failed to initialize controller paths! Exception: " + e);
       throw new ResourceHandlerException(
-          "Failed to initialize controller paths!");
-    } finally {
-      rwLock.writeLock().unlock();
+        "Failed to initialize controller paths!");
     }
   }
 
@@ -173,12 +181,13 @@ class CGroupsHandlerImpl implements CGroupsHandler {
    * for mounts with type "cgroup". Cgroup controllers will
    * appear in the list of options for a path.
    */
-  private Map<String, List<String>> parseMtab() throws IOException {
+  private static Map<String, List<String>> parseMtab(String mtab)
+      throws IOException {
     Map<String, List<String>> ret = new HashMap<String, List<String>>();
     BufferedReader in = null;
 
     try {
-      FileInputStream fis = new FileInputStream(new File(getMtabFileName()));
+      FileInputStream fis = new FileInputStream(new File(mtab));
       in = new BufferedReader(new InputStreamReader(fis, "UTF-8"));
 
       for (String str = in.readLine(); str != null;
@@ -197,7 +206,7 @@ class CGroupsHandlerImpl implements CGroupsHandler {
         }
       }
     } catch (IOException e) {
-      throw new IOException("Error while reading " + getMtabFileName(), e);
+      throw new IOException("Error while reading " + mtab, e);
     } finally {
       IOUtils.cleanup(LOG, in);
     }
@@ -205,7 +214,7 @@ class CGroupsHandlerImpl implements CGroupsHandler {
     return ret;
   }
 
-  private String findControllerInMtab(String controller,
+  private static String findControllerInMtab(String controller,
       Map<String, List<String>> entries) {
     for (Map.Entry<String, List<String>> e : entries.entrySet()) {
       if (e.getValue().contains(controller))
@@ -215,10 +224,6 @@ class CGroupsHandlerImpl implements CGroupsHandler {
     return null;
   }
 
-  String getMtabFileName() {
-    return MTAB_FILE;
-  }
-
   @Override
   public void mountCGroupController(CGroupController controller)
       throws ResourceHandlerException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2a548a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/DiskResourceHandler.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/DiskResourceHandler.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/DiskResourceHandler.java
new file mode 100644
index 0000000..ca08d89
--- /dev/null
+++ 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/DiskResourceHandler.java
@@ -0,0 +1,30 @@
+/**
+ * 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.server.nodemanager.containermanager.linux.resources;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Resource handler for disk resources.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public interface DiskResourceHandler extends ResourceHandler {
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2a548a5/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 30fc951..5dfd78c 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
@@ -20,6 +20,7 @@
 
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -31,25 +32,27 @@ import java.util.List;
 
 /**
  * Provides mechanisms to get various resource handlers - cpu, memory, network,
- * disk etc., - based on configuration
+ * disk etc., - based on configuration.
  */
 
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class ResourceHandlerModule {
-  private volatile static ResourceHandlerChain resourceHandlerChain;
+  private static volatile ResourceHandlerChain resourceHandlerChain;
 
   /**
    * This specific implementation might provide resource management as well
    * as resource metrics functionality. We need to ensure that the same
    * instance is used for both.
    */
-  private volatile static TrafficControlBandwidthHandlerImpl
+  private static volatile TrafficControlBandwidthHandlerImpl
       trafficControlBandwidthHandler;
-  private volatile static CGroupsHandler cGroupsHandler;
+  private static volatile CGroupsHandler cGroupsHandler;
+  private static volatile CGroupsBlkioResourceHandlerImpl
+      cGroupsBlkioResourceHandler;
 
   /**
-   * Returns an initialized, thread-safe CGroupsHandler instance
+   * Returns an initialized, thread-safe CGroupsHandler instance.
    */
   public static CGroupsHandler getCGroupsHandler(Configuration conf)
       throws ResourceHandlerException {
@@ -94,6 +97,28 @@ public class ResourceHandlerModule {
     return getTrafficControlBandwidthHandler(conf);
   }
 
+  public static DiskResourceHandler getDiskResourceHandler(Configuration conf)
+      throws ResourceHandlerException {
+    if (conf.getBoolean(YarnConfiguration.NM_DISK_RESOURCE_ENABLED,
+        YarnConfiguration.DEFAULT_NM_DISK_RESOURCE_ENABLED)) {
+      return getCgroupsBlkioResourceHandler(conf);
+    }
+    return null;
+  }
+
+  private static CGroupsBlkioResourceHandlerImpl getCgroupsBlkioResourceHandler(
+      Configuration conf) throws ResourceHandlerException {
+    if (cGroupsBlkioResourceHandler == null) {
+      synchronized (DiskResourceHandler.class) {
+        if (cGroupsBlkioResourceHandler == null) {
+          cGroupsBlkioResourceHandler =
+              new CGroupsBlkioResourceHandlerImpl(getCGroupsHandler(conf));
+        }
+      }
+    }
+    return cGroupsBlkioResourceHandler;
+  }
+
   private static void addHandlerIfNotNull(List<ResourceHandler> handlerList,
       ResourceHandler handler) {
     if (handler != null) {
@@ -106,11 +131,12 @@ public class ResourceHandlerModule {
     ArrayList<ResourceHandler> handlerList = new ArrayList<>();
 
     addHandlerIfNotNull(handlerList, getOutboundBandwidthResourceHandler(conf));
+    addHandlerIfNotNull(handlerList, getDiskResourceHandler(conf));
     resourceHandlerChain = new ResourceHandlerChain(handlerList);
   }
 
-  public static ResourceHandlerChain getConfiguredResourceHandlerChain
-      (Configuration conf) throws ResourceHandlerException {
+  public static ResourceHandlerChain getConfiguredResourceHandlerChain(
+      Configuration conf) throws ResourceHandlerException {
     if (resourceHandlerChain == null) {
       synchronized (ResourceHandlerModule.class) {
         if (resourceHandlerChain == null) {
@@ -125,4 +151,9 @@ public class ResourceHandlerModule {
       return null;
     }
   }
+
+  @VisibleForTesting
+  static void nullifyResourceHandlerChain() throws ResourceHandlerException {
+    resourceHandlerChain = null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2a548a5/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 ffa17ac..176b63c 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
@@ -30,6 +30,7 @@ import java.io.PrintWriter;
 import java.io.Writer;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -503,4 +504,9 @@ public class CgroupsLCEResourcesHandler implements LCEResourcesHandler {
   String getMtabFileName() {
     return MTAB_FILE;
   }
+
+  @VisibleForTesting
+  Map<String, String> getControllerPaths() {
+    return Collections.unmodifiableMap(controllerPaths);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2a548a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsBlkioResourceHandlerImpl.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/TestCGroupsBlkioResourceHandlerImpl.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/TestCGroupsBlkioResourceHandlerImpl.java
new file mode 100644
index 0000000..20aab69
--- /dev/null
+++ 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/TestCGroupsBlkioResourceHandlerImpl.java
@@ -0,0 +1,116 @@
+/**
+ * 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.server.nodemanager.containermanager.linux.resources;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.List;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.times;
+
+/**
+ * Tests for the cgroups disk handler implementation.
+ */
+public class TestCGroupsBlkioResourceHandlerImpl {
+
+  private CGroupsHandler mockCGroupsHandler;
+  private CGroupsBlkioResourceHandlerImpl cGroupsBlkioResourceHandlerImpl;
+
+  @Before
+  public void setup() {
+    mockCGroupsHandler = mock(CGroupsHandler.class);
+    cGroupsBlkioResourceHandlerImpl =
+        new CGroupsBlkioResourceHandlerImpl(mockCGroupsHandler);
+  }
+
+  @Test
+  public void testBootstrap() throws Exception {
+    Configuration conf = new YarnConfiguration();
+    List<PrivilegedOperation> ret =
+        cGroupsBlkioResourceHandlerImpl.bootstrap(conf);
+    verify(mockCGroupsHandler, times(1)).mountCGroupController(
+        CGroupsHandler.CGroupController.BLKIO);
+    Assert.assertNull(ret);
+  }
+
+  @Test
+  public void testPreStart() throws Exception {
+    String id = "container_01_01";
+    String path = "test-path/" + id;
+    ContainerId mockContainerId = mock(ContainerId.class);
+    when(mockContainerId.toString()).thenReturn(id);
+    Container mockContainer = mock(Container.class);
+    when(mockContainer.getContainerId()).thenReturn(mockContainerId);
+    when(
+      mockCGroupsHandler.getPathForCGroupTasks(
+        CGroupsHandler.CGroupController.BLKIO, id)).thenReturn(path);
+
+    List<PrivilegedOperation> ret =
+        cGroupsBlkioResourceHandlerImpl.preStart(mockContainer);
+    verify(mockCGroupsHandler, times(1)).createCGroup(
+        CGroupsHandler.CGroupController.BLKIO, id);
+    verify(mockCGroupsHandler, times(1)).updateCGroupParam(
+        CGroupsHandler.CGroupController.BLKIO, id,
+        CGroupsHandler.CGROUP_PARAM_BLKIO_WEIGHT,
+        CGroupsBlkioResourceHandlerImpl.DEFAULT_WEIGHT);
+    Assert.assertNotNull(ret);
+    Assert.assertEquals(1, ret.size());
+    PrivilegedOperation op = ret.get(0);
+    Assert.assertEquals(PrivilegedOperation.OperationType.ADD_PID_TO_CGROUP,
+        op.getOperationType());
+    List<String> args = op.getArguments();
+    Assert.assertEquals(1, args.size());
+    Assert.assertEquals(PrivilegedOperation.CGROUP_ARG_PREFIX + path,
+        args.get(0));
+  }
+
+  @Test
+  public void testReacquireContainer() throws Exception {
+    ContainerId containerIdMock = mock(ContainerId.class);
+    Assert.assertNull(cGroupsBlkioResourceHandlerImpl
+        .reacquireContainer(containerIdMock));
+  }
+
+  @Test
+  public void testPostComplete() throws Exception {
+    String id = "container_01_01";
+    ContainerId mockContainerId = mock(ContainerId.class);
+    when(mockContainerId.toString()).thenReturn(id);
+    Assert.assertNull(cGroupsBlkioResourceHandlerImpl
+        .postComplete(mockContainerId));
+    verify(mockCGroupsHandler, times(1)).deleteCGroup(
+        CGroupsHandler.CGroupController.BLKIO, id);
+  }
+
+  @Test
+  public void testTeardown() throws Exception {
+    Assert.assertNull(cGroupsBlkioResourceHandlerImpl.teardown());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2a548a5/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 0717447..50f8da6 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
@@ -20,6 +20,7 @@
 
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -35,18 +36,21 @@ import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 
 import java.io.File;
+import java.io.FileWriter;
 import java.io.IOException;
 import java.nio.file.Files;
-import java.util.List;
 import java.util.Map;
+import java.util.UUID;
 
 import static org.mockito.Matchers.eq;
-import static org.mockito.Matchers.isNull;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.verifyNoMoreInteractions;
 import static org.mockito.Mockito.verifyZeroInteractions;
 
+/**
+ * Tests for the CGroups handler implementation.
+ */
 public class TestCGroupsHandlerImpl {
   private static final Log LOG =
       LogFactory.getLog(TestCGroupsHandlerImpl.class);
@@ -84,8 +88,8 @@ public class TestCGroupsHandlerImpl {
     try {
       cGroupsHandler = new CGroupsHandlerImpl(conf,
           privilegedOperationExecutorMock);
-      PrivilegedOperation expectedOp = new PrivilegedOperation
-          (PrivilegedOperation.OperationType.MOUNT_CGROUPS, (String) null);
+      PrivilegedOperation expectedOp = new PrivilegedOperation(
+          PrivilegedOperation.OperationType.MOUNT_CGROUPS, (String) null);
       //This is expected to be of the form :
       //net_cls=<mount_path>/net_cls
       StringBuffer controllerKV = new StringBuffer(controller.getName())
@@ -94,8 +98,8 @@ public class TestCGroupsHandlerImpl {
 
       cGroupsHandler.mountCGroupController(controller);
       try {
-        ArgumentCaptor<PrivilegedOperation> opCaptor = ArgumentCaptor.forClass
-            (PrivilegedOperation.class);
+        ArgumentCaptor<PrivilegedOperation> opCaptor = ArgumentCaptor.forClass(
+            PrivilegedOperation.class);
         verify(privilegedOperationExecutorMock)
             .executePrivilegedOperation(opCaptor.capture(), eq(false));
 
@@ -200,17 +204,15 @@ public class TestCGroupsHandlerImpl {
 
       Assert.assertTrue(paramFile.exists());
       try {
-        Assert.assertEquals(paramValue, new String(Files.readAllBytes
-            (paramFile
-                .toPath())));
+        Assert.assertEquals(paramValue, new String(Files.readAllBytes(
+            paramFile.toPath())));
       } catch (IOException e) {
         LOG.error("Caught exception: " + e);
-        Assert.assertTrue("Unexpected IOException trying to read cgroup param!",
-            false);
+        Assert.fail("Unexpected IOException trying to read cgroup param!");
       }
 
-      Assert.assertEquals(paramValue, cGroupsHandler.getCGroupParam
-          (controller, testCGroup, param));
+      Assert.assertEquals(paramValue,
+          cGroupsHandler.getCGroupParam(controller, testCGroup, param));
 
       //We can't really do a delete test here. Linux cgroups
       //implementation provides additional semantics - the cgroup cannot be
@@ -222,10 +224,77 @@ public class TestCGroupsHandlerImpl {
       //delete is not possible with a regular non-empty directory.
     } catch (ResourceHandlerException e) {
       LOG.error("Caught exception: " + e);
-      Assert.assertTrue(
-          "Unexpected ResourceHandlerException during cgroup operations!",
-          false);
+      Assert
+        .fail("Unexpected ResourceHandlerException during cgroup operations!");
+    }
+  }
+
+  public static File createMockCgroupMount(File parentDir, String type)
+      throws IOException {
+    return createMockCgroupMount(parentDir, type, "hadoop-yarn");
+  }
+
+  public static File createMockCgroupMount(File parentDir, String type,
+      String hierarchy) throws IOException {
+    File cgroupMountDir =
+        new File(parentDir.getAbsolutePath(), type + "/" + hierarchy);
+    FileUtils.deleteQuietly(cgroupMountDir);
+    if (!cgroupMountDir.mkdirs()) {
+      String message =
+          "Could not create dir " + cgroupMountDir.getAbsolutePath();
+      throw new IOException(message);
     }
+    return cgroupMountDir;
+  }
+
+  public static File createMockMTab(File parentDir) throws IOException {
+    String cpuMtabContent =
+        "none " + parentDir.getAbsolutePath()
+            + "/cpu cgroup rw,relatime,cpu 0 0\n";
+    String blkioMtabContent =
+        "none " + parentDir.getAbsolutePath()
+            + "/blkio cgroup rw,relatime,blkio 0 0\n";
+
+    File mockMtab = new File(parentDir, UUID.randomUUID().toString());
+    if (!mockMtab.exists()) {
+      if (!mockMtab.createNewFile()) {
+        String message = "Could not create file " + mockMtab.getAbsolutePath();
+        throw new IOException(message);
+      }
+    }
+    FileWriter mtabWriter = new FileWriter(mockMtab.getAbsoluteFile());
+    mtabWriter.write(cpuMtabContent);
+    mtabWriter.write(blkioMtabContent);
+    mtabWriter.close();
+    mockMtab.deleteOnExit();
+    return mockMtab;
+  }
+
+
+  @Test
+  public void testMtabParsing() throws Exception {
+    File parentDir = new File(tmpPath);
+    // create mock cgroup
+    File cpuCgroupMountDir = createMockCgroupMount(parentDir, "cpu",
+        hierarchy);
+    Assert.assertTrue(cpuCgroupMountDir.exists());
+    File blkioCgroupMountDir = createMockCgroupMount(parentDir,
+        "blkio", hierarchy);
+    Assert.assertTrue(blkioCgroupMountDir.exists());
+    File mockMtabFile = createMockMTab(parentDir);
+    Map<CGroupsHandler.CGroupController, String> controllerPaths =
+        CGroupsHandlerImpl.initializeControllerPathsFromMtab(
+          mockMtabFile.getAbsolutePath(), hierarchy);
+    Assert.assertEquals(2, controllerPaths.size());
+    Assert.assertTrue(controllerPaths
+        .containsKey(CGroupsHandler.CGroupController.CPU));
+    Assert.assertTrue(controllerPaths
+        .containsKey(CGroupsHandler.CGroupController.BLKIO));
+    String cpuDir = controllerPaths.get(CGroupsHandler.CGroupController.CPU);
+    String blkioDir =
+        controllerPaths.get(CGroupsHandler.CGroupController.BLKIO);
+    Assert.assertEquals(parentDir.getAbsolutePath() + "/cpu", cpuDir);
+    Assert.assertEquals(parentDir.getAbsolutePath() + "/blkio", blkioDir);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2a548a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestResourceHandlerModule.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/TestResourceHandlerModule.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/TestResourceHandlerModule.java
index 939dfe7..69479d1 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/TestResourceHandlerModule.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/TestResourceHandlerModule.java
@@ -37,7 +37,7 @@ public class TestResourceHandlerModule {
   Configuration networkEnabledConf;
 
   @Before
-  public void setup() {
+  public void setup() throws Exception {
     emptyConf = new YarnConfiguration();
     networkEnabledConf = new YarnConfiguration();
 
@@ -46,6 +46,7 @@ public class TestResourceHandlerModule {
     //We need to bypass mtab parsing for figuring out cgroups mount locations
     networkEnabledConf.setBoolean(YarnConfiguration
         .NM_LINUX_CONTAINER_CGROUPS_MOUNT, true);
+    ResourceHandlerModule.nullifyResourceHandlerChain();
   }
 
   @Test
@@ -75,4 +76,27 @@ public class TestResourceHandlerModule {
       Assert.fail("Unexpected ResourceHandlerException: " + e);
     }
   }
+
+  @Test
+  public void testDiskResourceHandler() throws Exception {
+
+    DiskResourceHandler handler =
+        ResourceHandlerModule.getDiskResourceHandler(emptyConf);
+    Assert.assertNull(handler);
+
+    Configuration diskConf = new YarnConfiguration();
+    diskConf.setBoolean(YarnConfiguration.NM_DISK_RESOURCE_ENABLED, true);
+
+    handler = ResourceHandlerModule.getDiskResourceHandler(diskConf);
+    Assert.assertNotNull(handler);
+
+    ResourceHandlerChain resourceHandlerChain =
+        ResourceHandlerModule.getConfiguredResourceHandlerChain(diskConf);
+    List<ResourceHandler> resourceHandlers =
+        resourceHandlerChain.getResourceHandlerList();
+    // Exactly one resource handler in chain
+    Assert.assertEquals(resourceHandlers.size(), 1);
+    // Same instance is expected to be in the chain.
+    Assert.assertTrue(resourceHandlers.get(0) == handler);
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2a548a5/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 4e35169..8e9d787 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
@@ -21,6 +21,7 @@ import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.nodemanager.LinuxContainerExecutor;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.TestCGroupsHandlerImpl;
 import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
 import org.junit.Assert;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -33,7 +34,6 @@ import org.mockito.Mockito;
 import java.io.*;
 import java.util.List;
 import java.util.Scanner;
-import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
 
 public class TestCgroupsLCEResourcesHandler {
@@ -142,7 +142,7 @@ public class TestCgroupsLCEResourcesHandler {
 
     @Override
     int[] getOverallLimits(float x) {
-      if (generateLimitsMode == true) {
+      if (generateLimitsMode) {
         return super.getOverallLimits(x);
       }
       return limits;
@@ -172,10 +172,11 @@ public class TestCgroupsLCEResourcesHandler {
     handler.initConfig();
 
     // create mock cgroup
-    File cgroupMountDir = createMockCgroupMount(cgroupDir);
+    File cpuCgroupMountDir = TestCGroupsHandlerImpl.createMockCgroupMount(
+        cgroupDir, "cpu");
 
     // create mock mtab
-    File mockMtab = createMockMTab(cgroupDir);
+    File mockMtab = TestCGroupsHandlerImpl.createMockMTab(cgroupDir);
 
     // setup our handler and call init()
     handler.setMtabFile(mockMtab.getAbsolutePath());
@@ -184,8 +185,8 @@ public class TestCgroupsLCEResourcesHandler {
     // in this case, we're using all cpu so the files
     // shouldn't exist(because init won't create them
     handler.init(mockLCE, plugin);
-    File periodFile = new File(cgroupMountDir, "cpu.cfs_period_us");
-    File quotaFile = new File(cgroupMountDir, "cpu.cfs_quota_us");
+    File periodFile = new File(cpuCgroupMountDir, "cpu.cfs_period_us");
+    File quotaFile = new File(cpuCgroupMountDir, "cpu.cfs_quota_us");
     Assert.assertFalse(periodFile.exists());
     Assert.assertFalse(quotaFile.exists());
 
@@ -202,7 +203,7 @@ public class TestCgroupsLCEResourcesHandler {
 
     // set cpu back to 100, quota should be -1
     conf.setInt(YarnConfiguration.NM_RESOURCE_PERCENTAGE_PHYSICAL_CPU_LIMIT,
-      100);
+        100);
     handler.limits[0] = 100 * 1000;
     handler.limits[1] = 1000 * 1000;
     handler.init(mockLCE, plugin);
@@ -235,7 +236,7 @@ public class TestCgroupsLCEResourcesHandler {
     Assert.assertEquals(expectedQuota, ret[0]);
     Assert.assertEquals(-1, ret[1]);
 
-    int[] params = { 0, -1 };
+    int[] params = {0, -1};
     for (int cores : params) {
       try {
         handler.getOverallLimits(cores);
@@ -251,34 +252,6 @@ public class TestCgroupsLCEResourcesHandler {
     Assert.assertEquals(-1, ret[1]);
   }
 
-  private File createMockCgroupMount(File cgroupDir) throws IOException {
-    File cgroupMountDir = new File(cgroupDir.getAbsolutePath(), "hadoop-yarn");
-    FileUtils.deleteQuietly(cgroupDir);
-    if (!cgroupMountDir.mkdirs()) {
-      String message =
-          "Could not create dir " + cgroupMountDir.getAbsolutePath();
-      throw new IOException(message);
-    }
-    return cgroupMountDir;
-  }
-
-  private File createMockMTab(File cgroupDir) throws IOException {
-    String mtabContent =
-        "none " + cgroupDir.getAbsolutePath() + " cgroup rw,relatime,cpu 0 0";
-    File mockMtab = new File("target", UUID.randomUUID().toString());
-    if (!mockMtab.exists()) {
-      if (!mockMtab.createNewFile()) {
-        String message = "Could not create file " + mockMtab.getAbsolutePath();
-        throw new IOException(message);
-      }
-    }
-    FileWriter mtabWriter = new FileWriter(mockMtab.getAbsoluteFile());
-    mtabWriter.write(mtabContent);
-    mtabWriter.close();
-    mockMtab.deleteOnExit();
-    return mockMtab;
-  }
-
   @Test
   public void testContainerLimits() throws IOException {
     LinuxContainerExecutor mockLCE = new MockLinuxContainerExecutor();
@@ -286,6 +259,7 @@ public class TestCgroupsLCEResourcesHandler {
         new CustomCgroupsLCEResourceHandler();
     handler.generateLimitsMode = true;
     YarnConfiguration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.NM_DISK_RESOURCE_ENABLED, true);
     final int numProcessors = 4;
     ResourceCalculatorPlugin plugin =
         Mockito.mock(ResourceCalculatorPlugin.class);
@@ -294,71 +268,77 @@ public class TestCgroupsLCEResourcesHandler {
     handler.initConfig();
 
     // create mock cgroup
-    File cgroupMountDir = createMockCgroupMount(cgroupDir);
+    File cpuCgroupMountDir = TestCGroupsHandlerImpl.createMockCgroupMount(
+        cgroupDir, "cpu");
 
     // create mock mtab
-    File mockMtab = createMockMTab(cgroupDir);
+    File mockMtab = TestCGroupsHandlerImpl.createMockMTab(cgroupDir);
 
     // setup our handler and call init()
     handler.setMtabFile(mockMtab.getAbsolutePath());
     handler.init(mockLCE, plugin);
 
-    // check values
-    // default case - files shouldn't exist, strict mode off by default
+    // check the controller paths map isn't empty
     ContainerId id = ContainerId.fromString("container_1_1_1_1");
     handler.preExecute(id, Resource.newInstance(1024, 1));
-    File containerDir = new File(cgroupMountDir, id.toString());
-    Assert.assertTrue(containerDir.exists());
-    Assert.assertTrue(containerDir.isDirectory());
-    File periodFile = new File(containerDir, "cpu.cfs_period_us");
-    File quotaFile = new File(containerDir, "cpu.cfs_quota_us");
+    Assert.assertNotNull(handler.getControllerPaths());
+    // check values
+    // default case - files shouldn't exist, strict mode off by default
+    File containerCpuDir = new File(cpuCgroupMountDir, id.toString());
+    Assert.assertTrue(containerCpuDir.exists());
+    Assert.assertTrue(containerCpuDir.isDirectory());
+    File periodFile = new File(containerCpuDir, "cpu.cfs_period_us");
+    File quotaFile = new File(containerCpuDir, "cpu.cfs_quota_us");
     Assert.assertFalse(periodFile.exists());
     Assert.assertFalse(quotaFile.exists());
 
     // no files created because we're using all cpu
-    FileUtils.deleteQuietly(containerDir);
+    FileUtils.deleteQuietly(containerCpuDir);
     conf.setBoolean(
-      YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_STRICT_RESOURCE_USAGE, true);
+        YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_STRICT_RESOURCE_USAGE,
+        true);
     handler.initConfig();
     handler.preExecute(id,
-      Resource.newInstance(1024, YarnConfiguration.DEFAULT_NM_VCORES));
-    Assert.assertTrue(containerDir.exists());
-    Assert.assertTrue(containerDir.isDirectory());
-    periodFile = new File(containerDir, "cpu.cfs_period_us");
-    quotaFile = new File(containerDir, "cpu.cfs_quota_us");
+        Resource.newInstance(1024, YarnConfiguration.DEFAULT_NM_VCORES));
+    Assert.assertTrue(containerCpuDir.exists());
+    Assert.assertTrue(containerCpuDir.isDirectory());
+    periodFile = new File(containerCpuDir, "cpu.cfs_period_us");
+    quotaFile = new File(containerCpuDir, "cpu.cfs_quota_us");
     Assert.assertFalse(periodFile.exists());
     Assert.assertFalse(quotaFile.exists());
 
     // 50% of CPU
-    FileUtils.deleteQuietly(containerDir);
+    FileUtils.deleteQuietly(containerCpuDir);
     conf.setBoolean(
-      YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_STRICT_RESOURCE_USAGE, true);
+        YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_STRICT_RESOURCE_USAGE,
+        true);
     handler.initConfig();
     handler.preExecute(id,
-      Resource.newInstance(1024, YarnConfiguration.DEFAULT_NM_VCORES / 2));
-    Assert.assertTrue(containerDir.exists());
-    Assert.assertTrue(containerDir.isDirectory());
-    periodFile = new File(containerDir, "cpu.cfs_period_us");
-    quotaFile = new File(containerDir, "cpu.cfs_quota_us");
+        Resource.newInstance(1024, YarnConfiguration.DEFAULT_NM_VCORES / 2));
+    Assert.assertTrue(containerCpuDir.exists());
+    Assert.assertTrue(containerCpuDir.isDirectory());
+    periodFile = new File(containerCpuDir, "cpu.cfs_period_us");
+    quotaFile = new File(containerCpuDir, "cpu.cfs_quota_us");
     Assert.assertTrue(periodFile.exists());
     Assert.assertTrue(quotaFile.exists());
     Assert.assertEquals(500 * 1000, readIntFromFile(periodFile));
     Assert.assertEquals(1000 * 1000, readIntFromFile(quotaFile));
 
     // CGroups set to 50% of CPU, container set to 50% of YARN CPU
-    FileUtils.deleteQuietly(containerDir);
+    FileUtils.deleteQuietly(containerCpuDir);
     conf.setBoolean(
-      YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_STRICT_RESOURCE_USAGE, true);
+        YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_STRICT_RESOURCE_USAGE,
+        true);
     conf
       .setInt(YarnConfiguration.NM_RESOURCE_PERCENTAGE_PHYSICAL_CPU_LIMIT, 50);
     handler.initConfig();
     handler.init(mockLCE, plugin);
     handler.preExecute(id,
-      Resource.newInstance(1024, YarnConfiguration.DEFAULT_NM_VCORES / 2));
-    Assert.assertTrue(containerDir.exists());
-    Assert.assertTrue(containerDir.isDirectory());
-    periodFile = new File(containerDir, "cpu.cfs_period_us");
-    quotaFile = new File(containerDir, "cpu.cfs_quota_us");
+        Resource.newInstance(1024, YarnConfiguration.DEFAULT_NM_VCORES / 2));
+    Assert.assertTrue(containerCpuDir.exists());
+    Assert.assertTrue(containerCpuDir.isDirectory());
+    periodFile = new File(containerCpuDir, "cpu.cfs_period_us");
+    quotaFile = new File(containerCpuDir, "cpu.cfs_quota_us");
     Assert.assertTrue(periodFile.exists());
     Assert.assertTrue(quotaFile.exists());
     Assert.assertEquals(1000 * 1000, readIntFromFile(periodFile));


[02/50] [abbrv] hadoop git commit: MAPREDUCE-6343. JobConf.parseMaximumHeapSizeMB() fails to parse value greater than 2GB expressed in bytes. (Hao Xia via kasha)

Posted by zj...@apache.org.
MAPREDUCE-6343. JobConf.parseMaximumHeapSizeMB() fails to parse value greater than 2GB expressed in bytes. (Hao Xia via kasha)


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

Branch: refs/heads/YARN-2928
Commit: 0835aaf08238da3b0a6da2e792483f919ff73083
Parents: b42a6c5
Author: Karthik Kambatla <ka...@apache.org>
Authored: Tue Apr 28 14:05:26 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:52 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                |  3 +++
 .../main/java/org/apache/hadoop/mapred/JobConf.java | 10 +++++-----
 .../java/org/apache/hadoop/mapred/TestJobConf.java  | 16 ++++++++++++++++
 3 files changed, 24 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0835aaf0/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index f8e713d..57f6678 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -206,6 +206,9 @@ Trunk (Unreleased)
     MAPREDUCE-6234. TestHighRamJob fails due to the change in MAPREDUCE-5785. 
     (Masatake Iwasaki via kasha)
 
+    MAPREDUCE-6343. JobConf.parseMaximumHeapSizeMB() fails to parse value 
+    greater than 2GB expressed in bytes. (Hao Xia via kasha)
+
   BREAKDOWN OF MAPREDUCE-2841 (NATIVE TASK) SUBTASKS
 
     MAPREDUCE-5985. native-task: Fix build on macosx. Contributed by

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0835aaf0/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
index 9cac685..059593a 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
@@ -2080,28 +2080,28 @@ public class JobConf extends Configuration {
     // Find the last matching -Xmx following word boundaries
     Matcher m = JAVA_OPTS_XMX_PATTERN.matcher(javaOpts);
     if (m.matches()) {
-      int size = Integer.parseInt(m.group(1));
+      long size = Long.parseLong(m.group(1));
       if (size <= 0) {
         return -1;
       }
       if (m.group(2).isEmpty()) {
         // -Xmx specified in bytes
-        return size / (1024 * 1024);
+        return (int) (size / (1024 * 1024));
       }
       char unit = m.group(2).charAt(0);
       switch (unit) {
         case 'g':
         case 'G':
           // -Xmx specified in GB
-          return size * 1024;
+          return (int) (size * 1024);
         case 'm':
         case 'M':
           // -Xmx specified in MB
-          return size;
+          return (int) size;
         case 'k':
         case 'K':
           // -Xmx specified in KB
-          return size / 1024;
+          return (int) (size / 1024);
       }
     }
     // -Xmx not specified

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0835aaf0/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobConf.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobConf.java
index 0612ade..57fa4e6 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobConf.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobConf.java
@@ -361,4 +361,20 @@ public class TestJobConf {
       jobConf.getMaxTaskFailuresPerTracker() < jobConf.getMaxReduceAttempts()
       );
   }
+
+  /**
+   * Test parsing various types of Java heap options.
+   */
+  @Test
+  public void testParseMaximumHeapSizeMB() {
+    // happy cases
+    Assert.assertEquals(4096, JobConf.parseMaximumHeapSizeMB("-Xmx4294967296"));
+    Assert.assertEquals(4096, JobConf.parseMaximumHeapSizeMB("-Xmx4194304k"));
+    Assert.assertEquals(4096, JobConf.parseMaximumHeapSizeMB("-Xmx4096m"));
+    Assert.assertEquals(4096, JobConf.parseMaximumHeapSizeMB("-Xmx4g"));
+
+    // sad cases
+    Assert.assertEquals(-1, JobConf.parseMaximumHeapSizeMB("-Xmx4?"));
+    Assert.assertEquals(-1, JobConf.parseMaximumHeapSizeMB(""));
+  }
 }


[19/50] [abbrv] hadoop git commit: HDFS-7281. Missing block is marked as corrupted block (Ming Ma via Yongjun Zhang)

Posted by zj...@apache.org.
HDFS-7281. Missing block is marked as corrupted block (Ming Ma via Yongjun Zhang)


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

Branch: refs/heads/YARN-2928
Commit: 78bfaccefee55567a5b796d70e96c58b3a98c0da
Parents: f2a548a
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Fri May 1 08:42:00 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:56 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../server/blockmanagement/BlockManager.java    |  3 +-
 .../hdfs/server/namenode/NamenodeFsck.java      | 54 +++++++++++++++-----
 .../hadoop/hdfs/server/namenode/TestFsck.java   | 23 ++++++---
 4 files changed, 63 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/78bfacce/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 3bee852..9accdc0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -20,6 +20,9 @@ Trunk (Unreleased)
 
     HDFS-7985. WebHDFS should be always enabled. (Li Lu via wheat9)
 
+    HDFS-7281. Missing block is marked as corrupted block (Ming Ma via
+    Yongjun Zhang)
+ 
   NEW FEATURES
 
     HDFS-3125. Add JournalService to enable Journal Daemon. (suresh)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78bfacce/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 1db1356..53ffe0b 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
@@ -849,7 +849,8 @@ public class BlockManager {
     }
 
     final int numNodes = blocksMap.numNodes(blk);
-    final boolean isCorrupt = numCorruptNodes == numNodes;
+    final boolean isCorrupt = numCorruptNodes != 0 &&
+        numCorruptNodes == numNodes;
     final int numMachines = isCorrupt ? numNodes: numNodes - numCorruptNodes;
     final DatanodeStorageInfo[] machines = new DatanodeStorageInfo[numMachines];
     int j = 0;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78bfacce/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index 0cfe31a..ac77394 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -531,6 +531,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
     int missing = 0;
     int corrupt = 0;
     long missize = 0;
+    long corruptSize = 0;
     int underReplicatedPerFile = 0;
     int misReplicatedPerFile = 0;
     StringBuilder report = new StringBuilder();
@@ -570,10 +571,11 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       // count corrupt blocks
       boolean isCorrupt = lBlk.isCorrupt();
       if (isCorrupt) {
+        res.addCorrupt(block.getNumBytes());
         corrupt++;
-        res.corruptBlocks++;
-        out.print("\n" + path + ": CORRUPT blockpool " + block.getBlockPoolId() + 
-            " block " + block.getBlockName()+"\n");
+        corruptSize += block.getNumBytes();
+        out.print("\n" + path + ": CORRUPT blockpool " +
+            block.getBlockPoolId() + " block " + block.getBlockName() + "\n");
       }
 
       // count minimally replicated blocks
@@ -619,7 +621,11 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       // report
       String blkName = block.toString();
       report.append(blockNumber + ". " + blkName + " len=" + block.getNumBytes());
-      if (totalReplicasPerBlock == 0) {
+      if (totalReplicasPerBlock == 0 && !isCorrupt) {
+        // If the block is corrupted, it means all its available replicas are
+        // corrupted. We don't mark it as missing given these available replicas
+        // might still be accessible as the block might be incorrectly marked as
+        // corrupted by client machines.
         report.append(" MISSING!");
         res.addMissing(block.toString(), block.getNumBytes());
         missing++;
@@ -674,9 +680,15 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
 
     // count corrupt file & move or delete if necessary
     if ((missing > 0) || (corrupt > 0)) {
-      if (!showFiles && (missing > 0)) {
-        out.print("\n" + path + ": MISSING " + missing
-            + " blocks of total size " + missize + " B.");
+      if (!showFiles) {
+        if (missing > 0) {
+          out.print("\n" + path + ": MISSING " + missing
+              + " blocks of total size " + missize + " B.");
+        }
+        if (corrupt > 0) {
+          out.print("\n" + path + ": CORRUPT " + corrupt
+              + " blocks of total size " + corruptSize + " B.");
+        }
       }
       res.corruptFiles++;
       if (isOpen) {
@@ -688,9 +700,16 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
     }
 
     if (showFiles) {
-      if (missing > 0) {
-        out.print(" MISSING " + missing + " blocks of total size " + missize + " B\n");
-      }  else if (underReplicatedPerFile == 0 && misReplicatedPerFile == 0) {
+      if (missing > 0 || corrupt > 0) {
+        if (missing > 0) {
+          out.print(" MISSING " + missing + " blocks of total size " +
+              missize + " B\n");
+        }
+        if (corrupt > 0) {
+          out.print(" CORRUPT " + corrupt + " blocks of total size " +
+              corruptSize + " B\n");
+        }
+      } else if (underReplicatedPerFile == 0 && misReplicatedPerFile == 0) {
         out.print(" OK\n");
       }
       if (showBlocks) {
@@ -956,6 +975,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
     long missingSize = 0L;
     long corruptFiles = 0L;
     long corruptBlocks = 0L;
+    long corruptSize = 0L;
     long excessiveReplicas = 0L;
     long missingReplicas = 0L;
     long decommissionedReplicas = 0L;
@@ -998,7 +1018,13 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       missingIds.add(id);
       missingSize += size;
     }
-    
+
+    /** Add a corrupt block. */
+    void addCorrupt(long size) {
+      corruptBlocks++;
+      corruptSize += size;
+    }
+
     /** Return the actual replication factor. */
     float getReplicationFactor() {
       if (totalBlocks == 0)
@@ -1051,7 +1077,8 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
                 "\n  MISSING SIZE:\t\t").append(missingSize).append(" B");
           }
           if (corruptBlocks > 0) {
-            res.append("\n  CORRUPT BLOCKS: \t").append(corruptBlocks);
+            res.append("\n  CORRUPT BLOCKS: \t").append(corruptBlocks).append(
+                "\n  CORRUPT SIZE:\t\t").append(corruptSize).append(" B");
           }
         }
         res.append("\n  ********************************");
@@ -1086,7 +1113,8 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       }
       res.append("\n Default replication factor:\t").append(replication)
           .append("\n Average block replication:\t").append(
-              getReplicationFactor()).append("\n Corrupt blocks:\t\t").append(
+              getReplicationFactor()).append("\n Missing blocks:\t\t").append(
+              missingIds.size()).append("\n Corrupt blocks:\t\t").append(
               corruptBlocks).append("\n Missing replicas:\t\t").append(
               missingReplicas);
       if (totalReplicas > 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78bfacce/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 8fe273b..1ce09e1 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
@@ -120,7 +120,10 @@ public class TestFsck {
       "ip=/\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\s" + 
       "cmd=getfileinfo\\ssrc=\\/\\sdst=null\\s" + 
       "perm=null\\s" + "proto=.*");
-  
+
+  static final Pattern numMissingBlocksPattern = Pattern.compile(
+      ".*Missing blocks:\t\t([0123456789]*).*");
+
   static final Pattern numCorruptBlocksPattern = Pattern.compile(
       ".*Corrupt blocks:\t\t([0123456789]*).*");
   
@@ -360,19 +363,27 @@ public class TestFsck {
       // Wait for fsck to discover all the missing blocks
       while (true) {
         outStr = runFsck(conf, 1, false, "/");
+        String numMissing = null;
         String numCorrupt = null;
         for (String line : outStr.split(LINE_SEPARATOR)) {
-          Matcher m = numCorruptBlocksPattern.matcher(line);
+          Matcher m = numMissingBlocksPattern.matcher(line);
+          if (m.matches()) {
+            numMissing = m.group(1);
+          }
+          m = numCorruptBlocksPattern.matcher(line);
           if (m.matches()) {
             numCorrupt = m.group(1);
+          }
+          if (numMissing != null && numCorrupt != null) {
             break;
           }
         }
-        if (numCorrupt == null) {
-          throw new IOException("failed to find number of corrupt " +
-              "blocks in fsck output.");
+        if (numMissing == null || numCorrupt == null) {
+          throw new IOException("failed to find number of missing or corrupt" +
+              " blocks in fsck output.");
         }
-        if (numCorrupt.equals(Integer.toString(totalMissingBlocks))) {
+        if (numMissing.equals(Integer.toString(totalMissingBlocks))) {
+          assertTrue(numCorrupt.equals(Integer.toString(0)));
           assertTrue(outStr.contains(NamenodeFsck.CORRUPT_STATUS));
           break;
         }


[31/50] [abbrv] hadoop git commit: YARN-3363. add localization and container launch time to ContainerMetrics at NM to show these timing information for each active container. (zxu via rkanter)

Posted by zj...@apache.org.
YARN-3363. add localization and container launch time to ContainerMetrics at NM to show these timing information for each active container. (zxu via rkanter)


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

Branch: refs/heads/YARN-2928
Commit: c1c22df00851e1bee985de0fcf8ce7e609a58bb8
Parents: cb5ae084
Author: Robert Kanter <rk...@apache.org>
Authored: Fri May 1 16:39:21 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:58 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  4 +++
 .../container/ContainerImpl.java                | 29 ++++++++++++--------
 .../monitor/ContainerMetrics.java               | 20 ++++++++++++++
 .../monitor/ContainerStartMonitoringEvent.java  | 15 +++++++++-
 .../monitor/ContainersMonitorImpl.java          |  9 ++++++
 .../monitor/TestContainerMetrics.java           | 10 +++++++
 6 files changed, 74 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c1c22df0/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 8dff48d..a66024a 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -241,6 +241,10 @@ Release 2.8.0 - UNRELEASED
     YARN-3406. Display count of running containers in the RM's Web UI.
     (Ryu Kobayashi via ozawa)
 
+    YARN-3363. add localization and container launch time to ContainerMetrics
+    at NM to show these timing information for each active container.
+    (zxu via rkanter)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c1c22df0/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 68669aa..9997ca2 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
@@ -96,6 +96,7 @@ public class ContainerImpl implements Container {
   private int exitCode = ContainerExitStatus.INVALID;
   private final StringBuilder diagnostics;
   private boolean wasLaunched;
+  private long containerLocalizationStartTime;
   private long containerLaunchStartTime;
   private static Clock clock = new SystemClock();
 
@@ -493,16 +494,21 @@ public class ContainerImpl implements Container {
   // resource usage.
   @SuppressWarnings("unchecked") // dispatcher not typed
   private void sendContainerMonitorStartEvent() {
-      long pmemBytes = getResource().getMemory() * 1024 * 1024L;
-      float pmemRatio = daemonConf.getFloat(
-          YarnConfiguration.NM_VMEM_PMEM_RATIO,
-          YarnConfiguration.DEFAULT_NM_VMEM_PMEM_RATIO);
-      long vmemBytes = (long) (pmemRatio * pmemBytes);
-      int cpuVcores = getResource().getVirtualCores();
-
-      dispatcher.getEventHandler().handle(
-          new ContainerStartMonitoringEvent(containerId,
-              vmemBytes, pmemBytes, cpuVcores));
+    long launchDuration = clock.getTime() - containerLaunchStartTime;
+    metrics.addContainerLaunchDuration(launchDuration);
+
+    long pmemBytes = getResource().getMemory() * 1024 * 1024L;
+    float pmemRatio = daemonConf.getFloat(
+        YarnConfiguration.NM_VMEM_PMEM_RATIO,
+        YarnConfiguration.DEFAULT_NM_VMEM_PMEM_RATIO);
+    long vmemBytes = (long) (pmemRatio * pmemBytes);
+    int cpuVcores = getResource().getVirtualCores();
+    long localizationDuration = containerLaunchStartTime -
+        containerLocalizationStartTime;
+    dispatcher.getEventHandler().handle(
+        new ContainerStartMonitoringEvent(containerId,
+        vmemBytes, pmemBytes, cpuVcores, launchDuration,
+        localizationDuration));
   }
 
   private void addDiagnostics(String... diags) {
@@ -601,6 +607,7 @@ public class ContainerImpl implements Container {
         }
       }
 
+      container.containerLocalizationStartTime = clock.getTime();
       // Send requests for public, private resources
       Map<String,LocalResource> cntrRsrc = ctxt.getLocalResources();
       if (!cntrRsrc.isEmpty()) {
@@ -756,8 +763,6 @@ public class ContainerImpl implements Container {
       container.sendContainerMonitorStartEvent();
       container.metrics.runningContainer();
       container.wasLaunched  = true;
-      long duration = clock.getTime() - container.containerLaunchStartTime;
-      container.metrics.addContainerLaunchDuration(duration);
 
       if (container.recoveredAsKilled) {
         LOG.info("Killing " + container.containerId

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c1c22df0/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 ffa72a4..c364143 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
@@ -28,6 +28,7 @@ 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.MutableGaugeInt;
+import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
 import org.apache.hadoop.metrics2.lib.MutableStat;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 
@@ -46,6 +47,9 @@ public class ContainerMetrics implements MetricsSource {
   public static final String VMEM_LIMIT_METRIC_NAME = "vMemLimitMBs";
   public static final String VCORE_LIMIT_METRIC_NAME = "vCoreLimit";
   public static final String PMEM_USAGE_METRIC_NAME = "pMemUsageMBs";
+  public static final String LAUNCH_DURATION_METRIC_NAME = "launchDurationMs";
+  public static final String LOCALIZATION_DURATION_METRIC_NAME =
+      "localizationDurationMs";
   private static final String PHY_CPU_USAGE_METRIC_NAME = "pCpuUsagePercent";
 
   // Use a multiplier of 1000 to avoid losing too much precision when
@@ -74,6 +78,12 @@ public class ContainerMetrics implements MetricsSource {
   @Metric
   public MutableGaugeInt cpuVcoreLimit;
 
+  @Metric
+  public MutableGaugeLong launchDurationMs;
+
+  @Metric
+  public MutableGaugeLong localizationDurationMs;
+
   static final MetricsInfo RECORD_INFO =
       info("ContainerResource", "Resource limit and usage by container");
 
@@ -122,6 +132,10 @@ public class ContainerMetrics implements MetricsSource {
         VMEM_LIMIT_METRIC_NAME, "Virtual memory limit in MBs", 0);
     this.cpuVcoreLimit = registry.newGauge(
         VCORE_LIMIT_METRIC_NAME, "CPU limit in number of vcores", 0);
+    this.launchDurationMs = registry.newGauge(
+        LAUNCH_DURATION_METRIC_NAME, "Launch duration in MS", 0L);
+    this.localizationDurationMs = registry.newGauge(
+        LOCALIZATION_DURATION_METRIC_NAME, "Localization duration in MS", 0L);
   }
 
   ContainerMetrics tag(MetricsInfo info, ContainerId containerId) {
@@ -213,6 +227,12 @@ public class ContainerMetrics implements MetricsSource {
     this.cpuVcoreLimit.set(cpuVcores);
   }
 
+  public void recordStateChangeDurations(long launchDuration,
+      long localizationDuration) {
+    this.launchDurationMs.set(launchDuration);
+    this.localizationDurationMs.set(localizationDuration);
+  }
+
   private synchronized void scheduleTimerTaskIfRequired() {
     if (flushPeriodMs > 0) {
       // Lazily initialize timer

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c1c22df0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainerStartMonitoringEvent.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/ContainerStartMonitoringEvent.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/ContainerStartMonitoringEvent.java
index 56e2d8e..c09bebf 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/ContainerStartMonitoringEvent.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/ContainerStartMonitoringEvent.java
@@ -25,13 +25,18 @@ public class ContainerStartMonitoringEvent extends ContainersMonitorEvent {
   private final long vmemLimit;
   private final long pmemLimit;
   private final int cpuVcores;
+  private final long launchDuration;
+  private final long localizationDuration;
 
   public ContainerStartMonitoringEvent(ContainerId containerId,
-      long vmemLimit, long pmemLimit, int cpuVcores) {
+      long vmemLimit, long pmemLimit, int cpuVcores, long launchDuration,
+      long localizationDuration) {
     super(containerId, ContainersMonitorEventType.START_MONITORING_CONTAINER);
     this.vmemLimit = vmemLimit;
     this.pmemLimit = pmemLimit;
     this.cpuVcores = cpuVcores;
+    this.launchDuration = launchDuration;
+    this.localizationDuration = localizationDuration;
   }
 
   public long getVmemLimit() {
@@ -45,4 +50,12 @@ public class ContainerStartMonitoringEvent extends ContainersMonitorEvent {
   public int getCpuVcores() {
     return this.cpuVcores;
   }
+
+  public long getLaunchDuration() {
+    return this.launchDuration;
+  }
+
+  public long getLocalizationDuration() {
+    return this.localizationDuration;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c1c22df0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
----------------------------------------------------------------------
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/ContainersMonitorImpl.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/ContainersMonitorImpl.java
index bfd6b48..3af1839 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/ContainersMonitorImpl.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/ContainersMonitorImpl.java
@@ -729,6 +729,15 @@ public class ContainersMonitorImpl extends AbstractService implements
     case START_MONITORING_CONTAINER:
       ContainerStartMonitoringEvent startEvent =
           (ContainerStartMonitoringEvent) monitoringEvent;
+
+      if (containerMetricsEnabled) {
+        ContainerMetrics usageMetrics = ContainerMetrics
+            .forContainer(containerId, containerMetricsPeriodMs);
+        usageMetrics.recordStateChangeDurations(
+            startEvent.getLaunchDuration(),
+            startEvent.getLocalizationDuration());
+      }
+
       synchronized (this.containersToBeAdded) {
         ProcessTreeInfo processTreeInfo =
             new ProcessTreeInfo(containerId, null, null,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c1c22df0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainerMetrics.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/monitor/TestContainerMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainerMetrics.java
index c628648..bdf9994 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainerMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainerMetrics.java
@@ -87,10 +87,14 @@ public class TestContainerMetrics {
     int anyPmemLimit = 1024;
     int anyVmemLimit = 2048;
     int anyVcores = 10;
+    long anyLaunchDuration = 20L;
+    long anyLocalizationDuration = 1000L;
     String anyProcessId = "1234";
 
     metrics.recordResourceLimit(anyVmemLimit, anyPmemLimit, anyVcores);
     metrics.recordProcessId(anyProcessId);
+    metrics.recordStateChangeDurations(anyLaunchDuration,
+        anyLocalizationDuration);
 
     Thread.sleep(110);
     metrics.getMetrics(collector, true);
@@ -105,6 +109,12 @@ public class TestContainerMetrics {
     MetricsRecords.assertMetric(record, ContainerMetrics.VMEM_LIMIT_METRIC_NAME, anyVmemLimit);
     MetricsRecords.assertMetric(record, ContainerMetrics.VCORE_LIMIT_METRIC_NAME, anyVcores);
 
+    MetricsRecords.assertMetric(record,
+        ContainerMetrics.LAUNCH_DURATION_METRIC_NAME, anyLaunchDuration);
+    MetricsRecords.assertMetric(record,
+        ContainerMetrics.LOCALIZATION_DURATION_METRIC_NAME,
+        anyLocalizationDuration);
+
     collector.clear();
   }
 }


[11/50] [abbrv] hadoop git commit: MAPREDUCE-6339. Job history file is not flushed correctly because isTimerActive flag is not set true when flushTimerTask is scheduled. Contributed by zhihai xu.

Posted by zj...@apache.org.
MAPREDUCE-6339. Job history file is not flushed correctly because
isTimerActive flag is not set true when flushTimerTask is scheduled.
Contributed by zhihai xu.


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

Branch: refs/heads/YARN-2928
Commit: 26fee33c2cc3ad05c326d0c95a5aeffb992af1f9
Parents: 0953b5e
Author: Devaraj K <de...@apache.org>
Authored: Thu Apr 30 12:33:22 2015 +0530
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:54 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                          | 3 +++
 .../hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java   | 7 +++++++
 .../mapreduce/jobhistory/TestJobHistoryEventHandler.java      | 2 ++
 3 files changed, 12 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/26fee33c/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 57f6678..1acbca0 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -400,6 +400,9 @@ Release 2.7.1 - UNRELEASED
     MAPREDUCE-6334. Fetcher#copyMapOutput is leaking usedMemory upon
     IOException during InMemoryMapOutput shuffle handler (Eric Payne via jlowe)
 
+    MAPREDUCE-6339. Job history file is not flushed correctly because isTimerActive 
+    flag is not set true when flushTimerTask is scheduled. (zhihai xu via devaraj)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/26fee33c/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 6d72095..ce53537 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
@@ -75,6 +75,7 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
+import com.google.common.annotations.VisibleForTesting;
 /**
  * The job history events get routed to this class. This class writes the Job
  * history events to the DFS directly into a staging dir and then moved to a
@@ -1437,6 +1438,7 @@ public class JobHistoryEventHandler extends AbstractService
           if (!isTimerShutDown) {
             flushTimerTask = new FlushTimerTask(this);
             flushTimer.schedule(flushTimerTask, flushTimeout);
+            isTimerActive = true;
           }
         }
       }
@@ -1556,4 +1558,9 @@ public class JobHistoryEventHandler extends AbstractService
     }
     return JobState.KILLED.toString();
   }
+
+  @VisibleForTesting
+  boolean getFlushTimerStatus() {
+    return isTimerActive;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/26fee33c/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 7bbbb3e..ff756d2 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
@@ -224,10 +224,12 @@ public class TestJobHistoryEventHandler {
       }
 
       handleNextNEvents(jheh, 9);
+      Assert.assertTrue(jheh.getFlushTimerStatus());
       verify(mockWriter, times(0)).flush();
 
       Thread.sleep(2 * 4 * 1000l); // 4 seconds should be enough. Just be safe.
       verify(mockWriter).flush();
+      Assert.assertFalse(jheh.getFlushTimerStatus());
     } finally {
       jheh.stop();
       verify(mockWriter).close();


[40/50] [abbrv] hadoop git commit: HDFS-8249. Separate HdfsConstants into the client and the server side class. Contributed by Haohui Mai.

Posted by zj...@apache.org.
HDFS-8249. Separate HdfsConstants into the client and the server side class. Contributed by Haohui Mai.


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

Branch: refs/heads/YARN-2928
Commit: b125d0de0a90533f823898525acb515b9fd875d3
Parents: f5f6a68
Author: Haohui Mai <wh...@apache.org>
Authored: Fri May 1 15:27:28 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:59 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/HAUtilClient.java    |   2 +-
 .../org/apache/hadoop/hdfs/protocol/Block.java  |   4 +-
 .../hadoop/hdfs/protocol/HdfsConstants.java     | 109 ++++++++++++
 .../hdfs/protocol/HdfsConstantsClient.java      |  45 -----
 .../apache/hadoop/hdfs/web/JsonUtilClient.java  |   6 +-
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |   4 +-
 .../hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java    |   4 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../bkjournal/BookKeeperJournalManager.java     |  14 +-
 .../bkjournal/EditLogLedgerMetadata.java        |   8 +-
 .../bkjournal/TestBookKeeperEditLogStreams.java |   6 +-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |   8 +-
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |   8 +-
 .../org/apache/hadoop/hdfs/DataStreamer.java    |   5 +-
 .../org/apache/hadoop/hdfs/NameNodeProxies.java |   3 +-
 .../hadoop/hdfs/client/impl/LeaseRenewer.java   |   6 +-
 .../hadoop/hdfs/protocol/ClientProtocol.java    |   4 +-
 .../hadoop/hdfs/protocol/HdfsConstants.java     | 175 -------------------
 .../protocol/SnapshottableDirectoryStatus.java  |   2 +-
 .../protocolPB/ClientNamenodeProtocolPB.java    |   2 +-
 ...tNamenodeProtocolServerSideTranslatorPB.java |   4 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |   6 +-
 .../hdfs/qjournal/client/IPCLoggerChannel.java  |   4 +-
 .../hdfs/qjournal/protocol/RequestInfo.java     |   4 +-
 .../QJournalProtocolServerSideTranslatorPB.java |   4 +-
 .../hadoop/hdfs/qjournal/server/Journal.java    |  22 +--
 .../hadoop/hdfs/server/balancer/Dispatcher.java |   5 +-
 .../server/blockmanagement/BlockIdManager.java  |  10 +-
 .../BlockPlacementPolicyDefault.java            |   4 +-
 .../BlockStoragePolicySuite.java                |  19 +-
 .../hdfs/server/common/HdfsServerConstants.java | 109 +++++++++---
 .../hadoop/hdfs/server/common/StorageInfo.java  |   5 +-
 .../server/datanode/BlockMetadataHeader.java    |   4 +-
 .../server/datanode/BlockPoolSliceStorage.java  |  20 +--
 .../hdfs/server/datanode/BlockReceiver.java     |   4 +-
 .../hdfs/server/datanode/BlockSender.java       |   8 +-
 .../hadoop/hdfs/server/datanode/DataNode.java   |   2 +-
 .../hdfs/server/datanode/DataStorage.java       |  35 ++--
 .../hdfs/server/datanode/DataXceiver.java       |  16 +-
 .../hdfs/server/datanode/DirectoryScanner.java  |   4 +-
 .../datanode/fsdataset/impl/BlockPoolSlice.java |   4 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |   8 +-
 .../datanode/fsdataset/impl/FsDatasetUtil.java  |   4 +-
 .../apache/hadoop/hdfs/server/mover/Mover.java  |   2 +-
 .../hadoop/hdfs/server/namenode/BackupNode.java |   8 +-
 .../namenode/EditLogBackupInputStream.java      |   6 +-
 .../server/namenode/EditLogFileInputStream.java |  22 +--
 .../server/namenode/EditLogInputStream.java     |   4 +-
 .../hdfs/server/namenode/EditsDoubleBuffer.java |   6 +-
 .../server/namenode/FSDirStatAndListingOp.java  |  16 +-
 .../hdfs/server/namenode/FSDirectory.java       |   6 +-
 .../hadoop/hdfs/server/namenode/FSEditLog.java  |   8 +-
 .../hdfs/server/namenode/FSEditLogLoader.java   |  24 +--
 .../hdfs/server/namenode/FSEditLogOp.java       |  28 +--
 .../hadoop/hdfs/server/namenode/FSImage.java    |  18 +-
 .../hdfs/server/namenode/FSImageFormat.java     |   7 +-
 .../server/namenode/FSImageFormatProtobuf.java  |   8 +-
 ...FSImagePreTransactionalStorageInspector.java |   4 +-
 .../namenode/FSImageStorageInspector.java       |   4 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  19 +-
 .../server/namenode/FileJournalManager.java     |  22 +--
 .../hadoop/hdfs/server/namenode/INode.java      |  13 +-
 .../hdfs/server/namenode/INodeDirectory.java    |   2 +-
 .../hadoop/hdfs/server/namenode/INodeFile.java  |   2 +-
 .../hadoop/hdfs/server/namenode/INodeId.java    |   4 +-
 .../hadoop/hdfs/server/namenode/INodeMap.java   |   6 +-
 .../hdfs/server/namenode/INodesInPath.java      |   3 +-
 .../hdfs/server/namenode/LeaseManager.java      |   5 +-
 .../hadoop/hdfs/server/namenode/NNStorage.java  |  11 +-
 .../hadoop/hdfs/server/namenode/NameNode.java   |   2 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |  10 +-
 .../namenode/RedundantEditLogInputStream.java   |  12 +-
 .../hdfs/server/namenode/TransferFsImage.java   |   8 +-
 .../server/namenode/ha/BootstrapStandby.java    |   6 +-
 .../hdfs/server/namenode/ha/EditLogTailer.java  |   6 +-
 .../snapshot/FileWithSnapshotFeature.java       |   4 +-
 .../server/protocol/NNHAStatusHeartbeat.java    |   4 +-
 .../hdfs/server/protocol/NamespaceInfo.java     |   4 +-
 .../hdfs/server/protocol/RemoteEditLog.java     |  11 +-
 .../hadoop/hdfs/tools/StoragePolicyAdmin.java   |   4 +-
 .../offlineEditsViewer/OfflineEditsLoader.java  |   7 +-
 .../offlineImageViewer/ImageLoaderCurrent.java  |   4 +-
 .../org/apache/hadoop/fs/TestSymlinkHdfs.java   |   3 +-
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |   8 +-
 .../hadoop/hdfs/TestBlockStoragePolicy.java     |  39 +++--
 .../org/apache/hadoop/hdfs/TestDFSRollback.java |   6 +-
 .../hadoop/hdfs/TestDFSStartupVersions.java     |   8 +-
 .../org/apache/hadoop/hdfs/TestDFSUpgrade.java  |   4 +-
 .../hadoop/hdfs/TestDatanodeRegistration.java   |   9 +-
 .../org/apache/hadoop/hdfs/TestFileAppend4.java |   5 +-
 .../apache/hadoop/hdfs/TestFileCreation.java    |   5 +-
 .../org/apache/hadoop/hdfs/TestGetBlocks.java   |   4 +-
 .../java/org/apache/hadoop/hdfs/TestLease.java  |  11 +-
 .../apache/hadoop/hdfs/TestLeaseRecovery2.java  |   5 +-
 .../apache/hadoop/hdfs/UpgradeUtilities.java    |   4 +-
 .../hdfs/qjournal/server/TestJournalNode.java   |   4 +-
 .../blockmanagement/TestBlockManager.java       |   9 +-
 .../blockmanagement/TestReplicationPolicy.java  |  42 ++---
 .../TestReplicationPolicyConsiderLoad.java      |   6 +-
 .../TestReplicationPolicyWithNodeGroup.java     |  46 ++---
 .../server/datanode/TestDatanodeRegister.java   |   8 +-
 .../server/datanode/TestDirectoryScanner.java   |   6 +-
 .../hdfs/server/mover/TestStorageMover.java     |   8 +-
 .../server/namenode/NNThroughputBenchmark.java  |   7 +-
 .../hdfs/server/namenode/TestAddBlockRetry.java |  12 +-
 .../hdfs/server/namenode/TestEditLog.java       |   6 +-
 .../namenode/TestEditLogFileInputStream.java    |   4 +-
 .../server/namenode/TestFSEditLogLoader.java    |   6 +-
 .../namenode/TestFSPermissionChecker.java       |   6 +-
 .../hdfs/server/namenode/TestFileTruncate.java  |   5 +-
 .../hdfs/server/namenode/TestINodeFile.java     |  16 +-
 .../namenode/TestMetadataVersionOutput.java     |   4 +-
 .../namenode/TestNameNodeOptionParsing.java     |   6 +-
 .../namenode/TestTruncateQuotaUpdate.java       |   4 +-
 .../namenode/ha/TestDFSUpgradeWithHA.java       |   4 +-
 .../snapshot/TestOpenFilesWithSnapshot.java     |   4 +-
 .../server/namenode/snapshot/TestSnapshot.java  |   1 -
 .../apache/hadoop/hdfs/web/TestJsonUtil.java    |   4 +-
 118 files changed, 664 insertions(+), 728 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HAUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HAUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HAUtilClient.java
index 7956838..9f28cfc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HAUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HAUtilClient.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.security.token.Token;
 
 import java.net.URI;
 
-import static org.apache.hadoop.hdfs.protocol.HdfsConstantsClient.HA_DT_SERVICE_PREFIX;
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.HA_DT_SERVICE_PREFIX;
 
 @InterfaceAudience.Private
 public class HAUtilClient {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/Block.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/Block.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/Block.java
index 11daccc..0dac290 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/Block.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/Block.java
@@ -78,7 +78,7 @@ public class Block implements Writable, Comparable<Block> {
   public static long getGenerationStamp(String metaFile) {
     Matcher m = metaFilePattern.matcher(metaFile);
     return m.matches() ? Long.parseLong(m.group(2))
-        : HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP;
+        : HdfsConstants.GRANDFATHER_GENERATION_STAMP;
   }
 
   /**
@@ -100,7 +100,7 @@ public class Block implements Writable, Comparable<Block> {
   }
 
   public Block(final long blkid) {
-    this(blkid, 0, HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP);
+    this(blkid, 0, HdfsConstants.GRANDFATHER_GENERATION_STAMP);
   }
 
   public Block(Block blk) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/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
new file mode 100644
index 0000000..58c7ea1
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
@@ -0,0 +1,109 @@
+/**
+ * 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.protocol;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.StringUtils;
+
+@InterfaceAudience.Private
+public final class HdfsConstants {
+  // Long that indicates "leave current quota unchanged"
+  public static final long QUOTA_DONT_SET = Long.MAX_VALUE;
+  public static final long QUOTA_RESET = -1L;
+  public static final int BYTES_IN_INTEGER = Integer.SIZE / Byte.SIZE;
+  /**
+   * URI Scheme for hdfs://namenode/ URIs.
+   */
+  public static final String HDFS_URI_SCHEME = "hdfs";
+  public static final String MEMORY_STORAGE_POLICY_NAME = "LAZY_PERSIST";
+  public static final String ALLSSD_STORAGE_POLICY_NAME = "ALL_SSD";
+  public static final String ONESSD_STORAGE_POLICY_NAME = "ONE_SSD";
+  // TODO should be conf injected?
+  public static final int DEFAULT_DATA_SOCKET_SIZE = 128 * 1024;
+  /**
+   * A special path component contained in the path for a snapshot file/dir
+   */
+  public static final String DOT_SNAPSHOT_DIR = ".snapshot";
+  public static final String SEPARATOR_DOT_SNAPSHOT_DIR
+          = Path.SEPARATOR + DOT_SNAPSHOT_DIR;
+  public static final String SEPARATOR_DOT_SNAPSHOT_DIR_SEPARATOR
+      = Path.SEPARATOR + DOT_SNAPSHOT_DIR + Path.SEPARATOR;
+
+  /**
+   * Generation stamp of blocks that pre-date the introduction
+   * of a generation stamp.
+   */
+  public static final long GRANDFATHER_GENERATION_STAMP = 0;
+  /**
+   * The inode id validation of lease check will be skipped when the request
+   * uses GRANDFATHER_INODE_ID for backward compatibility.
+   */
+  public static final long GRANDFATHER_INODE_ID = 0;
+  public static final byte BLOCK_STORAGE_POLICY_ID_UNSPECIFIED = 0;
+  /**
+   * A prefix put before the namenode URI inside the "service" field
+   * of a delgation token, indicating that the URI is a logical (HA)
+   * URI.
+   */
+  public static final String HA_DT_SERVICE_PREFIX = "ha-";
+  // The name of the SafeModeException. FileSystem should retry if it sees
+  // the below exception in RPC
+  public static final String SAFEMODE_EXCEPTION_CLASS_NAME =
+      "org.apache.hadoop.hdfs.server.namenode.SafeModeException";
+  /**
+   * HDFS Protocol Names:
+   */
+  public static final String CLIENT_NAMENODE_PROTOCOL_NAME =
+      "org.apache.hadoop.hdfs.protocol.ClientProtocol";
+
+  // SafeMode actions
+  public enum SafeModeAction {
+    SAFEMODE_LEAVE, SAFEMODE_ENTER, SAFEMODE_GET
+  }
+
+  public enum RollingUpgradeAction {
+    QUERY, PREPARE, FINALIZE;
+
+    private static final Map<String, RollingUpgradeAction> MAP
+        = new HashMap<>();
+    static {
+      MAP.put("", QUERY);
+      for(RollingUpgradeAction a : values()) {
+        MAP.put(a.name(), a);
+      }
+    }
+
+    /** Covert the given String to a RollingUpgradeAction. */
+    public static RollingUpgradeAction fromString(String s) {
+      return MAP.get(StringUtils.toUpperCase(s));
+    }
+  }
+
+  // type of the datanode report
+  public enum DatanodeReportType {
+    ALL, LIVE, DEAD, DECOMMISSIONING
+  }
+
+  /* Hidden constructor */
+  protected HdfsConstants() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstantsClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstantsClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstantsClient.java
deleted file mode 100644
index 00f07e8..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstantsClient.java
+++ /dev/null
@@ -1,45 +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.protocol;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-@InterfaceAudience.Private
-public interface HdfsConstantsClient {
-  /**
-   * Generation stamp of blocks that pre-date the introduction
-   * of a generation stamp.
-   */
-  long GRANDFATHER_GENERATION_STAMP = 0;
-  /**
-   * The inode id validation of lease check will be skipped when the request
-   * uses GRANDFATHER_INODE_ID for backward compatibility.
-   */
-  long GRANDFATHER_INODE_ID = 0;
-  byte BLOCK_STORAGE_POLICY_ID_UNSPECIFIED = 0;
-  /**
-   * A prefix put before the namenode URI inside the "service" field
-   * of a delgation token, indicating that the URI is a logical (HA)
-   * URI.
-   */
-  String HA_DT_SERVICE_PREFIX = "ha-";
-  // The name of the SafeModeException. FileSystem should retry if it sees
-  // the below exception in RPC
-  String SAFEMODE_EXCEPTION_CLASS_NAME = "org.apache.hadoop.hdfs.server" +
-      ".namenode.SafeModeException";
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/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 e263a0a..ca94840 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
@@ -33,7 +33,7 @@ import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -123,11 +123,11 @@ class JsonUtilClient {
     final long blockSize = ((Number) m.get("blockSize")).longValue();
     final short replication = ((Number) m.get("replication")).shortValue();
     final long fileId = m.containsKey("fileId") ?
-        ((Number) m.get("fileId")).longValue() : HdfsConstantsClient.GRANDFATHER_INODE_ID;
+        ((Number) m.get("fileId")).longValue() : HdfsConstants.GRANDFATHER_INODE_ID;
     final int childrenNum = getInt(m, "childrenNum", -1);
     final byte storagePolicy = m.containsKey("storagePolicy") ?
         (byte) ((Number) m.get("storagePolicy")).longValue() :
-        HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
+        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),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/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 d28f571..aebd25a 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
@@ -59,7 +59,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HAUtilClient;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.web.resources.*;
@@ -171,7 +171,7 @@ public class WebHdfsFileSystem extends FileSystem
               HdfsClientConfigKeys.HttpClient.RETRY_POLICY_ENABLED_DEFAULT,
               HdfsClientConfigKeys.HttpClient.RETRY_POLICY_SPEC_KEY,
               HdfsClientConfigKeys.HttpClient.RETRY_POLICY_SPEC_DEFAULT,
-              HdfsConstantsClient.SAFEMODE_EXCEPTION_CLASS_NAME);
+              HdfsConstants.SAFEMODE_EXCEPTION_CLASS_NAME);
     } else {
 
       int maxFailoverAttempts = conf.getInt(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
index 161f3e9..05d806a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
@@ -44,8 +44,8 @@ import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
 import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.net.DNS;
@@ -2031,7 +2031,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       }
 
       return new PATHCONF3Response(Nfs3Status.NFS3_OK, attrs, 0,
-          HdfsConstants.MAX_PATH_LENGTH, true, false, false, true);
+          HdfsServerConstants.MAX_PATH_LENGTH, true, false, false, true);
     } catch (IOException e) {
       LOG.warn("Exception ", e);
       int status = mapErrorStatus(e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 1882df5..71873a4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -497,6 +497,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8086. Move LeaseRenewer to the hdfs.client.impl package.  (Takanobu
     Asanuma via szetszwo)
 
+    HDFS-8249. Separate HdfsConstants into the client and the server side
+    class. (wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
index 89fa84c..21fa4f2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.contrib.bkjournal;
 
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.JournalManager;
@@ -568,7 +568,7 @@ public class BookKeeperJournalManager implements JournalManager {
           return;
         }
         streams.add(elis);
-        if (elis.getLastTxId() == HdfsConstants.INVALID_TXID) {
+        if (elis.getLastTxId() == HdfsServerConstants.INVALID_TXID) {
           return;
         }
         fromTxId = elis.getLastTxId() + 1;
@@ -589,7 +589,7 @@ public class BookKeeperJournalManager implements JournalManager {
       long lastTxId = l.getLastTxId();
       if (l.isInProgress()) {
         lastTxId = recoverLastTxId(l, false);
-        if (lastTxId == HdfsConstants.INVALID_TXID) {
+        if (lastTxId == HdfsServerConstants.INVALID_TXID) {
           break;
         }
       }
@@ -634,7 +634,7 @@ public class BookKeeperJournalManager implements JournalManager {
           EditLogLedgerMetadata l = EditLogLedgerMetadata.read(zkc, znode);
           try {
             long endTxId = recoverLastTxId(l, true);
-            if (endTxId == HdfsConstants.INVALID_TXID) {
+            if (endTxId == HdfsServerConstants.INVALID_TXID) {
               LOG.error("Unrecoverable corruption has occurred in segment "
                   + l.toString() + " at path " + znode
                   + ". Unable to continue recovery.");
@@ -788,10 +788,10 @@ public class BookKeeperJournalManager implements JournalManager {
 
       in = new BookKeeperEditLogInputStream(lh, l, lastAddConfirmed);
 
-      long endTxId = HdfsConstants.INVALID_TXID;
+      long endTxId = HdfsServerConstants.INVALID_TXID;
       FSEditLogOp op = in.readOp();
       while (op != null) {
-        if (endTxId == HdfsConstants.INVALID_TXID
+        if (endTxId == HdfsServerConstants.INVALID_TXID
             || op.getTransactionId() == endTxId+1) {
           endTxId = op.getTransactionId();
         }
@@ -827,7 +827,7 @@ public class BookKeeperJournalManager implements JournalManager {
         try {
           EditLogLedgerMetadata editLogLedgerMetadata = EditLogLedgerMetadata
               .read(zkc, legderMetadataPath);
-          if (editLogLedgerMetadata.getLastTxId() != HdfsConstants.INVALID_TXID
+          if (editLogLedgerMetadata.getLastTxId() != HdfsServerConstants.INVALID_TXID
               && editLogLedgerMetadata.getLastTxId() < fromTxId) {
             // exclude already read closed edits, but include inprogress edits
             // as this will be handled in caller

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/EditLogLedgerMetadata.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/EditLogLedgerMetadata.java b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/EditLogLedgerMetadata.java
index 039cb3a..2d1f8b9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/EditLogLedgerMetadata.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/EditLogLedgerMetadata.java
@@ -19,13 +19,13 @@ package org.apache.hadoop.contrib.bkjournal;
 
 import java.io.IOException;
 import java.util.Comparator;
+
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.KeeperException;
 
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
@@ -67,7 +67,7 @@ public class EditLogLedgerMetadata {
     this.dataLayoutVersion = dataLayoutVersion;
     this.ledgerId = ledgerId;
     this.firstTxId = firstTxId;
-    this.lastTxId = HdfsConstants.INVALID_TXID;
+    this.lastTxId = HdfsServerConstants.INVALID_TXID;
     this.inprogress = true;
   }
   
@@ -107,7 +107,7 @@ public class EditLogLedgerMetadata {
   }
 
   void finalizeLedger(long newLastTxId) {
-    assert this.lastTxId == HdfsConstants.INVALID_TXID;
+    assert this.lastTxId == HdfsServerConstants.INVALID_TXID;
     this.lastTxId = newLastTxId;
     this.inprogress = false;      
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperEditLogStreams.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperEditLogStreams.java b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperEditLogStreams.java
index ec9692b..52e4568 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperEditLogStreams.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperEditLogStreams.java
@@ -27,7 +27,7 @@ import org.apache.bookkeeper.client.LedgerHandle;
 import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.zookeeper.ZooKeeper;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -68,7 +68,7 @@ public class TestBookKeeperEditLogStreams {
       lh.close();
 
       EditLogLedgerMetadata metadata = new EditLogLedgerMetadata("/foobar",
-          HdfsConstants.NAMENODE_LAYOUT_VERSION, lh.getId(), 0x1234);
+          HdfsServerConstants.NAMENODE_LAYOUT_VERSION, lh.getId(), 0x1234);
       try {
         new BookKeeperEditLogInputStream(lh, metadata, -1);
         fail("Shouldn't get this far, should have thrown");
@@ -77,7 +77,7 @@ public class TestBookKeeperEditLogStreams {
       }
 
       metadata = new EditLogLedgerMetadata("/foobar",
-          HdfsConstants.NAMENODE_LAYOUT_VERSION, lh.getId(), 0x1234);
+          HdfsServerConstants.NAMENODE_LAYOUT_VERSION, lh.getId(), 0x1234);
       try {
         new BookKeeperEditLogInputStream(lh, metadata, 0);
         fail("Shouldn't get this far, should have thrown");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index aaba543..99b8d2c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -543,10 +543,10 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       } catch (IOException e) {
         // Abort if the lease has already expired. 
         final long elapsed = Time.monotonicNow() - getLastLeaseRenewal();
-        if (elapsed > HdfsConstants.LEASE_HARDLIMIT_PERIOD) {
+        if (elapsed > HdfsServerConstants.LEASE_HARDLIMIT_PERIOD) {
           LOG.warn("Failed to renew lease for " + clientName + " for "
               + (elapsed/1000) + " seconds (>= hard-limit ="
-              + (HdfsConstants.LEASE_HARDLIMIT_PERIOD/1000) + " seconds.) "
+              + (HdfsServerConstants.LEASE_HARDLIMIT_PERIOD/1000) + " seconds.) "
               + "Closing all files being written ...", e);
           closeAllFilesBeingWritten(true);
         } else {
@@ -1902,7 +1902,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
           //connect to a datanode
           IOStreamPair pair = connectToDN(datanodes[j], timeout, lb);
           out = new DataOutputStream(new BufferedOutputStream(pair.out,
-              HdfsConstants.SMALL_BUFFER_SIZE));
+              HdfsServerConstants.SMALL_BUFFER_SIZE));
           in = new DataInputStream(pair.in);
 
           if (LOG.isDebugEnabled()) {
@@ -2067,7 +2067,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
 
     try {
       DataOutputStream out = new DataOutputStream(new BufferedOutputStream(pair.out,
-          HdfsConstants.SMALL_BUFFER_SIZE));
+          HdfsServerConstants.SMALL_BUFFER_SIZE));
       DataInputStream in = new DataInputStream(pair.in);
   
       new Sender(out).readBlock(lb.getBlock(), lb.getBlockToken(), clientName,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
index 2ec1c80..37f8c72 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
@@ -72,7 +72,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
 import org.apache.hadoop.crypto.key.KeyProviderFactory;
-import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
@@ -80,20 +79,17 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolTranslatorPB;
-import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
-import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.ToolRunner;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -262,7 +258,7 @@ public class DFSUtil {
    * @return true, if the component is reserved
    */
   public static boolean isReservedPathComponent(String component) {
-    for (String reserved : HdfsConstants.RESERVED_PATH_COMPONENTS) {
+    for (String reserved : HdfsServerConstants.RESERVED_PATH_COMPONENTS) {
       if (component.equals(reserved)) {
         return true;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
index 8e874eb..43787ab 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
@@ -71,6 +71,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
 import org.apache.hadoop.hdfs.util.ByteArrayManager;
@@ -1144,7 +1145,7 @@ class DataStreamer extends Daemon {
       unbufOut = saslStreams.out;
       unbufIn = saslStreams.in;
       out = new DataOutputStream(new BufferedOutputStream(unbufOut,
-          HdfsConstants.SMALL_BUFFER_SIZE));
+          HdfsServerConstants.SMALL_BUFFER_SIZE));
       in = new DataInputStream(unbufIn);
 
       //send the TRANSFER_BLOCK request
@@ -1424,7 +1425,7 @@ class DataStreamer extends Daemon {
         unbufOut = saslStreams.out;
         unbufIn = saslStreams.in;
         out = new DataOutputStream(new BufferedOutputStream(unbufOut,
-            HdfsConstants.SMALL_BUFFER_SIZE));
+            HdfsServerConstants.SMALL_BUFFER_SIZE));
         blockReplyStream = new DataInputStream(unbufIn);
 
         //

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
index d826755..77262bc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hdfs.protocolPB.JournalProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.JournalProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.hdfs.server.namenode.ha.AbstractNNFailoverProxyProvider;
@@ -417,7 +418,7 @@ public class NameNodeProxies {
 
       RetryPolicy createPolicy = RetryPolicies
           .retryUpToMaximumCountWithFixedSleep(5,
-              HdfsConstants.LEASE_SOFTLIMIT_PERIOD, TimeUnit.MILLISECONDS);
+              HdfsServerConstants.LEASE_SOFTLIMIT_PERIOD, TimeUnit.MILLISECONDS);
     
       Map<Class<? extends Exception>, RetryPolicy> remoteExceptionToPolicyMap 
                  = new HashMap<Class<? extends Exception>, RetryPolicy>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
index 4cdf168..99323bb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSOutputStream;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.StringUtils;
@@ -165,7 +165,7 @@ public class LeaseRenewer {
   /** The time in milliseconds that the map became empty. */
   private long emptyTime = Long.MAX_VALUE;
   /** A fixed lease renewal time period in milliseconds */
-  private long renewal = HdfsConstants.LEASE_SOFTLIMIT_PERIOD/2;
+  private long renewal = HdfsServerConstants.LEASE_SOFTLIMIT_PERIOD/2;
 
   /** A daemon for renewing lease */
   private Daemon daemon = null;
@@ -372,7 +372,7 @@ public class LeaseRenewer {
 
     //update renewal time
     if (renewal == dfsc.getConf().getHdfsTimeout()/2) {
-      long min = HdfsConstants.LEASE_SOFTLIMIT_PERIOD;
+      long min = HdfsServerConstants.LEASE_SOFTLIMIT_PERIOD;
       for(DFSClient c : dfsclients) {
         final int timeout = c.getConf().getHdfsTimeout();
         if (timeout > 0 && timeout < min) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index bafb02b..55faf16 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -967,8 +967,8 @@ public interface ClientProtocol {
    * <br><br>
    *                       
    * The quota can have three types of values : (1) 0 or more will set 
-   * the quota to that value, (2) {@link HdfsConstants#QUOTA_DONT_SET}  implies 
-   * the quota will not be changed, and (3) {@link HdfsConstants#QUOTA_RESET} 
+   * the quota to that value, (2) {@link HdfsConstants#QUOTA_DONT_SET}  implies
+   * the quota will not be changed, and (3) {@link HdfsConstants#QUOTA_RESET}
    * implies the quota will be reset. Any other value is a runtime error.
    * 
    * @throws AccessControlException permission denied

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
deleted file mode 100644
index 52259b0..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
+++ /dev/null
@@ -1,175 +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.protocol;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.server.datanode.DataNodeLayoutVersion;
-import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
-import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
-import org.apache.hadoop.util.StringUtils;
-
-/************************************
- * Some handy constants
- * 
- ************************************/
-@InterfaceAudience.Private
-public class HdfsConstants {
-  /* Hidden constructor */
-  protected HdfsConstants() {
-  }
-  
-  /**
-   * HDFS Protocol Names:  
-   */
-  public static final String CLIENT_NAMENODE_PROTOCOL_NAME = 
-      "org.apache.hadoop.hdfs.protocol.ClientProtocol";
-  public static final String CLIENT_DATANODE_PROTOCOL_NAME = 
-      "org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol";
-  
-  
-  public static final int MIN_BLOCKS_FOR_WRITE = 1;
-
-  // Long that indicates "leave current quota unchanged"
-  public static final long QUOTA_DONT_SET = Long.MAX_VALUE;
-  public static final long QUOTA_RESET = -1L;
-
-  //
-  // Timeouts, constants
-  //
-  public static final long LEASE_SOFTLIMIT_PERIOD = 60 * 1000;
-  public static final long LEASE_HARDLIMIT_PERIOD = 60 * LEASE_SOFTLIMIT_PERIOD;
-  public static final long LEASE_RECOVER_PERIOD = 10 * 1000; // in ms
-
-  // We need to limit the length and depth of a path in the filesystem.
-  // HADOOP-438
-  // Currently we set the maximum length to 8k characters and the maximum depth
-  // to 1k.
-  public static final int MAX_PATH_LENGTH = 8000;
-  public static final int MAX_PATH_DEPTH = 1000;
-
-  // TODO should be conf injected?
-  public static final int DEFAULT_DATA_SOCKET_SIZE = 128 * 1024;
-  public static final int IO_FILE_BUFFER_SIZE = new HdfsConfiguration().getInt(
-      DFSConfigKeys.IO_FILE_BUFFER_SIZE_KEY,
-      DFSConfigKeys.IO_FILE_BUFFER_SIZE_DEFAULT);
-  // Used for writing header etc.
-  public static final int SMALL_BUFFER_SIZE = Math.min(IO_FILE_BUFFER_SIZE / 2,
-      512);
-
-  public static final int BYTES_IN_INTEGER = Integer.SIZE / Byte.SIZE;
-
-  // SafeMode actions
-  public static enum SafeModeAction {
-    SAFEMODE_LEAVE, SAFEMODE_ENTER, SAFEMODE_GET;
-  }
-
-  public static enum RollingUpgradeAction {
-    QUERY, PREPARE, FINALIZE;
-    
-    private static final Map<String, RollingUpgradeAction> MAP
-        = new HashMap<String, RollingUpgradeAction>();
-    static {
-      MAP.put("", QUERY);
-      for(RollingUpgradeAction a : values()) {
-        MAP.put(a.name(), a);
-      }
-    }
-
-    /** Covert the given String to a RollingUpgradeAction. */
-    public static RollingUpgradeAction fromString(String s) {
-      return MAP.get(StringUtils.toUpperCase(s));
-    }
-  }
-
-  // type of the datanode report
-  public static enum DatanodeReportType {
-    ALL, LIVE, DEAD, DECOMMISSIONING
-  }
-
-  // An invalid transaction ID that will never be seen in a real namesystem.
-  public static final long INVALID_TXID = -12345;
-
-  // Number of generation stamps reserved for legacy blocks.
-  public static final long RESERVED_GENERATION_STAMPS_V1 =
-      1024L * 1024 * 1024 * 1024;
-
-  /**
-   * URI Scheme for hdfs://namenode/ URIs.
-   */
-  public static final String HDFS_URI_SCHEME = "hdfs";
-
-
-  /**
-   * Current layout version for NameNode.
-   * Please see {@link NameNodeLayoutVersion.Feature} on adding new layout version.
-   */
-  public static final int NAMENODE_LAYOUT_VERSION
-      = NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION;
-
-  /**
-   * Current layout version for DataNode.
-   * Please see {@link DataNodeLayoutVersion.Feature} on adding new layout version.
-   */
-  public static final int DATANODE_LAYOUT_VERSION
-      = DataNodeLayoutVersion.CURRENT_LAYOUT_VERSION;
-
-  /**
-   * Path components that are reserved in HDFS.
-   * <p>
-   * .reserved is only reserved under root ("/").
-   */
-  public static final String[] RESERVED_PATH_COMPONENTS = new String[] {
-    HdfsConstants.DOT_SNAPSHOT_DIR,
-    FSDirectory.DOT_RESERVED_STRING
-  };
-
-  /**
-   * A special path component contained in the path for a snapshot file/dir
-   */
-  public static final String DOT_SNAPSHOT_DIR = ".snapshot";
-
-  public static final byte[] DOT_SNAPSHOT_DIR_BYTES
-      = DFSUtil.string2Bytes(DOT_SNAPSHOT_DIR);
-  
-  public static final String SEPARATOR_DOT_SNAPSHOT_DIR
-      = Path.SEPARATOR + DOT_SNAPSHOT_DIR;
-
-  public static final String SEPARATOR_DOT_SNAPSHOT_DIR_SEPARATOR
-      = Path.SEPARATOR + DOT_SNAPSHOT_DIR + Path.SEPARATOR;
-
-  public static final String MEMORY_STORAGE_POLICY_NAME = "LAZY_PERSIST";
-  public static final String ALLSSD_STORAGE_POLICY_NAME = "ALL_SSD";
-  public static final String ONESSD_STORAGE_POLICY_NAME = "ONE_SSD";
-  public static final String HOT_STORAGE_POLICY_NAME = "HOT";
-  public static final String WARM_STORAGE_POLICY_NAME = "WARM";
-  public static final String COLD_STORAGE_POLICY_NAME = "COLD";
-
-  public static final byte MEMORY_STORAGE_POLICY_ID = 15;
-  public static final byte ALLSSD_STORAGE_POLICY_ID = 12;
-  public static final byte ONESSD_STORAGE_POLICY_ID = 10;
-  public static final byte HOT_STORAGE_POLICY_ID = 7;
-  public static final byte WARM_STORAGE_POLICY_ID = 5;
-  public static final byte COLD_STORAGE_POLICY_ID = 2;
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
index 518e91a..3067696 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
@@ -61,7 +61,7 @@ public class SnapshottableDirectoryStatus {
       int snapshotNumber, int snapshotQuota, byte[] parentFullPath) {
     this.dirStatus = new HdfsFileStatus(0, true, 0, 0, modification_time,
         access_time, permission, owner, group, null, localName, inodeId,
-        childrenNum, null, HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
+        childrenNum, null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
     this.snapshotNumber = snapshotNumber;
     this.snapshotQuota = snapshotQuota;
     this.parentFullPath = parentFullPath;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java
index 4f25694..0d3796c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.security.token.TokenInfo;
 @KerberosInfo(
     serverPrincipal = DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY)
 @TokenInfo(DelegationTokenSelector.class)
-@ProtocolInfo(protocolName = HdfsConstants.CLIENT_NAMENODE_PROTOCOL_NAME, 
+@ProtocolInfo(protocolName = HdfsConstants.CLIENT_NAMENODE_PROTOCOL_NAME,
     protocolVersion = 1)
 /**
  * Protocol that a clients use to communicate with the NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/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 563d10b..480b3d9 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
@@ -36,7 +36,7 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -533,7 +533,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       boolean result = 
           server.complete(req.getSrc(), req.getClientName(),
           req.hasLast() ? PBHelper.convert(req.getLast()) : null,
-          req.hasFileId() ? req.getFileId() : HdfsConstantsClient.GRANDFATHER_INODE_ID);
+          req.hasFileId() ? req.getFileId() : HdfsConstants.GRANDFATHER_INODE_ID);
       return CompleteResponseProto.newBuilder().setResult(result).build();
     } catch (IOException e) {
       throw new ServiceException(e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index b440e60..ba6670c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -73,10 +73,10 @@ import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -1436,12 +1436,12 @@ public class PBHelper {
         fs.getFileType().equals(FileType.IS_SYMLINK) ? 
             fs.getSymlink().toByteArray() : null,
         fs.getPath().toByteArray(),
-        fs.hasFileId()? fs.getFileId(): HdfsConstantsClient.GRANDFATHER_INODE_ID,
+        fs.hasFileId()? fs.getFileId(): HdfsConstants.GRANDFATHER_INODE_ID,
         fs.hasLocations() ? PBHelper.convert(fs.getLocations()) : null,
         fs.hasChildrenNum() ? fs.getChildrenNum() : -1,
         fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) : null,
         fs.hasStoragePolicy() ? (byte) fs.getStoragePolicy()
-            : HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
+            : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
   }
 
   public static SnapshottableDirectoryStatus convert(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java
index 6938f57..cbfe5b8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java
@@ -33,7 +33,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.qjournal.protocol.JournalOutOfSyncException;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
@@ -46,6 +45,7 @@ import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo;
 import org.apache.hadoop.hdfs.qjournal.protocolPB.QJournalProtocolPB;
 import org.apache.hadoop.hdfs.qjournal.protocolPB.QJournalProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.qjournal.server.GetJournalEditServlet;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
@@ -92,7 +92,7 @@ public class IPCLoggerChannel implements AsyncLogger {
   private final ListeningExecutorService parallelExecutor;
   private long ipcSerial = 0;
   private long epoch = -1;
-  private long committedTxId = HdfsConstants.INVALID_TXID;
+  private long committedTxId = HdfsServerConstants.INVALID_TXID;
   
   private final String journalId;
   private final NamespaceInfo nsInfo;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/RequestInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/RequestInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/RequestInfo.java
index dfd1e4d..2569aad 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/RequestInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/RequestInfo.java
@@ -18,7 +18,7 @@
 package org.apache.hadoop.hdfs.qjournal.protocol;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 
 @InterfaceAudience.Private
 public class RequestInfo {
@@ -60,6 +60,6 @@ public class RequestInfo {
   }
 
   public boolean hasCommittedTxId() {
-    return (committedTxId != HdfsConstants.INVALID_TXID);
+    return (committedTxId != HdfsServerConstants.INVALID_TXID);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java
index 9ff63d7..28f77f2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java
@@ -21,7 +21,6 @@ import java.io.IOException;
 import java.net.URL;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocolPB.JournalProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
@@ -66,6 +65,7 @@ import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PurgeLogs
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.StartLogSegmentRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.StartLogSegmentResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
@@ -263,7 +263,7 @@ public class QJournalProtocolServerSideTranslatorPB implements QJournalProtocolP
         reqInfo.getEpoch(),
         reqInfo.getIpcSerialNumber(),
         reqInfo.hasCommittedTxId() ?
-          reqInfo.getCommittedTxId() : HdfsConstants.INVALID_TXID);
+          reqInfo.getCommittedTxId() : HdfsServerConstants.INVALID_TXID);
   }
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/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 9d11ca5..2953055 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
@@ -34,7 +34,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.qjournal.protocol.JournalNotFormattedException;
 import org.apache.hadoop.hdfs.qjournal.protocol.JournalOutOfSyncException;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
@@ -44,6 +43,7 @@ import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.Persisted
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.StorageErrorReporter;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
@@ -81,8 +81,8 @@ public class Journal implements Closeable {
 
   // Current writing state
   private EditLogOutputStream curSegment;
-  private long curSegmentTxId = HdfsConstants.INVALID_TXID;
-  private long nextTxId = HdfsConstants.INVALID_TXID;
+  private long curSegmentTxId = HdfsServerConstants.INVALID_TXID;
+  private long nextTxId = HdfsServerConstants.INVALID_TXID;
   private long highestWrittenTxId = 0;
   
   private final String journalId;
@@ -170,7 +170,7 @@ public class Journal implements Closeable {
         new File(currentDir, LAST_WRITER_EPOCH), 0);
     this.committedTxnId = new BestEffortLongFile(
         new File(currentDir, COMMITTED_TXID_FILENAME),
-        HdfsConstants.INVALID_TXID);
+        HdfsServerConstants.INVALID_TXID);
   }
   
   /**
@@ -191,7 +191,7 @@ public class Journal implements Closeable {
       EditLogFile latestLog = files.remove(files.size() - 1);
       latestLog.scanLog();
       LOG.info("Latest log is " + latestLog);
-      if (latestLog.getLastTxId() == HdfsConstants.INVALID_TXID) {
+      if (latestLog.getLastTxId() == HdfsServerConstants.INVALID_TXID) {
         // the log contains no transactions
         LOG.warn("Latest log " + latestLog + " has no transactions. " +
             "moving it aside and looking for previous log");
@@ -327,7 +327,7 @@ public class Journal implements Closeable {
     
     curSegment.abort();
     curSegment = null;
-    curSegmentTxId = HdfsConstants.INVALID_TXID;
+    curSegmentTxId = HdfsServerConstants.INVALID_TXID;
   }
 
   /**
@@ -565,7 +565,7 @@ public class Journal implements Closeable {
       if (curSegment != null) {
         curSegment.close();
         curSegment = null;
-        curSegmentTxId = HdfsConstants.INVALID_TXID;
+        curSegmentTxId = HdfsServerConstants.INVALID_TXID;
       }
       
       checkSync(nextTxId == endTxId + 1,
@@ -677,7 +677,7 @@ public class Journal implements Closeable {
     if (elf.isInProgress()) {
       elf.scanLog();
     }
-    if (elf.getLastTxId() == HdfsConstants.INVALID_TXID) {
+    if (elf.getLastTxId() == HdfsServerConstants.INVALID_TXID) {
       LOG.info("Edit log file " + elf + " appears to be empty. " +
           "Moving it aside...");
       elf.moveAsideEmptyFile();
@@ -727,7 +727,7 @@ public class Journal implements Closeable {
     }
     
     builder.setLastWriterEpoch(lastWriterEpoch.get());
-    if (committedTxnId.get() != HdfsConstants.INVALID_TXID) {
+    if (committedTxnId.get() != HdfsServerConstants.INVALID_TXID) {
       builder.setLastCommittedTxId(committedTxnId.get());
     }
     
@@ -1021,7 +1021,7 @@ public class Journal implements Closeable {
         new File(previousDir, LAST_WRITER_EPOCH), 0);
     BestEffortLongFile prevCommittedTxnId = new BestEffortLongFile(
         new File(previousDir, COMMITTED_TXID_FILENAME),
-        HdfsConstants.INVALID_TXID);
+        HdfsServerConstants.INVALID_TXID);
 
     lastPromisedEpoch = new PersistentLongFile(
         new File(currentDir, LAST_PROMISED_FILENAME), 0);
@@ -1029,7 +1029,7 @@ public class Journal implements Closeable {
         new File(currentDir, LAST_WRITER_EPOCH), 0);
     committedTxnId = new BestEffortLongFile(
         new File(currentDir, COMMITTED_TXID_FILENAME),
-        HdfsConstants.INVALID_TXID);
+        HdfsServerConstants.INVALID_TXID);
 
     try {
       lastPromisedEpoch.set(prevLastPromisedEpoch.get());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
index 5d02bf9..ba16905 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
@@ -52,7 +52,6 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
@@ -309,9 +308,9 @@ public class Dispatcher {
         unbufOut = saslStreams.out;
         unbufIn = saslStreams.in;
         out = new DataOutputStream(new BufferedOutputStream(unbufOut,
-            HdfsConstants.IO_FILE_BUFFER_SIZE));
+            HdfsServerConstants.IO_FILE_BUFFER_SIZE));
         in = new DataInputStream(new BufferedInputStream(unbufIn,
-            HdfsConstants.IO_FILE_BUFFER_SIZE));
+            HdfsServerConstants.IO_FILE_BUFFER_SIZE));
 
         sendRequest(out, eb, accessToken);
         receiveResponse(in);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
index eac6ed2..feebd87 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
@@ -21,8 +21,8 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 
 import java.io.IOException;
 
@@ -56,7 +56,7 @@ public class BlockIdManager {
   private final SequentialBlockIdGenerator blockIdGenerator;
 
   public BlockIdManager(BlockManager blockManager) {
-    this.generationStampV1Limit = HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP;
+    this.generationStampV1Limit = HdfsConstants.GRANDFATHER_GENERATION_STAMP;
     this.blockIdGenerator = new SequentialBlockIdGenerator(blockManager);
   }
 
@@ -70,7 +70,7 @@ public class BlockIdManager {
     Preconditions.checkState(generationStampV2.getCurrentValue() ==
       GenerationStamp.LAST_RESERVED_STAMP);
     generationStampV2.skipTo(generationStampV1.getCurrentValue() +
-      HdfsConstants.RESERVED_GENERATION_STAMPS_V1);
+      HdfsServerConstants.RESERVED_GENERATION_STAMPS_V1);
 
     generationStampV1Limit = generationStampV2.getCurrentValue();
     return generationStampV2.getCurrentValue();
@@ -83,7 +83,7 @@ public class BlockIdManager {
    * @param stamp set generation stamp limit to this value
    */
   public void setGenerationStampV1Limit(long stamp) {
-    Preconditions.checkState(generationStampV1Limit == HdfsConstantsClient
+    Preconditions.checkState(generationStampV1Limit == HdfsConstants
       .GRANDFATHER_GENERATION_STAMP);
     generationStampV1Limit = stamp;
   }
@@ -204,6 +204,6 @@ public class BlockIdManager {
     generationStampV2.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
     getBlockIdGenerator().setCurrentValue(SequentialBlockIdGenerator
       .LAST_RESERVED_BLOCK_ID);
-    generationStampV1Limit = HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP;
+    generationStampV1Limit = HdfsConstants.GRANDFATHER_GENERATION_STAMP;
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
index c2752ac..620d2a6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
@@ -29,8 +29,8 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.Node;
@@ -792,7 +792,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       }
     }
     
-    final long requiredSize = blockSize * HdfsConstants.MIN_BLOCKS_FOR_WRITE;
+    final long requiredSize = blockSize * HdfsServerConstants.MIN_BLOCKS_FOR_WRITE;
     final long scheduledSize = blockSize * node.getBlocksScheduled(storage.getStorageType());
     final long remaining = node.getRemaining(storage.getStorageType());
     if (requiredSize > remaining - scheduledSize) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java
index 6c352f3..2a71c29 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -47,39 +48,39 @@ public class BlockStoragePolicySuite {
   public static BlockStoragePolicySuite createDefaultSuite() {
     final BlockStoragePolicy[] policies =
         new BlockStoragePolicy[1 << ID_BIT_LENGTH];
-    final byte lazyPersistId = HdfsConstants.MEMORY_STORAGE_POLICY_ID;
+    final byte lazyPersistId = HdfsServerConstants.MEMORY_STORAGE_POLICY_ID;
     policies[lazyPersistId] = new BlockStoragePolicy(lazyPersistId,
         HdfsConstants.MEMORY_STORAGE_POLICY_NAME,
         new StorageType[]{StorageType.RAM_DISK, StorageType.DISK},
         new StorageType[]{StorageType.DISK},
         new StorageType[]{StorageType.DISK},
         true);    // Cannot be changed on regular files, but inherited.
-    final byte allssdId = HdfsConstants.ALLSSD_STORAGE_POLICY_ID;
+    final byte allssdId = HdfsServerConstants.ALLSSD_STORAGE_POLICY_ID;
     policies[allssdId] = new BlockStoragePolicy(allssdId,
         HdfsConstants.ALLSSD_STORAGE_POLICY_NAME,
         new StorageType[]{StorageType.SSD},
         new StorageType[]{StorageType.DISK},
         new StorageType[]{StorageType.DISK});
-    final byte onessdId = HdfsConstants.ONESSD_STORAGE_POLICY_ID;
+    final byte onessdId = HdfsServerConstants.ONESSD_STORAGE_POLICY_ID;
     policies[onessdId] = new BlockStoragePolicy(onessdId,
         HdfsConstants.ONESSD_STORAGE_POLICY_NAME,
         new StorageType[]{StorageType.SSD, StorageType.DISK},
         new StorageType[]{StorageType.SSD, StorageType.DISK},
         new StorageType[]{StorageType.SSD, StorageType.DISK});
-    final byte hotId = HdfsConstants.HOT_STORAGE_POLICY_ID;
+    final byte hotId = HdfsServerConstants.HOT_STORAGE_POLICY_ID;
     policies[hotId] = new BlockStoragePolicy(hotId,
-        HdfsConstants.HOT_STORAGE_POLICY_NAME,
+        HdfsServerConstants.HOT_STORAGE_POLICY_NAME,
         new StorageType[]{StorageType.DISK}, StorageType.EMPTY_ARRAY,
         new StorageType[]{StorageType.ARCHIVE});
-    final byte warmId = HdfsConstants.WARM_STORAGE_POLICY_ID;
+    final byte warmId = HdfsServerConstants.WARM_STORAGE_POLICY_ID;
     policies[warmId] = new BlockStoragePolicy(warmId,
-        HdfsConstants.WARM_STORAGE_POLICY_NAME,
+        HdfsServerConstants.WARM_STORAGE_POLICY_NAME,
         new StorageType[]{StorageType.DISK, StorageType.ARCHIVE},
         new StorageType[]{StorageType.DISK, StorageType.ARCHIVE},
         new StorageType[]{StorageType.DISK, StorageType.ARCHIVE});
-    final byte coldId = HdfsConstants.COLD_STORAGE_POLICY_ID;
+    final byte coldId = HdfsServerConstants.COLD_STORAGE_POLICY_ID;
     policies[coldId] = new BlockStoragePolicy(coldId,
-        HdfsConstants.COLD_STORAGE_POLICY_NAME,
+        HdfsServerConstants.COLD_STORAGE_POLICY_NAME,
         new StorageType[]{StorageType.ARCHIVE}, StorageType.EMPTY_ARRAY,
         StorageType.EMPTY_ARRAY);
     return new BlockStoragePolicySuite(hotId, policies);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
index 2d267ce..47b0818 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
@@ -24,9 +24,16 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeLayoutVersion;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.MetaRecoveryContext;
 
 import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
 import org.apache.hadoop.util.StringUtils;
 
 /************************************
@@ -35,21 +42,75 @@ import org.apache.hadoop.util.StringUtils;
  ************************************/
 
 @InterfaceAudience.Private
-public final class HdfsServerConstants {
-  /* Hidden constructor */
-  private HdfsServerConstants() { }
-  
+public interface HdfsServerConstants {
+  int MIN_BLOCKS_FOR_WRITE = 1;
+  //
+  // Timeouts, constants
+  //
+  long LEASE_SOFTLIMIT_PERIOD = 60 * 1000;
+  long LEASE_HARDLIMIT_PERIOD = 60 * LEASE_SOFTLIMIT_PERIOD;
+  long LEASE_RECOVER_PERIOD = 10 * 1000; // in ms
+  // We need to limit the length and depth of a path in the filesystem.
+  // HADOOP-438
+  // Currently we set the maximum length to 8k characters and the maximum depth
+  // to 1k.
+  int MAX_PATH_LENGTH = 8000;
+  int MAX_PATH_DEPTH = 1000;
+  int IO_FILE_BUFFER_SIZE = new HdfsConfiguration().getInt(
+      CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY,
+      CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT);
+  // Used for writing header etc.
+  int SMALL_BUFFER_SIZE = Math.min(IO_FILE_BUFFER_SIZE / 2,
+      512);
+  // An invalid transaction ID that will never be seen in a real namesystem.
+  long INVALID_TXID = -12345;
+  // Number of generation stamps reserved for legacy blocks.
+  long RESERVED_GENERATION_STAMPS_V1 =
+      1024L * 1024 * 1024 * 1024;
+  /**
+   * Current layout version for NameNode.
+   * Please see {@link NameNodeLayoutVersion.Feature} on adding new layout version.
+   */
+  int NAMENODE_LAYOUT_VERSION
+      = NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION;
+  /**
+   * Current layout version for DataNode.
+   * Please see {@link DataNodeLayoutVersion.Feature} on adding new layout version.
+   */
+  int DATANODE_LAYOUT_VERSION
+      = DataNodeLayoutVersion.CURRENT_LAYOUT_VERSION;
+  /**
+   * Path components that are reserved in HDFS.
+   * <p>
+   * .reserved is only reserved under root ("/").
+   */
+  String[] RESERVED_PATH_COMPONENTS = new String[] {
+      HdfsConstants.DOT_SNAPSHOT_DIR,
+      FSDirectory.DOT_RESERVED_STRING
+  };
+  byte[] DOT_SNAPSHOT_DIR_BYTES
+              = DFSUtil.string2Bytes(HdfsConstants.DOT_SNAPSHOT_DIR);
+  String HOT_STORAGE_POLICY_NAME = "HOT";
+  String WARM_STORAGE_POLICY_NAME = "WARM";
+  String COLD_STORAGE_POLICY_NAME = "COLD";
+  byte MEMORY_STORAGE_POLICY_ID = 15;
+  byte ALLSSD_STORAGE_POLICY_ID = 12;
+  byte ONESSD_STORAGE_POLICY_ID = 10;
+  byte HOT_STORAGE_POLICY_ID = 7;
+  byte WARM_STORAGE_POLICY_ID = 5;
+  byte COLD_STORAGE_POLICY_ID = 2;
+
   /**
    * Type of the node
    */
-  static public enum NodeType {
+  enum NodeType {
     NAME_NODE,
     DATA_NODE,
-    JOURNAL_NODE;
+    JOURNAL_NODE
   }
 
   /** Startup options for rolling upgrade. */
-  public static enum RollingUpgradeStartupOption{
+  enum RollingUpgradeStartupOption{
     ROLLBACK, STARTED;
 
     public String getOptionString() {
@@ -93,7 +154,7 @@ public final class HdfsServerConstants {
   }
 
   /** Startup options */
-  static public enum StartupOption{
+  enum StartupOption{
     FORMAT  ("-format"),
     CLUSTERID ("-clusterid"),
     GENCLUSTERID ("-genclusterid"),
@@ -137,7 +198,7 @@ public final class HdfsServerConstants {
     // Used only with recovery option
     private int force = 0;
 
-    private StartupOption(String arg) {this.name = arg;}
+    StartupOption(String arg) {this.name = arg;}
     public String getName() {return name;}
     public NamenodeRole toNodeRole() {
       switch(this) {
@@ -221,21 +282,21 @@ public final class HdfsServerConstants {
   }
 
   // Timeouts for communicating with DataNode for streaming writes/reads
-  public static final int READ_TIMEOUT = 60 * 1000;
-  public static final int READ_TIMEOUT_EXTENSION = 5 * 1000;
-  public static final int WRITE_TIMEOUT = 8 * 60 * 1000;
-  public static final int WRITE_TIMEOUT_EXTENSION = 5 * 1000; //for write pipeline
+  int READ_TIMEOUT = 60 * 1000;
+  int READ_TIMEOUT_EXTENSION = 5 * 1000;
+  int WRITE_TIMEOUT = 8 * 60 * 1000;
+  int WRITE_TIMEOUT_EXTENSION = 5 * 1000; //for write pipeline
 
   /**
    * Defines the NameNode role.
    */
-  static public enum NamenodeRole {
+  enum NamenodeRole {
     NAMENODE  ("NameNode"),
     BACKUP    ("Backup Node"),
     CHECKPOINT("Checkpoint Node");
 
     private String description = null;
-    private NamenodeRole(String arg) {this.description = arg;}
+    NamenodeRole(String arg) {this.description = arg;}
   
     @Override
     public String toString() {
@@ -246,7 +307,7 @@ public final class HdfsServerConstants {
   /**
    * Block replica states, which it can go through while being constructed.
    */
-  static public enum ReplicaState {
+  enum ReplicaState {
     /** Replica is finalized. The state when replica is not modified. */
     FINALIZED(0),
     /** Replica is being written to. */
@@ -260,7 +321,7 @@ public final class HdfsServerConstants {
 
     private final int value;
 
-    private ReplicaState(int v) {
+    ReplicaState(int v) {
       value = v;
     }
 
@@ -286,7 +347,7 @@ public final class HdfsServerConstants {
   /**
    * States, which a block can go through while it is under construction.
    */
-  static public enum BlockUCState {
+  enum BlockUCState {
     /**
      * Block construction completed.<br>
      * The block has at least the configured minimal replication number
@@ -315,16 +376,16 @@ public final class HdfsServerConstants {
      * {@link ReplicaState#FINALIZED} 
      * replicas has yet been reported by data-nodes themselves.
      */
-    COMMITTED;
+    COMMITTED
   }
   
-  public static final String NAMENODE_LEASE_HOLDER = "HDFS_NameNode";
-  public static final long NAMENODE_LEASE_RECHECK_INTERVAL = 2000;
+  String NAMENODE_LEASE_HOLDER = "HDFS_NameNode";
+  long NAMENODE_LEASE_RECHECK_INTERVAL = 2000;
 
-  public static final String CRYPTO_XATTR_ENCRYPTION_ZONE =
+  String CRYPTO_XATTR_ENCRYPTION_ZONE =
       "raw.hdfs.crypto.encryption.zone";
-  public static final String CRYPTO_XATTR_FILE_ENCRYPTION_INFO =
+  String CRYPTO_XATTR_FILE_ENCRYPTION_INFO =
       "raw.hdfs.crypto.file.encryption.info";
-  public static final String SECURITY_XATTR_UNREADABLE_BY_SUPERUSER =
+  String SECURITY_XATTR_UNREADABLE_BY_SUPERUSER =
       "security.hdfs.unreadable.by.superuser";
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java
index f40b079..50363c9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java
@@ -26,7 +26,6 @@ import java.util.Properties;
 import java.util.SortedSet;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.LayoutFeature;
@@ -222,8 +221,8 @@ public class StorageInfo {
   }
 
   public int getServiceLayoutVersion() {
-    return storageType == NodeType.DATA_NODE ? HdfsConstants.DATANODE_LAYOUT_VERSION
-        : HdfsConstants.NAMENODE_LAYOUT_VERSION;
+    return storageType == NodeType.DATA_NODE ? HdfsServerConstants.DATANODE_LAYOUT_VERSION
+        : HdfsServerConstants.NAMENODE_LAYOUT_VERSION;
   }
 
   public Map<Integer, SortedSet<LayoutFeature>> getServiceLayoutFeatureMap() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
index 94493aa..04700b8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
@@ -33,7 +33,7 @@ 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.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.DataChecksum;
 
@@ -85,7 +85,7 @@ public class BlockMetadataHeader {
     DataInputStream in = null;
     try {
       in = new DataInputStream(new BufferedInputStream(
-        new FileInputStream(metaFile), HdfsConstants.IO_FILE_BUFFER_SIZE));
+        new FileInputStream(metaFile), HdfsServerConstants.IO_FILE_BUFFER_SIZE));
       return readDataChecksum(in, metaFile);
     } finally {
       IOUtils.closeStream(in);


[39/50] [abbrv] hadoop git commit: HDFS-8249. Separate HdfsConstants into the client and the server side class. Contributed by Haohui Mai.

Posted by zj...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
index d26a9a5..09a2d8b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
@@ -24,8 +24,8 @@ import com.google.common.collect.Lists;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.HardLink;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
@@ -267,7 +267,7 @@ public class BlockPoolSliceStorage extends Storage {
     LOG.info("Formatting block pool " + blockpoolID + " directory "
         + bpSdir.getCurrentDir());
     bpSdir.clearDirectory(); // create directory
-    this.layoutVersion = HdfsConstants.DATANODE_LAYOUT_VERSION;
+    this.layoutVersion = HdfsServerConstants.DATANODE_LAYOUT_VERSION;
     this.cTime = nsInfo.getCTime();
     this.namespaceID = nsInfo.getNamespaceID();
     this.blockpoolID = nsInfo.getBlockPoolID();
@@ -361,7 +361,7 @@ public class BlockPoolSliceStorage extends Storage {
     }
     readProperties(sd);
     checkVersionUpgradable(this.layoutVersion);
-    assert this.layoutVersion >= HdfsConstants.DATANODE_LAYOUT_VERSION 
+    assert this.layoutVersion >= HdfsServerConstants.DATANODE_LAYOUT_VERSION
        : "Future version is not allowed";
     if (getNamespaceID() != nsInfo.getNamespaceID()) {
       throw new IOException("Incompatible namespaceIDs in "
@@ -375,17 +375,17 @@ public class BlockPoolSliceStorage extends Storage {
           + nsInfo.getBlockPoolID() + "; datanode blockpoolID = "
           + blockpoolID);
     }
-    if (this.layoutVersion == HdfsConstants.DATANODE_LAYOUT_VERSION
+    if (this.layoutVersion == HdfsServerConstants.DATANODE_LAYOUT_VERSION
         && this.cTime == nsInfo.getCTime()) {
       return; // regular startup
     }
-    if (this.layoutVersion > HdfsConstants.DATANODE_LAYOUT_VERSION) {
+    if (this.layoutVersion > HdfsServerConstants.DATANODE_LAYOUT_VERSION) {
       int restored = restoreBlockFilesFromTrash(getTrashRootDir(sd));
       LOG.info("Restored " + restored + " block files from trash " +
         "before the layout upgrade. These blocks will be moved to " +
         "the previous directory during the upgrade");
     }
-    if (this.layoutVersion > HdfsConstants.DATANODE_LAYOUT_VERSION
+    if (this.layoutVersion > HdfsServerConstants.DATANODE_LAYOUT_VERSION
         || this.cTime < nsInfo.getCTime()) {
       doUpgrade(datanode, sd, nsInfo); // upgrade
       return;
@@ -425,7 +425,7 @@ public class BlockPoolSliceStorage extends Storage {
     }
     LOG.info("Upgrading block pool storage directory " + bpSd.getRoot()
         + ".\n   old LV = " + this.getLayoutVersion() + "; old CTime = "
-        + this.getCTime() + ".\n   new LV = " + HdfsConstants.DATANODE_LAYOUT_VERSION
+        + this.getCTime() + ".\n   new LV = " + HdfsServerConstants.DATANODE_LAYOUT_VERSION
         + "; new CTime = " + nsInfo.getCTime());
     // get <SD>/previous directory
     String dnRoot = getDataNodeStorageRoot(bpSd.getRoot().getCanonicalPath());
@@ -454,7 +454,7 @@ public class BlockPoolSliceStorage extends Storage {
     
     // 3. Create new <SD>/current with block files hardlinks and VERSION
     linkAllBlocks(datanode, bpTmpDir, bpCurDir);
-    this.layoutVersion = HdfsConstants.DATANODE_LAYOUT_VERSION;
+    this.layoutVersion = HdfsServerConstants.DATANODE_LAYOUT_VERSION;
     assert this.namespaceID == nsInfo.getNamespaceID() 
         : "Data-node and name-node layout versions must be the same.";
     this.cTime = nsInfo.getCTime();
@@ -563,13 +563,13 @@ public class BlockPoolSliceStorage extends Storage {
     // the namespace state or can be further upgraded to it.
     // In another word, we can only roll back when ( storedLV >= software LV)
     // && ( DN.previousCTime <= NN.ctime)
-    if (!(prevInfo.getLayoutVersion() >= HdfsConstants.DATANODE_LAYOUT_VERSION && 
+    if (!(prevInfo.getLayoutVersion() >= HdfsServerConstants.DATANODE_LAYOUT_VERSION &&
         prevInfo.getCTime() <= nsInfo.getCTime())) { // cannot rollback
       throw new InconsistentFSStateException(bpSd.getRoot(),
           "Cannot rollback to a newer state.\nDatanode previous state: LV = "
               + prevInfo.getLayoutVersion() + " CTime = " + prevInfo.getCTime()
               + " is newer than the namespace state: LV = "
-              + HdfsConstants.DATANODE_LAYOUT_VERSION + " CTime = " + nsInfo.getCTime());
+              + HdfsServerConstants.DATANODE_LAYOUT_VERSION + " CTime = " + nsInfo.getCTime());
     }
     
     LOG.info("Rolling back storage directory " + bpSd.getRoot()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
index c0be956..10692d4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
@@ -41,13 +41,13 @@ import org.apache.hadoop.fs.FSOutputSummer;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -246,7 +246,7 @@ class BlockReceiver implements Closeable {
             out.getClass());
       }
       this.checksumOut = new DataOutputStream(new BufferedOutputStream(
-          streams.getChecksumOut(), HdfsConstants.SMALL_BUFFER_SIZE));
+          streams.getChecksumOut(), HdfsServerConstants.SMALL_BUFFER_SIZE));
       // write data chunk header if creating a new replica
       if (isCreate) {
         BlockMetadataHeader.writeHeader(checksumOut, diskChecksum);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
index e76b93a..6b958a2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
@@ -35,8 +35,8 @@ import java.util.Arrays;
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
@@ -105,7 +105,7 @@ class BlockSender implements java.io.Closeable {
    */
   private static final int MIN_BUFFER_WITH_TRANSFERTO = 64*1024;
   private static final int TRANSFERTO_BUFFER_SIZE = Math.max(
-      HdfsConstants.IO_FILE_BUFFER_SIZE, MIN_BUFFER_WITH_TRANSFERTO);
+      HdfsServerConstants.IO_FILE_BUFFER_SIZE, MIN_BUFFER_WITH_TRANSFERTO);
   
   /** the block to read from */
   private final ExtendedBlock block;
@@ -298,7 +298,7 @@ class BlockSender implements java.io.Closeable {
             // storage and computes the checksum.
             if (metaIn.getLength() > BlockMetadataHeader.getHeaderSize()) {
               checksumIn = new DataInputStream(new BufferedInputStream(
-                  metaIn, HdfsConstants.IO_FILE_BUFFER_SIZE));
+                  metaIn, HdfsServerConstants.IO_FILE_BUFFER_SIZE));
   
               csum = BlockMetadataHeader.readDataChecksum(checksumIn, block);
               keepMetaInOpen = true;
@@ -747,7 +747,7 @@ class BlockSender implements java.io.Closeable {
         pktBufSize += checksumSize * maxChunksPerPacket;
       } else {
         maxChunksPerPacket = Math.max(1,
-            numberOfChunks(HdfsConstants.IO_FILE_BUFFER_SIZE));
+            numberOfChunks(HdfsServerConstants.IO_FILE_BUFFER_SIZE));
         // Packet size includes both checksum and data
         pktBufSize += (chunkSize + checksumSize) * maxChunksPerPacket;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index f042dff..22b1520 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -2156,7 +2156,7 @@ public class DataNode extends ReconfigurableBase
         unbufIn = saslStreams.in;
         
         out = new DataOutputStream(new BufferedOutputStream(unbufOut,
-            HdfsConstants.SMALL_BUFFER_SIZE));
+            HdfsServerConstants.SMALL_BUFFER_SIZE));
         in = new DataInputStream(unbufIn);
         blockSender = new BlockSender(b, 0, b.getNumBytes(), 
             false, false, true, DataNode.this, null, cachingStrategy);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
index 77fcfed..089e032 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
@@ -35,8 +35,8 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
@@ -60,7 +60,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
@@ -466,7 +465,7 @@ public class DataStorage extends Storage {
   void recoverTransitionRead(DataNode datanode, NamespaceInfo nsInfo,
       Collection<StorageLocation> dataDirs, StartupOption startOpt) throws IOException {
     if (this.initialized) {
-      LOG.info("DataNode version: " + HdfsConstants.DATANODE_LAYOUT_VERSION
+      LOG.info("DataNode version: " + HdfsServerConstants.DATANODE_LAYOUT_VERSION
           + " and NameNode layout version: " + nsInfo.getLayoutVersion());
       this.storageDirs = new ArrayList<StorageDirectory>(dataDirs.size());
       // mark DN storage is initialized
@@ -509,7 +508,7 @@ public class DataStorage extends Storage {
   void format(StorageDirectory sd, NamespaceInfo nsInfo,
               String datanodeUuid) throws IOException {
     sd.clearDirectory(); // create directory
-    this.layoutVersion = HdfsConstants.DATANODE_LAYOUT_VERSION;
+    this.layoutVersion = HdfsServerConstants.DATANODE_LAYOUT_VERSION;
     this.clusterID = nsInfo.getClusterID();
     this.namespaceID = nsInfo.getNamespaceID();
     this.cTime = 0;
@@ -664,7 +663,7 @@ public class DataStorage extends Storage {
     }
     readProperties(sd);
     checkVersionUpgradable(this.layoutVersion);
-    assert this.layoutVersion >= HdfsConstants.DATANODE_LAYOUT_VERSION :
+    assert this.layoutVersion >= HdfsServerConstants.DATANODE_LAYOUT_VERSION :
       "Future version is not allowed";
     
     boolean federationSupported = 
@@ -696,13 +695,13 @@ public class DataStorage extends Storage {
             DatanodeStorage.isValidStorageId(sd.getStorageUuid());
 
     // regular start up.
-    if (this.layoutVersion == HdfsConstants.DATANODE_LAYOUT_VERSION) {
+    if (this.layoutVersion == HdfsServerConstants.DATANODE_LAYOUT_VERSION) {
       createStorageID(sd, !haveValidStorageId);
       return; // regular startup
     }
 
     // do upgrade
-    if (this.layoutVersion > HdfsConstants.DATANODE_LAYOUT_VERSION) {
+    if (this.layoutVersion > HdfsServerConstants.DATANODE_LAYOUT_VERSION) {
       doUpgrade(datanode, sd, nsInfo);  // upgrade
       createStorageID(sd, !haveValidStorageId);
       return;
@@ -714,7 +713,7 @@ public class DataStorage extends Storage {
     // failed.
     throw new IOException("BUG: The stored LV = " + this.getLayoutVersion()
         + " is newer than the supported LV = "
-        + HdfsConstants.DATANODE_LAYOUT_VERSION);
+        + HdfsServerConstants.DATANODE_LAYOUT_VERSION);
   }
 
   /**
@@ -749,9 +748,9 @@ public class DataStorage extends Storage {
       // field and overwrite the file. The upgrade work is handled by
       // {@link BlockPoolSliceStorage#doUpgrade}
       LOG.info("Updating layout version from " + layoutVersion + " to "
-          + HdfsConstants.DATANODE_LAYOUT_VERSION + " for storage "
+          + HdfsServerConstants.DATANODE_LAYOUT_VERSION + " for storage "
           + sd.getRoot());
-      layoutVersion = HdfsConstants.DATANODE_LAYOUT_VERSION;
+      layoutVersion = HdfsServerConstants.DATANODE_LAYOUT_VERSION;
       writeProperties(sd);
       return;
     }
@@ -759,7 +758,7 @@ public class DataStorage extends Storage {
     LOG.info("Upgrading storage directory " + sd.getRoot()
              + ".\n   old LV = " + this.getLayoutVersion()
              + "; old CTime = " + this.getCTime()
-             + ".\n   new LV = " + HdfsConstants.DATANODE_LAYOUT_VERSION
+             + ".\n   new LV = " + HdfsServerConstants.DATANODE_LAYOUT_VERSION
              + "; new CTime = " + nsInfo.getCTime());
     
     File curDir = sd.getCurrentDir();
@@ -790,7 +789,7 @@ public class DataStorage extends Storage {
         STORAGE_DIR_CURRENT));
     
     // 4. Write version file under <SD>/current
-    layoutVersion = HdfsConstants.DATANODE_LAYOUT_VERSION;
+    layoutVersion = HdfsServerConstants.DATANODE_LAYOUT_VERSION;
     clusterID = nsInfo.getClusterID();
     writeProperties(sd);
     
@@ -848,11 +847,11 @@ public class DataStorage extends Storage {
     // This is a regular startup or a post-federation rollback
     if (!prevDir.exists()) {
       if (DataNodeLayoutVersion.supports(LayoutVersion.Feature.FEDERATION,
-          HdfsConstants.DATANODE_LAYOUT_VERSION)) {
-        readProperties(sd, HdfsConstants.DATANODE_LAYOUT_VERSION);
+          HdfsServerConstants.DATANODE_LAYOUT_VERSION)) {
+        readProperties(sd, HdfsServerConstants.DATANODE_LAYOUT_VERSION);
         writeProperties(sd);
         LOG.info("Layout version rolled back to "
-            + HdfsConstants.DATANODE_LAYOUT_VERSION + " for storage "
+            + HdfsServerConstants.DATANODE_LAYOUT_VERSION + " for storage "
             + sd.getRoot());
       }
       return;
@@ -862,16 +861,16 @@ public class DataStorage extends Storage {
 
     // We allow rollback to a state, which is either consistent with
     // the namespace state or can be further upgraded to it.
-    if (!(prevInfo.getLayoutVersion() >= HdfsConstants.DATANODE_LAYOUT_VERSION
+    if (!(prevInfo.getLayoutVersion() >= HdfsServerConstants.DATANODE_LAYOUT_VERSION
           && prevInfo.getCTime() <= nsInfo.getCTime()))  // cannot rollback
       throw new InconsistentFSStateException(sd.getRoot(),
           "Cannot rollback to a newer state.\nDatanode previous state: LV = "
               + prevInfo.getLayoutVersion() + " CTime = " + prevInfo.getCTime()
               + " is newer than the namespace state: LV = "
-              + HdfsConstants.DATANODE_LAYOUT_VERSION + " CTime = "
+              + HdfsServerConstants.DATANODE_LAYOUT_VERSION + " CTime = "
               + nsInfo.getCTime());
     LOG.info("Rolling back storage directory " + sd.getRoot()
-        + ".\n   target LV = " + HdfsConstants.DATANODE_LAYOUT_VERSION
+        + ".\n   target LV = " + HdfsServerConstants.DATANODE_LAYOUT_VERSION
         + "; target CTime = " + nsInfo.getCTime());
     File tmpDir = sd.getRemovedTmp();
     assert !tmpDir.exists() : "removed.tmp directory must not exist.";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
index 3b014e8..de25579 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
@@ -191,7 +191,7 @@ class DataXceiver extends Receiver implements Runnable {
           socketIn, datanode.getXferAddress().getPort(),
           datanode.getDatanodeId());
         input = new BufferedInputStream(saslStreams.in,
-          HdfsConstants.SMALL_BUFFER_SIZE);
+          HdfsServerConstants.SMALL_BUFFER_SIZE);
         socketOut = saslStreams.out;
       } catch (InvalidMagicNumberException imne) {
         if (imne.isHandshake4Encryption()) {
@@ -514,7 +514,7 @@ class DataXceiver extends Receiver implements Runnable {
     long read = 0;
     OutputStream baseStream = getOutputStream();
     DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
-        baseStream, HdfsConstants.SMALL_BUFFER_SIZE));
+        baseStream, HdfsServerConstants.SMALL_BUFFER_SIZE));
     checkAccess(out, true, block, blockToken,
         Op.READ_BLOCK, BlockTokenIdentifier.AccessMode.READ);
   
@@ -658,7 +658,7 @@ class DataXceiver extends Receiver implements Runnable {
     final DataOutputStream replyOut = new DataOutputStream(
         new BufferedOutputStream(
             getOutputStream(),
-            HdfsConstants.SMALL_BUFFER_SIZE));
+            HdfsServerConstants.SMALL_BUFFER_SIZE));
     checkAccess(replyOut, isClient, block, blockToken,
         Op.WRITE_BLOCK, BlockTokenIdentifier.AccessMode.WRITE);
 
@@ -717,7 +717,7 @@ class DataXceiver extends Receiver implements Runnable {
           unbufMirrorOut = saslStreams.out;
           unbufMirrorIn = saslStreams.in;
           mirrorOut = new DataOutputStream(new BufferedOutputStream(unbufMirrorOut,
-              HdfsConstants.SMALL_BUFFER_SIZE));
+              HdfsServerConstants.SMALL_BUFFER_SIZE));
           mirrorIn = new DataInputStream(unbufMirrorIn);
 
           // Do not propagate allowLazyPersist to downstream DataNodes.
@@ -932,7 +932,7 @@ class DataXceiver extends Receiver implements Runnable {
         .getMetaDataInputStream(block);
     
     final DataInputStream checksumIn = new DataInputStream(
-        new BufferedInputStream(metadataIn, HdfsConstants.IO_FILE_BUFFER_SIZE));
+        new BufferedInputStream(metadataIn, HdfsServerConstants.IO_FILE_BUFFER_SIZE));
     updateCurrentThreadName("Getting checksum for block " + block);
     try {
       //read metadata file
@@ -1024,7 +1024,7 @@ class DataXceiver extends Receiver implements Runnable {
       // set up response stream
       OutputStream baseStream = getOutputStream();
       reply = new DataOutputStream(new BufferedOutputStream(
-          baseStream, HdfsConstants.SMALL_BUFFER_SIZE));
+          baseStream, HdfsServerConstants.SMALL_BUFFER_SIZE));
 
       // send status first
       writeSuccessWithChecksumInfo(blockSender, reply);
@@ -1132,9 +1132,9 @@ class DataXceiver extends Receiver implements Runnable {
         unbufProxyIn = saslStreams.in;
         
         proxyOut = new DataOutputStream(new BufferedOutputStream(unbufProxyOut, 
-            HdfsConstants.SMALL_BUFFER_SIZE));
+            HdfsServerConstants.SMALL_BUFFER_SIZE));
         proxyReply = new DataInputStream(new BufferedInputStream(unbufProxyIn,
-            HdfsConstants.IO_FILE_BUFFER_SIZE));
+            HdfsServerConstants.IO_FILE_BUFFER_SIZE));
         
         /* send request to the proxy */
         IoeDuringCopyBlockOperation = true;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
index f6b058b..62885a9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
@@ -43,7 +43,7 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.util.Daemon;
@@ -305,7 +305,7 @@ public class DirectoryScanner implements Runnable {
     public long getGenStamp() {
       return metaSuffix != null ? Block.getGenerationStamp(
           getMetaFile().getName()) : 
-            HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP;
+            HdfsConstants.GRANDFATHER_GENERATION_STAMP;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
index 6daf039..414d078 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
@@ -41,7 +41,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
@@ -612,7 +612,7 @@ class BlockPoolSlice {
       }
       checksumIn = new DataInputStream(
           new BufferedInputStream(new FileInputStream(metaFile),
-              HdfsConstants.IO_FILE_BUFFER_SIZE));
+              HdfsServerConstants.IO_FILE_BUFFER_SIZE));
 
       // read and handle the common header here. For now just a version
       final DataChecksum checksum = BlockMetadataHeader.readDataChecksum(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/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 b87daec..d72b8bf 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
@@ -65,8 +65,8 @@ import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
@@ -957,7 +957,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
         }
       }
       metaOut = new DataOutputStream(new BufferedOutputStream(
-          new FileOutputStream(dstMeta), HdfsConstants.SMALL_BUFFER_SIZE));
+          new FileOutputStream(dstMeta), HdfsServerConstants.SMALL_BUFFER_SIZE));
       BlockMetadataHeader.writeHeader(metaOut, checksum);
 
       int offset = 0;
@@ -2128,7 +2128,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
 
       final long diskGS = diskMetaFile != null && diskMetaFile.exists() ?
           Block.getGenerationStamp(diskMetaFile.getName()) :
-            HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP;
+            HdfsConstants.GRANDFATHER_GENERATION_STAMP;
 
       if (diskFile == null || !diskFile.exists()) {
         if (memBlockInfo == null) {
@@ -2229,7 +2229,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           // as the block file, then use the generation stamp from it
           long gs = diskMetaFile != null && diskMetaFile.exists()
               && diskMetaFile.getParent().equals(memFile.getParent()) ? diskGS
-              : HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP;
+              : HdfsConstants.GRANDFATHER_GENERATION_STAMP;
 
           LOG.warn("Updating generation stamp for block " + blockId
               + " from " + memBlockInfo.getGenerationStamp() + " to " + gs);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java
index 1932706..8c5b4a1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java
@@ -24,7 +24,7 @@ import java.util.Arrays;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
 
 /** Utility methods. */
@@ -88,7 +88,7 @@ public class FsDatasetUtil {
       return Block.getGenerationStamp(listdir[j].getName());
     }
     FsDatasetImpl.LOG.warn("Block " + blockFile + " does not have a metafile!");
-    return HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP;
+    return HdfsConstants.GRANDFATHER_GENERATION_STAMP;
   }
 
   /** Find the corresponding meta data file from a given block file */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
index f1beb75..b32220c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
@@ -330,7 +330,7 @@ public class Mover {
     private boolean processFile(String fullPath, HdfsLocatedFileStatus status) {
       final byte policyId = status.getStoragePolicy();
       // currently we ignore files with unspecified storage policy
-      if (policyId == HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED) {
+      if (policyId == HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED) {
         return false;
       }
       final BlockStoragePolicy policy = blockStoragePolicies[policyId];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/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 430f00c..e6560ee 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
@@ -29,12 +29,12 @@ import org.apache.hadoop.ha.ServiceFailedException;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.NameNodeProxies;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalProtocolService;
 import org.apache.hadoop.hdfs.protocolPB.JournalProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.JournalProtocolServerSideTranslatorPB;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
@@ -156,7 +156,7 @@ public class BackupNode extends NameNode {
     // Backup node should never do lease recovery,
     // therefore lease hard limit should never expire.
     namesystem.leaseManager.setLeasePeriod(
-        HdfsConstants.LEASE_SOFTLIMIT_PERIOD, Long.MAX_VALUE);
+        HdfsServerConstants.LEASE_SOFTLIMIT_PERIOD, Long.MAX_VALUE);
 
     // register with the active name-node 
     registerWith(nsInfo);
@@ -414,9 +414,9 @@ public class BackupNode extends NameNode {
       LOG.error(errorMsg);
       throw new IOException(errorMsg);
     }
-    assert HdfsConstants.NAMENODE_LAYOUT_VERSION == nsInfo.getLayoutVersion() :
+    assert HdfsServerConstants.NAMENODE_LAYOUT_VERSION == nsInfo.getLayoutVersion() :
       "Active and backup node layout versions must be the same. Expected: "
-      + HdfsConstants.NAMENODE_LAYOUT_VERSION + " actual "+ nsInfo.getLayoutVersion();
+      + HdfsServerConstants.NAMENODE_LAYOUT_VERSION + " actual "+ nsInfo.getLayoutVersion();
     return nsInfo;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java
index 3649437..689cacc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java
@@ -21,8 +21,8 @@ import java.io.DataInputStream;
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
 
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 
 /**
  * An implementation of the abstract class {@link EditLogInputStream},
@@ -130,12 +130,12 @@ class EditLogBackupInputStream extends EditLogInputStream {
 
   @Override
   public long getFirstTxId() {
-    return HdfsConstants.INVALID_TXID;
+    return HdfsServerConstants.INVALID_TXID;
   }
 
   @Override
   public long getLastTxId() {
-    return HdfsConstants.INVALID_TXID;
+    return HdfsServerConstants.INVALID_TXID;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
index 974860c..3e21c24 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
@@ -33,9 +33,9 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutFlags;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation;
 import org.apache.hadoop.hdfs.server.namenode.TransferFsImage.HttpGetFailedException;
@@ -85,7 +85,7 @@ public class EditLogFileInputStream extends EditLogInputStream {
    */
   EditLogFileInputStream(File name)
       throws LogHeaderCorruptException, IOException {
-    this(name, HdfsConstants.INVALID_TXID, HdfsConstants.INVALID_TXID, false);
+    this(name, HdfsServerConstants.INVALID_TXID, HdfsServerConstants.INVALID_TXID, false);
   }
 
   /**
@@ -203,7 +203,7 @@ public class EditLogFileInputStream extends EditLogInputStream {
       if ((op != null) && (op.hasTransactionId())) {
         long txId = op.getTransactionId();
         if ((txId >= lastTxId) &&
-            (lastTxId != HdfsConstants.INVALID_TXID)) {
+            (lastTxId != HdfsServerConstants.INVALID_TXID)) {
           //
           // Sometimes, the NameNode crashes while it's writing to the
           // edit log.  In that case, you can end up with an unfinalized edit log
@@ -310,7 +310,7 @@ public class EditLogFileInputStream extends EditLogInputStream {
       // If the header is malformed or the wrong value, this indicates a corruption
       LOG.warn("Log file " + file + " has no valid header", e);
       return new FSEditLogLoader.EditLogValidation(0,
-          HdfsConstants.INVALID_TXID, true);
+          HdfsServerConstants.INVALID_TXID, true);
     }
     
     try {
@@ -331,18 +331,18 @@ public class EditLogFileInputStream extends EditLogInputStream {
     } catch (LogHeaderCorruptException e) {
       LOG.warn("Log file " + file + " has no valid header", e);
       return new FSEditLogLoader.EditLogValidation(0,
-          HdfsConstants.INVALID_TXID, true);
+          HdfsServerConstants.INVALID_TXID, true);
     }
 
     long lastPos = 0;
-    long lastTxId = HdfsConstants.INVALID_TXID;
+    long lastTxId = HdfsServerConstants.INVALID_TXID;
     long numValid = 0;
     try {
       while (true) {
-        long txid = HdfsConstants.INVALID_TXID;
+        long txid = HdfsServerConstants.INVALID_TXID;
         lastPos = in.getPosition();
         try {
-          if ((txid = in.scanNextOp()) == HdfsConstants.INVALID_TXID) {
+          if ((txid = in.scanNextOp()) == HdfsServerConstants.INVALID_TXID) {
             break;
           }
         } catch (Throwable t) {
@@ -354,7 +354,7 @@ public class EditLogFileInputStream extends EditLogInputStream {
           FSImage.LOG.warn("After resync, position is " + in.getPosition());
           continue;
         }
-        if (lastTxId == HdfsConstants.INVALID_TXID || txid > lastTxId) {
+        if (lastTxId == HdfsServerConstants.INVALID_TXID || txid > lastTxId) {
           lastTxId = txid;
         }
         numValid++;
@@ -382,12 +382,12 @@ public class EditLogFileInputStream extends EditLogInputStream {
           "Reached EOF when reading log header");
     }
     if (verifyLayoutVersion &&
-        (logVersion < HdfsConstants.NAMENODE_LAYOUT_VERSION || // future version
+        (logVersion < HdfsServerConstants.NAMENODE_LAYOUT_VERSION || // future version
          logVersion > Storage.LAST_UPGRADABLE_LAYOUT_VERSION)) { // unsupported
       throw new LogHeaderCorruptException(
           "Unexpected version of the file system log file: "
           + logVersion + ". Current version = "
-          + HdfsConstants.NAMENODE_LAYOUT_VERSION + ".");
+          + HdfsServerConstants.NAMENODE_LAYOUT_VERSION + ".");
     }
     return logVersion;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java
index ac58616..a4377cd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 
 import java.io.Closeable;
 import java.io.IOException;
@@ -112,7 +112,7 @@ public abstract class EditLogInputStream implements Closeable {
    */
   protected long scanNextOp() throws IOException {
     FSEditLogOp next = readOp();
-    return next != null ? next.txid : HdfsConstants.INVALID_TXID;
+    return next != null ? next.txid : HdfsServerConstants.INVALID_TXID;
   }
   
   /** 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditsDoubleBuffer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditsDoubleBuffer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditsDoubleBuffer.java
index f1da3fb..76a9eac 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditsDoubleBuffer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditsDoubleBuffer.java
@@ -21,7 +21,7 @@ import java.io.IOException;
 import java.io.OutputStream;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.Writer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.IOUtils;
@@ -141,7 +141,7 @@ public class EditsDoubleBuffer {
     }
 
     public void writeOp(FSEditLogOp op) throws IOException {
-      if (firstTxId == HdfsConstants.INVALID_TXID) {
+      if (firstTxId == HdfsServerConstants.INVALID_TXID) {
         firstTxId = op.txid;
       } else {
         assert op.txid > firstTxId;
@@ -153,7 +153,7 @@ public class EditsDoubleBuffer {
     @Override
     public DataOutputBuffer reset() {
       super.reset();
-      firstTxId = HdfsConstants.INVALID_TXID;
+      firstTxId = HdfsServerConstants.INVALID_TXID;
       numTxns = 0;
       return this;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/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 e77ddae..c636d93 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
@@ -30,7 +30,6 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -139,7 +138,7 @@ class FSDirStatAndListingOp {
   }
 
   private static byte getStoragePolicyID(byte inodePolicy, byte parentPolicy) {
-    return inodePolicy != HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED ? inodePolicy :
+    return inodePolicy != HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED ? inodePolicy :
         parentPolicy;
   }
 
@@ -175,7 +174,7 @@ class FSDirStatAndListingOp {
       if (targetNode == null)
         return null;
       byte parentStoragePolicy = isSuperUser ?
-          targetNode.getStoragePolicyID() : HdfsConstantsClient
+          targetNode.getStoragePolicyID() : HdfsConstants
           .BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
 
       if (!targetNode.isDirectory()) {
@@ -202,7 +201,7 @@ class FSDirStatAndListingOp {
         INode cur = contents.get(startChild+i);
         byte curPolicy = isSuperUser && !cur.isSymlink()?
             cur.getLocalStoragePolicyID():
-            HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
+            HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
         INodeAttributes nodeAttrs = getINodeAttributes(
             fsd, src, cur.getLocalNameBytes(), cur,
             snapshot);
@@ -265,7 +264,7 @@ class FSDirStatAndListingOp {
       listing[i] = createFileStatus(
           fsd, sRoot.getLocalNameBytes(),
           sRoot, nodeAttrs,
-          HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
+          HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
           Snapshot.CURRENT_STATE_ID, false,
           INodesInPath.fromINode(sRoot));
     }
@@ -293,8 +292,8 @@ class FSDirStatAndListingOp {
       }
 
       byte policyId = includeStoragePolicy && !i.isSymlink() ?
-          i.getStoragePolicyID() : HdfsConstantsClient
-          .BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
+          i.getStoragePolicyID() :
+          HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
       INodeAttributes nodeAttrs = getINodeAttributes(
           fsd, path, HdfsFileStatus.EMPTY_NAME, i, src.getPathSnapshotId());
       return createFileStatus(
@@ -316,7 +315,7 @@ class FSDirStatAndListingOp {
       if (fsd.getINode4DotSnapshot(srcs) != null) {
         return new HdfsFileStatus(0, true, 0, 0, 0, 0, null, null, null, null,
             HdfsFileStatus.EMPTY_NAME, -1L, 0, null,
-            HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
+            HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
       }
       return null;
     }
@@ -330,7 +329,6 @@ class FSDirStatAndListingOp {
     }
   }
 
-
   /**
    * create an hdfs file status from an inode
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/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 0189f63..bf538ed 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
@@ -49,7 +49,6 @@ import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
@@ -60,6 +59,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderCon
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.util.ByteArray;
@@ -742,7 +742,7 @@ public class FSDirectory implements Closeable {
     EnumCounters<StorageType> typeSpaceDeltas =
         new EnumCounters<StorageType>(StorageType.class);
     // Storage type and its quota are only available when storage policy is set
-    if (storagePolicyID != HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED) {
+    if (storagePolicyID != HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED) {
       BlockStoragePolicy storagePolicy = getBlockManager().getStoragePolicy(storagePolicyID);
 
       if (oldRep != newRep) {
@@ -884,7 +884,7 @@ public class FSDirectory implements Closeable {
 
   /** Verify if the inode name is legal. */
   void verifyINodeName(byte[] childName) throws HadoopIllegalArgumentException {
-    if (Arrays.equals(HdfsConstants.DOT_SNAPSHOT_DIR_BYTES, childName)) {
+    if (Arrays.equals(HdfsServerConstants.DOT_SNAPSHOT_DIR_BYTES, childName)) {
       String s = "\"" + HdfsConstants.DOT_SNAPSHOT_DIR + "\" is a reserved name.";
       if (!namesystem.isImageLoaded()) {
         s += "  Please rename it before upgrade.";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
index df9c585..bda827a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
@@ -43,9 +43,9 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.Storage.FormatConfirmable;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@@ -159,7 +159,7 @@ public class FSEditLog implements LogsPurgeable {
 
   // the first txid of the log that's currently open for writing.
   // If this value is N, we are currently writing to edits_inprogress_N
-  private long curSegmentTxId = HdfsConstants.INVALID_TXID;
+  private long curSegmentTxId = HdfsServerConstants.INVALID_TXID;
 
   // the time of printing the statistics to the log file.
   private long lastPrintTime;
@@ -1338,7 +1338,7 @@ public class FSEditLog implements LogsPurgeable {
       return;
     }
     
-    assert curSegmentTxId == HdfsConstants.INVALID_TXID || // on format this is no-op
+    assert curSegmentTxId == HdfsServerConstants.INVALID_TXID || // on format this is no-op
       minTxIdToKeep <= curSegmentTxId :
       "cannot purge logs older than txid " + minTxIdToKeep +
       " when current segment starts at " + curSegmentTxId;
@@ -1602,7 +1602,7 @@ public class FSEditLog implements LogsPurgeable {
       EditLogInputStream elis = iter.next();
       if (elis.getFirstTxId() > txId) break;
       long next = elis.getLastTxId();
-      if (next == HdfsConstants.INVALID_TXID) {
+      if (next == HdfsServerConstants.INVALID_TXID) {
         if (!inProgressOk) {
           throw new RuntimeException("inProgressOk = false, but " +
               "selectInputStreams returned an in-progress edit " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index d234cf2..eaa2e77 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -35,16 +35,16 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.XAttrSetFlag;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
@@ -302,7 +302,7 @@ public class FSEditLogLoader {
       long lastInodeId) throws IOException {
     long inodeId = inodeIdFromOp;
 
-    if (inodeId == HdfsConstantsClient.GRANDFATHER_INODE_ID) {
+    if (inodeId == HdfsConstants.GRANDFATHER_INODE_ID) {
       if (NameNodeLayoutVersion.supports(
           LayoutVersion.Feature.ADD_INODE_ID, logVersion)) {
         throw new IOException("The layout version " + logVersion
@@ -322,7 +322,7 @@ public class FSEditLogLoader {
   @SuppressWarnings("deprecation")
   private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir,
       StartupOption startOpt, int logVersion, long lastInodeId) throws IOException {
-    long inodeId = HdfsConstantsClient.GRANDFATHER_INODE_ID;
+    long inodeId = HdfsConstants.GRANDFATHER_INODE_ID;
     if (LOG.isTraceEnabled()) {
       LOG.trace("replaying edit log: " + op);
     }
@@ -380,7 +380,7 @@ public class FSEditLogLoader {
         if (toAddRetryCache) {
           HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatusForEditLog(
               fsNamesys.dir, path, HdfsFileStatus.EMPTY_NAME, newFile,
-              HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID,
+              HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID,
               false, iip);
           fsNamesys.addCacheEntryWithPayload(addCloseOp.rpcClientId,
               addCloseOp.rpcCallId, stat);
@@ -400,7 +400,7 @@ public class FSEditLogLoader {
             HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatusForEditLog(
                 fsNamesys.dir, path,
                 HdfsFileStatus.EMPTY_NAME, newFile,
-                HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
+                HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
                 Snapshot.CURRENT_STATE_ID, false, iip);
             fsNamesys.addCacheEntryWithPayload(addCloseOp.rpcClientId,
                 addCloseOp.rpcCallId, new LastBlockWithStatus(lb, stat));
@@ -473,7 +473,7 @@ public class FSEditLogLoader {
         if (toAddRetryCache) {
           HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatusForEditLog(
               fsNamesys.dir, path, HdfsFileStatus.EMPTY_NAME, file,
-              HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
+              HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
               Snapshot.CURRENT_STATE_ID, false, iip);
           fsNamesys.addCacheEntryWithPayload(appendOp.rpcClientId,
               appendOp.rpcCallId, new LastBlockWithStatus(lb, stat));
@@ -1095,7 +1095,7 @@ public class FSEditLogLoader {
     // The editlog must be emptied by restarting the namenode, before proceeding
     // with the upgrade.
     if (Storage.is203LayoutVersion(logVersion)
-        && logVersion != HdfsConstants.NAMENODE_LAYOUT_VERSION) {
+        && logVersion != HdfsServerConstants.NAMENODE_LAYOUT_VERSION) {
       String msg = "During upgrade failed to load the editlog version "
           + logVersion + " from release 0.20.203. Please go back to the old "
           + " release and restart the namenode. This empties the editlog "
@@ -1112,7 +1112,7 @@ public class FSEditLogLoader {
    */
   static EditLogValidation validateEditLog(EditLogInputStream in) {
     long lastPos = 0;
-    long lastTxId = HdfsConstants.INVALID_TXID;
+    long lastTxId = HdfsServerConstants.INVALID_TXID;
     long numValid = 0;
     FSEditLogOp op = null;
     while (true) {
@@ -1129,7 +1129,7 @@ public class FSEditLogLoader {
         FSImage.LOG.warn("After resync, position is " + in.getPosition());
         continue;
       }
-      if (lastTxId == HdfsConstants.INVALID_TXID
+      if (lastTxId == HdfsServerConstants.INVALID_TXID
           || op.getTransactionId() > lastTxId) {
         lastTxId = op.getTransactionId();
       }
@@ -1140,7 +1140,7 @@ public class FSEditLogLoader {
 
   static EditLogValidation scanEditLog(EditLogInputStream in) {
     long lastPos = 0;
-    long lastTxId = HdfsConstants.INVALID_TXID;
+    long lastTxId = HdfsServerConstants.INVALID_TXID;
     long numValid = 0;
     FSEditLogOp op = null;
     while (true) {
@@ -1157,7 +1157,7 @@ public class FSEditLogLoader {
         FSImage.LOG.warn("After resync, position is " + in.getPosition());
         continue;
       }
-      if (lastTxId == HdfsConstants.INVALID_TXID
+      if (lastTxId == HdfsServerConstants.INVALID_TXID
           || op.getTransactionId() > lastTxId) {
         lastTxId = op.getTransactionId();
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
index 0daf12f..c8e565e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
@@ -101,13 +101,13 @@ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEditLogProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.XAttrEditLogProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.util.XMLUtils;
 import org.apache.hadoop.hdfs.util.XMLUtils.InvalidXmlException;
 import org.apache.hadoop.hdfs.util.XMLUtils.Stanza;
@@ -148,7 +148,7 @@ public abstract class FSEditLogOp {
   int rpcCallId;
 
   final void reset() {
-    txid = HdfsConstants.INVALID_TXID;
+    txid = HdfsServerConstants.INVALID_TXID;
     rpcClientId = RpcConstants.DUMMY_CLIENT_ID;
     rpcCallId = RpcConstants.INVALID_CALL_ID;
     resetSubFields();
@@ -241,16 +241,16 @@ public abstract class FSEditLogOp {
   }
 
   public long getTransactionId() {
-    Preconditions.checkState(txid != HdfsConstants.INVALID_TXID);
+    Preconditions.checkState(txid != HdfsServerConstants.INVALID_TXID);
     return txid;
   }
 
   public String getTransactionIdStr() {
-    return (txid == HdfsConstants.INVALID_TXID) ? "(none)" : "" + txid;
+    return (txid == HdfsServerConstants.INVALID_TXID) ? "(none)" : "" + txid;
   }
   
   public boolean hasTransactionId() {
-    return (txid != HdfsConstants.INVALID_TXID);
+    return (txid != HdfsServerConstants.INVALID_TXID);
   }
 
   public void setTransactionId(long txid) {
@@ -433,7 +433,7 @@ public abstract class FSEditLogOp {
     
     private AddCloseOp(FSEditLogOpCodes opCode) {
       super(opCode);
-      storagePolicyId = HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
+      storagePolicyId = HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
       assert(opCode == OP_ADD || opCode == OP_CLOSE || opCode == OP_APPEND);
     }
 
@@ -577,7 +577,7 @@ public abstract class FSEditLogOp {
         this.inodeId = in.readLong();
       } else {
         // The inodeId should be updated when this editLogOp is applied
-        this.inodeId = HdfsConstantsClient.GRANDFATHER_INODE_ID;
+        this.inodeId = HdfsConstants.GRANDFATHER_INODE_ID;
       }
       if ((-17 < logVersion && length != 4) ||
           (logVersion <= -17 && length != 5 && !NameNodeLayoutVersion.supports(
@@ -635,7 +635,7 @@ public abstract class FSEditLogOp {
             NameNodeLayoutVersion.Feature.BLOCK_STORAGE_POLICY, logVersion)) {
           this.storagePolicyId = FSImageSerialization.readByte(in);
         } else {
-          this.storagePolicyId = HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
+          this.storagePolicyId = HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
         }
         // read clientId and callId
         readRpcIds(in, logVersion);
@@ -1652,7 +1652,7 @@ public abstract class FSEditLogOp {
         this.inodeId = FSImageSerialization.readLong(in);
       } else {
         // This id should be updated when this editLogOp is applied
-        this.inodeId = HdfsConstantsClient.GRANDFATHER_INODE_ID;
+        this.inodeId = HdfsConstants.GRANDFATHER_INODE_ID;
       }
       this.path = FSImageSerialization.readString(in);
       if (NameNodeLayoutVersion.supports(
@@ -2545,7 +2545,7 @@ public abstract class FSEditLogOp {
         this.inodeId = FSImageSerialization.readLong(in);
       } else {
         // This id should be updated when the editLogOp is applied
-        this.inodeId = HdfsConstantsClient.GRANDFATHER_INODE_ID;
+        this.inodeId = HdfsConstants.GRANDFATHER_INODE_ID;
       }
       this.path = FSImageSerialization.readString(in);
       this.value = FSImageSerialization.readString(in);
@@ -4689,7 +4689,7 @@ public abstract class FSEditLogOp {
         // Read the txid
         op.setTransactionId(in.readLong());
       } else {
-        op.setTransactionId(HdfsConstants.INVALID_TXID);
+        op.setTransactionId(HdfsServerConstants.INVALID_TXID);
       }
 
       op.readFields(in, logVersion);
@@ -4712,13 +4712,13 @@ public abstract class FSEditLogOp {
         try {
           opCodeByte = in.readByte(); // op code
         } catch (EOFException e) {
-          return HdfsConstants.INVALID_TXID;
+          return HdfsServerConstants.INVALID_TXID;
         }
 
         FSEditLogOpCodes opCode = FSEditLogOpCodes.fromByte(opCodeByte);
         if (opCode == OP_INVALID) {
           verifyTerminator();
-          return HdfsConstants.INVALID_TXID;
+          return HdfsServerConstants.INVALID_TXID;
         }
 
         int length = in.readInt(); // read the length of the op
@@ -4730,7 +4730,7 @@ public abstract class FSEditLogOp {
         return txid;
       } else {
         FSEditLogOp op = decodeOp();
-        return op == null ? HdfsConstants.INVALID_TXID : op.getTransactionId();
+        return op == null ? HdfsServerConstants.INVALID_TXID : op.getTransactionId();
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
index 433b05a..6a54967 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
@@ -44,9 +44,9 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@@ -230,7 +230,7 @@ public class FSImage implements Closeable {
     if (startOpt == StartupOption.METADATAVERSION) {
       System.out.println("HDFS Image Version: " + layoutVersion);
       System.out.println("Software format version: " +
-        HdfsConstants.NAMENODE_LAYOUT_VERSION);
+        HdfsServerConstants.NAMENODE_LAYOUT_VERSION);
       return false;
     }
 
@@ -241,11 +241,11 @@ public class FSImage implements Closeable {
         && startOpt != StartupOption.UPGRADEONLY
         && !RollingUpgradeStartupOption.STARTED.matches(startOpt)
         && layoutVersion < Storage.LAST_PRE_UPGRADE_LAYOUT_VERSION
-        && layoutVersion != HdfsConstants.NAMENODE_LAYOUT_VERSION) {
+        && layoutVersion != HdfsServerConstants.NAMENODE_LAYOUT_VERSION) {
       throw new IOException(
           "\nFile system image contains an old layout version " 
           + storage.getLayoutVersion() + ".\nAn upgrade to version "
-          + HdfsConstants.NAMENODE_LAYOUT_VERSION + " is required.\n"
+          + HdfsServerConstants.NAMENODE_LAYOUT_VERSION + " is required.\n"
           + "Please restart NameNode with the \""
           + RollingUpgradeStartupOption.STARTED.getOptionString()
           + "\" option if a rolling upgrade is already started;"
@@ -397,7 +397,7 @@ public class FSImage implements Closeable {
     long oldCTime = storage.getCTime();
     storage.cTime = now();  // generate new cTime for the state
     int oldLV = storage.getLayoutVersion();
-    storage.layoutVersion = HdfsConstants.NAMENODE_LAYOUT_VERSION;
+    storage.layoutVersion = HdfsServerConstants.NAMENODE_LAYOUT_VERSION;
     
     List<StorageDirectory> errorSDs =
       Collections.synchronizedList(new ArrayList<StorageDirectory>());
@@ -458,11 +458,11 @@ public class FSImage implements Closeable {
     boolean canRollback = false;
     FSImage prevState = new FSImage(conf);
     try {
-      prevState.getStorage().layoutVersion = HdfsConstants.NAMENODE_LAYOUT_VERSION;
+      prevState.getStorage().layoutVersion = HdfsServerConstants.NAMENODE_LAYOUT_VERSION;
       for (Iterator<StorageDirectory> it = storage.dirIterator(false); it.hasNext();) {
         StorageDirectory sd = it.next();
         if (!NNUpgradeUtil.canRollBack(sd, storage, prevState.getStorage(),
-            HdfsConstants.NAMENODE_LAYOUT_VERSION)) {
+            HdfsServerConstants.NAMENODE_LAYOUT_VERSION)) {
           continue;
         }
         LOG.info("Can perform rollback for " + sd);
@@ -473,7 +473,7 @@ public class FSImage implements Closeable {
         // If HA is enabled, check if the shared log can be rolled back as well.
         editLog.initJournalsForWrite();
         boolean canRollBackSharedEditLog = editLog.canRollBackSharedLog(
-            prevState.getStorage(), HdfsConstants.NAMENODE_LAYOUT_VERSION);
+            prevState.getStorage(), HdfsServerConstants.NAMENODE_LAYOUT_VERSION);
         if (canRollBackSharedEditLog) {
           LOG.info("Can perform rollback for shared edit log.");
           canRollback = true;
@@ -839,7 +839,7 @@ public class FSImage implements Closeable {
           lastAppliedTxId = loader.getLastAppliedTxId();
         }
         // If we are in recovery mode, we may have skipped over some txids.
-        if (editIn.getLastTxId() != HdfsConstants.INVALID_TXID) {
+        if (editIn.getLastTxId() != HdfsServerConstants.INVALID_TXID) {
           lastAppliedTxId = editIn.getLastTxId();
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
index 906213e..d1d007f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
@@ -55,6 +55,7 @@ import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
@@ -1046,10 +1047,10 @@ public class FSImageFormat {
   @VisibleForTesting
   public static void useDefaultRenameReservedPairs() {
     renameReservedMap.clear();
-    for (String key: HdfsConstants.RESERVED_PATH_COMPONENTS) {
+    for (String key: HdfsServerConstants.RESERVED_PATH_COMPONENTS) {
       renameReservedMap.put(
           key,
-          key + "." + HdfsConstants.NAMENODE_LAYOUT_VERSION + "."
+          key + "." + HdfsServerConstants.NAMENODE_LAYOUT_VERSION + "."
               + "UPGRADE_RENAMED");
     }
   }
@@ -1147,7 +1148,7 @@ public class FSImageFormat {
       final int layoutVersion) {
     // If the LV doesn't support snapshots, we're doing an upgrade
     if (!NameNodeLayoutVersion.supports(Feature.SNAPSHOT, layoutVersion)) {
-      if (Arrays.equals(component, HdfsConstants.DOT_SNAPSHOT_DIR_BYTES)) {
+      if (Arrays.equals(component, HdfsServerConstants.DOT_SNAPSHOT_DIR_BYTES)) {
         Preconditions.checkArgument(
             renameReservedMap.containsKey(HdfsConstants.DOT_SNAPSHOT_DIR),
             RESERVED_ERROR_MSG);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
index 3ee848a..24afcae 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
@@ -42,11 +42,11 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.CacheManagerSection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.NameSystemSection;
@@ -143,7 +143,7 @@ public final class FSImageFormatProtobuf {
     private long imgTxId;
     /**
      * Whether the image's layout version must be the same with
-     * {@link HdfsConstants#NAMENODE_LAYOUT_VERSION}. This is only set to true
+     * {@link HdfsServerConstants#NAMENODE_LAYOUT_VERSION}. This is only set to true
      * when we're doing (rollingUpgrade rollback).
      */
     private final boolean requireSameLayoutVersion;
@@ -192,10 +192,10 @@ public final class FSImageFormatProtobuf {
       }
       FileSummary summary = FSImageUtil.loadSummary(raFile);
       if (requireSameLayoutVersion && summary.getLayoutVersion() !=
-          HdfsConstants.NAMENODE_LAYOUT_VERSION) {
+          HdfsServerConstants.NAMENODE_LAYOUT_VERSION) {
         throw new IOException("Image version " + summary.getLayoutVersion() +
             " is not equal to the software version " +
-            HdfsConstants.NAMENODE_LAYOUT_VERSION);
+            HdfsServerConstants.NAMENODE_LAYOUT_VERSION);
       }
 
       FileChannel channel = fin.getChannel();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImagePreTransactionalStorageInspector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImagePreTransactionalStorageInspector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImagePreTransactionalStorageInspector.java
index d5243e0..3cd5ae1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImagePreTransactionalStorageInspector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImagePreTransactionalStorageInspector.java
@@ -34,7 +34,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
@@ -180,7 +180,7 @@ class FSImagePreTransactionalStorageInspector extends FSImageStorageInspector {
     
     FSImageFile file = new FSImageFile(latestNameSD, 
         NNStorage.getStorageFile(latestNameSD, NameNodeFile.IMAGE),
-        HdfsConstants.INVALID_TXID);
+        HdfsServerConstants.INVALID_TXID);
     LinkedList<FSImageFile> ret = new LinkedList<FSImageFile>();
     ret.add(file);
     return ret;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageStorageInspector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageStorageInspector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageStorageInspector.java
index cb1351b..dfda34e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageStorageInspector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageStorageInspector.java
@@ -21,9 +21,9 @@ import java.io.File;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 
 /**
@@ -69,7 +69,7 @@ abstract class FSImageStorageInspector {
     private final File file;
     
     FSImageFile(StorageDirectory sd, File file, long txId) {
-      assert txId >= 0 || txId == HdfsConstants.INVALID_TXID 
+      assert txId >= 0 || txId == HdfsServerConstants.INVALID_TXID
         : "Invalid txid on " + file +": " + txId;
       
       this.sd = sd;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/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 809d594..72e1d32 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
@@ -184,7 +184,6 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
@@ -212,6 +211,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
@@ -1021,7 +1021,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   private void updateStorageVersionForRollingUpgrade(final long layoutVersion,
       StartupOption startOpt) throws IOException {
     boolean rollingStarted = RollingUpgradeStartupOption.STARTED
-        .matches(startOpt) && layoutVersion > HdfsConstants
+        .matches(startOpt) && layoutVersion > HdfsServerConstants
         .NAMENODE_LAYOUT_VERSION;
     boolean rollingRollback = RollingUpgradeStartupOption.ROLLBACK
         .matches(startOpt);
@@ -3234,7 +3234,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     Block previousBlock = ExtendedBlock.getLocalBlock(previous);
     final INode inode;
     final INodesInPath iip;
-    if (fileId == HdfsConstantsClient.GRANDFATHER_INODE_ID) {
+    if (fileId == HdfsConstants.GRANDFATHER_INODE_ID) {
       // Older clients may not have given us an inode ID to work with.
       // In this case, we have to try to resolve the path and hope it
       // hasn't changed or been deleted since the file was opened for write.
@@ -3354,7 +3354,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
       //check lease
       final INode inode;
-      if (fileId == HdfsConstantsClient.GRANDFATHER_INODE_ID) {
+      if (fileId == HdfsConstants.GRANDFATHER_INODE_ID) {
         // Older clients may not have given us an inode ID to work with.
         // In this case, we have to try to resolve the path and hope it
         // hasn't changed or been deleted since the file was opened for write.
@@ -3411,7 +3411,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
       final INode inode;
       final INodesInPath iip;
-      if (fileId == HdfsConstantsClient.GRANDFATHER_INODE_ID) {
+      if (fileId == HdfsConstants.GRANDFATHER_INODE_ID) {
         // Older clients may not have given us an inode ID to work with.
         // In this case, we have to try to resolve the path and hope it
         // hasn't changed or been deleted since the file was opened for write.
@@ -3530,7 +3530,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final INodesInPath iip;
     INode inode = null;
     try {
-      if (fileId == HdfsConstantsClient.GRANDFATHER_INODE_ID) {
+      if (fileId == HdfsConstants.GRANDFATHER_INODE_ID) {
         // Older clients may not have given us an inode ID to work with.
         // In this case, we have to try to resolve the path and hope it
         // hasn't changed or been deleted since the file was opened for write.
@@ -3981,7 +3981,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkNameNodeSafeMode("Cannot fsync file " + src);
       src = dir.resolvePath(pc, src, pathComponents);
       final INode inode;
-      if (fileId == HdfsConstantsClient.GRANDFATHER_INODE_ID) {
+      if (fileId == HdfsConstants.GRANDFATHER_INODE_ID) {
         // Older clients may not have given us an inode ID to work with.
         // In this case, we have to try to resolve the path and hope it
         // hasn't changed or been deleted since the file was opened for write.
@@ -4439,7 +4439,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     //remove lease, close file
     finalizeINodeFileUnderConstruction(src, pendingFile,
-        Snapshot.findLatestSnapshot(pendingFile, Snapshot.CURRENT_STATE_ID));
+                                       Snapshot.findLatestSnapshot(pendingFile,
+                                                                   Snapshot.CURRENT_STATE_ID));
 
     return src;
   }
@@ -5731,7 +5732,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null)
       return;
-    safeMode.setBlockTotal((int)getCompleteBlocksTotal());
+    safeMode.setBlockTotal((int) getCompleteBlocksTotal());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b125d0de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
index b760dd8..ebd7475 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
@@ -32,7 +32,7 @@ import java.util.regex.Pattern;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@@ -286,7 +286,7 @@ public class FileJournalManager implements JournalManager {
         try {
           long startTxId = Long.parseLong(inProgressEditsMatch.group(1));
           ret.add(
-              new EditLogFile(f, startTxId, HdfsConstants.INVALID_TXID, true));
+              new EditLogFile(f, startTxId, HdfsServerConstants.INVALID_TXID, true));
           continue;
         } catch (NumberFormatException nfe) {
           LOG.error("In-progress edits file " + f + " has improperly " +
@@ -301,7 +301,7 @@ public class FileJournalManager implements JournalManager {
         if (staleInprogressEditsMatch.matches()) {
           try {
             long startTxId = Long.parseLong(staleInprogressEditsMatch.group(1));
-            ret.add(new EditLogFile(f, startTxId, HdfsConstants.INVALID_TXID,
+            ret.add(new EditLogFile(f, startTxId, HdfsServerConstants.INVALID_TXID,
                 true));
             continue;
           } catch (NumberFormatException nfe) {
@@ -348,7 +348,7 @@ public class FileJournalManager implements JournalManager {
         }
       }
       if (elf.lastTxId < fromTxId) {
-        assert elf.lastTxId != HdfsConstants.INVALID_TXID;
+        assert elf.lastTxId != HdfsServerConstants.INVALID_TXID;
         if (LOG.isDebugEnabled()) {
           LOG.debug("passing over " + elf + " because it ends at " +
               elf.lastTxId + ", but we only care about transactions " +
@@ -391,7 +391,7 @@ public class FileJournalManager implements JournalManager {
           throw new CorruptionException("In-progress edit log file is corrupt: "
               + elf);
         }
-        if (elf.getLastTxId() == HdfsConstants.INVALID_TXID) {
+        if (elf.getLastTxId() == HdfsServerConstants.INVALID_TXID) {
           // If the file has a valid header (isn't corrupt) but contains no
           // transactions, we likely just crashed after opening the file and
           // writing the header, but before syncing any transactions. Safe to
@@ -480,19 +480,19 @@ public class FileJournalManager implements JournalManager {
     EditLogFile(File file,
         long firstTxId, long lastTxId) {
       this(file, firstTxId, lastTxId, false);
-      assert (lastTxId != HdfsConstants.INVALID_TXID)
+      assert (lastTxId != HdfsServerConstants.INVALID_TXID)
         && (lastTxId >= firstTxId);
     }
     
     EditLogFile(File file, long firstTxId, 
                 long lastTxId, boolean isInProgress) { 
-      assert (lastTxId == HdfsConstants.INVALID_TXID && isInProgress)
-        || (lastTxId != HdfsConstants.INVALID_TXID && lastTxId >= firstTxId);
-      assert (firstTxId > 0) || (firstTxId == HdfsConstants.INVALID_TXID);
+      assert (lastTxId == HdfsServerConstants.INVALID_TXID && isInProgress)
+        || (lastTxId != HdfsServerConstants.INVALID_TXID && lastTxId >= firstTxId);
+      assert (firstTxId > 0) || (firstTxId == HdfsServerConstants.INVALID_TXID);
       assert file != null;
       
       Preconditions.checkArgument(!isInProgress ||
-          lastTxId == HdfsConstants.INVALID_TXID);
+          lastTxId == HdfsServerConstants.INVALID_TXID);
       
       this.firstTxId = firstTxId;
       this.lastTxId = lastTxId;
@@ -552,7 +552,7 @@ public class FileJournalManager implements JournalManager {
     }
 
     public void moveAsideEmptyFile() throws IOException {
-      assert lastTxId == HdfsConstants.INVALID_TXID;
+      assert lastTxId == HdfsServerConstants.INVALID_TXID;
       renameSelf(".empty");
     }
       


[15/50] [abbrv] hadoop git commit: HADOOP-11891. OsSecureRandom should lazily fill its reservoir (asuresh)

Posted by zj...@apache.org.
HADOOP-11891. OsSecureRandom should lazily fill its reservoir (asuresh)


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

Branch: refs/heads/YARN-2928
Commit: 6f6b313bc072a8be7663e51a21a955b70240376d
Parents: 9840141
Author: Arun Suresh <as...@apache.org>
Authored: Thu Apr 30 13:59:43 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:55 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt  |  2 ++
 .../hadoop/crypto/random/OsSecureRandom.java     | 19 ++++---------------
 2 files changed, 6 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f6b313b/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 373b33e..d2e1d4a 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -603,6 +603,8 @@ Release 2.7.1 - UNRELEASED
     HADOOP-11802. DomainSocketWatcher thread terminates sometimes after there
     is an I/O error during requestShortCircuitShm (cmccabe)
 
+    HADOOP-11891. OsSecureRandom should lazily fill its reservoir (asuresh)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f6b313b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/random/OsSecureRandom.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/random/OsSecureRandom.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/random/OsSecureRandom.java
index fee4186..2010e6d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/random/OsSecureRandom.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/random/OsSecureRandom.java
@@ -58,6 +58,9 @@ public class OsSecureRandom extends Random implements Closeable, Configurable {
   private void fillReservoir(int min) {
     if (pos >= reservoir.length - min) {
       try {
+        if (stream == null) {
+          stream = new FileInputStream(new File(randomDevPath));
+        }
         IOUtils.readFully(stream, reservoir, 0, reservoir.length);
       } catch (IOException e) {
         throw new RuntimeException("failed to fill reservoir", e);
@@ -75,21 +78,7 @@ public class OsSecureRandom extends Random implements Closeable, Configurable {
     this.randomDevPath = conf.get(
         HADOOP_SECURITY_SECURE_RANDOM_DEVICE_FILE_PATH_KEY,
         HADOOP_SECURITY_SECURE_RANDOM_DEVICE_FILE_PATH_DEFAULT);
-    File randomDevFile = new File(randomDevPath);
-
-    try {
-      close();
-      this.stream = new FileInputStream(randomDevFile);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-
-    try {
-      fillReservoir(0);
-    } catch (RuntimeException e) {
-      close();
-      throw e;
-    }
+    close();
   }
 
   @Override