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 ji...@apache.org on 2015/05/05 01:43:05 UTC

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

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 1b3b9e5c3 -> d701acc9c


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/279958b7
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/279958b7
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/279958b7

Branch: refs/heads/HDFS-7240
Commit: 279958b772c25e0633bd967828b7d27d5c0a6a56
Parents: 1b3b9e5
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Fri May 1 08:42:00 2015 -0700
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Fri May 1 08:42:00 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/279958b7/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/279958b7/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/279958b7/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/279958b7/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;
         }


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

Posted by ji...@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/3ba18362
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3ba18362
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3ba18362

Branch: refs/heads/HDFS-7240
Commit: 3ba18362f2a4b83635b89aa0adc5ebaf27d9ca83
Parents: a319771
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Mon May 4 15:02:21 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Mon May 4 15:02:21 2015 +0900

----------------------------------------------------------------------
 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/3ba18362/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 2d87444..062042c 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -359,6 +359,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/3ba18362/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/3ba18362/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);


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

Posted by ji...@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/f8204e24
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f8204e24
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f8204e24

Branch: refs/heads/HDFS-7240
Commit: f8204e241d9271497defd4d42646fb89c61cefe3
Parents: 6f541ed
Author: Gera Shegalov <ge...@apache.org>
Authored: Fri May 1 14:49:09 2015 -0700
Committer: Gera Shegalov <ge...@apache.org>
Committed: Fri May 1 18:18:55 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/f8204e24/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 6f38201..c110f88 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -287,6 +287,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/f8204e24/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 e511ff0..ca21f11 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/f8204e24/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 0dd9ba1..b44d13b 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;
@@ -200,7 +201,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();
@@ -220,10 +223,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/f8204e24/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/f8204e24/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);
+    }
+  }
 }


[33/33] hadoop git commit: YARN-2725. Added test cases of retrying creating znode in ZKRMStateStore. Contributed by Tsuyoshi Ozawa

Posted by ji...@apache.org.
YARN-2725. Added test cases of retrying creating znode in ZKRMStateStore. Contributed by Tsuyoshi Ozawa


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

Branch: refs/heads/HDFS-7240
Commit: d701acc9c67adc578ba18035edde1166eedaae98
Parents: 71f4de2
Author: Jian He <ji...@apache.org>
Authored: Mon May 4 16:13:29 2015 -0700
Committer: Jian He <ji...@apache.org>
Committed: Mon May 4 16:13:29 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                          |  3 +++
 .../server/resourcemanager/recovery/ZKRMStateStore.java  |  2 +-
 .../resourcemanager/recovery/TestZKRMStateStore.java     | 11 +++++++++++
 3 files changed, 15 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d701acc9/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 09ec41a..287205d 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -302,6 +302,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3375. NodeHealthScriptRunner.shouldRun() check is performing 3 times for 
     starting NodeHealthScriptRunner. (Devaraj K via wangda)
 
+    YARN-2725. Added test cases of retrying creating znode in ZKRMStateStore.
+    (Tsuyoshi Ozawa via jianhe)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d701acc9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.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/recovery/ZKRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
index 9da6400..97dd029 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
@@ -297,7 +297,7 @@ public class ZKRMStateStore extends RMStateStore {
     createRootDir(amrmTokenSecretManagerRoot);
   }
 
-  private void createRootDir(final String rootPath) throws Exception {
+  protected void createRootDir(final String rootPath) throws Exception {
     // For root dirs, we shouldn't use the doMulti helper methods
     new ZKAction<String>() {
       @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d701acc9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.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/recovery/TestZKRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java
index 6b09d39..c632a06 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java
@@ -100,6 +100,15 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
         return workingZnode + "/" + ROOT_ZNODE_NAME + "/" + RM_APP_ROOT + "/"
             + appId;
       }
+
+      /**
+       * Emulating retrying createRootDir not to raise NodeExist exception
+       * @throws Exception
+       */
+      public void testRetryingCreateRootDir() throws Exception {
+        createRootDir(znodeWorkingPath);
+      }
+
     }
 
     public RMStateStore getRMStateStore() throws Exception {
@@ -148,6 +157,8 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
     testDeleteStore(zkTester);
     testRemoveApplication(zkTester);
     testAMRMTokenSecretManagerStateStore(zkTester);
+    ((TestZKRMStateStoreTester.TestZKRMStateStoreInternal)
+        zkTester.getRMStateStore()).testRetryingCreateRootDir();
   }
 
   @Test (timeout = 60000)


[22/33] 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 ji...@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/bb6ef298
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/bb6ef298
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/bb6ef298

Branch: refs/heads/HDFS-7240
Commit: bb6ef2984d8f117711b806c4ebdc757bd182c06e
Parents: 3ba1836
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Mon May 4 15:36:33 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Mon May 4 15:36:33 2015 +0900

----------------------------------------------------------------------
 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/bb6ef298/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 062042c..19f95fc 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -362,6 +362,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/bb6ef298/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 = 


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

Posted by ji...@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/6ae2a0d0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6ae2a0d0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6ae2a0d0

Branch: refs/heads/HDFS-7240
Commit: 6ae2a0d048e133b43249c248a75a4d77d9abb80d
Parents: 57d9a97
Author: Haohui Mai <wh...@apache.org>
Authored: Fri May 1 15:27:28 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Sat May 2 10:03:23 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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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);


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

Posted by ji...@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/8f65c793
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8f65c793
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8f65c793

Branch: refs/heads/HDFS-7240
Commit: 8f65c793f2930bfd16885a2ab188a9970b754974
Parents: bb9ddef
Author: Jason Lowe <jl...@apache.org>
Authored: Mon May 4 15:31:15 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Mon May 4 15:31:15 2015 +0000

----------------------------------------------------------------------
 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/8f65c793/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 684efc5..08762e3 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -296,6 +296,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/8f65c793/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()));
     }


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

Posted by ji...@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/64d30a61
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/64d30a61
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/64d30a61

Branch: refs/heads/HDFS-7240
Commit: 64d30a61867e5cb0a2acaa7ae4fa4d3eb3be8edd
Parents: 3393461
Author: Arpit Agarwal <ar...@apache.org>
Authored: Fri May 1 11:11:48 2015 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Fri May 1 14:43:09 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/64d30a61/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/64d30a61/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/64d30a61/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/64d30a61/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/33] hadoop git commit: HDFS-8249. Separate HdfsConstants into the client and the server side class. Contributed by Haohui Mai.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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);


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

Posted by ji...@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/f343f865
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f343f865
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f343f865

Branch: refs/heads/HDFS-7240
Commit: f343f8657e2b01773a32c2c7d960dc368954b42e
Parents: f8204e2
Author: Gera Shegalov <ge...@apache.org>
Authored: Fri May 1 15:44:36 2015 -0700
Committer: Gera Shegalov <ge...@apache.org>
Committed: Fri May 1 18:18:55 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/f343f865/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/f343f865/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/f343f865/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);
+  }
+
 }


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

Posted by ji...@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/76fa606e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/76fa606e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/76fa606e

Branch: refs/heads/HDFS-7240
Commit: 76fa606e2d3d04407f2f6b4ea276cce0f60db4be
Parents: bb6ef29
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Mon May 4 17:05:00 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Mon May 4 17:05:00 2015 +0900

----------------------------------------------------------------------
 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/76fa606e/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/76fa606e/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.");


[31/33] hadoop git commit: HDFS-8237. Move all protocol classes used by ClientProtocol to hdfs-client. Contributed by Haohui Mai.

Posted by ji...@apache.org.
HDFS-8237. Move all protocol classes used by ClientProtocol to hdfs-client. 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/0d6aa5d6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/0d6aa5d6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/0d6aa5d6

Branch: refs/heads/HDFS-7240
Commit: 0d6aa5d60948a7966da0ca1c3344a37c1d32f2e9
Parents: bf70c5a
Author: Haohui Mai <wh...@apache.org>
Authored: Mon May 4 15:00:29 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Mon May 4 15:00:29 2015 -0700

----------------------------------------------------------------------
 .../dev-support/findbugsExcludeFile.xml         |   3 +
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |  85 +++++
 .../hdfs/client/HdfsClientConfigKeys.java       |   1 +
 .../hdfs/protocol/CacheDirectiveEntry.java      |  45 +++
 .../hdfs/protocol/CacheDirectiveInfo.java       | 358 +++++++++++++++++++
 .../hdfs/protocol/CacheDirectiveStats.java      | 169 +++++++++
 .../hadoop/hdfs/protocol/CachePoolEntry.java    |  45 +++
 .../hadoop/hdfs/protocol/CachePoolInfo.java     | 229 ++++++++++++
 .../hadoop/hdfs/protocol/CachePoolStats.java    | 115 ++++++
 .../hdfs/protocol/SnapshotDiffReport.java       | 215 +++++++++++
 .../protocol/SnapshottableDirectoryStatus.java  | 218 +++++++++++
 .../delegation/DelegationTokenSelector.java     |  74 ++++
 .../namenode/NotReplicatedYetException.java     |  37 ++
 .../hdfs/server/protocol/DatanodeStorage.java   | 126 +++++++
 .../server/protocol/DatanodeStorageReport.java  |  42 +++
 .../hdfs/server/protocol/StorageReport.java     |  66 ++++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |  75 +---
 .../hdfs/protocol/CacheDirectiveEntry.java      |  45 ---
 .../hdfs/protocol/CacheDirectiveInfo.java       | 358 -------------------
 .../hdfs/protocol/CacheDirectiveStats.java      | 169 ---------
 .../hadoop/hdfs/protocol/CachePoolEntry.java    |  45 ---
 .../hadoop/hdfs/protocol/CachePoolInfo.java     | 229 ------------
 .../hadoop/hdfs/protocol/CachePoolStats.java    | 115 ------
 .../hdfs/protocol/SnapshotDiffReport.java       | 215 -----------
 .../protocol/SnapshottableDirectoryStatus.java  | 218 -----------
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  10 +-
 .../delegation/DelegationTokenSelector.java     |  74 ----
 .../hadoop/hdfs/server/namenode/INode.java      |   3 +-
 .../namenode/NotReplicatedYetException.java     |  37 --
 .../namenode/snapshot/SnapshotManager.java      |   3 +-
 .../hdfs/server/protocol/DatanodeStorage.java   | 126 -------
 .../server/protocol/DatanodeStorageReport.java  |  42 ---
 .../hdfs/server/protocol/StorageReport.java     |  66 ----
 34 files changed, 1842 insertions(+), 1819 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml b/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
index 7aade70..be2911f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
@@ -11,6 +11,9 @@
       <Class name="org.apache.hadoop.hdfs.protocol.DirectoryListing"/>
       <Class name="org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier"/>
       <Class name="org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey"/>
+      <Class name="org.apache.hadoop.hdfs.protocol.SnapshotDiffReport$DiffReportEntry"/>
+      <Class name="org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus"/>
+      <Class name="org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport"/>
     </Or>
     <Bug pattern="EI_EXPOSE_REP,EI_EXPOSE_REP2" />
   </Match>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
index 97d3408..eda135e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs;
 
 import com.google.common.base.Joiner;
 import com.google.common.collect.Maps;
+import com.google.common.primitives.SignedBytes;
 import org.apache.commons.io.Charsets;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
@@ -36,15 +37,19 @@ import org.slf4j.LoggerFactory;
 
 import java.io.UnsupportedEncodingException;
 import java.net.InetSocketAddress;
+import java.text.SimpleDateFormat;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Date;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_NAMESERVICES;
 
 public class DFSUtilClient {
+  public static final byte[] EMPTY_BYTES = {};
   private static final Logger LOG = LoggerFactory.getLogger(
       DFSUtilClient.class);
   /**
@@ -184,6 +189,48 @@ public class DFSUtilClient {
     return blkLocations;
   }
 
+  /** Compare two byte arrays by lexicographical order. */
+  public static int compareBytes(byte[] left, byte[] right) {
+    if (left == null) {
+      left = EMPTY_BYTES;
+    }
+    if (right == null) {
+      right = EMPTY_BYTES;
+    }
+    return SignedBytes.lexicographicalComparator().compare(left, right);
+  }
+
+  /**
+   * Given a list of path components returns a byte array
+   */
+  public static byte[] byteArray2bytes(byte[][] pathComponents) {
+    if (pathComponents.length == 0) {
+      return EMPTY_BYTES;
+    } else if (pathComponents.length == 1
+        && (pathComponents[0] == null || pathComponents[0].length == 0)) {
+      return new byte[]{(byte) Path.SEPARATOR_CHAR};
+    }
+    int length = 0;
+    for (int i = 0; i < pathComponents.length; i++) {
+      length += pathComponents[i].length;
+      if (i < pathComponents.length - 1) {
+        length++; // for SEPARATOR
+      }
+    }
+    byte[] path = new byte[length];
+    int index = 0;
+    for (int i = 0; i < pathComponents.length; i++) {
+      System.arraycopy(pathComponents[i], 0, path, index,
+          pathComponents[i].length);
+      index += pathComponents[i].length;
+      if (i < pathComponents.length - 1) {
+        path[index] = (byte) Path.SEPARATOR_CHAR;
+        index++;
+      }
+    }
+    return path;
+  }
+
   /**
    * Decode a specific range of bytes of the given byte array to a string
    * using UTF8.
@@ -343,4 +390,42 @@ public class DFSUtilClient {
     }
     return true;
   }
+
+  /**
+   * Converts a time duration in milliseconds into DDD:HH:MM:SS format.
+   */
+  public static String durationToString(long durationMs) {
+    boolean negative = false;
+    if (durationMs < 0) {
+      negative = true;
+      durationMs = -durationMs;
+    }
+    // Chop off the milliseconds
+    long durationSec = durationMs / 1000;
+    final int secondsPerMinute = 60;
+    final int secondsPerHour = 60*60;
+    final int secondsPerDay = 60*60*24;
+    final long days = durationSec / secondsPerDay;
+    durationSec -= days * secondsPerDay;
+    final long hours = durationSec / secondsPerHour;
+    durationSec -= hours * secondsPerHour;
+    final long minutes = durationSec / secondsPerMinute;
+    durationSec -= minutes * secondsPerMinute;
+    final long seconds = durationSec;
+    final long milliseconds = durationMs % 1000;
+    String format = "%03d:%02d:%02d:%02d.%03d";
+    if (negative)  {
+      format = "-" + format;
+    }
+    return String.format(format, days, hours, minutes, seconds, milliseconds);
+  }
+
+  /**
+   * Converts a Date into an ISO-8601 formatted datetime string.
+   */
+  public static String dateToIso8601String(Date date) {
+    SimpleDateFormat df =
+        new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ssZ", Locale.ENGLISH);
+    return df.format(date);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
index 86c8a87..26283aa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
@@ -38,6 +38,7 @@ public interface HdfsClientConfigKeys {
   int     DFS_NAMENODE_HTTPS_PORT_DEFAULT = 50470;
   String  DFS_NAMENODE_HTTPS_ADDRESS_KEY = "dfs.namenode.https-address";
   String DFS_HA_NAMENODES_KEY_PREFIX = "dfs.ha.namenodes";
+  int DFS_NAMENODE_RPC_PORT_DEFAULT = 8020;
 
   /** dfs.client.retry configuration properties */
   interface Retry {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveEntry.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveEntry.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveEntry.java
new file mode 100644
index 0000000..fe3215f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveEntry.java
@@ -0,0 +1,45 @@
+/**
+ * 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;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Describes a path-based cache directive entry.
+ */
+@InterfaceStability.Evolving
+@InterfaceAudience.Public
+public class CacheDirectiveEntry {
+  private final CacheDirectiveInfo info;
+  private final CacheDirectiveStats stats;
+
+  public CacheDirectiveEntry(CacheDirectiveInfo info,
+      CacheDirectiveStats stats) {
+    this.info = info;
+    this.stats = stats;
+  }
+
+  public CacheDirectiveInfo getInfo() {
+    return info;
+  }
+
+  public CacheDirectiveStats getStats() {
+    return stats;
+  }
+};

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveInfo.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveInfo.java
new file mode 100644
index 0000000..2305c59
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveInfo.java
@@ -0,0 +1,358 @@
+/**
+ * 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.Date;
+
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.Path;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+
+/**
+ * Describes a path-based cache directive.
+ */
+@InterfaceStability.Evolving
+@InterfaceAudience.Public
+public class CacheDirectiveInfo {
+  /**
+   * A builder for creating new CacheDirectiveInfo instances.
+   */
+  public static class Builder {
+    private Long id;
+    private Path path;
+    private Short replication;
+    private String pool;
+    private Expiration expiration;
+
+    /**
+     * Builds a new CacheDirectiveInfo populated with the set properties.
+     * 
+     * @return New CacheDirectiveInfo.
+     */
+    public CacheDirectiveInfo build() {
+      return new CacheDirectiveInfo(id, path, replication, pool, expiration);
+    }
+
+    /**
+     * Creates an empty builder.
+     */
+    public Builder() {
+    }
+
+    /**
+     * Creates a builder with all elements set to the same values as the
+     * given CacheDirectiveInfo.
+     */
+    public Builder(CacheDirectiveInfo directive) {
+      this.id = directive.getId();
+      this.path = directive.getPath();
+      this.replication = directive.getReplication();
+      this.pool = directive.getPool();
+      this.expiration = directive.getExpiration();
+    }
+
+    /**
+     * Sets the id used in this request.
+     * 
+     * @param id The id used in this request.
+     * @return This builder, for call chaining.
+     */
+    public Builder setId(Long id) {
+      this.id = id;
+      return this;
+    }
+
+    /**
+     * Sets the path used in this request.
+     * 
+     * @param path The path used in this request.
+     * @return This builder, for call chaining.
+     */
+    public Builder setPath(Path path) {
+      this.path = path;
+      return this;
+    }
+
+    /**
+     * Sets the replication used in this request.
+     * 
+     * @param replication The replication used in this request.
+     * @return This builder, for call chaining.
+     */
+    public Builder setReplication(Short replication) {
+      this.replication = replication;
+      return this;
+    }
+
+    /**
+     * Sets the pool used in this request.
+     * 
+     * @param pool The pool used in this request.
+     * @return This builder, for call chaining.
+     */
+    public Builder setPool(String pool) {
+      this.pool = pool;
+      return this;
+    }
+
+    /**
+     * Sets when the CacheDirective should expire. A
+     * {@link CacheDirectiveInfo.Expiration} can specify either an absolute or
+     * relative expiration time.
+     * 
+     * @param expiration when this CacheDirective should expire
+     * @return This builder, for call chaining
+     */
+    public Builder setExpiration(Expiration expiration) {
+      this.expiration = expiration;
+      return this;
+    }
+  }
+
+  /**
+   * Denotes a relative or absolute expiration time for a CacheDirective. Use
+   * factory methods {@link CacheDirectiveInfo.Expiration#newAbsolute(Date)} and
+   * {@link CacheDirectiveInfo.Expiration#newRelative(long)} to create an
+   * Expiration.
+   * <p>
+   * In either case, the server-side clock is used to determine when a
+   * CacheDirective expires.
+   */
+  public static class Expiration {
+
+    /**
+     * The maximum value we accept for a relative expiry.
+     */
+    public static final long MAX_RELATIVE_EXPIRY_MS =
+        Long.MAX_VALUE / 4; // This helps prevent weird overflow bugs
+
+    /**
+     * An relative Expiration that never expires.
+     */
+    public static final Expiration NEVER = newRelative(MAX_RELATIVE_EXPIRY_MS);
+
+    /**
+     * Create a new relative Expiration.
+     * <p>
+     * Use {@link Expiration#NEVER} to indicate an Expiration that never
+     * expires.
+     * 
+     * @param ms how long until the CacheDirective expires, in milliseconds
+     * @return A relative Expiration
+     */
+    public static Expiration newRelative(long ms) {
+      return new Expiration(ms, true);
+    }
+
+    /**
+     * Create a new absolute Expiration.
+     * <p>
+     * Use {@link Expiration#NEVER} to indicate an Expiration that never
+     * expires.
+     * 
+     * @param date when the CacheDirective expires
+     * @return An absolute Expiration
+     */
+    public static Expiration newAbsolute(Date date) {
+      return new Expiration(date.getTime(), false);
+    }
+
+    /**
+     * Create a new absolute Expiration.
+     * <p>
+     * Use {@link Expiration#NEVER} to indicate an Expiration that never
+     * expires.
+     * 
+     * @param ms when the CacheDirective expires, in milliseconds since the Unix
+     *          epoch.
+     * @return An absolute Expiration
+     */
+    public static Expiration newAbsolute(long ms) {
+      return new Expiration(ms, false);
+    }
+
+    private final long ms;
+    private final boolean isRelative;
+
+    private Expiration(long ms, boolean isRelative) {
+      if (isRelative) {
+        Preconditions.checkArgument(ms <= MAX_RELATIVE_EXPIRY_MS,
+            "Expiration time is too far in the future!");
+      }
+      this.ms = ms;
+      this.isRelative = isRelative;
+    }
+
+    /**
+     * @return true if Expiration was specified as a relative duration, false if
+     *         specified as an absolute time.
+     */
+    public boolean isRelative() {
+      return isRelative;
+    }
+
+    /**
+     * @return The raw underlying millisecond value, either a relative duration
+     *         or an absolute time as milliseconds since the Unix epoch.
+     */
+    public long getMillis() {
+      return ms;
+    }
+
+    /**
+     * @return Expiration time as a {@link Date} object. This converts a
+     *         relative Expiration into an absolute Date based on the local
+     *         clock.
+     */
+    public Date getAbsoluteDate() {
+      return new Date(getAbsoluteMillis());
+    }
+
+    /**
+     * @return Expiration time in milliseconds from the Unix epoch. This
+     *         converts a relative Expiration into an absolute time based on the
+     *         local clock.
+     */
+    public long getAbsoluteMillis() {
+      if (!isRelative) {
+        return ms;
+      } else {
+        return new Date().getTime() + ms;
+      }
+    }
+
+    @Override
+    public String toString() {
+      if (isRelative) {
+        return DFSUtilClient.durationToString(ms);
+      }
+      return DFSUtilClient.dateToIso8601String(new Date(ms));
+    }
+  }
+
+  private final Long id;
+  private final Path path;
+  private final Short replication;
+  private final String pool;
+  private final Expiration expiration;
+
+  CacheDirectiveInfo(Long id, Path path, Short replication, String pool,
+      Expiration expiration) {
+    this.id = id;
+    this.path = path;
+    this.replication = replication;
+    this.pool = pool;
+    this.expiration = expiration;
+  }
+
+  /**
+   * @return The ID of this directive.
+   */
+  public Long getId() {
+    return id;
+  }
+
+  /**
+   * @return The path used in this request.
+   */
+  public Path getPath() {
+    return path;
+  }
+
+  /**
+   * @return The number of times the block should be cached.
+   */
+  public Short getReplication() {
+    return replication;
+  }
+
+  /**
+   * @return The pool used in this request.
+   */
+  public String getPool() {
+    return pool;
+  }
+
+  /**
+   * @return When this directive expires.
+   */
+  public Expiration getExpiration() {
+    return expiration;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o == null) {
+      return false;
+    }
+    if (getClass() != o.getClass()) {
+      return false;
+    }
+    CacheDirectiveInfo other = (CacheDirectiveInfo)o;
+    return new EqualsBuilder().append(getId(), other.getId()).
+        append(getPath(), other.getPath()).
+        append(getReplication(), other.getReplication()).
+        append(getPool(), other.getPool()).
+        append(getExpiration(), other.getExpiration()).
+        isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder().append(id).
+        append(path).
+        append(replication).
+        append(pool).
+        append(expiration).
+        hashCode();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append("{");
+    String prefix = "";
+    if (id != null) {
+      builder.append(prefix).append("id: ").append(id);
+      prefix = ", ";
+    }
+    if (path != null) {
+      builder.append(prefix).append("path: ").append(path);
+      prefix = ", ";
+    }
+    if (replication != null) {
+      builder.append(prefix).append("replication: ").append(replication);
+      prefix = ", ";
+    }
+    if (pool != null) {
+      builder.append(prefix).append("pool: ").append(pool);
+      prefix = ", ";
+    }
+    if (expiration != null) {
+      builder.append(prefix).append("expiration: ").append(expiration);
+      prefix = ", ";
+    }
+    builder.append("}");
+    return builder.toString();
+  }
+};

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveStats.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveStats.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveStats.java
new file mode 100644
index 0000000..0fd4ca2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveStats.java
@@ -0,0 +1,169 @@
+/**
+ * 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;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Describes a path-based cache directive.
+ */
+@InterfaceStability.Evolving
+@InterfaceAudience.Public
+public class CacheDirectiveStats {
+  public static class Builder {
+    private long bytesNeeded;
+    private long bytesCached;
+    private long filesNeeded;
+    private long filesCached;
+    private boolean hasExpired;
+
+    /**
+     * Builds a new CacheDirectiveStats populated with the set properties.
+     * 
+     * @return New CacheDirectiveStats.
+     */
+    public CacheDirectiveStats build() {
+      return new CacheDirectiveStats(bytesNeeded, bytesCached, filesNeeded,
+          filesCached, hasExpired);
+    }
+
+    /**
+     * Creates an empty builder.
+     */
+    public Builder() {
+    }
+
+    /**
+     * Sets the bytes needed by this directive.
+     * 
+     * @param bytesNeeded The bytes needed.
+     * @return This builder, for call chaining.
+     */
+    public Builder setBytesNeeded(long bytesNeeded) {
+      this.bytesNeeded = bytesNeeded;
+      return this;
+    }
+
+    /**
+     * Sets the bytes cached by this directive.
+     * 
+     * @param bytesCached The bytes cached.
+     * @return This builder, for call chaining.
+     */
+    public Builder setBytesCached(long bytesCached) {
+      this.bytesCached = bytesCached;
+      return this;
+    }
+
+    /**
+     * Sets the files needed by this directive.
+     * @param filesNeeded The number of files needed
+     * @return This builder, for call chaining.
+     */
+    public Builder setFilesNeeded(long filesNeeded) {
+      this.filesNeeded = filesNeeded;
+      return this;
+    }
+
+    /**
+     * Sets the files cached by this directive.
+     * 
+     * @param filesCached The number of files cached.
+     * @return This builder, for call chaining.
+     */
+    public Builder setFilesCached(long filesCached) {
+      this.filesCached = filesCached;
+      return this;
+    }
+
+    /**
+     * Sets whether this directive has expired.
+     * 
+     * @param hasExpired if this directive has expired
+     * @return This builder, for call chaining.
+     */
+    public Builder setHasExpired(boolean hasExpired) {
+      this.hasExpired = hasExpired;
+      return this;
+    }
+  }
+
+  private final long bytesNeeded;
+  private final long bytesCached;
+  private final long filesNeeded;
+  private final long filesCached;
+  private final boolean hasExpired;
+
+  private CacheDirectiveStats(long bytesNeeded, long bytesCached,
+      long filesNeeded, long filesCached, boolean hasExpired) {
+    this.bytesNeeded = bytesNeeded;
+    this.bytesCached = bytesCached;
+    this.filesNeeded = filesNeeded;
+    this.filesCached = filesCached;
+    this.hasExpired = hasExpired;
+  }
+
+  /**
+   * @return The bytes needed.
+   */
+  public long getBytesNeeded() {
+    return bytesNeeded;
+  }
+
+  /**
+   * @return The bytes cached.
+   */
+  public long getBytesCached() {
+    return bytesCached;
+  }
+
+  /**
+   * @return The number of files needed.
+   */
+  public long getFilesNeeded() {
+    return filesNeeded;
+  }
+
+  /**
+   * @return The number of files cached.
+   */
+  public long getFilesCached() {
+    return filesCached;
+  }
+
+  /**
+   * @return Whether this directive has expired.
+   */
+  public boolean hasExpired() {
+    return hasExpired;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append("{");
+    builder.append("bytesNeeded: ").append(bytesNeeded);
+    builder.append(", ").append("bytesCached: ").append(bytesCached);
+    builder.append(", ").append("filesNeeded: ").append(filesNeeded);
+    builder.append(", ").append("filesCached: ").append(filesCached);
+    builder.append(", ").append("hasExpired: ").append(hasExpired);
+    builder.append("}");
+    return builder.toString();
+  }
+};

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolEntry.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolEntry.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolEntry.java
new file mode 100644
index 0000000..3c1e345
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolEntry.java
@@ -0,0 +1,45 @@
+/**
+ * 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;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Describes a Cache Pool entry.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class CachePoolEntry {
+  private final CachePoolInfo info;
+  private final CachePoolStats stats;
+
+  public CachePoolEntry(CachePoolInfo info, CachePoolStats stats) {
+    this.info = info;
+    this.stats = stats;
+  }
+
+  public CachePoolInfo getInfo() {
+    return info;
+  }
+
+  public CachePoolStats getStats() {
+    return stats;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
new file mode 100644
index 0000000..61bbe38
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
@@ -0,0 +1,229 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.protocol;
+
+import java.io.IOException;
+
+import javax.annotation.Nullable;
+
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+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.fs.InvalidRequestException;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo.Expiration;
+
+/**
+ * CachePoolInfo describes a cache pool.
+ *
+ * This class is used in RPCs to create and modify cache pools.
+ * It is serializable and can be stored in the edit log.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class CachePoolInfo {
+  public static final Log LOG = LogFactory.getLog(CachePoolInfo.class);
+
+  /**
+   * Indicates that the pool does not have a maximum relative expiry.
+   */
+  public static final long RELATIVE_EXPIRY_NEVER =
+      Expiration.MAX_RELATIVE_EXPIRY_MS;
+  /**
+   * Default max relative expiry for cache pools.
+   */
+  public static final long DEFAULT_MAX_RELATIVE_EXPIRY =
+      RELATIVE_EXPIRY_NEVER;
+
+  public static final long LIMIT_UNLIMITED = Long.MAX_VALUE;
+  public static final long DEFAULT_LIMIT = LIMIT_UNLIMITED;
+
+  final String poolName;
+
+  @Nullable
+  String ownerName;
+
+  @Nullable
+  String groupName;
+
+  @Nullable
+  FsPermission mode;
+
+  @Nullable
+  Long limit;
+
+  @Nullable
+  Long maxRelativeExpiryMs;
+
+  public CachePoolInfo(String poolName) {
+    this.poolName = poolName;
+  }
+
+  /**
+   * @return Name of the pool.
+   */
+  public String getPoolName() {
+    return poolName;
+  }
+
+  /**
+   * @return The owner of the pool. Along with the group and mode, determines
+   *         who has access to view and modify the pool.
+   */
+  public String getOwnerName() {
+    return ownerName;
+  }
+
+  public CachePoolInfo setOwnerName(String ownerName) {
+    this.ownerName = ownerName;
+    return this;
+  }
+
+  /**
+   * @return The group of the pool. Along with the owner and mode, determines
+   *         who has access to view and modify the pool.
+   */
+  public String getGroupName() {
+    return groupName;
+  }
+
+  public CachePoolInfo setGroupName(String groupName) {
+    this.groupName = groupName;
+    return this;
+  }
+
+  /**
+   * @return Unix-style permissions of the pool. Along with the owner and group,
+   *         determines who has access to view and modify the pool.
+   */
+  public FsPermission getMode() {
+    return mode;
+  }
+
+  public CachePoolInfo setMode(FsPermission mode) {
+    this.mode = mode;
+    return this;
+  }
+
+  /**
+   * @return The maximum aggregate number of bytes that can be cached by
+   *         directives in this pool.
+   */
+  public Long getLimit() {
+    return limit;
+  }
+
+  public CachePoolInfo setLimit(Long bytes) {
+    this.limit = bytes;
+    return this;
+  }
+
+  /**
+   * @return The maximum relative expiration of directives of this pool in
+   *         milliseconds
+   */
+  public Long getMaxRelativeExpiryMs() {
+    return maxRelativeExpiryMs;
+  }
+
+  /**
+   * Set the maximum relative expiration of directives of this pool in
+   * milliseconds.
+   * 
+   * @param ms in milliseconds
+   * @return This builder, for call chaining.
+   */
+  public CachePoolInfo setMaxRelativeExpiryMs(Long ms) {
+    this.maxRelativeExpiryMs = ms;
+    return this;
+  }
+
+  public String toString() {
+    return new StringBuilder().append("{").
+      append("poolName:").append(poolName).
+      append(", ownerName:").append(ownerName).
+      append(", groupName:").append(groupName).
+      append(", mode:").append((mode == null) ? "null" :
+          String.format("0%03o", mode.toShort())).
+      append(", limit:").append(limit).
+      append(", maxRelativeExpiryMs:").append(maxRelativeExpiryMs).
+      append("}").toString();
+  }
+  
+  @Override
+  public boolean equals(Object o) {
+    if (o == null) { return false; }
+    if (o == this) { return true; }
+    if (o.getClass() != getClass()) {
+      return false;
+    }
+    CachePoolInfo other = (CachePoolInfo)o;
+    return new EqualsBuilder().
+        append(poolName, other.poolName).
+        append(ownerName, other.ownerName).
+        append(groupName, other.groupName).
+        append(mode, other.mode).
+        append(limit, other.limit).
+        append(maxRelativeExpiryMs, other.maxRelativeExpiryMs).
+        isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder().
+        append(poolName).
+        append(ownerName).
+        append(groupName).
+        append(mode).
+        append(limit).
+        append(maxRelativeExpiryMs).
+        hashCode();
+  }
+
+  public static void validate(CachePoolInfo info) throws IOException {
+    if (info == null) {
+      throw new InvalidRequestException("CachePoolInfo is null");
+    }
+    if ((info.getLimit() != null) && (info.getLimit() < 0)) {
+      throw new InvalidRequestException("Limit is negative.");
+    }
+    if (info.getMaxRelativeExpiryMs() != null) {
+      long maxRelativeExpiryMs = info.getMaxRelativeExpiryMs();
+      if (maxRelativeExpiryMs < 0l) {
+        throw new InvalidRequestException("Max relative expiry is negative.");
+      }
+      if (maxRelativeExpiryMs > Expiration.MAX_RELATIVE_EXPIRY_MS) {
+        throw new InvalidRequestException("Max relative expiry is too big.");
+      }
+    }
+    validateName(info.poolName);
+  }
+
+  public static void validateName(String poolName) throws IOException {
+    if (poolName == null || poolName.isEmpty()) {
+      // Empty pool names are not allowed because they would be highly
+      // confusing.  They would also break the ability to list all pools
+      // by starting with prevKey = ""
+      throw new IOException("invalid empty cache pool name");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolStats.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolStats.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolStats.java
new file mode 100644
index 0000000..c552652
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolStats.java
@@ -0,0 +1,115 @@
+/**
+ * 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;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * CachePoolStats describes cache pool statistics.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class CachePoolStats {
+  public static class Builder {
+    private long bytesNeeded;
+    private long bytesCached;
+    private long bytesOverlimit;
+    private long filesNeeded;
+    private long filesCached;
+
+    public Builder() {
+    }
+
+    public Builder setBytesNeeded(long bytesNeeded) {
+      this.bytesNeeded = bytesNeeded;
+      return this;
+    }
+
+    public Builder setBytesCached(long bytesCached) {
+      this.bytesCached = bytesCached;
+      return this;
+    }
+
+    public Builder setBytesOverlimit(long bytesOverlimit) {
+      this.bytesOverlimit = bytesOverlimit;
+      return this;
+    }
+
+    public Builder setFilesNeeded(long filesNeeded) {
+      this.filesNeeded = filesNeeded;
+      return this;
+    }
+
+    public Builder setFilesCached(long filesCached) {
+      this.filesCached = filesCached;
+      return this;
+    }
+
+    public CachePoolStats build() {
+      return new CachePoolStats(bytesNeeded, bytesCached, bytesOverlimit,
+          filesNeeded, filesCached);
+    }
+  };
+
+  private final long bytesNeeded;
+  private final long bytesCached;
+  private final long bytesOverlimit;
+  private final long filesNeeded;
+  private final long filesCached;
+
+  private CachePoolStats(long bytesNeeded, long bytesCached,
+      long bytesOverlimit, long filesNeeded, long filesCached) {
+    this.bytesNeeded = bytesNeeded;
+    this.bytesCached = bytesCached;
+    this.bytesOverlimit = bytesOverlimit;
+    this.filesNeeded = filesNeeded;
+    this.filesCached = filesCached;
+  }
+
+  public long getBytesNeeded() {
+    return bytesNeeded;
+  }
+
+  public long getBytesCached() {
+    return bytesCached;
+  }
+
+  public long getBytesOverlimit() {
+    return bytesOverlimit;
+  }
+
+  public long getFilesNeeded() {
+    return filesNeeded;
+  }
+
+  public long getFilesCached() {
+    return filesCached;
+  }
+
+  public String toString() {
+    return new StringBuilder().append("{").
+      append("bytesNeeded:").append(bytesNeeded).
+      append(", bytesCached:").append(bytesCached).
+      append(", bytesOverlimit:").append(bytesOverlimit).
+      append(", filesNeeded:").append(filesNeeded).
+      append(", filesCached:").append(filesCached).
+      append("}").toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReport.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReport.java
new file mode 100644
index 0000000..b58ed36
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReport.java
@@ -0,0 +1,215 @@
+/**
+ * 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.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.fs.Path;
+
+import com.google.common.base.Objects;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+
+/**
+ * This class represents to end users the difference between two snapshots of 
+ * the same directory, or the difference between a snapshot of the directory and
+ * its current state. Instead of capturing all the details of the diff, this
+ * class only lists where the changes happened and their types.
+ */
+public class SnapshotDiffReport {
+  private final static String LINE_SEPARATOR = System.getProperty(
+      "line.separator", "\n");
+
+  /**
+   * Types of the difference, which include CREATE, MODIFY, DELETE, and RENAME.
+   * Each type has a label for representation: +/M/-/R represent CREATE, MODIFY,
+   * DELETE, and RENAME respectively.
+   */
+  public enum DiffType {
+    CREATE("+"),     
+    MODIFY("M"),    
+    DELETE("-"), 
+    RENAME("R");
+    
+    private final String label;
+    
+    private DiffType(String label) {
+      this.label = label;
+    }
+    
+    public String getLabel() {
+      return label;
+    }
+    
+    public static DiffType getTypeFromLabel(String label) {
+      if (label.equals(CREATE.getLabel())) {
+        return CREATE;
+      } else if (label.equals(MODIFY.getLabel())) {
+        return MODIFY;
+      } else if (label.equals(DELETE.getLabel())) {
+        return DELETE;
+      } else if (label.equals(RENAME.getLabel())) {
+        return RENAME;
+      }
+      return null;
+    }
+  };
+  
+  /**
+   * Representing the full path and diff type of a file/directory where changes
+   * have happened.
+   */
+  public static class DiffReportEntry {
+    /** The type of the difference. */
+    private final DiffType type;
+    /**
+     * The relative path (related to the snapshot root) of 1) the file/directory
+     * where changes have happened, or 2) the source file/dir of a rename op.
+     */
+    private final byte[] sourcePath;
+    private final byte[] targetPath;
+
+    public DiffReportEntry(DiffType type, byte[] sourcePath) {
+      this(type, sourcePath, null);
+    }
+
+    public DiffReportEntry(DiffType type, byte[][] sourcePathComponents) {
+      this(type, sourcePathComponents, null);
+    }
+
+    public DiffReportEntry(DiffType type, byte[] sourcePath, byte[] targetPath) {
+      this.type = type;
+      this.sourcePath = sourcePath;
+      this.targetPath = targetPath;
+    }
+    
+    public DiffReportEntry(DiffType type, byte[][] sourcePathComponents,
+        byte[][] targetPathComponents) {
+      this.type = type;
+      this.sourcePath = DFSUtilClient.byteArray2bytes(sourcePathComponents);
+      this.targetPath = targetPathComponents == null ? null : DFSUtilClient
+          .byteArray2bytes(targetPathComponents);
+    }
+    
+    @Override
+    public String toString() {
+      String str = type.getLabel() + "\t" + getPathString(sourcePath);
+      if (type == DiffType.RENAME) {
+        str += " -> " + getPathString(targetPath);
+      }
+      return str;
+    }
+    
+    public DiffType getType() {
+      return type;
+    }
+
+    static String getPathString(byte[] path) {
+      String pathStr = DFSUtilClient.bytes2String(path);
+      if (pathStr.isEmpty()) {
+        return Path.CUR_DIR;
+      } else {
+        return Path.CUR_DIR + Path.SEPARATOR + pathStr;
+      }
+    }
+
+    public byte[] getSourcePath() {
+      return sourcePath;
+    }
+
+    public byte[] getTargetPath() {
+      return targetPath;
+    }
+
+    @Override
+    public boolean equals(Object other) {
+      if (this == other) {
+        return true;
+      } 
+      if (other != null && other instanceof DiffReportEntry) {
+        DiffReportEntry entry = (DiffReportEntry) other;
+        return type.equals(entry.getType())
+            && Arrays.equals(sourcePath, entry.getSourcePath())
+            && Arrays.equals(targetPath, entry.getTargetPath());
+      }
+      return false;
+    }
+    
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(getSourcePath(), getTargetPath());
+    }
+  }
+  
+  /** snapshot root full path */
+  private final String snapshotRoot;
+
+  /** start point of the diff */
+  private final String fromSnapshot;
+  
+  /** end point of the diff */
+  private final String toSnapshot;
+  
+  /** list of diff */
+  private final List<DiffReportEntry> diffList;
+  
+  public SnapshotDiffReport(String snapshotRoot, String fromSnapshot,
+      String toSnapshot, List<DiffReportEntry> entryList) {
+    this.snapshotRoot = snapshotRoot;
+    this.fromSnapshot = fromSnapshot;
+    this.toSnapshot = toSnapshot;
+    this.diffList = entryList != null ? entryList : Collections
+        .<DiffReportEntry> emptyList();
+  }
+  
+  /** @return {@link #snapshotRoot}*/
+  public String getSnapshotRoot() {
+    return snapshotRoot;
+  }
+
+  /** @return {@link #fromSnapshot} */
+  public String getFromSnapshot() {
+    return fromSnapshot;
+  }
+
+  /** @return {@link #toSnapshot} */
+  public String getLaterSnapshotName() {
+    return toSnapshot;
+  }
+  
+  /** @return {@link #diffList} */
+  public List<DiffReportEntry> getDiffList() {
+    return diffList;
+  }
+  
+  @Override
+  public String toString() {
+    StringBuilder str = new StringBuilder();
+    String from = fromSnapshot == null || fromSnapshot.isEmpty() ? 
+        "current directory" : "snapshot " + fromSnapshot;
+    String to = toSnapshot == null || toSnapshot.isEmpty() ? "current directory"
+        : "snapshot " + toSnapshot;
+    str.append("Difference between " + from + " and " + to
+        + " under directory " + snapshotRoot + ":" + LINE_SEPARATOR);
+    for (DiffReportEntry entry : diffList) {
+      str.append(entry.toString() + LINE_SEPARATOR);
+    }
+    return str.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
new file mode 100644
index 0000000..ac19d44
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
@@ -0,0 +1,218 @@
+/**
+ * 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.io.PrintStream;
+import java.text.SimpleDateFormat;
+import java.util.Comparator;
+import java.util.Date;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+
+/**
+ * Metadata about a snapshottable directory
+ */
+public class SnapshottableDirectoryStatus {
+  /** Compare the statuses by full paths. */
+  public static final Comparator<SnapshottableDirectoryStatus> COMPARATOR
+      = new Comparator<SnapshottableDirectoryStatus>() {
+    @Override
+    public int compare(SnapshottableDirectoryStatus left,
+                       SnapshottableDirectoryStatus right) {
+      int d = DFSUtilClient.compareBytes(left.parentFullPath, right.parentFullPath);
+      return d != 0? d
+          : DFSUtilClient.compareBytes(left.dirStatus.getLocalNameInBytes(),
+              right.dirStatus.getLocalNameInBytes());
+    }
+  };
+
+  /** Basic information of the snapshottable directory */
+  private final HdfsFileStatus dirStatus;
+  
+  /** Number of snapshots that have been taken*/
+  private final int snapshotNumber;
+  
+  /** Number of snapshots allowed. */
+  private final int snapshotQuota;
+  
+  /** Full path of the parent. */
+  private final byte[] parentFullPath;
+  
+  public SnapshottableDirectoryStatus(long modification_time, long access_time,
+      FsPermission permission, String owner, String group, byte[] localName,
+      long inodeId, int childrenNum,
+      int snapshotNumber, int snapshotQuota, byte[] parentFullPath) {
+    this.dirStatus = new HdfsFileStatus(0, true, 0, 0, modification_time,
+        access_time, permission, owner, group, null, localName, inodeId,
+        childrenNum, null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
+    this.snapshotNumber = snapshotNumber;
+    this.snapshotQuota = snapshotQuota;
+    this.parentFullPath = parentFullPath;
+  }
+
+  /**
+   * @return Number of snapshots that have been taken for the directory
+   */
+  public int getSnapshotNumber() {
+    return snapshotNumber;
+  }
+
+  /**
+   * @return Number of snapshots allowed for the directory
+   */
+  public int getSnapshotQuota() {
+    return snapshotQuota;
+  }
+  
+  /**
+   * @return Full path of the parent
+   */
+  public byte[] getParentFullPath() {
+    return parentFullPath;
+  }
+
+  /**
+   * @return The basic information of the directory
+   */
+  public HdfsFileStatus getDirStatus() {
+    return dirStatus;
+  }
+  
+  /**
+   * @return Full path of the file
+   */
+  public Path getFullPath() {
+    String parentFullPathStr = 
+        (parentFullPath == null || parentFullPath.length == 0) ? 
+            null : DFSUtilClient.bytes2String(parentFullPath);
+    if (parentFullPathStr == null
+        && dirStatus.getLocalNameInBytes().length == 0) {
+      // root
+      return new Path("/");
+    } else {
+      return parentFullPathStr == null ? new Path(dirStatus.getLocalName())
+          : new Path(parentFullPathStr, dirStatus.getLocalName());
+    }
+  }
+  
+  /**
+   * Print a list of {@link SnapshottableDirectoryStatus} out to a given stream.
+   * @param stats The list of {@link SnapshottableDirectoryStatus}
+   * @param out The given stream for printing.
+   */
+  public static void print(SnapshottableDirectoryStatus[] stats, 
+      PrintStream out) {
+    if (stats == null || stats.length == 0) {
+      out.println();
+      return;
+    }
+    int maxRepl = 0, maxLen = 0, maxOwner = 0, maxGroup = 0;
+    int maxSnapshotNum = 0, maxSnapshotQuota = 0;
+    for (SnapshottableDirectoryStatus status : stats) {
+      maxRepl = maxLength(maxRepl, status.dirStatus.getReplication());
+      maxLen = maxLength(maxLen, status.dirStatus.getLen());
+      maxOwner = maxLength(maxOwner, status.dirStatus.getOwner());
+      maxGroup = maxLength(maxGroup, status.dirStatus.getGroup());
+      maxSnapshotNum = maxLength(maxSnapshotNum, status.snapshotNumber);
+      maxSnapshotQuota = maxLength(maxSnapshotQuota, status.snapshotQuota);
+    }
+    
+    StringBuilder fmt = new StringBuilder();
+    fmt.append("%s%s "); // permission string
+    fmt.append("%"  + maxRepl  + "s ");
+    fmt.append((maxOwner > 0) ? "%-" + maxOwner + "s " : "%s");
+    fmt.append((maxGroup > 0) ? "%-" + maxGroup + "s " : "%s");
+    fmt.append("%"  + maxLen   + "s ");
+    fmt.append("%s "); // mod time
+    fmt.append("%"  + maxSnapshotNum  + "s ");
+    fmt.append("%"  + maxSnapshotQuota  + "s ");
+    fmt.append("%s"); // path
+    
+    String lineFormat = fmt.toString();
+    SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm");
+         
+    for (SnapshottableDirectoryStatus status : stats) {
+      String line = String.format(lineFormat, "d", 
+          status.dirStatus.getPermission(),
+          status.dirStatus.getReplication(),
+          status.dirStatus.getOwner(),
+          status.dirStatus.getGroup(),
+          String.valueOf(status.dirStatus.getLen()),
+          dateFormat.format(new Date(status.dirStatus.getModificationTime())),
+          status.snapshotNumber, status.snapshotQuota, 
+          status.getFullPath().toString()
+      );
+      out.println(line);
+    }
+  }
+
+  private static int maxLength(int n, Object value) {
+    return Math.max(n, String.valueOf(value).length());
+  }
+
+  public static class Bean {
+    private final String path;
+    private final int snapshotNumber;
+    private final int snapshotQuota;
+    private final long modificationTime;
+    private final short permission;
+    private final String owner;
+    private final String group;
+
+    public Bean(String path, int snapshotNumber, int snapshotQuota,
+        long modificationTime, short permission, String owner, String group) {
+      this.path = path;
+      this.snapshotNumber = snapshotNumber;
+      this.snapshotQuota = snapshotQuota;
+      this.modificationTime = modificationTime;
+      this.permission = permission;
+      this.owner = owner;
+      this.group = group;
+    }
+
+    public String getPath() {
+      return path;
+    }
+
+    public int getSnapshotNumber() {
+      return snapshotNumber;
+    }
+
+    public int getSnapshotQuota() {
+      return snapshotQuota;
+    }
+
+    public long getModificationTime() {
+      return modificationTime;
+    }
+
+    public short getPermission() {
+      return permission;
+    }
+
+    public String getOwner() {
+      return owner;
+    }
+
+    public String getGroup() {
+      return group;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java
new file mode 100644
index 0000000..9cf44d7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java
@@ -0,0 +1,74 @@
+/**
+ * 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.security.token.delegation;
+
+import java.net.URI;
+import java.util.Collection;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector;
+
+/**
+ * A delegation token that is specialized for HDFS
+ */
+@InterfaceAudience.Private
+public class DelegationTokenSelector
+    extends AbstractDelegationTokenSelector<DelegationTokenIdentifier>{
+  public static final String SERVICE_NAME_KEY = "hdfs.service.host_";
+
+  /**
+   * Select the delegation token for hdfs.  The port will be rewritten to
+   * the port of hdfs.service.host_$nnAddr, or the default rpc namenode port. 
+   * This method should only be called by non-hdfs filesystems that do not
+   * use the rpc port to acquire tokens.  Ex. webhdfs
+   * @param nnUri of the remote namenode
+   * @param tokens as a collection
+   * @param conf hadoop configuration
+   * @return Token
+   */
+  public Token<DelegationTokenIdentifier> selectToken(
+      final URI nnUri, Collection<Token<?>> tokens,
+      final Configuration conf) {
+    // this guesses the remote cluster's rpc service port.
+    // the current token design assumes it's the same as the local cluster's
+    // rpc port unless a config key is set.  there should be a way to automatic
+    // and correctly determine the value
+    Text serviceName = SecurityUtil.buildTokenService(nnUri);
+    final String nnServiceName = conf.get(SERVICE_NAME_KEY + serviceName);
+    
+    int nnRpcPort = HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT;
+    if (nnServiceName != null) {
+      nnRpcPort = NetUtils.createSocketAddr(nnServiceName, nnRpcPort).getPort(); 
+    }
+    // use original hostname from the uri to avoid unintentional host resolving
+    serviceName = SecurityUtil.buildTokenService(
+    		NetUtils.createSocketAddrForHost(nnUri.getHost(), nnRpcPort));
+    
+    return selectToken(serviceName, tokens);
+  }
+
+  public DelegationTokenSelector() {
+    super(DelegationTokenIdentifier.HDFS_DELEGATION_KIND);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/NotReplicatedYetException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/NotReplicatedYetException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/NotReplicatedYetException.java
new file mode 100644
index 0000000..712676d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/NotReplicatedYetException.java
@@ -0,0 +1,37 @@
+/**
+ * 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 java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * The file has not finished being written to enough datanodes yet.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class NotReplicatedYetException extends IOException {
+  private static final long serialVersionUID = 1L;
+
+  public NotReplicatedYetException(String msg) {
+    super(msg);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java
new file mode 100644
index 0000000..0c8b6c93
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java
@@ -0,0 +1,126 @@
+/**
+ * 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.protocol;
+
+import org.apache.hadoop.fs.StorageType;
+
+import java.util.UUID;
+
+/**
+ * Class captures information of a storage in Datanode.
+ */
+public class DatanodeStorage {
+  /** The state of the storage. */
+  public enum State {
+    NORMAL,
+    
+    /**
+     * A storage that represents a read-only path to replicas stored on a shared storage device.
+     * Replicas on {@link #READ_ONLY_SHARED} storage are not counted towards live replicas.
+     * 
+     * <p>
+     * In certain implementations, a {@link #READ_ONLY_SHARED} storage may be correlated to 
+     * its {@link #NORMAL} counterpart using the {@link DatanodeStorage#storageID}.  This
+     * property should be used for debugging purposes only.
+     * </p> 
+     */
+    READ_ONLY_SHARED,
+
+    FAILED;
+  }
+  
+  private final String storageID;
+  private final State state;
+  private final StorageType storageType;
+  private static final String STORAGE_ID_PREFIX = "DS-";
+
+  /**
+   * Create a storage with {@link State#NORMAL} and {@link StorageType#DEFAULT}.
+   */
+  public DatanodeStorage(String storageID) {
+    this(storageID, State.NORMAL, StorageType.DEFAULT);
+  }
+
+  public DatanodeStorage(String sid, State s, StorageType sm) {
+    this.storageID = sid;
+    this.state = s;
+    this.storageType = sm;
+  }
+
+  public String getStorageID() {
+    return storageID;
+  }
+
+  public State getState() {
+    return state;
+  }
+
+  public StorageType getStorageType() {
+    return storageType;
+  }
+
+  /**
+   * Generate new storage ID. The format of this string can be changed
+   * in the future without requiring that old storage IDs be updated.
+   *
+   * @return unique storage ID
+   */
+  public static String generateUuid() {
+    return STORAGE_ID_PREFIX + UUID.randomUUID();
+  }
+
+  /**
+   * Verify that a given string is a storage ID in the "DS-..uuid.." format.
+   */
+  public static boolean isValidStorageId(final String storageID) {
+    try {
+      // Attempt to parse the UUID.
+      if (storageID != null && storageID.indexOf(STORAGE_ID_PREFIX) == 0) {
+        UUID.fromString(storageID.substring(STORAGE_ID_PREFIX.length()));
+        return true;
+      }
+    } catch (IllegalArgumentException iae) {
+    }
+
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return "DatanodeStorage["+ storageID + "," + storageType + "," + state +"]";
+  }
+  
+  @Override
+  public boolean equals(Object other){
+    if (other == this) {
+      return true;
+    }
+
+    if ((other == null) ||
+        !(other instanceof DatanodeStorage)) {
+      return false;
+    }
+    DatanodeStorage otherStorage = (DatanodeStorage) other;
+    return otherStorage.getStorageID().compareTo(getStorageID()) == 0;
+  }
+
+  @Override
+  public int hashCode() {
+    return getStorageID().hashCode();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorageReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorageReport.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorageReport.java
new file mode 100644
index 0000000..6a956a0
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorageReport.java
@@ -0,0 +1,42 @@
+/**
+ * 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.protocol;
+
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+
+/**
+ * Class captures information of a datanode and its storages.
+ */
+public class DatanodeStorageReport {
+  final DatanodeInfo datanodeInfo;
+  final StorageReport[] storageReports;
+
+  public DatanodeStorageReport(DatanodeInfo datanodeInfo,
+      StorageReport[] storageReports) {
+    this.datanodeInfo = datanodeInfo;
+    this.storageReports = storageReports;
+  }
+
+  public DatanodeInfo getDatanodeInfo() {
+    return datanodeInfo;
+  }
+
+  public StorageReport[] getStorageReports() {
+    return storageReports;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java
new file mode 100644
index 0000000..5fd5733
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java
@@ -0,0 +1,66 @@
+/**
+ * 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.protocol;
+
+/**
+ * Utilization report for a Datanode storage
+ */
+public class StorageReport {
+  private final DatanodeStorage storage;
+  private final boolean failed;
+  private final long capacity;
+  private final long dfsUsed;
+  private final long remaining;
+  private final long blockPoolUsed;
+
+  public static final StorageReport[] EMPTY_ARRAY = {};
+  
+  public StorageReport(DatanodeStorage storage, boolean failed,
+      long capacity, long dfsUsed, long remaining, long bpUsed) {
+    this.storage = storage;
+    this.failed = failed;
+    this.capacity = capacity;
+    this.dfsUsed = dfsUsed;
+    this.remaining = remaining;
+    this.blockPoolUsed = bpUsed;
+  }
+
+  public DatanodeStorage getStorage() {
+    return storage;
+  }
+
+  public boolean isFailed() {
+    return failed;
+  }
+
+  public long getCapacity() {
+    return capacity;
+  }
+
+  public long getDfsUsed() {
+    return dfsUsed;
+  }
+
+  public long getRemaining() {
+    return remaining;
+  }
+
+  public long getBlockPoolUsed() {
+    return blockPoolUsed;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/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 21d73ba..365b005 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -504,6 +504,9 @@ Release 2.8.0 - UNRELEASED
     "dfs.client.read.shortcircuit.streams.cache.size" (Brahma Reddy Battula via
     Colin P. McCabe)
 
+    HDFS-8237. Move all protocol classes used by ClientProtocol to hdfs-client.
+    (wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/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 37f8c72..42ff7fa 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
@@ -43,14 +43,12 @@ import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.security.SecureRandom;
-import java.text.SimpleDateFormat;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.Date;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Locale;
 import java.util.Map;
 import java.util.Random;
 import java.util.Set;
@@ -97,26 +95,12 @@ import com.google.common.base.Charsets;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import com.google.common.primitives.SignedBytes;
 import com.google.protobuf.BlockingService;
 
 @InterfaceAudience.Private
 public class DFSUtil {
   public static final Log LOG = LogFactory.getLog(DFSUtil.class.getName());
   
-  public static final byte[] EMPTY_BYTES = {};
-
-  /** Compare two byte arrays by lexicographical order. */
-  public static int compareBytes(byte[] left, byte[] right) {
-    if (left == null) {
-      left = EMPTY_BYTES;
-    }
-    if (right == null) {
-      right = EMPTY_BYTES;
-    }
-    return SignedBytes.lexicographicalComparator().compare(left, right);
-  }
-
   private DFSUtil() { /* Hidden constructor */ }
   private static final ThreadLocal<Random> RANDOM = new ThreadLocal<Random>() {
     @Override
@@ -345,37 +329,6 @@ public class DFSUtil {
     return Joiner.on(Path.SEPARATOR).join(components);
   }
 
-  /**
-   * Given a list of path components returns a byte array
-   */
-  public static byte[] byteArray2bytes(byte[][] pathComponents) {
-    if (pathComponents.length == 0) {
-      return EMPTY_BYTES;
-    } else if (pathComponents.length == 1
-        && (pathComponents[0] == null || pathComponents[0].length == 0)) {
-      return new byte[]{(byte) Path.SEPARATOR_CHAR};
-    }
-    int length = 0;
-    for (int i = 0; i < pathComponents.length; i++) {
-      length += pathComponents[i].length;
-      if (i < pathComponents.length - 1) {
-        length++; // for SEPARATOR
-      }
-    }
-    byte[] path = new byte[length];
-    int index = 0;
-    for (int i = 0; i < pathComponents.length; i++) {
-      System.arraycopy(pathComponents[i], 0, path, index,
-          pathComponents[i].length);
-      index += pathComponents[i].length;
-      if (i < pathComponents.length - 1) {
-        path[index] = (byte) Path.SEPARATOR_CHAR;
-        index++;
-      }
-    }
-    return path;
-  }
-
   /** Convert an object representing a path to a string. */
   public static String path2String(final Object path) {
     return path == null? null
@@ -1377,38 +1330,14 @@ public class DFSUtil {
    * Converts a Date into an ISO-8601 formatted datetime string.
    */
   public static String dateToIso8601String(Date date) {
-    SimpleDateFormat df =
-        new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ssZ", Locale.ENGLISH);
-    return df.format(date);
+    return DFSUtilClient.dateToIso8601String(date);
   }
 
   /**
    * Converts a time duration in milliseconds into DDD:HH:MM:SS format.
    */
   public static String durationToString(long durationMs) {
-    boolean negative = false;
-    if (durationMs < 0) {
-      negative = true;
-      durationMs = -durationMs;
-    }
-    // Chop off the milliseconds
-    long durationSec = durationMs / 1000;
-    final int secondsPerMinute = 60;
-    final int secondsPerHour = 60*60;
-    final int secondsPerDay = 60*60*24;
-    final long days = durationSec / secondsPerDay;
-    durationSec -= days * secondsPerDay;
-    final long hours = durationSec / secondsPerHour;
-    durationSec -= hours * secondsPerHour;
-    final long minutes = durationSec / secondsPerMinute;
-    durationSec -= minutes * secondsPerMinute;
-    final long seconds = durationSec;
-    final long milliseconds = durationMs % 1000;
-    String format = "%03d:%02d:%02d:%02d.%03d";
-    if (negative)  {
-      format = "-" + format;
-    }
-    return String.format(format, days, hours, minutes, seconds, milliseconds);
+    return DFSUtilClient.durationToString(durationMs);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveEntry.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveEntry.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveEntry.java
deleted file mode 100644
index fe3215f..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveEntry.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;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * Describes a path-based cache directive entry.
- */
-@InterfaceStability.Evolving
-@InterfaceAudience.Public
-public class CacheDirectiveEntry {
-  private final CacheDirectiveInfo info;
-  private final CacheDirectiveStats stats;
-
-  public CacheDirectiveEntry(CacheDirectiveInfo info,
-      CacheDirectiveStats stats) {
-    this.info = info;
-    this.stats = stats;
-  }
-
-  public CacheDirectiveInfo getInfo() {
-    return info;
-  }
-
-  public CacheDirectiveStats getStats() {
-    return stats;
-  }
-};


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

Posted by ji...@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/6f541edc
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6f541edc
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6f541edc

Branch: refs/heads/HDFS-7240
Commit: 6f541edce0ed64bf316276715c4bc07794ff20ac
Parents: 7d46a80
Author: Arpit Agarwal <ar...@apache.org>
Authored: Fri May 1 16:30:51 2015 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Fri May 1 16:42:19 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/6f541edc/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/6f541edc/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/6f541edc/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/6f541edc/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.
    */


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

Posted by ji...@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/a319771d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a319771d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a319771d

Branch: refs/heads/HDFS-7240
Commit: a319771d1d9eebaf8e4165dba73383a229cb1525
Parents: e8d0ee5
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Sat May 2 22:15:24 2015 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Sat May 2 22:15:24 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/a319771d/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/a319771d/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/a319771d/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/a319771d/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);


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

Posted by ji...@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/bb9ddef0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/bb9ddef0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/bb9ddef0

Branch: refs/heads/HDFS-7240
Commit: bb9ddef0e7603b60d25250bb53a7ae9f147cd3cd
Parents: 76fa606
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Mon May 4 17:47:37 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Mon May 4 17:47:37 2015 +0900

----------------------------------------------------------------------
 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/bb9ddef0/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/bb9ddef0/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();
     }
   }


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

Posted by ji...@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/57d9a972
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/57d9a972
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/57d9a972

Branch: refs/heads/HDFS-7240
Commit: 57d9a972cbd62aae0ab010d38a0973619972edd6
Parents: f1a152c
Author: Junping Du <ju...@apache.org>
Authored: Sat May 2 10:18:44 2015 -0700
Committer: Junping Du <ju...@apache.org>
Committed: Sat May 2 10:18:44 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/57d9a972/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index c110f88..899310e 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -290,6 +290,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/57d9a972/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/57d9a972/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);
+  }
+  
+}


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

Posted by ji...@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/b82567d4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b82567d4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b82567d4

Branch: refs/heads/HDFS-7240
Commit: b82567d45507c50d2f28eff4bbdf3b1a69d4bf1b
Parents: 2d7363b
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Fri May 1 11:19:40 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Fri May 1 11:19:40 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/b82567d4/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/b82567d4/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/b82567d4/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/b82567d4/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/b82567d4/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/b82567d4/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/b82567d4/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/b82567d4/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/b82567d4/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/b82567d4/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/b82567d4/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);


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

Posted by ji...@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/3ddb260e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3ddb260e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3ddb260e

Branch: refs/heads/HDFS-7240
Commit: 3ddb260e30e7424363bcde68176bcea1ad8d6392
Parents: f343f86
Author: Gera Shegalov <ge...@apache.org>
Authored: Fri May 1 15:57:42 2015 -0700
Committer: Gera Shegalov <ge...@apache.org>
Committed: Fri May 1 18:18:55 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/3ddb260e/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/3ddb260e/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


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

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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);


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

Posted by ji...@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/f1a152cc
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f1a152cc
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f1a152cc

Branch: refs/heads/HDFS-7240
Commit: f1a152cc0adc071277c80637ea6f5faa0bf06a1a
Parents: 3ddb260
Author: Gera Shegalov <ge...@apache.org>
Authored: Fri May 1 16:13:56 2015 -0700
Committer: Gera Shegalov <ge...@apache.org>
Committed: Fri May 1 18:18:56 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/f1a152cc/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index c073470..2d87444 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -356,6 +356,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/f1a152cc/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 ff4511f..d776d44 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
@@ -739,7 +739,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>


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

Posted by ji...@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/e8d0ee5f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e8d0ee5f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e8d0ee5f

Branch: refs/heads/HDFS-7240
Commit: e8d0ee5fc9af612d7abc9ab2c201434e7102d092
Parents: 6ae2a0d
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Sun May 3 10:51:17 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Sun May 3 10:51:17 2015 +0900

----------------------------------------------------------------------
 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/e8d0ee5f/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 899310e..684efc5 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -293,6 +293,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/e8d0ee5f/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);
     }
   }
 


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

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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/6ae2a0d0/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");
     }
       


[28/33] 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 ji...@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/3fe79e1d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3fe79e1d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3fe79e1d

Branch: refs/heads/HDFS-7240
Commit: 3fe79e1db84391cb17dbed6b579fe9c803b3d1c2
Parents: 7dc3c120
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Mon May 4 12:50:29 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Mon May 4 12:50:29 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/3fe79e1d/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/3fe79e1d/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.


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

Posted by ji...@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/c4578760
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c4578760
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c4578760

Branch: refs/heads/HDFS-7240
Commit: c4578760b67d5b5169949a1b059f4472a268ff1b
Parents: 8f65c79
Author: cnauroth <cn...@apache.org>
Authored: Mon May 4 11:35:04 2015 -0700
Committer: cnauroth <cn...@apache.org>
Committed: Mon May 4 11:35:04 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/c4578760/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/c4578760/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/c4578760/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);
+  }
+}


[29/33] hadoop git commit: MAPREDUCE-6259. IllegalArgumentException due to missing job submit time. Contributed by zhihai xu

Posted by ji...@apache.org.
MAPREDUCE-6259. IllegalArgumentException due to missing job submit time. 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/bf70c5ae
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/bf70c5ae
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/bf70c5ae

Branch: refs/heads/HDFS-7240
Commit: bf70c5ae2824a9139c1aa9d7c14020018881cec2
Parents: 3fe79e1
Author: Jason Lowe <jl...@apache.org>
Authored: Mon May 4 20:39:18 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Mon May 4 20:39:18 2015 +0000

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |  3 ++
 .../jobhistory/JobHistoryEventHandler.java      | 15 ++++--
 .../hadoop/mapreduce/v2/app/MRAppMaster.java    |  4 +-
 .../jobhistory/TestJobHistoryEventHandler.java  | 57 +++++++++++++++++---
 .../mapreduce/jobhistory/AMStartedEvent.java    | 16 ++++--
 5 files changed, 77 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf70c5ae/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index f7e3bde..481757a 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -397,6 +397,9 @@ Release 2.7.1 - UNRELEASED
     MAPREDUCE-6339. Job history file is not flushed correctly because isTimerActive 
     flag is not set true when flushTimerTask is scheduled. (zhihai xu via devaraj)
 
+    MAPREDUCE-6259. IllegalArgumentException due to missing job submit time
+    (zhihai xu via jlowe)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf70c5ae/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 6b0ea79..bf32888 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
@@ -426,10 +426,10 @@ public class JobHistoryEventHandler extends AbstractService
    * This should be the first call to history for a job
    * 
    * @param jobId the jobId.
-   * @param forcedJobStateOnShutDown
+   * @param amStartedEvent
    * @throws IOException
    */
-  protected void setupEventWriter(JobId jobId, String forcedJobStateOnShutDown)
+  protected void setupEventWriter(JobId jobId, AMStartedEvent amStartedEvent)
       throws IOException {
     if (stagingDirPath == null) {
       LOG.error("Log Directory is null, returning");
@@ -489,8 +489,13 @@ public class JobHistoryEventHandler extends AbstractService
     }
 
     MetaInfo fi = new MetaInfo(historyFile, logDirConfPath, writer,
-        user, jobName, jobId, forcedJobStateOnShutDown, queueName);
+        user, jobName, jobId, amStartedEvent.getForcedJobStateOnShutDown(),
+        queueName);
     fi.getJobSummary().setJobId(jobId);
+    fi.getJobSummary().setJobLaunchTime(amStartedEvent.getStartTime());
+    fi.getJobSummary().setJobSubmitTime(amStartedEvent.getSubmitTime());
+    fi.getJobIndexInfo().setJobStartTime(amStartedEvent.getStartTime());
+    fi.getJobIndexInfo().setSubmitTime(amStartedEvent.getSubmitTime());
     fileMap.put(jobId, fi);
   }
 
@@ -541,8 +546,7 @@ public class JobHistoryEventHandler extends AbstractService
         try {
           AMStartedEvent amStartedEvent =
               (AMStartedEvent) event.getHistoryEvent();
-          setupEventWriter(event.getJobID(),
-              amStartedEvent.getForcedJobStateOnShutDown());
+          setupEventWriter(event.getJobID(), amStartedEvent);
         } catch (IOException ioe) {
           LOG.error("Error JobHistoryEventHandler in handleEvent: " + event,
               ioe);
@@ -982,6 +986,7 @@ public class JobHistoryEventHandler extends AbstractService
         tEvent.addEventInfo("NODE_MANAGER_HTTP_PORT",
                 ase.getNodeManagerHttpPort());
         tEvent.addEventInfo("START_TIME", ase.getStartTime());
+        tEvent.addEventInfo("SUBMIT_TIME", ase.getSubmitTime());
         tEntity.addEvent(tEvent);
         tEntity.setEntityId(jobId.toString());
         tEntity.setEntityType(MAPREDUCE_JOB_ENTITY_TYPE);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf70c5ae/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
index 9908ea5..c41f679 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
@@ -1052,7 +1052,7 @@ public class MRAppMaster extends CompositeService {
           new JobHistoryEvent(job.getID(), new AMStartedEvent(info
               .getAppAttemptId(), info.getStartTime(), info.getContainerId(),
               info.getNodeManagerHost(), info.getNodeManagerPort(), info
-                  .getNodeManagerHttpPort())));
+                  .getNodeManagerHttpPort(), appSubmitTime)));
     }
 
     // Send out an MR AM inited event for this AM.
@@ -1061,7 +1061,7 @@ public class MRAppMaster extends CompositeService {
             .getAppAttemptId(), amInfo.getStartTime(), amInfo.getContainerId(),
             amInfo.getNodeManagerHost(), amInfo.getNodeManagerPort(), amInfo
                 .getNodeManagerHttpPort(), this.forcedState == null ? null
-                    : this.forcedState.toString())));
+                    : this.forcedState.toString(), appSubmitTime)));
     amInfos.add(amInfo);
 
     // metrics system init is really init & start.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf70c5ae/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 49be35b..2b07efb 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
@@ -125,7 +125,7 @@ public class TestJobHistoryEventHandler {
     try {
       jheh.start();
       handleEvent(jheh, new JobHistoryEvent(t.jobId, new AMStartedEvent(
-          t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000)));
+          t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000, -1)));
       mockWriter = jheh.getEventWriter();
       verify(mockWriter).write(any(HistoryEvent.class));
 
@@ -168,7 +168,7 @@ public class TestJobHistoryEventHandler {
     try {
       jheh.start();
       handleEvent(jheh, new JobHistoryEvent(t.jobId, new AMStartedEvent(
-          t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000)));
+          t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000, -1)));
       mockWriter = jheh.getEventWriter();
       verify(mockWriter).write(any(HistoryEvent.class));
 
@@ -213,7 +213,7 @@ public class TestJobHistoryEventHandler {
     try {
       jheh.start();
       handleEvent(jheh, new JobHistoryEvent(t.jobId, new AMStartedEvent(
-          t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000)));
+          t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000, -1)));
       mockWriter = jheh.getEventWriter();
       verify(mockWriter).write(any(HistoryEvent.class));
 
@@ -256,7 +256,7 @@ public class TestJobHistoryEventHandler {
     try {
       jheh.start();
       handleEvent(jheh, new JobHistoryEvent(t.jobId, new AMStartedEvent(
-          t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000)));
+          t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000, -1)));
       mockWriter = jheh.getEventWriter();
       verify(mockWriter).write(any(HistoryEvent.class));
 
@@ -293,7 +293,7 @@ public class TestJobHistoryEventHandler {
     try {
       jheh.start();
       handleEvent(jheh, new JobHistoryEvent(t.jobId, new AMStartedEvent(
-        t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000)));
+        t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000, -1)));
       verify(jheh, times(0)).processDoneFiles(any(JobId.class));
 
       handleEvent(jheh, new JobHistoryEvent(t.jobId,
@@ -338,7 +338,7 @@ public class TestJobHistoryEventHandler {
     try {
       jheh.start();
       handleEvent(jheh, new JobHistoryEvent(t.jobId, new AMStartedEvent(
-        t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000)));
+        t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000, -1)));
       verify(jheh, times(0)).processDoneFiles(t.jobId);
 
       // skip processing done files
@@ -395,7 +395,7 @@ public class TestJobHistoryEventHandler {
     try {
       jheh.start();
       handleEvent(jheh, new JobHistoryEvent(t.jobId, new AMStartedEvent(
-          t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000)));
+          t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000, -1)));
 
       handleEvent(jheh, new JobHistoryEvent(t.jobId, new JobFinishedEvent(
           TypeConverter.fromYarn(t.jobId), 0, 0, 0, 0, 0, new Counters(),
@@ -441,6 +441,47 @@ public class TestJobHistoryEventHandler {
         pathStr);
   }
 
+  // test AMStartedEvent for submitTime and startTime
+  @Test (timeout=50000)
+  public void testAMStartedEvent() throws Exception {
+    TestParams t = new TestParams();
+    Configuration conf = new Configuration();
+
+    JHEvenHandlerForTest realJheh =
+        new JHEvenHandlerForTest(t.mockAppContext, 0);
+    JHEvenHandlerForTest jheh = spy(realJheh);
+    jheh.init(conf);
+
+    EventWriter mockWriter = null;
+    try {
+      jheh.start();
+      handleEvent(jheh, new JobHistoryEvent(t.jobId, new AMStartedEvent(
+          t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000, 100)));
+
+      JobHistoryEventHandler.MetaInfo mi =
+          JobHistoryEventHandler.fileMap.get(t.jobId);
+      Assert.assertEquals(mi.getJobIndexInfo().getSubmitTime(), 100);
+      Assert.assertEquals(mi.getJobIndexInfo().getJobStartTime(), 200);
+      Assert.assertEquals(mi.getJobSummary().getJobSubmitTime(), 100);
+      Assert.assertEquals(mi.getJobSummary().getJobLaunchTime(), 200);
+
+      handleEvent(jheh, new JobHistoryEvent(t.jobId,
+        new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(t.jobId), 0,
+          0, 0, JobStateInternal.FAILED.toString())));
+
+      Assert.assertEquals(mi.getJobIndexInfo().getSubmitTime(), 100);
+      Assert.assertEquals(mi.getJobIndexInfo().getJobStartTime(), 200);
+      Assert.assertEquals(mi.getJobSummary().getJobSubmitTime(), 100);
+      Assert.assertEquals(mi.getJobSummary().getJobLaunchTime(), 200);
+      verify(jheh, times(1)).processDoneFiles(t.jobId);
+
+      mockWriter = jheh.getEventWriter();
+      verify(mockWriter, times(2)).write(any(HistoryEvent.class));
+    } finally {
+      jheh.stop();
+    }
+  }
+
   // Have JobHistoryEventHandler handle some events and make sure they get
   // stored to the Timeline store
   @Test (timeout=50000)
@@ -463,7 +504,7 @@ public class TestJobHistoryEventHandler {
               .getTimelineStore();
 
       handleEvent(jheh, new JobHistoryEvent(t.jobId, new AMStartedEvent(
-              t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000),
+              t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000, -1),
               currentTime - 10));
       TimelineEntities entities = ts.getEntities("MAPREDUCE_JOB", null, null,
               null, null, null, null, null, null, null);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf70c5ae/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java
index d1a378b..e1465f5 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java
@@ -35,6 +35,7 @@ import org.apache.avro.util.Utf8;
 public class AMStartedEvent implements HistoryEvent {
   private AMStarted datum = new AMStarted();
   private String forcedJobStateOnShutDown;
+  private long submitTime;
 
   /**
    * Create an event to record the start of an MR AppMaster
@@ -54,9 +55,9 @@ public class AMStartedEvent implements HistoryEvent {
    */
   public AMStartedEvent(ApplicationAttemptId appAttemptId, long startTime,
       ContainerId containerId, String nodeManagerHost, int nodeManagerPort,
-      int nodeManagerHttpPort) {
+      int nodeManagerHttpPort, long submitTime) {
     this(appAttemptId, startTime, containerId, nodeManagerHost,
-        nodeManagerPort, nodeManagerHttpPort, null);
+        nodeManagerPort, nodeManagerHttpPort, null, submitTime);
   }
 
   /**
@@ -79,7 +80,8 @@ public class AMStartedEvent implements HistoryEvent {
    */
   public AMStartedEvent(ApplicationAttemptId appAttemptId, long startTime,
       ContainerId containerId, String nodeManagerHost, int nodeManagerPort,
-      int nodeManagerHttpPort, String forcedJobStateOnShutDown) {
+      int nodeManagerHttpPort, String forcedJobStateOnShutDown,
+      long submitTime) {
     datum.applicationAttemptId = new Utf8(appAttemptId.toString());
     datum.startTime = startTime;
     datum.containerId = new Utf8(containerId.toString());
@@ -87,6 +89,7 @@ public class AMStartedEvent implements HistoryEvent {
     datum.nodeManagerPort = nodeManagerPort;
     datum.nodeManagerHttpPort = nodeManagerHttpPort;
     this.forcedJobStateOnShutDown = forcedJobStateOnShutDown;
+    this.submitTime = submitTime;
   }
 
   AMStartedEvent() {
@@ -150,6 +153,13 @@ public class AMStartedEvent implements HistoryEvent {
     return this.forcedJobStateOnShutDown;
   }
 
+  /**
+   * @return the submit time for the Application(Job)
+   */
+  public long getSubmitTime() {
+    return this.submitTime;
+  }
+
   /** Get the attempt id */
 
   @Override


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

Posted by ji...@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/d3d019c3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d3d019c3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d3d019c3

Branch: refs/heads/HDFS-7240
Commit: d3d019c337ecc10e9c6bbefc3a97c6cd1f5283c3
Parents: 64d30a6
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Fri May 1 15:11:09 2015 -0700
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Fri May 1 15:12:18 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/d3d019c3/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/d3d019c3/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/d3d019c3/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/d3d019c3/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/d3d019c3/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/d3d019c3/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/d3d019c3/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/d3d019c3/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());
+  }
+
+}


[07/33] 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 ji...@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/ac7d1529
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ac7d1529
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ac7d1529

Branch: refs/heads/HDFS-7240
Commit: ac7d152901e29b1f444507fe4e421eb6e1402b5a
Parents: d3d019c
Author: Robert Kanter <rk...@apache.org>
Authored: Fri May 1 16:39:21 2015 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Fri May 1 16:39:21 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/ac7d1529/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index dcf3538..a21df9e 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -177,6 +177,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/ac7d1529/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/ac7d1529/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/ac7d1529/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/ac7d1529/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 cce749e..76bbda1 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
@@ -617,6 +617,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/ac7d1529/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();
   }
 }


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

Posted by ji...@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/7dc3c120
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7dc3c120
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7dc3c120

Branch: refs/heads/HDFS-7240
Commit: 7dc3c1203d1ab14c09d0aaf0869a5bcdfafb0a5a
Parents: c457876
Author: Jason Lowe <jl...@apache.org>
Authored: Mon May 4 19:02:39 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Mon May 4 19:02:39 2015 +0000

----------------------------------------------------------------------
 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/7dc3c120/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 19f95fc..f7e3bde 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -365,6 +365,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/7dc3c120/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/7dc3c120/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);
+  }
 }


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

Posted by ji...@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/33934611
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/33934611
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/33934611

Branch: refs/heads/HDFS-7240
Commit: 3393461197da869c3f1e47888e03670106bda023
Parents: b82567d
Author: Arun Suresh <as...@apache.org>
Authored: Fri May 1 11:48:54 2015 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Fri May 1 11:48:54 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/33934611/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


[30/33] hadoop git commit: HDFS-8237. Move all protocol classes used by ClientProtocol to hdfs-client. Contributed by Haohui Mai.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveInfo.java
deleted file mode 100644
index f6b3c34..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveInfo.java
+++ /dev/null
@@ -1,358 +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.Date;
-
-import org.apache.commons.lang.builder.EqualsBuilder;
-import org.apache.commons.lang.builder.HashCodeBuilder;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSUtil;
-
-import com.google.common.base.Preconditions;
-
-/**
- * Describes a path-based cache directive.
- */
-@InterfaceStability.Evolving
-@InterfaceAudience.Public
-public class CacheDirectiveInfo {
-  /**
-   * A builder for creating new CacheDirectiveInfo instances.
-   */
-  public static class Builder {
-    private Long id;
-    private Path path;
-    private Short replication;
-    private String pool;
-    private Expiration expiration;
-
-    /**
-     * Builds a new CacheDirectiveInfo populated with the set properties.
-     * 
-     * @return New CacheDirectiveInfo.
-     */
-    public CacheDirectiveInfo build() {
-      return new CacheDirectiveInfo(id, path, replication, pool, expiration);
-    }
-
-    /**
-     * Creates an empty builder.
-     */
-    public Builder() {
-    }
-
-    /**
-     * Creates a builder with all elements set to the same values as the
-     * given CacheDirectiveInfo.
-     */
-    public Builder(CacheDirectiveInfo directive) {
-      this.id = directive.getId();
-      this.path = directive.getPath();
-      this.replication = directive.getReplication();
-      this.pool = directive.getPool();
-      this.expiration = directive.getExpiration();
-    }
-
-    /**
-     * Sets the id used in this request.
-     * 
-     * @param id The id used in this request.
-     * @return This builder, for call chaining.
-     */
-    public Builder setId(Long id) {
-      this.id = id;
-      return this;
-    }
-
-    /**
-     * Sets the path used in this request.
-     * 
-     * @param path The path used in this request.
-     * @return This builder, for call chaining.
-     */
-    public Builder setPath(Path path) {
-      this.path = path;
-      return this;
-    }
-
-    /**
-     * Sets the replication used in this request.
-     * 
-     * @param replication The replication used in this request.
-     * @return This builder, for call chaining.
-     */
-    public Builder setReplication(Short replication) {
-      this.replication = replication;
-      return this;
-    }
-
-    /**
-     * Sets the pool used in this request.
-     * 
-     * @param pool The pool used in this request.
-     * @return This builder, for call chaining.
-     */
-    public Builder setPool(String pool) {
-      this.pool = pool;
-      return this;
-    }
-
-    /**
-     * Sets when the CacheDirective should expire. A
-     * {@link CacheDirectiveInfo.Expiration} can specify either an absolute or
-     * relative expiration time.
-     * 
-     * @param expiration when this CacheDirective should expire
-     * @return This builder, for call chaining
-     */
-    public Builder setExpiration(Expiration expiration) {
-      this.expiration = expiration;
-      return this;
-    }
-  }
-
-  /**
-   * Denotes a relative or absolute expiration time for a CacheDirective. Use
-   * factory methods {@link CacheDirectiveInfo.Expiration#newAbsolute(Date)} and
-   * {@link CacheDirectiveInfo.Expiration#newRelative(long)} to create an
-   * Expiration.
-   * <p>
-   * In either case, the server-side clock is used to determine when a
-   * CacheDirective expires.
-   */
-  public static class Expiration {
-
-    /**
-     * The maximum value we accept for a relative expiry.
-     */
-    public static final long MAX_RELATIVE_EXPIRY_MS =
-        Long.MAX_VALUE / 4; // This helps prevent weird overflow bugs
-
-    /**
-     * An relative Expiration that never expires.
-     */
-    public static final Expiration NEVER = newRelative(MAX_RELATIVE_EXPIRY_MS);
-
-    /**
-     * Create a new relative Expiration.
-     * <p>
-     * Use {@link Expiration#NEVER} to indicate an Expiration that never
-     * expires.
-     * 
-     * @param ms how long until the CacheDirective expires, in milliseconds
-     * @return A relative Expiration
-     */
-    public static Expiration newRelative(long ms) {
-      return new Expiration(ms, true);
-    }
-
-    /**
-     * Create a new absolute Expiration.
-     * <p>
-     * Use {@link Expiration#NEVER} to indicate an Expiration that never
-     * expires.
-     * 
-     * @param date when the CacheDirective expires
-     * @return An absolute Expiration
-     */
-    public static Expiration newAbsolute(Date date) {
-      return new Expiration(date.getTime(), false);
-    }
-
-    /**
-     * Create a new absolute Expiration.
-     * <p>
-     * Use {@link Expiration#NEVER} to indicate an Expiration that never
-     * expires.
-     * 
-     * @param ms when the CacheDirective expires, in milliseconds since the Unix
-     *          epoch.
-     * @return An absolute Expiration
-     */
-    public static Expiration newAbsolute(long ms) {
-      return new Expiration(ms, false);
-    }
-
-    private final long ms;
-    private final boolean isRelative;
-
-    private Expiration(long ms, boolean isRelative) {
-      if (isRelative) {
-        Preconditions.checkArgument(ms <= MAX_RELATIVE_EXPIRY_MS,
-            "Expiration time is too far in the future!");
-      }
-      this.ms = ms;
-      this.isRelative = isRelative;
-    }
-
-    /**
-     * @return true if Expiration was specified as a relative duration, false if
-     *         specified as an absolute time.
-     */
-    public boolean isRelative() {
-      return isRelative;
-    }
-
-    /**
-     * @return The raw underlying millisecond value, either a relative duration
-     *         or an absolute time as milliseconds since the Unix epoch.
-     */
-    public long getMillis() {
-      return ms;
-    }
-
-    /**
-     * @return Expiration time as a {@link Date} object. This converts a
-     *         relative Expiration into an absolute Date based on the local
-     *         clock.
-     */
-    public Date getAbsoluteDate() {
-      return new Date(getAbsoluteMillis());
-    }
-
-    /**
-     * @return Expiration time in milliseconds from the Unix epoch. This
-     *         converts a relative Expiration into an absolute time based on the
-     *         local clock.
-     */
-    public long getAbsoluteMillis() {
-      if (!isRelative) {
-        return ms;
-      } else {
-        return new Date().getTime() + ms;
-      }
-    }
-
-    @Override
-    public String toString() {
-      if (isRelative) {
-        return DFSUtil.durationToString(ms);
-      }
-      return DFSUtil.dateToIso8601String(new Date(ms));
-    }
-  }
-
-  private final Long id;
-  private final Path path;
-  private final Short replication;
-  private final String pool;
-  private final Expiration expiration;
-
-  CacheDirectiveInfo(Long id, Path path, Short replication, String pool,
-      Expiration expiration) {
-    this.id = id;
-    this.path = path;
-    this.replication = replication;
-    this.pool = pool;
-    this.expiration = expiration;
-  }
-
-  /**
-   * @return The ID of this directive.
-   */
-  public Long getId() {
-    return id;
-  }
-
-  /**
-   * @return The path used in this request.
-   */
-  public Path getPath() {
-    return path;
-  }
-
-  /**
-   * @return The number of times the block should be cached.
-   */
-  public Short getReplication() {
-    return replication;
-  }
-
-  /**
-   * @return The pool used in this request.
-   */
-  public String getPool() {
-    return pool;
-  }
-
-  /**
-   * @return When this directive expires.
-   */
-  public Expiration getExpiration() {
-    return expiration;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (o == null) {
-      return false;
-    }
-    if (getClass() != o.getClass()) {
-      return false;
-    }
-    CacheDirectiveInfo other = (CacheDirectiveInfo)o;
-    return new EqualsBuilder().append(getId(), other.getId()).
-        append(getPath(), other.getPath()).
-        append(getReplication(), other.getReplication()).
-        append(getPool(), other.getPool()).
-        append(getExpiration(), other.getExpiration()).
-        isEquals();
-  }
-
-  @Override
-  public int hashCode() {
-    return new HashCodeBuilder().append(id).
-        append(path).
-        append(replication).
-        append(pool).
-        append(expiration).
-        hashCode();
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder builder = new StringBuilder();
-    builder.append("{");
-    String prefix = "";
-    if (id != null) {
-      builder.append(prefix).append("id: ").append(id);
-      prefix = ", ";
-    }
-    if (path != null) {
-      builder.append(prefix).append("path: ").append(path);
-      prefix = ", ";
-    }
-    if (replication != null) {
-      builder.append(prefix).append("replication: ").append(replication);
-      prefix = ", ";
-    }
-    if (pool != null) {
-      builder.append(prefix).append("pool: ").append(pool);
-      prefix = ", ";
-    }
-    if (expiration != null) {
-      builder.append(prefix).append("expiration: ").append(expiration);
-      prefix = ", ";
-    }
-    builder.append("}");
-    return builder.toString();
-  }
-};

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveStats.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveStats.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveStats.java
deleted file mode 100644
index 0fd4ca2..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveStats.java
+++ /dev/null
@@ -1,169 +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;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * Describes a path-based cache directive.
- */
-@InterfaceStability.Evolving
-@InterfaceAudience.Public
-public class CacheDirectiveStats {
-  public static class Builder {
-    private long bytesNeeded;
-    private long bytesCached;
-    private long filesNeeded;
-    private long filesCached;
-    private boolean hasExpired;
-
-    /**
-     * Builds a new CacheDirectiveStats populated with the set properties.
-     * 
-     * @return New CacheDirectiveStats.
-     */
-    public CacheDirectiveStats build() {
-      return new CacheDirectiveStats(bytesNeeded, bytesCached, filesNeeded,
-          filesCached, hasExpired);
-    }
-
-    /**
-     * Creates an empty builder.
-     */
-    public Builder() {
-    }
-
-    /**
-     * Sets the bytes needed by this directive.
-     * 
-     * @param bytesNeeded The bytes needed.
-     * @return This builder, for call chaining.
-     */
-    public Builder setBytesNeeded(long bytesNeeded) {
-      this.bytesNeeded = bytesNeeded;
-      return this;
-    }
-
-    /**
-     * Sets the bytes cached by this directive.
-     * 
-     * @param bytesCached The bytes cached.
-     * @return This builder, for call chaining.
-     */
-    public Builder setBytesCached(long bytesCached) {
-      this.bytesCached = bytesCached;
-      return this;
-    }
-
-    /**
-     * Sets the files needed by this directive.
-     * @param filesNeeded The number of files needed
-     * @return This builder, for call chaining.
-     */
-    public Builder setFilesNeeded(long filesNeeded) {
-      this.filesNeeded = filesNeeded;
-      return this;
-    }
-
-    /**
-     * Sets the files cached by this directive.
-     * 
-     * @param filesCached The number of files cached.
-     * @return This builder, for call chaining.
-     */
-    public Builder setFilesCached(long filesCached) {
-      this.filesCached = filesCached;
-      return this;
-    }
-
-    /**
-     * Sets whether this directive has expired.
-     * 
-     * @param hasExpired if this directive has expired
-     * @return This builder, for call chaining.
-     */
-    public Builder setHasExpired(boolean hasExpired) {
-      this.hasExpired = hasExpired;
-      return this;
-    }
-  }
-
-  private final long bytesNeeded;
-  private final long bytesCached;
-  private final long filesNeeded;
-  private final long filesCached;
-  private final boolean hasExpired;
-
-  private CacheDirectiveStats(long bytesNeeded, long bytesCached,
-      long filesNeeded, long filesCached, boolean hasExpired) {
-    this.bytesNeeded = bytesNeeded;
-    this.bytesCached = bytesCached;
-    this.filesNeeded = filesNeeded;
-    this.filesCached = filesCached;
-    this.hasExpired = hasExpired;
-  }
-
-  /**
-   * @return The bytes needed.
-   */
-  public long getBytesNeeded() {
-    return bytesNeeded;
-  }
-
-  /**
-   * @return The bytes cached.
-   */
-  public long getBytesCached() {
-    return bytesCached;
-  }
-
-  /**
-   * @return The number of files needed.
-   */
-  public long getFilesNeeded() {
-    return filesNeeded;
-  }
-
-  /**
-   * @return The number of files cached.
-   */
-  public long getFilesCached() {
-    return filesCached;
-  }
-
-  /**
-   * @return Whether this directive has expired.
-   */
-  public boolean hasExpired() {
-    return hasExpired;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder builder = new StringBuilder();
-    builder.append("{");
-    builder.append("bytesNeeded: ").append(bytesNeeded);
-    builder.append(", ").append("bytesCached: ").append(bytesCached);
-    builder.append(", ").append("filesNeeded: ").append(filesNeeded);
-    builder.append(", ").append("filesCached: ").append(filesCached);
-    builder.append(", ").append("hasExpired: ").append(hasExpired);
-    builder.append("}");
-    return builder.toString();
-  }
-};

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolEntry.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolEntry.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolEntry.java
deleted file mode 100644
index 3c1e345..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolEntry.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;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * Describes a Cache Pool entry.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class CachePoolEntry {
-  private final CachePoolInfo info;
-  private final CachePoolStats stats;
-
-  public CachePoolEntry(CachePoolInfo info, CachePoolStats stats) {
-    this.info = info;
-    this.stats = stats;
-  }
-
-  public CachePoolInfo getInfo() {
-    return info;
-  }
-
-  public CachePoolStats getStats() {
-    return stats;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
deleted file mode 100644
index 61bbe38..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
+++ /dev/null
@@ -1,229 +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.io.IOException;
-
-import javax.annotation.Nullable;
-
-import org.apache.commons.lang.builder.EqualsBuilder;
-import org.apache.commons.lang.builder.HashCodeBuilder;
-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.fs.InvalidRequestException;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo.Expiration;
-
-/**
- * CachePoolInfo describes a cache pool.
- *
- * This class is used in RPCs to create and modify cache pools.
- * It is serializable and can be stored in the edit log.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class CachePoolInfo {
-  public static final Log LOG = LogFactory.getLog(CachePoolInfo.class);
-
-  /**
-   * Indicates that the pool does not have a maximum relative expiry.
-   */
-  public static final long RELATIVE_EXPIRY_NEVER =
-      Expiration.MAX_RELATIVE_EXPIRY_MS;
-  /**
-   * Default max relative expiry for cache pools.
-   */
-  public static final long DEFAULT_MAX_RELATIVE_EXPIRY =
-      RELATIVE_EXPIRY_NEVER;
-
-  public static final long LIMIT_UNLIMITED = Long.MAX_VALUE;
-  public static final long DEFAULT_LIMIT = LIMIT_UNLIMITED;
-
-  final String poolName;
-
-  @Nullable
-  String ownerName;
-
-  @Nullable
-  String groupName;
-
-  @Nullable
-  FsPermission mode;
-
-  @Nullable
-  Long limit;
-
-  @Nullable
-  Long maxRelativeExpiryMs;
-
-  public CachePoolInfo(String poolName) {
-    this.poolName = poolName;
-  }
-
-  /**
-   * @return Name of the pool.
-   */
-  public String getPoolName() {
-    return poolName;
-  }
-
-  /**
-   * @return The owner of the pool. Along with the group and mode, determines
-   *         who has access to view and modify the pool.
-   */
-  public String getOwnerName() {
-    return ownerName;
-  }
-
-  public CachePoolInfo setOwnerName(String ownerName) {
-    this.ownerName = ownerName;
-    return this;
-  }
-
-  /**
-   * @return The group of the pool. Along with the owner and mode, determines
-   *         who has access to view and modify the pool.
-   */
-  public String getGroupName() {
-    return groupName;
-  }
-
-  public CachePoolInfo setGroupName(String groupName) {
-    this.groupName = groupName;
-    return this;
-  }
-
-  /**
-   * @return Unix-style permissions of the pool. Along with the owner and group,
-   *         determines who has access to view and modify the pool.
-   */
-  public FsPermission getMode() {
-    return mode;
-  }
-
-  public CachePoolInfo setMode(FsPermission mode) {
-    this.mode = mode;
-    return this;
-  }
-
-  /**
-   * @return The maximum aggregate number of bytes that can be cached by
-   *         directives in this pool.
-   */
-  public Long getLimit() {
-    return limit;
-  }
-
-  public CachePoolInfo setLimit(Long bytes) {
-    this.limit = bytes;
-    return this;
-  }
-
-  /**
-   * @return The maximum relative expiration of directives of this pool in
-   *         milliseconds
-   */
-  public Long getMaxRelativeExpiryMs() {
-    return maxRelativeExpiryMs;
-  }
-
-  /**
-   * Set the maximum relative expiration of directives of this pool in
-   * milliseconds.
-   * 
-   * @param ms in milliseconds
-   * @return This builder, for call chaining.
-   */
-  public CachePoolInfo setMaxRelativeExpiryMs(Long ms) {
-    this.maxRelativeExpiryMs = ms;
-    return this;
-  }
-
-  public String toString() {
-    return new StringBuilder().append("{").
-      append("poolName:").append(poolName).
-      append(", ownerName:").append(ownerName).
-      append(", groupName:").append(groupName).
-      append(", mode:").append((mode == null) ? "null" :
-          String.format("0%03o", mode.toShort())).
-      append(", limit:").append(limit).
-      append(", maxRelativeExpiryMs:").append(maxRelativeExpiryMs).
-      append("}").toString();
-  }
-  
-  @Override
-  public boolean equals(Object o) {
-    if (o == null) { return false; }
-    if (o == this) { return true; }
-    if (o.getClass() != getClass()) {
-      return false;
-    }
-    CachePoolInfo other = (CachePoolInfo)o;
-    return new EqualsBuilder().
-        append(poolName, other.poolName).
-        append(ownerName, other.ownerName).
-        append(groupName, other.groupName).
-        append(mode, other.mode).
-        append(limit, other.limit).
-        append(maxRelativeExpiryMs, other.maxRelativeExpiryMs).
-        isEquals();
-  }
-
-  @Override
-  public int hashCode() {
-    return new HashCodeBuilder().
-        append(poolName).
-        append(ownerName).
-        append(groupName).
-        append(mode).
-        append(limit).
-        append(maxRelativeExpiryMs).
-        hashCode();
-  }
-
-  public static void validate(CachePoolInfo info) throws IOException {
-    if (info == null) {
-      throw new InvalidRequestException("CachePoolInfo is null");
-    }
-    if ((info.getLimit() != null) && (info.getLimit() < 0)) {
-      throw new InvalidRequestException("Limit is negative.");
-    }
-    if (info.getMaxRelativeExpiryMs() != null) {
-      long maxRelativeExpiryMs = info.getMaxRelativeExpiryMs();
-      if (maxRelativeExpiryMs < 0l) {
-        throw new InvalidRequestException("Max relative expiry is negative.");
-      }
-      if (maxRelativeExpiryMs > Expiration.MAX_RELATIVE_EXPIRY_MS) {
-        throw new InvalidRequestException("Max relative expiry is too big.");
-      }
-    }
-    validateName(info.poolName);
-  }
-
-  public static void validateName(String poolName) throws IOException {
-    if (poolName == null || poolName.isEmpty()) {
-      // Empty pool names are not allowed because they would be highly
-      // confusing.  They would also break the ability to list all pools
-      // by starting with prevKey = ""
-      throw new IOException("invalid empty cache pool name");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolStats.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolStats.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolStats.java
deleted file mode 100644
index c552652..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolStats.java
+++ /dev/null
@@ -1,115 +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;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * CachePoolStats describes cache pool statistics.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class CachePoolStats {
-  public static class Builder {
-    private long bytesNeeded;
-    private long bytesCached;
-    private long bytesOverlimit;
-    private long filesNeeded;
-    private long filesCached;
-
-    public Builder() {
-    }
-
-    public Builder setBytesNeeded(long bytesNeeded) {
-      this.bytesNeeded = bytesNeeded;
-      return this;
-    }
-
-    public Builder setBytesCached(long bytesCached) {
-      this.bytesCached = bytesCached;
-      return this;
-    }
-
-    public Builder setBytesOverlimit(long bytesOverlimit) {
-      this.bytesOverlimit = bytesOverlimit;
-      return this;
-    }
-
-    public Builder setFilesNeeded(long filesNeeded) {
-      this.filesNeeded = filesNeeded;
-      return this;
-    }
-
-    public Builder setFilesCached(long filesCached) {
-      this.filesCached = filesCached;
-      return this;
-    }
-
-    public CachePoolStats build() {
-      return new CachePoolStats(bytesNeeded, bytesCached, bytesOverlimit,
-          filesNeeded, filesCached);
-    }
-  };
-
-  private final long bytesNeeded;
-  private final long bytesCached;
-  private final long bytesOverlimit;
-  private final long filesNeeded;
-  private final long filesCached;
-
-  private CachePoolStats(long bytesNeeded, long bytesCached,
-      long bytesOverlimit, long filesNeeded, long filesCached) {
-    this.bytesNeeded = bytesNeeded;
-    this.bytesCached = bytesCached;
-    this.bytesOverlimit = bytesOverlimit;
-    this.filesNeeded = filesNeeded;
-    this.filesCached = filesCached;
-  }
-
-  public long getBytesNeeded() {
-    return bytesNeeded;
-  }
-
-  public long getBytesCached() {
-    return bytesCached;
-  }
-
-  public long getBytesOverlimit() {
-    return bytesOverlimit;
-  }
-
-  public long getFilesNeeded() {
-    return filesNeeded;
-  }
-
-  public long getFilesCached() {
-    return filesCached;
-  }
-
-  public String toString() {
-    return new StringBuilder().append("{").
-      append("bytesNeeded:").append(bytesNeeded).
-      append(", bytesCached:").append(bytesCached).
-      append(", bytesOverlimit:").append(bytesOverlimit).
-      append(", filesNeeded:").append(filesNeeded).
-      append(", filesCached:").append(filesCached).
-      append("}").toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReport.java
deleted file mode 100644
index b0db838..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReport.java
+++ /dev/null
@@ -1,215 +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.Arrays;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSUtil;
-
-import com.google.common.base.Objects;
-
-/**
- * This class represents to end users the difference between two snapshots of 
- * the same directory, or the difference between a snapshot of the directory and
- * its current state. Instead of capturing all the details of the diff, this
- * class only lists where the changes happened and their types.
- */
-public class SnapshotDiffReport {
-  private final static String LINE_SEPARATOR = System.getProperty(
-      "line.separator", "\n");
-
-  /**
-   * Types of the difference, which include CREATE, MODIFY, DELETE, and RENAME.
-   * Each type has a label for representation: +/M/-/R represent CREATE, MODIFY,
-   * DELETE, and RENAME respectively.
-   */
-  public enum DiffType {
-    CREATE("+"),     
-    MODIFY("M"),    
-    DELETE("-"), 
-    RENAME("R");
-    
-    private final String label;
-    
-    private DiffType(String label) {
-      this.label = label;
-    }
-    
-    public String getLabel() {
-      return label;
-    }
-    
-    public static DiffType getTypeFromLabel(String label) {
-      if (label.equals(CREATE.getLabel())) {
-        return CREATE;
-      } else if (label.equals(MODIFY.getLabel())) {
-        return MODIFY;
-      } else if (label.equals(DELETE.getLabel())) {
-        return DELETE;
-      } else if (label.equals(RENAME.getLabel())) {
-        return RENAME;
-      }
-      return null;
-    }
-  };
-  
-  /**
-   * Representing the full path and diff type of a file/directory where changes
-   * have happened.
-   */
-  public static class DiffReportEntry {
-    /** The type of the difference. */
-    private final DiffType type;
-    /**
-     * The relative path (related to the snapshot root) of 1) the file/directory
-     * where changes have happened, or 2) the source file/dir of a rename op.
-     */
-    private final byte[] sourcePath;
-    private final byte[] targetPath;
-
-    public DiffReportEntry(DiffType type, byte[] sourcePath) {
-      this(type, sourcePath, null);
-    }
-
-    public DiffReportEntry(DiffType type, byte[][] sourcePathComponents) {
-      this(type, sourcePathComponents, null);
-    }
-
-    public DiffReportEntry(DiffType type, byte[] sourcePath, byte[] targetPath) {
-      this.type = type;
-      this.sourcePath = sourcePath;
-      this.targetPath = targetPath;
-    }
-    
-    public DiffReportEntry(DiffType type, byte[][] sourcePathComponents,
-        byte[][] targetPathComponents) {
-      this.type = type;
-      this.sourcePath = DFSUtil.byteArray2bytes(sourcePathComponents);
-      this.targetPath = targetPathComponents == null ? null : DFSUtil
-          .byteArray2bytes(targetPathComponents);
-    }
-    
-    @Override
-    public String toString() {
-      String str = type.getLabel() + "\t" + getPathString(sourcePath);
-      if (type == DiffType.RENAME) {
-        str += " -> " + getPathString(targetPath);
-      }
-      return str;
-    }
-    
-    public DiffType getType() {
-      return type;
-    }
-
-    static String getPathString(byte[] path) {
-      String pathStr = DFSUtil.bytes2String(path);
-      if (pathStr.isEmpty()) {
-        return Path.CUR_DIR;
-      } else {
-        return Path.CUR_DIR + Path.SEPARATOR + pathStr;
-      }
-    }
-
-    public byte[] getSourcePath() {
-      return sourcePath;
-    }
-
-    public byte[] getTargetPath() {
-      return targetPath;
-    }
-
-    @Override
-    public boolean equals(Object other) {
-      if (this == other) {
-        return true;
-      } 
-      if (other != null && other instanceof DiffReportEntry) {
-        DiffReportEntry entry = (DiffReportEntry) other;
-        return type.equals(entry.getType())
-            && Arrays.equals(sourcePath, entry.getSourcePath())
-            && Arrays.equals(targetPath, entry.getTargetPath());
-      }
-      return false;
-    }
-    
-    @Override
-    public int hashCode() {
-      return Objects.hashCode(getSourcePath(), getTargetPath());
-    }
-  }
-  
-  /** snapshot root full path */
-  private final String snapshotRoot;
-
-  /** start point of the diff */
-  private final String fromSnapshot;
-  
-  /** end point of the diff */
-  private final String toSnapshot;
-  
-  /** list of diff */
-  private final List<DiffReportEntry> diffList;
-  
-  public SnapshotDiffReport(String snapshotRoot, String fromSnapshot,
-      String toSnapshot, List<DiffReportEntry> entryList) {
-    this.snapshotRoot = snapshotRoot;
-    this.fromSnapshot = fromSnapshot;
-    this.toSnapshot = toSnapshot;
-    this.diffList = entryList != null ? entryList : Collections
-        .<DiffReportEntry> emptyList();
-  }
-  
-  /** @return {@link #snapshotRoot}*/
-  public String getSnapshotRoot() {
-    return snapshotRoot;
-  }
-
-  /** @return {@link #fromSnapshot} */
-  public String getFromSnapshot() {
-    return fromSnapshot;
-  }
-
-  /** @return {@link #toSnapshot} */
-  public String getLaterSnapshotName() {
-    return toSnapshot;
-  }
-  
-  /** @return {@link #diffList} */
-  public List<DiffReportEntry> getDiffList() {
-    return diffList;
-  }
-  
-  @Override
-  public String toString() {
-    StringBuilder str = new StringBuilder();
-    String from = fromSnapshot == null || fromSnapshot.isEmpty() ? 
-        "current directory" : "snapshot " + fromSnapshot;
-    String to = toSnapshot == null || toSnapshot.isEmpty() ? "current directory"
-        : "snapshot " + toSnapshot;
-    str.append("Difference between " + from + " and " + to
-        + " under directory " + snapshotRoot + ":" + LINE_SEPARATOR);
-    for (DiffReportEntry entry : diffList) {
-      str.append(entry.toString() + LINE_SEPARATOR);
-    }
-    return str.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/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
deleted file mode 100644
index 3067696..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
+++ /dev/null
@@ -1,218 +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.io.PrintStream;
-import java.text.SimpleDateFormat;
-import java.util.Comparator;
-import java.util.Date;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.DFSUtil;
-
-/**
- * Metadata about a snapshottable directory
- */
-public class SnapshottableDirectoryStatus {
-  /** Compare the statuses by full paths. */
-  public static final Comparator<SnapshottableDirectoryStatus> COMPARATOR
-      = new Comparator<SnapshottableDirectoryStatus>() {
-    @Override
-    public int compare(SnapshottableDirectoryStatus left,
-                       SnapshottableDirectoryStatus right) {
-      int d = DFSUtil.compareBytes(left.parentFullPath, right.parentFullPath);
-      return d != 0? d
-          : DFSUtil.compareBytes(left.dirStatus.getLocalNameInBytes(),
-              right.dirStatus.getLocalNameInBytes());
-    }
-  };
-
-  /** Basic information of the snapshottable directory */
-  private final HdfsFileStatus dirStatus;
-  
-  /** Number of snapshots that have been taken*/
-  private final int snapshotNumber;
-  
-  /** Number of snapshots allowed. */
-  private final int snapshotQuota;
-  
-  /** Full path of the parent. */
-  private final byte[] parentFullPath;
-  
-  public SnapshottableDirectoryStatus(long modification_time, long access_time,
-      FsPermission permission, String owner, String group, byte[] localName,
-      long inodeId, int childrenNum,
-      int snapshotNumber, int snapshotQuota, byte[] parentFullPath) {
-    this.dirStatus = new HdfsFileStatus(0, true, 0, 0, modification_time,
-        access_time, permission, owner, group, null, localName, inodeId,
-        childrenNum, null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
-    this.snapshotNumber = snapshotNumber;
-    this.snapshotQuota = snapshotQuota;
-    this.parentFullPath = parentFullPath;
-  }
-
-  /**
-   * @return Number of snapshots that have been taken for the directory
-   */
-  public int getSnapshotNumber() {
-    return snapshotNumber;
-  }
-
-  /**
-   * @return Number of snapshots allowed for the directory
-   */
-  public int getSnapshotQuota() {
-    return snapshotQuota;
-  }
-  
-  /**
-   * @return Full path of the parent
-   */
-  public byte[] getParentFullPath() {
-    return parentFullPath;
-  }
-
-  /**
-   * @return The basic information of the directory
-   */
-  public HdfsFileStatus getDirStatus() {
-    return dirStatus;
-  }
-  
-  /**
-   * @return Full path of the file
-   */
-  public Path getFullPath() {
-    String parentFullPathStr = 
-        (parentFullPath == null || parentFullPath.length == 0) ? 
-            null : DFSUtil.bytes2String(parentFullPath);
-    if (parentFullPathStr == null
-        && dirStatus.getLocalNameInBytes().length == 0) {
-      // root
-      return new Path("/");
-    } else {
-      return parentFullPathStr == null ? new Path(dirStatus.getLocalName())
-          : new Path(parentFullPathStr, dirStatus.getLocalName());
-    }
-  }
-  
-  /**
-   * Print a list of {@link SnapshottableDirectoryStatus} out to a given stream.
-   * @param stats The list of {@link SnapshottableDirectoryStatus}
-   * @param out The given stream for printing.
-   */
-  public static void print(SnapshottableDirectoryStatus[] stats, 
-      PrintStream out) {
-    if (stats == null || stats.length == 0) {
-      out.println();
-      return;
-    }
-    int maxRepl = 0, maxLen = 0, maxOwner = 0, maxGroup = 0;
-    int maxSnapshotNum = 0, maxSnapshotQuota = 0;
-    for (SnapshottableDirectoryStatus status : stats) {
-      maxRepl = maxLength(maxRepl, status.dirStatus.getReplication());
-      maxLen = maxLength(maxLen, status.dirStatus.getLen());
-      maxOwner = maxLength(maxOwner, status.dirStatus.getOwner());
-      maxGroup = maxLength(maxGroup, status.dirStatus.getGroup());
-      maxSnapshotNum = maxLength(maxSnapshotNum, status.snapshotNumber);
-      maxSnapshotQuota = maxLength(maxSnapshotQuota, status.snapshotQuota);
-    }
-    
-    StringBuilder fmt = new StringBuilder();
-    fmt.append("%s%s "); // permission string
-    fmt.append("%"  + maxRepl  + "s ");
-    fmt.append((maxOwner > 0) ? "%-" + maxOwner + "s " : "%s");
-    fmt.append((maxGroup > 0) ? "%-" + maxGroup + "s " : "%s");
-    fmt.append("%"  + maxLen   + "s ");
-    fmt.append("%s "); // mod time
-    fmt.append("%"  + maxSnapshotNum  + "s ");
-    fmt.append("%"  + maxSnapshotQuota  + "s ");
-    fmt.append("%s"); // path
-    
-    String lineFormat = fmt.toString();
-    SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm");
-         
-    for (SnapshottableDirectoryStatus status : stats) {
-      String line = String.format(lineFormat, "d", 
-          status.dirStatus.getPermission(),
-          status.dirStatus.getReplication(),
-          status.dirStatus.getOwner(),
-          status.dirStatus.getGroup(),
-          String.valueOf(status.dirStatus.getLen()),
-          dateFormat.format(new Date(status.dirStatus.getModificationTime())),
-          status.snapshotNumber, status.snapshotQuota, 
-          status.getFullPath().toString()
-      );
-      out.println(line);
-    }
-  }
-
-  private static int maxLength(int n, Object value) {
-    return Math.max(n, String.valueOf(value).length());
-  }
-
-  public static class Bean {
-    private final String path;
-    private final int snapshotNumber;
-    private final int snapshotQuota;
-    private final long modificationTime;
-    private final short permission;
-    private final String owner;
-    private final String group;
-
-    public Bean(String path, int snapshotNumber, int snapshotQuota,
-        long modificationTime, short permission, String owner, String group) {
-      this.path = path;
-      this.snapshotNumber = snapshotNumber;
-      this.snapshotQuota = snapshotQuota;
-      this.modificationTime = modificationTime;
-      this.permission = permission;
-      this.owner = owner;
-      this.group = group;
-    }
-
-    public String getPath() {
-      return path;
-    }
-
-    public int getSnapshotNumber() {
-      return snapshotNumber;
-    }
-
-    public int getSnapshotQuota() {
-      return snapshotQuota;
-    }
-
-    public long getModificationTime() {
-      return modificationTime;
-    }
-
-    public short getPermission() {
-      return permission;
-    }
-
-    public String getOwner() {
-      return owner;
-    }
-
-    public String getGroup() {
-      return group;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/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 ba6670c..c9a9c33 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
@@ -47,9 +47,9 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.proto.HAServiceProtocolProtos;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.inotify.EventBatch;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
-import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.inotify.Event;
 import org.apache.hadoop.hdfs.inotify.EventBatchList;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -469,7 +469,7 @@ public class PBHelper {
   public static BlockKeyProto convert(BlockKey key) {
     byte[] encodedKey = key.getEncodedKey();
     ByteString keyBytes = ByteString.copyFrom(encodedKey == null ? 
-        DFSUtil.EMPTY_BYTES : encodedKey);
+        DFSUtilClient.EMPTY_BYTES : encodedKey);
     return BlockKeyProto.newBuilder().setKeyId(key.getKeyId())
         .setKeyBytes(keyBytes).setExpiryDate(key.getExpiryDate()).build();
   }
@@ -1514,7 +1514,7 @@ public class PBHelper {
     int snapshotQuota = status.getSnapshotQuota();
     byte[] parentFullPath = status.getParentFullPath();
     ByteString parentFullPathBytes = ByteString.copyFrom(
-        parentFullPath == null ? DFSUtil.EMPTY_BYTES : parentFullPath);
+        parentFullPath == null ? DFSUtilClient.EMPTY_BYTES : parentFullPath);
     HdfsFileStatusProto fs = convert(status.getDirStatus());
     SnapshottableDirectoryStatusProto.Builder builder = 
         SnapshottableDirectoryStatusProto
@@ -2024,7 +2024,7 @@ public class PBHelper {
       return null;
     }
     ByteString sourcePath = ByteString
-        .copyFrom(entry.getSourcePath() == null ? DFSUtil.EMPTY_BYTES : entry
+        .copyFrom(entry.getSourcePath() == null ? DFSUtilClient.EMPTY_BYTES : entry
             .getSourcePath());
     String modification = entry.getType().getLabel();
     SnapshotDiffReportEntryProto.Builder builder = SnapshotDiffReportEntryProto
@@ -2032,7 +2032,7 @@ public class PBHelper {
         .setModificationLabel(modification);
     if (entry.getType() == DiffType.RENAME) {
       ByteString targetPath = ByteString
-          .copyFrom(entry.getTargetPath() == null ? DFSUtil.EMPTY_BYTES : entry
+          .copyFrom(entry.getTargetPath() == null ? DFSUtilClient.EMPTY_BYTES : entry
               .getTargetPath());
       builder.setTargetPath(targetPath);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java
deleted file mode 100644
index e9b24ca..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java
+++ /dev/null
@@ -1,74 +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.security.token.delegation;
-
-import java.net.URI;
-import java.util.Collection;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector;
-
-/**
- * A delegation token that is specialized for HDFS
- */
-@InterfaceAudience.Private
-public class DelegationTokenSelector
-    extends AbstractDelegationTokenSelector<DelegationTokenIdentifier>{
-  public static final String SERVICE_NAME_KEY = "hdfs.service.host_";
-
-  /**
-   * Select the delegation token for hdfs.  The port will be rewritten to
-   * the port of hdfs.service.host_$nnAddr, or the default rpc namenode port. 
-   * This method should only be called by non-hdfs filesystems that do not
-   * use the rpc port to acquire tokens.  Ex. webhdfs
-   * @param nnUri of the remote namenode
-   * @param tokens as a collection
-   * @param conf hadoop configuration
-   * @return Token
-   */
-  public Token<DelegationTokenIdentifier> selectToken(
-      final URI nnUri, Collection<Token<?>> tokens,
-      final Configuration conf) {
-    // this guesses the remote cluster's rpc service port.
-    // the current token design assumes it's the same as the local cluster's
-    // rpc port unless a config key is set.  there should be a way to automatic
-    // and correctly determine the value
-    Text serviceName = SecurityUtil.buildTokenService(nnUri);
-    final String nnServiceName = conf.get(SERVICE_NAME_KEY + serviceName);
-    
-    int nnRpcPort = NameNode.DEFAULT_PORT;
-    if (nnServiceName != null) {
-      nnRpcPort = NetUtils.createSocketAddr(nnServiceName, nnRpcPort).getPort(); 
-    }
-    // use original hostname from the uri to avoid unintentional host resolving
-    serviceName = SecurityUtil.buildTokenService(
-    		NetUtils.createSocketAddrForHost(nnUri.getHost(), nnRpcPort));
-    
-    return selectToken(serviceName, tokens);
-  }
-
-  public DelegationTokenSelector() {
-    super(DelegationTokenIdentifier.HDFS_DELEGATION_KIND);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/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 409967e..f8efd76 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
@@ -29,6 +29,7 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.DFSUtil;
@@ -765,7 +766,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
 
   @Override
   public final int compareTo(byte[] bytes) {
-    return DFSUtil.compareBytes(getLocalNameBytes(), bytes);
+    return DFSUtilClient.compareBytes(getLocalNameBytes(), bytes);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NotReplicatedYetException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NotReplicatedYetException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NotReplicatedYetException.java
deleted file mode 100644
index 712676d..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NotReplicatedYetException.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs.server.namenode;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * The file has not finished being written to enough datanodes yet.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class NotReplicatedYetException extends IOException {
-  private static final long serialVersionUID = 1L;
-
-  public NotReplicatedYetException(String msg) {
-    super(msg);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
index c3b7523..802d64a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
@@ -30,6 +30,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import javax.management.ObjectName;
 
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.protocol.SnapshotInfo;
@@ -342,7 +343,7 @@ public class SnapshotManager implements SnapshotStatsMXBean {
             dir.getChildrenNum(Snapshot.CURRENT_STATE_ID),
             dir.getDirectorySnapshottableFeature().getNumSnapshots(),
             dir.getDirectorySnapshottableFeature().getSnapshotQuota(),
-            dir.getParent() == null ? DFSUtil.EMPTY_BYTES :
+            dir.getParent() == null ? DFSUtilClient.EMPTY_BYTES :
                 DFSUtil.string2Bytes(dir.getParent().getFullPathName()));
         statusList.add(status);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java
deleted file mode 100644
index 0c8b6c93..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.protocol;
-
-import org.apache.hadoop.fs.StorageType;
-
-import java.util.UUID;
-
-/**
- * Class captures information of a storage in Datanode.
- */
-public class DatanodeStorage {
-  /** The state of the storage. */
-  public enum State {
-    NORMAL,
-    
-    /**
-     * A storage that represents a read-only path to replicas stored on a shared storage device.
-     * Replicas on {@link #READ_ONLY_SHARED} storage are not counted towards live replicas.
-     * 
-     * <p>
-     * In certain implementations, a {@link #READ_ONLY_SHARED} storage may be correlated to 
-     * its {@link #NORMAL} counterpart using the {@link DatanodeStorage#storageID}.  This
-     * property should be used for debugging purposes only.
-     * </p> 
-     */
-    READ_ONLY_SHARED,
-
-    FAILED;
-  }
-  
-  private final String storageID;
-  private final State state;
-  private final StorageType storageType;
-  private static final String STORAGE_ID_PREFIX = "DS-";
-
-  /**
-   * Create a storage with {@link State#NORMAL} and {@link StorageType#DEFAULT}.
-   */
-  public DatanodeStorage(String storageID) {
-    this(storageID, State.NORMAL, StorageType.DEFAULT);
-  }
-
-  public DatanodeStorage(String sid, State s, StorageType sm) {
-    this.storageID = sid;
-    this.state = s;
-    this.storageType = sm;
-  }
-
-  public String getStorageID() {
-    return storageID;
-  }
-
-  public State getState() {
-    return state;
-  }
-
-  public StorageType getStorageType() {
-    return storageType;
-  }
-
-  /**
-   * Generate new storage ID. The format of this string can be changed
-   * in the future without requiring that old storage IDs be updated.
-   *
-   * @return unique storage ID
-   */
-  public static String generateUuid() {
-    return STORAGE_ID_PREFIX + UUID.randomUUID();
-  }
-
-  /**
-   * Verify that a given string is a storage ID in the "DS-..uuid.." format.
-   */
-  public static boolean isValidStorageId(final String storageID) {
-    try {
-      // Attempt to parse the UUID.
-      if (storageID != null && storageID.indexOf(STORAGE_ID_PREFIX) == 0) {
-        UUID.fromString(storageID.substring(STORAGE_ID_PREFIX.length()));
-        return true;
-      }
-    } catch (IllegalArgumentException iae) {
-    }
-
-    return false;
-  }
-
-  @Override
-  public String toString() {
-    return "DatanodeStorage["+ storageID + "," + storageType + "," + state +"]";
-  }
-  
-  @Override
-  public boolean equals(Object other){
-    if (other == this) {
-      return true;
-    }
-
-    if ((other == null) ||
-        !(other instanceof DatanodeStorage)) {
-      return false;
-    }
-    DatanodeStorage otherStorage = (DatanodeStorage) other;
-    return otherStorage.getStorageID().compareTo(getStorageID()) == 0;
-  }
-
-  @Override
-  public int hashCode() {
-    return getStorageID().hashCode();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorageReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorageReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorageReport.java
deleted file mode 100644
index 6a956a0..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorageReport.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.protocol;
-
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-
-/**
- * Class captures information of a datanode and its storages.
- */
-public class DatanodeStorageReport {
-  final DatanodeInfo datanodeInfo;
-  final StorageReport[] storageReports;
-
-  public DatanodeStorageReport(DatanodeInfo datanodeInfo,
-      StorageReport[] storageReports) {
-    this.datanodeInfo = datanodeInfo;
-    this.storageReports = storageReports;
-  }
-
-  public DatanodeInfo getDatanodeInfo() {
-    return datanodeInfo;
-  }
-
-  public StorageReport[] getStorageReports() {
-    return storageReports;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0d6aa5d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java
deleted file mode 100644
index 5fd5733..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.protocol;
-
-/**
- * Utilization report for a Datanode storage
- */
-public class StorageReport {
-  private final DatanodeStorage storage;
-  private final boolean failed;
-  private final long capacity;
-  private final long dfsUsed;
-  private final long remaining;
-  private final long blockPoolUsed;
-
-  public static final StorageReport[] EMPTY_ARRAY = {};
-  
-  public StorageReport(DatanodeStorage storage, boolean failed,
-      long capacity, long dfsUsed, long remaining, long bpUsed) {
-    this.storage = storage;
-    this.failed = failed;
-    this.capacity = capacity;
-    this.dfsUsed = dfsUsed;
-    this.remaining = remaining;
-    this.blockPoolUsed = bpUsed;
-  }
-
-  public DatanodeStorage getStorage() {
-    return storage;
-  }
-
-  public boolean isFailed() {
-    return failed;
-  }
-
-  public long getCapacity() {
-    return capacity;
-  }
-
-  public long getDfsUsed() {
-    return dfsUsed;
-  }
-
-  public long getRemaining() {
-    return remaining;
-  }
-
-  public long getBlockPoolUsed() {
-    return blockPoolUsed;
-  }
-}


[32/33] hadoop git commit: YARN-3375. NodeHealthScriptRunner.shouldRun() check is performing 3 times for starting NodeHealthScriptRunner (Devaraj K via wangda)

Posted by ji...@apache.org.
YARN-3375. NodeHealthScriptRunner.shouldRun() check is performing 3 times for starting NodeHealthScriptRunner (Devaraj K via wangda)


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

Branch: refs/heads/HDFS-7240
Commit: 71f4de220c74bf2c90630bd0442979d92380d304
Parents: 0d6aa5d
Author: Wangda Tan <wa...@apache.org>
Authored: Mon May 4 15:49:19 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Mon May 4 15:49:19 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/util/NodeHealthScriptRunner.java  | 8 --------
 hadoop-yarn-project/CHANGES.txt                              | 3 +++
 .../yarn/server/nodemanager/NodeHealthCheckerService.java    | 4 +---
 .../apache/hadoop/yarn/server/nodemanager/NodeManager.java   | 4 +++-
 4 files changed, 7 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/71f4de22/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NodeHealthScriptRunner.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NodeHealthScriptRunner.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NodeHealthScriptRunner.java
index 568ad5b..fc392c4 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NodeHealthScriptRunner.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NodeHealthScriptRunner.java
@@ -214,11 +214,6 @@ public class NodeHealthScriptRunner extends AbstractService {
    */
   @Override
   protected void serviceStart() throws Exception {
-    // if health script path is not configured don't start the thread.
-    if (!shouldRun(nodeHealthScript)) {
-      LOG.info("Not starting node health monitor");
-      return;
-    }
     nodeHealthScriptScheduler = new Timer("NodeHealthMonitor-Timer", true);
     // Start the timer task immediately and
     // then periodically at interval time.
@@ -232,9 +227,6 @@ public class NodeHealthScriptRunner extends AbstractService {
    */
   @Override
   protected void serviceStop() {
-    if (!shouldRun(nodeHealthScript)) {
-      return;
-    }
     if (nodeHealthScriptScheduler != null) {
       nodeHealthScriptScheduler.cancel();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/71f4de22/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 08762e3..09ec41a 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -299,6 +299,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3097. Logging of resource recovery on NM restart has redundancies
     (Eric Payne via jlowe)
 
+    YARN-3375. NodeHealthScriptRunner.shouldRun() check is performing 3 times for 
+    starting NodeHealthScriptRunner. (Devaraj K via wangda)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/71f4de22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeHealthCheckerService.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/NodeHealthCheckerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeHealthCheckerService.java
index 02b318a..c1a159a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeHealthCheckerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeHealthCheckerService.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.yarn.server.nodemanager;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.util.NodeHealthScriptRunner;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
 
 /**
  * The class which provides functionality of checking the health of the node and
@@ -44,8 +43,7 @@ public class NodeHealthCheckerService extends CompositeService {
 
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
-    if (NodeHealthScriptRunner.shouldRun(
-        conf.get(YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_PATH))) {
+    if (nodeHealthScriptRunner != null) {
       addService(nodeHealthScriptRunner);
     }
     addService(dirsHandler);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/71f4de22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.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/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
index 6718b53..4ac06d0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
@@ -210,7 +210,9 @@ public class NodeManager extends CompositeService
     String nodeHealthScript = 
         conf.get(YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_PATH);
     if(!NodeHealthScriptRunner.shouldRun(nodeHealthScript)) {
-      LOG.info("Abey khali");
+      LOG.info("Node Manager health check script is not available "
+          + "or doesn't have execute permission, so not "
+          + "starting the node health script runner.");
       return null;
     }
     long nmCheckintervalTime = conf.getLong(


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

Posted by ji...@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/7d46a806
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7d46a806
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7d46a806

Branch: refs/heads/HDFS-7240
Commit: 7d46a806e71de6692cd230e64e7de18a8252019d
Parents: ac7d152
Author: Wangda Tan <wa...@apache.org>
Authored: Fri May 1 16:40:44 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri May 1 16:40:55 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/7d46a806/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/7d46a806/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index a21df9e..6f38201 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -189,6 +189,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 


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

Posted by ji...@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/2d7363b2
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2d7363b2
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2d7363b2

Branch: refs/heads/HDFS-7240
Commit: 2d7363b27360e36fdd62546c0f9d0b1d78133f29
Parents: 279958b
Author: Jonathan Eagles <je...@gmail.com>
Authored: Fri May 1 13:11:50 2015 -0500
Committer: Jonathan Eagles <je...@gmail.com>
Committed: Fri May 1 13:11:50 2015 -0500

----------------------------------------------------------------------
 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/2d7363b2/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/2d7363b2/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/2d7363b2/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/2d7363b2/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/2d7363b2/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/2d7363b2/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/2d7363b2/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/2d7363b2/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/2d7363b2/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 9674de1..f23a2dd 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>
@@ -1024,11 +1023,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/2d7363b2/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/2d7363b2/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/2d7363b2/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>