Browse Source

HDFS-8249. Separate HdfsConstants into the client and the server side class. Contributed by Haohui Mai.

Haohui Mai 10 years ago
parent
commit
6ae2a0d048
100 changed files with 610 additions and 671 deletions
  1. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HAUtilClient.java
  2. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/Block.java
  3. 109 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
  4. 0 45
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstantsClient.java
  5. 3 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
  6. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  7. 2 2
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
  8. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  9. 7 7
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
  10. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/EditLogLedgerMetadata.java
  11. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperEditLogStreams.java
  12. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  13. 2 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  14. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
  15. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
  16. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
  17. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  18. 0 175
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
  19. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
  20. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java
  21. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  22. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  23. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java
  24. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/RequestInfo.java
  25. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java
  26. 11 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
  27. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
  28. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
  29. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
  30. 10 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java
  31. 85 24
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
  32. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java
  33. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
  34. 10 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
  35. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
  36. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
  37. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  38. 17 18
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
  39. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
  40. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
  41. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
  42. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
  43. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java
  44. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
  45. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
  46. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java
  47. 11 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
  48. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java
  49. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditsDoubleBuffer.java
  50. 7 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
  51. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  52. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  53. 12 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  54. 14 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
  55. 9 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
  56. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  57. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
  58. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImagePreTransactionalStorageInspector.java
  59. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageStorageInspector.java
  60. 10 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  61. 11 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
  62. 6 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  63. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  64. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  65. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeId.java
  66. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
  67. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
  68. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
  69. 5 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
  70. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  71. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  72. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RedundantEditLogInputStream.java
  73. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
  74. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
  75. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
  76. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
  77. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NNHAStatusHeartbeat.java
  78. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
  79. 5 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RemoteEditLog.java
  80. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java
  81. 3 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsLoader.java
  82. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
  83. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfs.java
  84. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
  85. 20 19
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
  86. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java
  87. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStartupVersions.java
  88. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java
  89. 4 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeRegistration.java
  90. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
  91. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
  92. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
  93. 3 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
  94. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery2.java
  95. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java
  96. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
  97. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
  98. 21 21
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
  99. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyConsiderLoad.java
  100. 23 23
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java

+ 1 - 1
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 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
 @InterfaceAudience.Private
 public class HAUtilClient {
 public class HAUtilClient {

+ 2 - 2
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) {
   public static long getGenerationStamp(String metaFile) {
     Matcher m = metaFilePattern.matcher(metaFile);
     Matcher m = metaFilePattern.matcher(metaFile);
     return m.matches() ? Long.parseLong(m.group(2))
     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) {
   public Block(final long blkid) {
-    this(blkid, 0, HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP);
+    this(blkid, 0, HdfsConstants.GRANDFATHER_GENERATION_STAMP);
   }
   }
 
 
   public Block(Block blk) {
   public Block(Block blk) {

+ 109 - 0
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() {
+  }
+}

+ 0 - 45
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstantsClient.java

@@ -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";
-}

+ 3 - 3
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.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 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.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -123,11 +123,11 @@ class JsonUtilClient {
     final long blockSize = ((Number) m.get("blockSize")).longValue();
     final long blockSize = ((Number) m.get("blockSize")).longValue();
     final short replication = ((Number) m.get("replication")).shortValue();
     final short replication = ((Number) m.get("replication")).shortValue();
     final long fileId = m.containsKey("fileId") ?
     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 int childrenNum = getInt(m, "childrenNum", -1);
     final byte storagePolicy = m.containsKey("storagePolicy") ?
     final byte storagePolicy = m.containsKey("storagePolicy") ?
         (byte) ((Number) m.get("storagePolicy")).longValue() :
         (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,
     return new HdfsFileStatus(len, type == WebHdfsConstants.PathType.DIRECTORY, replication,
         blockSize, mTime, aTime, permission, owner, group,
         blockSize, mTime, aTime, permission, owner, group,
         symlink, DFSUtilClient.string2Bytes(localName),
         symlink, DFSUtilClient.string2Bytes(localName),

+ 2 - 2
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.DFSUtilClient;
 import org.apache.hadoop.hdfs.HAUtilClient;
 import org.apache.hadoop.hdfs.HAUtilClient;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 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.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.web.resources.*;
 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_ENABLED_DEFAULT,
               HdfsClientConfigKeys.HttpClient.RETRY_POLICY_SPEC_KEY,
               HdfsClientConfigKeys.HttpClient.RETRY_POLICY_SPEC_KEY,
               HdfsClientConfigKeys.HttpClient.RETRY_POLICY_SPEC_DEFAULT,
               HdfsClientConfigKeys.HttpClient.RETRY_POLICY_SPEC_DEFAULT,
-              HdfsConstantsClient.SAFEMODE_EXCEPTION_CLASS_NAME);
+              HdfsConstants.SAFEMODE_EXCEPTION_CLASS_NAME);
     } else {
     } else {
 
 
       int maxFailoverAttempts = conf.getInt(
       int maxFailoverAttempts = conf.getInt(

+ 2 - 2
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.NfsConfigKeys;
 import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
 import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 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.HdfsFileStatus;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.net.DNS;
 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,
       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) {
     } catch (IOException e) {
       LOG.warn("Exception ", e);
       LOG.warn("Exception ", e);
       int status = mapErrorStatus(e);
       int status = mapErrorStatus(e);

+ 3 - 0
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
     HDFS-8086. Move LeaseRenewer to the hdfs.client.impl package.  (Takanobu
     Asanuma via szetszwo)
     Asanuma via szetszwo)
 
 
+    HDFS-8249. Separate HdfsConstants into the client and the server side
+    class. (wheat9)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

+ 7 - 7
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;
 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.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.JournalManager;
 import org.apache.hadoop.hdfs.server.namenode.JournalManager;
@@ -568,7 +568,7 @@ public class BookKeeperJournalManager implements JournalManager {
           return;
           return;
         }
         }
         streams.add(elis);
         streams.add(elis);
-        if (elis.getLastTxId() == HdfsConstants.INVALID_TXID) {
+        if (elis.getLastTxId() == HdfsServerConstants.INVALID_TXID) {
           return;
           return;
         }
         }
         fromTxId = elis.getLastTxId() + 1;
         fromTxId = elis.getLastTxId() + 1;
@@ -589,7 +589,7 @@ public class BookKeeperJournalManager implements JournalManager {
       long lastTxId = l.getLastTxId();
       long lastTxId = l.getLastTxId();
       if (l.isInProgress()) {
       if (l.isInProgress()) {
         lastTxId = recoverLastTxId(l, false);
         lastTxId = recoverLastTxId(l, false);
-        if (lastTxId == HdfsConstants.INVALID_TXID) {
+        if (lastTxId == HdfsServerConstants.INVALID_TXID) {
           break;
           break;
         }
         }
       }
       }
@@ -634,7 +634,7 @@ public class BookKeeperJournalManager implements JournalManager {
           EditLogLedgerMetadata l = EditLogLedgerMetadata.read(zkc, znode);
           EditLogLedgerMetadata l = EditLogLedgerMetadata.read(zkc, znode);
           try {
           try {
             long endTxId = recoverLastTxId(l, true);
             long endTxId = recoverLastTxId(l, true);
-            if (endTxId == HdfsConstants.INVALID_TXID) {
+            if (endTxId == HdfsServerConstants.INVALID_TXID) {
               LOG.error("Unrecoverable corruption has occurred in segment "
               LOG.error("Unrecoverable corruption has occurred in segment "
                   + l.toString() + " at path " + znode
                   + l.toString() + " at path " + znode
                   + ". Unable to continue recovery.");
                   + ". Unable to continue recovery.");
@@ -788,10 +788,10 @@ public class BookKeeperJournalManager implements JournalManager {
 
 
       in = new BookKeeperEditLogInputStream(lh, l, lastAddConfirmed);
       in = new BookKeeperEditLogInputStream(lh, l, lastAddConfirmed);
 
 
-      long endTxId = HdfsConstants.INVALID_TXID;
+      long endTxId = HdfsServerConstants.INVALID_TXID;
       FSEditLogOp op = in.readOp();
       FSEditLogOp op = in.readOp();
       while (op != null) {
       while (op != null) {
-        if (endTxId == HdfsConstants.INVALID_TXID
+        if (endTxId == HdfsServerConstants.INVALID_TXID
             || op.getTransactionId() == endTxId+1) {
             || op.getTransactionId() == endTxId+1) {
           endTxId = op.getTransactionId();
           endTxId = op.getTransactionId();
         }
         }
@@ -827,7 +827,7 @@ public class BookKeeperJournalManager implements JournalManager {
         try {
         try {
           EditLogLedgerMetadata editLogLedgerMetadata = EditLogLedgerMetadata
           EditLogLedgerMetadata editLogLedgerMetadata = EditLogLedgerMetadata
               .read(zkc, legderMetadataPath);
               .read(zkc, legderMetadataPath);
-          if (editLogLedgerMetadata.getLastTxId() != HdfsConstants.INVALID_TXID
+          if (editLogLedgerMetadata.getLastTxId() != HdfsServerConstants.INVALID_TXID
               && editLogLedgerMetadata.getLastTxId() < fromTxId) {
               && editLogLedgerMetadata.getLastTxId() < fromTxId) {
             // exclude already read closed edits, but include inprogress edits
             // exclude already read closed edits, but include inprogress edits
             // as this will be handled in caller
             // as this will be handled in caller

+ 4 - 4
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.io.IOException;
 import java.util.Comparator;
 import java.util.Comparator;
+
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException;
 
 
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 
 
@@ -67,7 +67,7 @@ public class EditLogLedgerMetadata {
     this.dataLayoutVersion = dataLayoutVersion;
     this.dataLayoutVersion = dataLayoutVersion;
     this.ledgerId = ledgerId;
     this.ledgerId = ledgerId;
     this.firstTxId = firstTxId;
     this.firstTxId = firstTxId;
-    this.lastTxId = HdfsConstants.INVALID_TXID;
+    this.lastTxId = HdfsServerConstants.INVALID_TXID;
     this.inprogress = true;
     this.inprogress = true;
   }
   }
   
   
@@ -107,7 +107,7 @@ public class EditLogLedgerMetadata {
   }
   }
 
 
   void finalizeLedger(long newLastTxId) {
   void finalizeLedger(long newLastTxId) {
-    assert this.lastTxId == HdfsConstants.INVALID_TXID;
+    assert this.lastTxId == HdfsServerConstants.INVALID_TXID;
     this.lastTxId = newLastTxId;
     this.lastTxId = newLastTxId;
     this.inprogress = false;      
     this.inprogress = false;      
   }
   }

+ 3 - 3
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.bookkeeper.conf.ClientConfiguration;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 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.apache.zookeeper.ZooKeeper;
 import org.junit.AfterClass;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.BeforeClass;
@@ -68,7 +68,7 @@ public class TestBookKeeperEditLogStreams {
       lh.close();
       lh.close();
 
 
       EditLogLedgerMetadata metadata = new EditLogLedgerMetadata("/foobar",
       EditLogLedgerMetadata metadata = new EditLogLedgerMetadata("/foobar",
-          HdfsConstants.NAMENODE_LAYOUT_VERSION, lh.getId(), 0x1234);
+          HdfsServerConstants.NAMENODE_LAYOUT_VERSION, lh.getId(), 0x1234);
       try {
       try {
         new BookKeeperEditLogInputStream(lh, metadata, -1);
         new BookKeeperEditLogInputStream(lh, metadata, -1);
         fail("Shouldn't get this far, should have thrown");
         fail("Shouldn't get this far, should have thrown");
@@ -77,7 +77,7 @@ public class TestBookKeeperEditLogStreams {
       }
       }
 
 
       metadata = new EditLogLedgerMetadata("/foobar",
       metadata = new EditLogLedgerMetadata("/foobar",
-          HdfsConstants.NAMENODE_LAYOUT_VERSION, lh.getId(), 0x1234);
+          HdfsServerConstants.NAMENODE_LAYOUT_VERSION, lh.getId(), 0x1234);
       try {
       try {
         new BookKeeperEditLogInputStream(lh, metadata, 0);
         new BookKeeperEditLogInputStream(lh, metadata, 0);
         fail("Shouldn't get this far, should have thrown");
         fail("Shouldn't get this far, should have thrown");

+ 4 - 4
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) {
       } catch (IOException e) {
         // Abort if the lease has already expired. 
         // Abort if the lease has already expired. 
         final long elapsed = Time.monotonicNow() - getLastLeaseRenewal();
         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 "
           LOG.warn("Failed to renew lease for " + clientName + " for "
               + (elapsed/1000) + " seconds (>= hard-limit ="
               + (elapsed/1000) + " seconds (>= hard-limit ="
-              + (HdfsConstants.LEASE_HARDLIMIT_PERIOD/1000) + " seconds.) "
+              + (HdfsServerConstants.LEASE_HARDLIMIT_PERIOD/1000) + " seconds.) "
               + "Closing all files being written ...", e);
               + "Closing all files being written ...", e);
           closeAllFilesBeingWritten(true);
           closeAllFilesBeingWritten(true);
         } else {
         } else {
@@ -1902,7 +1902,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
           //connect to a datanode
           //connect to a datanode
           IOStreamPair pair = connectToDN(datanodes[j], timeout, lb);
           IOStreamPair pair = connectToDN(datanodes[j], timeout, lb);
           out = new DataOutputStream(new BufferedOutputStream(pair.out,
           out = new DataOutputStream(new BufferedOutputStream(pair.out,
-              HdfsConstants.SMALL_BUFFER_SIZE));
+              HdfsServerConstants.SMALL_BUFFER_SIZE));
           in = new DataInputStream(pair.in);
           in = new DataInputStream(pair.in);
 
 
           if (LOG.isDebugEnabled()) {
           if (LOG.isDebugEnabled()) {
@@ -2067,7 +2067,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
 
 
     try {
     try {
       DataOutputStream out = new DataOutputStream(new BufferedOutputStream(pair.out,
       DataOutputStream out = new DataOutputStream(new BufferedOutputStream(pair.out,
-          HdfsConstants.SMALL_BUFFER_SIZE));
+          HdfsServerConstants.SMALL_BUFFER_SIZE));
       DataInputStream in = new DataInputStream(pair.in);
       DataInputStream in = new DataInputStream(pair.in);
   
   
       new Sender(out).readBlock(lb.getBlock(), lb.getBlockToken(), clientName,
       new Sender(out).readBlock(lb.getBlock(), lb.getBlockToken(), clientName,

+ 2 - 6
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.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
 import org.apache.hadoop.crypto.key.KeyProviderFactory;
 import org.apache.hadoop.crypto.key.KeyProviderFactory;
-import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 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.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 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.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.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.authorize.AccessControlList;
-import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.util.ToolRunner;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
@@ -262,7 +258,7 @@ public class DFSUtil {
    * @return true, if the component is reserved
    * @return true, if the component is reserved
    */
    */
   public static boolean isReservedPathComponent(String component) {
   public static boolean isReservedPathComponent(String component) {
-    for (String reserved : HdfsConstants.RESERVED_PATH_COMPONENTS) {
+    for (String reserved : HdfsServerConstants.RESERVED_PATH_COMPONENTS) {
       if (component.equals(reserved)) {
       if (component.equals(reserved)) {
         return true;
         return true;
       }
       }

+ 3 - 2
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.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 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.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
 import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
 import org.apache.hadoop.hdfs.util.ByteArrayManager;
 import org.apache.hadoop.hdfs.util.ByteArrayManager;
@@ -1144,7 +1145,7 @@ class DataStreamer extends Daemon {
       unbufOut = saslStreams.out;
       unbufOut = saslStreams.out;
       unbufIn = saslStreams.in;
       unbufIn = saslStreams.in;
       out = new DataOutputStream(new BufferedOutputStream(unbufOut,
       out = new DataOutputStream(new BufferedOutputStream(unbufOut,
-          HdfsConstants.SMALL_BUFFER_SIZE));
+          HdfsServerConstants.SMALL_BUFFER_SIZE));
       in = new DataInputStream(unbufIn);
       in = new DataInputStream(unbufIn);
 
 
       //send the TRANSFER_BLOCK request
       //send the TRANSFER_BLOCK request
@@ -1424,7 +1425,7 @@ class DataStreamer extends Daemon {
         unbufOut = saslStreams.out;
         unbufOut = saslStreams.out;
         unbufIn = saslStreams.in;
         unbufIn = saslStreams.in;
         out = new DataOutputStream(new BufferedOutputStream(unbufOut,
         out = new DataOutputStream(new BufferedOutputStream(unbufOut,
-            HdfsConstants.SMALL_BUFFER_SIZE));
+            HdfsServerConstants.SMALL_BUFFER_SIZE));
         blockReplyStream = new DataInputStream(unbufIn);
         blockReplyStream = new DataInputStream(unbufIn);
 
 
         //
         //

+ 2 - 1
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.JournalProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB;
 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.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.hdfs.server.namenode.ha.AbstractNNFailoverProxyProvider;
 import org.apache.hadoop.hdfs.server.namenode.ha.AbstractNNFailoverProxyProvider;
@@ -417,7 +418,7 @@ public class NameNodeProxies {
 
 
       RetryPolicy createPolicy = RetryPolicies
       RetryPolicy createPolicy = RetryPolicies
           .retryUpToMaximumCountWithFixedSleep(5,
           .retryUpToMaximumCountWithFixedSleep(5,
-              HdfsConstants.LEASE_SOFTLIMIT_PERIOD, TimeUnit.MILLISECONDS);
+              HdfsServerConstants.LEASE_SOFTLIMIT_PERIOD, TimeUnit.MILLISECONDS);
     
     
       Map<Class<? extends Exception>, RetryPolicy> remoteExceptionToPolicyMap 
       Map<Class<? extends Exception>, RetryPolicy> remoteExceptionToPolicyMap 
                  = new HashMap<Class<? extends Exception>, RetryPolicy>();
                  = new HashMap<Class<? extends Exception>, RetryPolicy>();

+ 3 - 3
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.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSOutputStream;
 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.security.UserGroupInformation;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
@@ -165,7 +165,7 @@ public class LeaseRenewer {
   /** The time in milliseconds that the map became empty. */
   /** The time in milliseconds that the map became empty. */
   private long emptyTime = Long.MAX_VALUE;
   private long emptyTime = Long.MAX_VALUE;
   /** A fixed lease renewal time period in milliseconds */
   /** 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 */
   /** A daemon for renewing lease */
   private Daemon daemon = null;
   private Daemon daemon = null;
@@ -372,7 +372,7 @@ public class LeaseRenewer {
 
 
     //update renewal time
     //update renewal time
     if (renewal == dfsc.getConf().getHdfsTimeout()/2) {
     if (renewal == dfsc.getConf().getHdfsTimeout()/2) {
-      long min = HdfsConstants.LEASE_SOFTLIMIT_PERIOD;
+      long min = HdfsServerConstants.LEASE_SOFTLIMIT_PERIOD;
       for(DFSClient c : dfsclients) {
       for(DFSClient c : dfsclients) {
         final int timeout = c.getConf().getHdfsTimeout();
         final int timeout = c.getConf().getHdfsTimeout();
         if (timeout > 0 && timeout < min) {
         if (timeout > 0 && timeout < min) {

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

@@ -967,8 +967,8 @@ public interface ClientProtocol {
    * <br><br>
    * <br><br>
    *                       
    *                       
    * The quota can have three types of values : (1) 0 or more will set 
    * 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.
    * implies the quota will be reset. Any other value is a runtime error.
    * 
    * 
    * @throws AccessControlException permission denied
    * @throws AccessControlException permission denied

+ 0 - 175
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java

@@ -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;
-}

+ 1 - 1
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) {
       int snapshotNumber, int snapshotQuota, byte[] parentFullPath) {
     this.dirStatus = new HdfsFileStatus(0, true, 0, 0, modification_time,
     this.dirStatus = new HdfsFileStatus(0, true, 0, 0, modification_time,
         access_time, permission, owner, group, null, localName, inodeId,
         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.snapshotNumber = snapshotNumber;
     this.snapshotQuota = snapshotQuota;
     this.snapshotQuota = snapshotQuota;
     this.parentFullPath = parentFullPath;
     this.parentFullPath = parentFullPath;

+ 1 - 1
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(
 @KerberosInfo(
     serverPrincipal = DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY)
     serverPrincipal = DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY)
 @TokenInfo(DelegationTokenSelector.class)
 @TokenInfo(DelegationTokenSelector.class)
-@ProtocolInfo(protocolName = HdfsConstants.CLIENT_NAMENODE_PROTOCOL_NAME, 
+@ProtocolInfo(protocolName = HdfsConstants.CLIENT_NAMENODE_PROTOCOL_NAME,
     protocolVersion = 1)
     protocolVersion = 1)
 /**
 /**
  * Protocol that a clients use to communicate with the NameNode.
  * Protocol that a clients use to communicate with the NameNode.

+ 2 - 2
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.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 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.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -533,7 +533,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       boolean result = 
       boolean result = 
           server.complete(req.getSrc(), req.getClientName(),
           server.complete(req.getSrc(), req.getClientName(),
           req.hasLast() ? PBHelper.convert(req.getLast()) : null,
           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();
       return CompleteResponseProto.newBuilder().setResult(result).build();
     } catch (IOException e) {
     } catch (IOException e) {
       throw new ServiceException(e);
       throw new ServiceException(e);

+ 3 - 3
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.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 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.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 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.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -1436,12 +1436,12 @@ public class PBHelper {
         fs.getFileType().equals(FileType.IS_SYMLINK) ? 
         fs.getFileType().equals(FileType.IS_SYMLINK) ? 
             fs.getSymlink().toByteArray() : null,
             fs.getSymlink().toByteArray() : null,
         fs.getPath().toByteArray(),
         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.hasLocations() ? PBHelper.convert(fs.getLocations()) : null,
         fs.hasChildrenNum() ? fs.getChildrenNum() : -1,
         fs.hasChildrenNum() ? fs.getChildrenNum() : -1,
         fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) : null,
         fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) : null,
         fs.hasStoragePolicy() ? (byte) fs.getStoragePolicy()
         fs.hasStoragePolicy() ? (byte) fs.getStoragePolicy()
-            : HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
+            : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
   }
   }
 
 
   public static SnapshottableDirectoryStatus convert(
   public static SnapshottableDirectoryStatus convert(

+ 2 - 2
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.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 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.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.qjournal.protocol.JournalOutOfSyncException;
 import org.apache.hadoop.hdfs.qjournal.protocol.JournalOutOfSyncException;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
 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.QJournalProtocolPB;
 import org.apache.hadoop.hdfs.qjournal.protocolPB.QJournalProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.qjournal.protocolPB.QJournalProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.qjournal.server.GetJournalEditServlet;
 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.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
@@ -92,7 +92,7 @@ public class IPCLoggerChannel implements AsyncLogger {
   private final ListeningExecutorService parallelExecutor;
   private final ListeningExecutorService parallelExecutor;
   private long ipcSerial = 0;
   private long ipcSerial = 0;
   private long epoch = -1;
   private long epoch = -1;
-  private long committedTxId = HdfsConstants.INVALID_TXID;
+  private long committedTxId = HdfsServerConstants.INVALID_TXID;
   
   
   private final String journalId;
   private final String journalId;
   private final NamespaceInfo nsInfo;
   private final NamespaceInfo nsInfo;

+ 2 - 2
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;
 package org.apache.hadoop.hdfs.qjournal.protocol;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 
 
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 public class RequestInfo {
 public class RequestInfo {
@@ -60,6 +60,6 @@ public class RequestInfo {
   }
   }
 
 
   public boolean hasCommittedTxId() {
   public boolean hasCommittedTxId() {
-    return (committedTxId != HdfsConstants.INVALID_TXID);
+    return (committedTxId != HdfsServerConstants.INVALID_TXID);
   }
   }
 }
 }

+ 2 - 2
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 java.net.URL;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 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.JournalProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
 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.StartLogSegmentRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.StartLogSegmentResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.StartLogSegmentResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo;
 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.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
@@ -263,7 +263,7 @@ public class QJournalProtocolServerSideTranslatorPB implements QJournalProtocolP
         reqInfo.getEpoch(),
         reqInfo.getEpoch(),
         reqInfo.getIpcSerialNumber(),
         reqInfo.getIpcSerialNumber(),
         reqInfo.hasCommittedTxId() ?
         reqInfo.hasCommittedTxId() ?
-          reqInfo.getCommittedTxId() : HdfsConstants.INVALID_TXID);
+          reqInfo.getCommittedTxId() : HdfsServerConstants.INVALID_TXID);
   }
   }
 
 
 
 

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

+ 2 - 3
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.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 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.DataTransferProtoUtil;
 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
@@ -309,9 +308,9 @@ public class Dispatcher {
         unbufOut = saslStreams.out;
         unbufOut = saslStreams.out;
         unbufIn = saslStreams.in;
         unbufIn = saslStreams.in;
         out = new DataOutputStream(new BufferedOutputStream(unbufOut,
         out = new DataOutputStream(new BufferedOutputStream(unbufOut,
-            HdfsConstants.IO_FILE_BUFFER_SIZE));
+            HdfsServerConstants.IO_FILE_BUFFER_SIZE));
         in = new DataInputStream(new BufferedInputStream(unbufIn,
         in = new DataInputStream(new BufferedInputStream(unbufIn,
-            HdfsConstants.IO_FILE_BUFFER_SIZE));
+            HdfsServerConstants.IO_FILE_BUFFER_SIZE));
 
 
         sendRequest(out, eb, accessToken);
         sendRequest(out, eb, accessToken);
         receiveResponse(in);
         receiveResponse(in);

+ 5 - 5
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 com.google.common.base.Preconditions;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 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.GenerationStamp;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 
 
 import java.io.IOException;
 import java.io.IOException;
 
 
@@ -56,7 +56,7 @@ public class BlockIdManager {
   private final SequentialBlockIdGenerator blockIdGenerator;
   private final SequentialBlockIdGenerator blockIdGenerator;
 
 
   public BlockIdManager(BlockManager blockManager) {
   public BlockIdManager(BlockManager blockManager) {
-    this.generationStampV1Limit = HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP;
+    this.generationStampV1Limit = HdfsConstants.GRANDFATHER_GENERATION_STAMP;
     this.blockIdGenerator = new SequentialBlockIdGenerator(blockManager);
     this.blockIdGenerator = new SequentialBlockIdGenerator(blockManager);
   }
   }
 
 
@@ -70,7 +70,7 @@ public class BlockIdManager {
     Preconditions.checkState(generationStampV2.getCurrentValue() ==
     Preconditions.checkState(generationStampV2.getCurrentValue() ==
       GenerationStamp.LAST_RESERVED_STAMP);
       GenerationStamp.LAST_RESERVED_STAMP);
     generationStampV2.skipTo(generationStampV1.getCurrentValue() +
     generationStampV2.skipTo(generationStampV1.getCurrentValue() +
-      HdfsConstants.RESERVED_GENERATION_STAMPS_V1);
+      HdfsServerConstants.RESERVED_GENERATION_STAMPS_V1);
 
 
     generationStampV1Limit = generationStampV2.getCurrentValue();
     generationStampV1Limit = generationStampV2.getCurrentValue();
     return generationStampV2.getCurrentValue();
     return generationStampV2.getCurrentValue();
@@ -83,7 +83,7 @@ public class BlockIdManager {
    * @param stamp set generation stamp limit to this value
    * @param stamp set generation stamp limit to this value
    */
    */
   public void setGenerationStampV1Limit(long stamp) {
   public void setGenerationStampV1Limit(long stamp) {
-    Preconditions.checkState(generationStampV1Limit == HdfsConstantsClient
+    Preconditions.checkState(generationStampV1Limit == HdfsConstants
       .GRANDFATHER_GENERATION_STAMP);
       .GRANDFATHER_GENERATION_STAMP);
     generationStampV1Limit = stamp;
     generationStampV1Limit = stamp;
   }
   }
@@ -204,6 +204,6 @@ public class BlockIdManager {
     generationStampV2.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
     generationStampV2.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
     getBlockIdGenerator().setCurrentValue(SequentialBlockIdGenerator
     getBlockIdGenerator().setCurrentValue(SequentialBlockIdGenerator
       .LAST_RESERVED_BLOCK_ID);
       .LAST_RESERVED_BLOCK_ID);
-    generationStampV1Limit = HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP;
+    generationStampV1Limit = HdfsConstants.GRANDFATHER_GENERATION_STAMP;
   }
   }
 }
 }

+ 2 - 2
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.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 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.LocatedBlock;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.Node;
 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 scheduledSize = blockSize * node.getBlocksScheduled(storage.getStorageType());
     final long remaining = node.getRemaining(storage.getStorageType());
     final long remaining = node.getRemaining(storage.getStorageType());
     if (requiredSize > remaining - scheduledSize) {
     if (requiredSize > remaining - scheduledSize) {

+ 10 - 9
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.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
@@ -47,39 +48,39 @@ public class BlockStoragePolicySuite {
   public static BlockStoragePolicySuite createDefaultSuite() {
   public static BlockStoragePolicySuite createDefaultSuite() {
     final BlockStoragePolicy[] policies =
     final BlockStoragePolicy[] policies =
         new BlockStoragePolicy[1 << ID_BIT_LENGTH];
         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,
     policies[lazyPersistId] = new BlockStoragePolicy(lazyPersistId,
         HdfsConstants.MEMORY_STORAGE_POLICY_NAME,
         HdfsConstants.MEMORY_STORAGE_POLICY_NAME,
         new StorageType[]{StorageType.RAM_DISK, StorageType.DISK},
         new StorageType[]{StorageType.RAM_DISK, StorageType.DISK},
         new StorageType[]{StorageType.DISK},
         new StorageType[]{StorageType.DISK},
         new StorageType[]{StorageType.DISK},
         new StorageType[]{StorageType.DISK},
         true);    // Cannot be changed on regular files, but inherited.
         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,
     policies[allssdId] = new BlockStoragePolicy(allssdId,
         HdfsConstants.ALLSSD_STORAGE_POLICY_NAME,
         HdfsConstants.ALLSSD_STORAGE_POLICY_NAME,
         new StorageType[]{StorageType.SSD},
         new StorageType[]{StorageType.SSD},
         new StorageType[]{StorageType.DISK},
         new StorageType[]{StorageType.DISK},
         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,
     policies[onessdId] = new BlockStoragePolicy(onessdId,
         HdfsConstants.ONESSD_STORAGE_POLICY_NAME,
         HdfsConstants.ONESSD_STORAGE_POLICY_NAME,
         new StorageType[]{StorageType.SSD, StorageType.DISK},
         new StorageType[]{StorageType.SSD, StorageType.DISK},
         new StorageType[]{StorageType.SSD, StorageType.DISK},
         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,
     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.DISK}, StorageType.EMPTY_ARRAY,
         new StorageType[]{StorageType.ARCHIVE});
         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,
     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},
         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,
     policies[coldId] = new BlockStoragePolicy(coldId,
-        HdfsConstants.COLD_STORAGE_POLICY_NAME,
+        HdfsServerConstants.COLD_STORAGE_POLICY_NAME,
         new StorageType[]{StorageType.ARCHIVE}, StorageType.EMPTY_ARRAY,
         new StorageType[]{StorageType.ARCHIVE}, StorageType.EMPTY_ARRAY,
         StorageType.EMPTY_ARRAY);
         StorageType.EMPTY_ARRAY);
     return new BlockStoragePolicySuite(hotId, policies);
     return new BlockStoragePolicySuite(hotId, policies);

+ 85 - 24
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 java.util.regex.Pattern;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 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 org.apache.hadoop.hdfs.server.namenode.MetaRecoveryContext;
 
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 
 
 /************************************
 /************************************
@@ -35,21 +42,75 @@ import org.apache.hadoop.util.StringUtils;
  ************************************/
  ************************************/
 
 
 @InterfaceAudience.Private
 @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
    * Type of the node
    */
    */
-  static public enum NodeType {
+  enum NodeType {
     NAME_NODE,
     NAME_NODE,
     DATA_NODE,
     DATA_NODE,
-    JOURNAL_NODE;
+    JOURNAL_NODE
   }
   }
 
 
   /** Startup options for rolling upgrade. */
   /** Startup options for rolling upgrade. */
-  public static enum RollingUpgradeStartupOption{
+  enum RollingUpgradeStartupOption{
     ROLLBACK, STARTED;
     ROLLBACK, STARTED;
 
 
     public String getOptionString() {
     public String getOptionString() {
@@ -93,7 +154,7 @@ public final class HdfsServerConstants {
   }
   }
 
 
   /** Startup options */
   /** Startup options */
-  static public enum StartupOption{
+  enum StartupOption{
     FORMAT  ("-format"),
     FORMAT  ("-format"),
     CLUSTERID ("-clusterid"),
     CLUSTERID ("-clusterid"),
     GENCLUSTERID ("-genclusterid"),
     GENCLUSTERID ("-genclusterid"),
@@ -137,7 +198,7 @@ public final class HdfsServerConstants {
     // Used only with recovery option
     // Used only with recovery option
     private int force = 0;
     private int force = 0;
 
 
-    private StartupOption(String arg) {this.name = arg;}
+    StartupOption(String arg) {this.name = arg;}
     public String getName() {return name;}
     public String getName() {return name;}
     public NamenodeRole toNodeRole() {
     public NamenodeRole toNodeRole() {
       switch(this) {
       switch(this) {
@@ -221,21 +282,21 @@ public final class HdfsServerConstants {
   }
   }
 
 
   // Timeouts for communicating with DataNode for streaming writes/reads
   // 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.
    * Defines the NameNode role.
    */
    */
-  static public enum NamenodeRole {
+  enum NamenodeRole {
     NAMENODE  ("NameNode"),
     NAMENODE  ("NameNode"),
     BACKUP    ("Backup Node"),
     BACKUP    ("Backup Node"),
     CHECKPOINT("Checkpoint Node");
     CHECKPOINT("Checkpoint Node");
 
 
     private String description = null;
     private String description = null;
-    private NamenodeRole(String arg) {this.description = arg;}
+    NamenodeRole(String arg) {this.description = arg;}
   
   
     @Override
     @Override
     public String toString() {
     public String toString() {
@@ -246,7 +307,7 @@ public final class HdfsServerConstants {
   /**
   /**
    * Block replica states, which it can go through while being constructed.
    * 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. */
     /** Replica is finalized. The state when replica is not modified. */
     FINALIZED(0),
     FINALIZED(0),
     /** Replica is being written to. */
     /** Replica is being written to. */
@@ -260,7 +321,7 @@ public final class HdfsServerConstants {
 
 
     private final int value;
     private final int value;
 
 
-    private ReplicaState(int v) {
+    ReplicaState(int v) {
       value = v;
       value = v;
     }
     }
 
 
@@ -286,7 +347,7 @@ public final class HdfsServerConstants {
   /**
   /**
    * States, which a block can go through while it is under construction.
    * States, which a block can go through while it is under construction.
    */
    */
-  static public enum BlockUCState {
+  enum BlockUCState {
     /**
     /**
      * Block construction completed.<br>
      * Block construction completed.<br>
      * The block has at least the configured minimal replication number
      * The block has at least the configured minimal replication number
@@ -315,16 +376,16 @@ public final class HdfsServerConstants {
      * {@link ReplicaState#FINALIZED} 
      * {@link ReplicaState#FINALIZED} 
      * replicas has yet been reported by data-nodes themselves.
      * 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";
       "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";
       "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";
       "security.hdfs.unreadable.by.superuser";
 }
 }

+ 2 - 3
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 java.util.SortedSet;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 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;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.LayoutFeature;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.LayoutFeature;
@@ -222,8 +221,8 @@ public class StorageInfo {
   }
   }
 
 
   public int getServiceLayoutVersion() {
   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() {
   public Map<Integer, SortedSet<LayoutFeature>> getServiceLayoutFeatureMap() {

+ 2 - 2
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.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 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.io.IOUtils;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DataChecksum;
 
 
@@ -85,7 +85,7 @@ public class BlockMetadataHeader {
     DataInputStream in = null;
     DataInputStream in = null;
     try {
     try {
       in = new DataInputStream(new BufferedInputStream(
       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);
       return readDataChecksum(in, metaFile);
     } finally {
     } finally {
       IOUtils.closeStream(in);
       IOUtils.closeStream(in);

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

+ 2 - 2
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.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 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.BlockConstructionStage;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;
 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.BlockOpResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 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.ReplicaInputStreams;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -246,7 +246,7 @@ class BlockReceiver implements Closeable {
             out.getClass());
             out.getClass());
       }
       }
       this.checksumOut = new DataOutputStream(new BufferedOutputStream(
       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
       // write data chunk header if creating a new replica
       if (isCreate) {
       if (isCreate) {
         BlockMetadataHeader.writeHeader(checksumOut, diskChecksum);
         BlockMetadataHeader.writeHeader(checksumOut, diskChecksum);

+ 4 - 4
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.commons.logging.Log;
 import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 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.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.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 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 MIN_BUFFER_WITH_TRANSFERTO = 64*1024;
   private static final int TRANSFERTO_BUFFER_SIZE = Math.max(
   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 */
   /** the block to read from */
   private final ExtendedBlock block;
   private final ExtendedBlock block;
@@ -298,7 +298,7 @@ class BlockSender implements java.io.Closeable {
             // storage and computes the checksum.
             // storage and computes the checksum.
             if (metaIn.getLength() > BlockMetadataHeader.getHeaderSize()) {
             if (metaIn.getLength() > BlockMetadataHeader.getHeaderSize()) {
               checksumIn = new DataInputStream(new BufferedInputStream(
               checksumIn = new DataInputStream(new BufferedInputStream(
-                  metaIn, HdfsConstants.IO_FILE_BUFFER_SIZE));
+                  metaIn, HdfsServerConstants.IO_FILE_BUFFER_SIZE));
   
   
               csum = BlockMetadataHeader.readDataChecksum(checksumIn, block);
               csum = BlockMetadataHeader.readDataChecksum(checksumIn, block);
               keepMetaInOpen = true;
               keepMetaInOpen = true;
@@ -747,7 +747,7 @@ class BlockSender implements java.io.Closeable {
         pktBufSize += checksumSize * maxChunksPerPacket;
         pktBufSize += checksumSize * maxChunksPerPacket;
       } else {
       } else {
         maxChunksPerPacket = Math.max(1,
         maxChunksPerPacket = Math.max(1,
-            numberOfChunks(HdfsConstants.IO_FILE_BUFFER_SIZE));
+            numberOfChunks(HdfsServerConstants.IO_FILE_BUFFER_SIZE));
         // Packet size includes both checksum and data
         // Packet size includes both checksum and data
         pktBufSize += (chunkSize + checksumSize) * maxChunksPerPacket;
         pktBufSize += (chunkSize + checksumSize) * maxChunksPerPacket;
       }
       }

+ 1 - 1
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;
         unbufIn = saslStreams.in;
         
         
         out = new DataOutputStream(new BufferedOutputStream(unbufOut,
         out = new DataOutputStream(new BufferedOutputStream(unbufOut,
-            HdfsConstants.SMALL_BUFFER_SIZE));
+            HdfsServerConstants.SMALL_BUFFER_SIZE));
         in = new DataInputStream(unbufIn);
         in = new DataInputStream(unbufIn);
         blockSender = new BlockSender(b, 0, b.getNumBytes(), 
         blockSender = new BlockSender(b, 0, b.getNumBytes(), 
             false, false, true, DataNode.this, null, cachingStrategy);
             false, false, true, DataNode.this, null, cachingStrategy);

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

+ 8 - 8
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(),
           socketIn, datanode.getXferAddress().getPort(),
           datanode.getDatanodeId());
           datanode.getDatanodeId());
         input = new BufferedInputStream(saslStreams.in,
         input = new BufferedInputStream(saslStreams.in,
-          HdfsConstants.SMALL_BUFFER_SIZE);
+          HdfsServerConstants.SMALL_BUFFER_SIZE);
         socketOut = saslStreams.out;
         socketOut = saslStreams.out;
       } catch (InvalidMagicNumberException imne) {
       } catch (InvalidMagicNumberException imne) {
         if (imne.isHandshake4Encryption()) {
         if (imne.isHandshake4Encryption()) {
@@ -514,7 +514,7 @@ class DataXceiver extends Receiver implements Runnable {
     long read = 0;
     long read = 0;
     OutputStream baseStream = getOutputStream();
     OutputStream baseStream = getOutputStream();
     DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
     DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
-        baseStream, HdfsConstants.SMALL_BUFFER_SIZE));
+        baseStream, HdfsServerConstants.SMALL_BUFFER_SIZE));
     checkAccess(out, true, block, blockToken,
     checkAccess(out, true, block, blockToken,
         Op.READ_BLOCK, BlockTokenIdentifier.AccessMode.READ);
         Op.READ_BLOCK, BlockTokenIdentifier.AccessMode.READ);
   
   
@@ -658,7 +658,7 @@ class DataXceiver extends Receiver implements Runnable {
     final DataOutputStream replyOut = new DataOutputStream(
     final DataOutputStream replyOut = new DataOutputStream(
         new BufferedOutputStream(
         new BufferedOutputStream(
             getOutputStream(),
             getOutputStream(),
-            HdfsConstants.SMALL_BUFFER_SIZE));
+            HdfsServerConstants.SMALL_BUFFER_SIZE));
     checkAccess(replyOut, isClient, block, blockToken,
     checkAccess(replyOut, isClient, block, blockToken,
         Op.WRITE_BLOCK, BlockTokenIdentifier.AccessMode.WRITE);
         Op.WRITE_BLOCK, BlockTokenIdentifier.AccessMode.WRITE);
 
 
@@ -717,7 +717,7 @@ class DataXceiver extends Receiver implements Runnable {
           unbufMirrorOut = saslStreams.out;
           unbufMirrorOut = saslStreams.out;
           unbufMirrorIn = saslStreams.in;
           unbufMirrorIn = saslStreams.in;
           mirrorOut = new DataOutputStream(new BufferedOutputStream(unbufMirrorOut,
           mirrorOut = new DataOutputStream(new BufferedOutputStream(unbufMirrorOut,
-              HdfsConstants.SMALL_BUFFER_SIZE));
+              HdfsServerConstants.SMALL_BUFFER_SIZE));
           mirrorIn = new DataInputStream(unbufMirrorIn);
           mirrorIn = new DataInputStream(unbufMirrorIn);
 
 
           // Do not propagate allowLazyPersist to downstream DataNodes.
           // Do not propagate allowLazyPersist to downstream DataNodes.
@@ -932,7 +932,7 @@ class DataXceiver extends Receiver implements Runnable {
         .getMetaDataInputStream(block);
         .getMetaDataInputStream(block);
     
     
     final DataInputStream checksumIn = new DataInputStream(
     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);
     updateCurrentThreadName("Getting checksum for block " + block);
     try {
     try {
       //read metadata file
       //read metadata file
@@ -1024,7 +1024,7 @@ class DataXceiver extends Receiver implements Runnable {
       // set up response stream
       // set up response stream
       OutputStream baseStream = getOutputStream();
       OutputStream baseStream = getOutputStream();
       reply = new DataOutputStream(new BufferedOutputStream(
       reply = new DataOutputStream(new BufferedOutputStream(
-          baseStream, HdfsConstants.SMALL_BUFFER_SIZE));
+          baseStream, HdfsServerConstants.SMALL_BUFFER_SIZE));
 
 
       // send status first
       // send status first
       writeSuccessWithChecksumInfo(blockSender, reply);
       writeSuccessWithChecksumInfo(blockSender, reply);
@@ -1132,9 +1132,9 @@ class DataXceiver extends Receiver implements Runnable {
         unbufProxyIn = saslStreams.in;
         unbufProxyIn = saslStreams.in;
         
         
         proxyOut = new DataOutputStream(new BufferedOutputStream(unbufProxyOut, 
         proxyOut = new DataOutputStream(new BufferedOutputStream(unbufProxyOut, 
-            HdfsConstants.SMALL_BUFFER_SIZE));
+            HdfsServerConstants.SMALL_BUFFER_SIZE));
         proxyReply = new DataInputStream(new BufferedInputStream(unbufProxyIn,
         proxyReply = new DataInputStream(new BufferedInputStream(unbufProxyIn,
-            HdfsConstants.IO_FILE_BUFFER_SIZE));
+            HdfsServerConstants.IO_FILE_BUFFER_SIZE));
         
         
         /* send request to the proxy */
         /* send request to the proxy */
         IoeDuringCopyBlockOperation = true;
         IoeDuringCopyBlockOperation = true;

+ 2 - 2
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.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 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.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.Daemon;
@@ -305,7 +305,7 @@ public class DirectoryScanner implements Runnable {
     public long getGenStamp() {
     public long getGenStamp() {
       return metaSuffix != null ? Block.getGenerationStamp(
       return metaSuffix != null ? Block.getGenerationStamp(
           getMetaFile().getName()) : 
           getMetaFile().getName()) : 
-            HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP;
+            HdfsConstants.GRANDFATHER_GENERATION_STAMP;
     }
     }
   }
   }
 
 

+ 2 - 2
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.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
 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.BlockMetadataHeader;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
@@ -612,7 +612,7 @@ class BlockPoolSlice {
       }
       }
       checksumIn = new DataInputStream(
       checksumIn = new DataInputStream(
           new BufferedInputStream(new FileInputStream(metaFile),
           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
       // read and handle the common header here. For now just a version
       final DataChecksum checksum = BlockMetadataHeader.readDataChecksum(
       final DataChecksum checksum = BlockMetadataHeader.readDataChecksum(

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

+ 2 - 2
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.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
 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;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
 
 
 /** Utility methods. */
 /** Utility methods. */
@@ -88,7 +88,7 @@ public class FsDatasetUtil {
       return Block.getGenerationStamp(listdir[j].getName());
       return Block.getGenerationStamp(listdir[j].getName());
     }
     }
     FsDatasetImpl.LOG.warn("Block " + blockFile + " does not have a metafile!");
     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 */
   /** Find the corresponding meta data file from a given block file */

+ 1 - 1
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) {
     private boolean processFile(String fullPath, HdfsLocatedFileStatus status) {
       final byte policyId = status.getStoragePolicy();
       final byte policyId = status.getStoragePolicy();
       // currently we ignore files with unspecified storage policy
       // 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;
         return false;
       }
       }
       final BlockStoragePolicy policy = blockStoragePolicies[policyId];
       final BlockStoragePolicy policy = blockStoragePolicies[policyId];

+ 4 - 4
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.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.NameNodeProxies;
 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.UnregisteredNodeException;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalProtocolService;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalProtocolService;
 import org.apache.hadoop.hdfs.protocolPB.JournalProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.JournalProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.JournalProtocolServerSideTranslatorPB;
 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.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
@@ -156,7 +156,7 @@ public class BackupNode extends NameNode {
     // Backup node should never do lease recovery,
     // Backup node should never do lease recovery,
     // therefore lease hard limit should never expire.
     // therefore lease hard limit should never expire.
     namesystem.leaseManager.setLeasePeriod(
     namesystem.leaseManager.setLeasePeriod(
-        HdfsConstants.LEASE_SOFTLIMIT_PERIOD, Long.MAX_VALUE);
+        HdfsServerConstants.LEASE_SOFTLIMIT_PERIOD, Long.MAX_VALUE);
 
 
     // register with the active name-node 
     // register with the active name-node 
     registerWith(nsInfo);
     registerWith(nsInfo);
@@ -414,9 +414,9 @@ public class BackupNode extends NameNode {
       LOG.error(errorMsg);
       LOG.error(errorMsg);
       throw new IOException(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: "
       "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;
     return nsInfo;
   }
   }
 
 

+ 3 - 3
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.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.IOException;
 
 
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 
 
 /**
 /**
  * An implementation of the abstract class {@link EditLogInputStream},
  * An implementation of the abstract class {@link EditLogInputStream},
@@ -130,12 +130,12 @@ class EditLogBackupInputStream extends EditLogInputStream {
 
 
   @Override
   @Override
   public long getFirstTxId() {
   public long getFirstTxId() {
-    return HdfsConstants.INVALID_TXID;
+    return HdfsServerConstants.INVALID_TXID;
   }
   }
 
 
   @Override
   @Override
   public long getLastTxId() {
   public long getLastTxId() {
-    return HdfsConstants.INVALID_TXID;
+    return HdfsServerConstants.INVALID_TXID;
   }
   }
 
 
   @Override
   @Override

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

+ 2 - 2
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.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 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.Closeable;
 import java.io.IOException;
 import java.io.IOException;
@@ -112,7 +112,7 @@ public abstract class EditLogInputStream implements Closeable {
    */
    */
   protected long scanNextOp() throws IOException {
   protected long scanNextOp() throws IOException {
     FSEditLogOp next = readOp();
     FSEditLogOp next = readOp();
-    return next != null ? next.txid : HdfsConstants.INVALID_TXID;
+    return next != null ? next.txid : HdfsServerConstants.INVALID_TXID;
   }
   }
   
   
   /** 
   /** 

+ 3 - 3
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 java.io.OutputStream;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 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.hdfs.server.namenode.FSEditLogOp.Writer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
@@ -141,7 +141,7 @@ public class EditsDoubleBuffer {
     }
     }
 
 
     public void writeOp(FSEditLogOp op) throws IOException {
     public void writeOp(FSEditLogOp op) throws IOException {
-      if (firstTxId == HdfsConstants.INVALID_TXID) {
+      if (firstTxId == HdfsServerConstants.INVALID_TXID) {
         firstTxId = op.txid;
         firstTxId = op.txid;
       } else {
       } else {
         assert op.txid > firstTxId;
         assert op.txid > firstTxId;
@@ -153,7 +153,7 @@ public class EditsDoubleBuffer {
     @Override
     @Override
     public DataOutputBuffer reset() {
     public DataOutputBuffer reset() {
       super.reset();
       super.reset();
-      firstTxId = HdfsConstants.INVALID_TXID;
+      firstTxId = HdfsServerConstants.INVALID_TXID;
       numTxns = 0;
       numTxns = 0;
       return this;
       return this;
     }
     }

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

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

+ 4 - 4
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.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 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.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 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.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.Storage.FormatConfirmable;
 import org.apache.hadoop.hdfs.server.common.Storage.FormatConfirmable;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 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.
   // 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
   // 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.
   // the time of printing the statistics to the log file.
   private long lastPrintTime;
   private long lastPrintTime;
@@ -1338,7 +1338,7 @@ public class FSEditLog implements LogsPurgeable {
       return;
       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 :
       minTxIdToKeep <= curSegmentTxId :
       "cannot purge logs older than txid " + minTxIdToKeep +
       "cannot purge logs older than txid " + minTxIdToKeep +
       " when current segment starts at " + curSegmentTxId;
       " when current segment starts at " + curSegmentTxId;
@@ -1602,7 +1602,7 @@ public class FSEditLog implements LogsPurgeable {
       EditLogInputStream elis = iter.next();
       EditLogInputStream elis = iter.next();
       if (elis.getFirstTxId() > txId) break;
       if (elis.getFirstTxId() > txId) break;
       long next = elis.getLastTxId();
       long next = elis.getLastTxId();
-      if (next == HdfsConstants.INVALID_TXID) {
+      if (next == HdfsServerConstants.INVALID_TXID) {
         if (!inProgressOk) {
         if (!inProgressOk) {
           throw new RuntimeException("inProgressOk = false, but " +
           throw new RuntimeException("inProgressOk = false, but " +
               "selectInputStreams returned an in-progress edit " +
               "selectInputStreams returned an in-progress edit " +

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

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

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

+ 4 - 3
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.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 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.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
@@ -1046,10 +1047,10 @@ public class FSImageFormat {
   @VisibleForTesting
   @VisibleForTesting
   public static void useDefaultRenameReservedPairs() {
   public static void useDefaultRenameReservedPairs() {
     renameReservedMap.clear();
     renameReservedMap.clear();
-    for (String key: HdfsConstants.RESERVED_PATH_COMPONENTS) {
+    for (String key: HdfsServerConstants.RESERVED_PATH_COMPONENTS) {
       renameReservedMap.put(
       renameReservedMap.put(
           key,
           key,
-          key + "." + HdfsConstants.NAMENODE_LAYOUT_VERSION + "."
+          key + "." + HdfsServerConstants.NAMENODE_LAYOUT_VERSION + "."
               + "UPGRADE_RENAMED");
               + "UPGRADE_RENAMED");
     }
     }
   }
   }
@@ -1147,7 +1148,7 @@ public class FSImageFormat {
       final int layoutVersion) {
       final int layoutVersion) {
     // If the LV doesn't support snapshots, we're doing an upgrade
     // If the LV doesn't support snapshots, we're doing an upgrade
     if (!NameNodeLayoutVersion.supports(Feature.SNAPSHOT, layoutVersion)) {
     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(
         Preconditions.checkArgument(
             renameReservedMap.containsKey(HdfsConstants.DOT_SNAPSHOT_DIR),
             renameReservedMap.containsKey(HdfsConstants.DOT_SNAPSHOT_DIR),
             RESERVED_ERROR_MSG);
             RESERVED_ERROR_MSG);

+ 4 - 4
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.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 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.CacheDirectiveInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
 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.CacheManagerSection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.NameSystemSection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.NameSystemSection;
@@ -143,7 +143,7 @@ public final class FSImageFormatProtobuf {
     private long imgTxId;
     private long imgTxId;
     /**
     /**
      * Whether the image's layout version must be the same with
      * 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).
      * when we're doing (rollingUpgrade rollback).
      */
      */
     private final boolean requireSameLayoutVersion;
     private final boolean requireSameLayoutVersion;
@@ -192,10 +192,10 @@ public final class FSImageFormatProtobuf {
       }
       }
       FileSummary summary = FSImageUtil.loadSummary(raFile);
       FileSummary summary = FSImageUtil.loadSummary(raFile);
       if (requireSameLayoutVersion && summary.getLayoutVersion() !=
       if (requireSameLayoutVersion && summary.getLayoutVersion() !=
-          HdfsConstants.NAMENODE_LAYOUT_VERSION) {
+          HdfsServerConstants.NAMENODE_LAYOUT_VERSION) {
         throw new IOException("Image version " + summary.getLayoutVersion() +
         throw new IOException("Image version " + summary.getLayoutVersion() +
             " is not equal to the software version " +
             " is not equal to the software version " +
-            HdfsConstants.NAMENODE_LAYOUT_VERSION);
+            HdfsServerConstants.NAMENODE_LAYOUT_VERSION);
       }
       }
 
 
       FileChannel channel = fin.getChannel();
       FileChannel channel = fin.getChannel();

+ 2 - 2
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.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileUtil;
 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.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
@@ -180,7 +180,7 @@ class FSImagePreTransactionalStorageInspector extends FSImageStorageInspector {
     
     
     FSImageFile file = new FSImageFile(latestNameSD, 
     FSImageFile file = new FSImageFile(latestNameSD, 
         NNStorage.getStorageFile(latestNameSD, NameNodeFile.IMAGE),
         NNStorage.getStorageFile(latestNameSD, NameNodeFile.IMAGE),
-        HdfsConstants.INVALID_TXID);
+        HdfsServerConstants.INVALID_TXID);
     LinkedList<FSImageFile> ret = new LinkedList<FSImageFile>();
     LinkedList<FSImageFile> ret = new LinkedList<FSImageFile>();
     ret.add(file);
     ret.add(file);
     return ret;
     return ret;

+ 2 - 2
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.io.IOException;
 import java.util.List;
 import java.util.List;
 
 
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 
 
 /**
 /**
@@ -69,7 +69,7 @@ abstract class FSImageStorageInspector {
     private final File file;
     private final File file;
     
     
     FSImageFile(StorageDirectory sd, File file, long txId) {
     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;
         : "Invalid txid on " + file +": " + txId;
       
       
       this.sd = sd;
       this.sd = sd;

+ 10 - 9
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.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 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.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 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.DatanodeManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 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.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 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.RollingUpgradeStartupOption;
@@ -1021,7 +1021,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   private void updateStorageVersionForRollingUpgrade(final long layoutVersion,
   private void updateStorageVersionForRollingUpgrade(final long layoutVersion,
       StartupOption startOpt) throws IOException {
       StartupOption startOpt) throws IOException {
     boolean rollingStarted = RollingUpgradeStartupOption.STARTED
     boolean rollingStarted = RollingUpgradeStartupOption.STARTED
-        .matches(startOpt) && layoutVersion > HdfsConstants
+        .matches(startOpt) && layoutVersion > HdfsServerConstants
         .NAMENODE_LAYOUT_VERSION;
         .NAMENODE_LAYOUT_VERSION;
     boolean rollingRollback = RollingUpgradeStartupOption.ROLLBACK
     boolean rollingRollback = RollingUpgradeStartupOption.ROLLBACK
         .matches(startOpt);
         .matches(startOpt);
@@ -3234,7 +3234,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     Block previousBlock = ExtendedBlock.getLocalBlock(previous);
     Block previousBlock = ExtendedBlock.getLocalBlock(previous);
     final INode inode;
     final INode inode;
     final INodesInPath iip;
     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.
       // 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
       // 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.
       // 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
       //check lease
       final INode inode;
       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.
         // 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
         // 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.
         // 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 INode inode;
       final INodesInPath iip;
       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.
         // 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
         // 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.
         // 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;
     final INodesInPath iip;
     INode inode = null;
     INode inode = null;
     try {
     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.
         // 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
         // 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.
         // 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);
       checkNameNodeSafeMode("Cannot fsync file " + src);
       src = dir.resolvePath(pc, src, pathComponents);
       src = dir.resolvePath(pc, src, pathComponents);
       final INode inode;
       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.
         // 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
         // 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.
         // 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
     //remove lease, close file
     finalizeINodeFileUnderConstruction(src, pendingFile,
     finalizeINodeFileUnderConstruction(src, pendingFile,
-        Snapshot.findLatestSnapshot(pendingFile, Snapshot.CURRENT_STATE_ID));
+                                       Snapshot.findLatestSnapshot(pendingFile,
+                                                                   Snapshot.CURRENT_STATE_ID));
 
 
     return src;
     return src;
   }
   }
@@ -5731,7 +5732,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     SafeModeInfo safeMode = this.safeMode;
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null)
     if (safeMode == null)
       return;
       return;
-    safeMode.setBlockTotal((int)getCompleteBlocksTotal());
+    safeMode.setBlockTotal((int) getCompleteBlocksTotal());
   }
   }
 
 
   /**
   /**

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

+ 6 - 7
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.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 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.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 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) {
   public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps) {
     final byte storagePolicyId = isSymlink() ?
     final byte storagePolicyId = isSymlink() ?
-        HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED : getStoragePolicyID();
+        HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED : getStoragePolicyID();
     return computeQuotaUsage(bsps, storagePolicyId,
     return computeQuotaUsage(bsps, storagePolicyId,
         new QuotaCounts.Builder().build(), true, Snapshot.CURRENT_STATE_ID);
         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(
   public final QuotaCounts computeQuotaUsage(
     BlockStoragePolicySuite bsps, QuotaCounts counts, boolean useCache) {
     BlockStoragePolicySuite bsps, QuotaCounts counts, boolean useCache) {
     final byte storagePolicyId = isSymlink() ?
     final byte storagePolicyId = isSymlink() ?
-        HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED : getStoragePolicyID();
+        HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED : getStoragePolicyID();
     return computeQuotaUsage(bsps, storagePolicyId, counts,
     return computeQuotaUsage(bsps, storagePolicyId, counts,
         useCache, Snapshot.CURRENT_STATE_ID);
         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
    * @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.
    * been specified.
    */
    */
   public abstract byte getLocalStoragePolicyID();
   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
    * Get the storage policy ID while computing quota usage
    * @param parentStoragePolicyId the storage policy ID of the parent directory
    * @param parentStoragePolicyId the storage policy ID of the parent directory
    * @return the storage policy ID of this INode. Note that for an
    * @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
    * instead of throwing Exception
    */
    */
   public byte getStoragePolicyIDForQuota(byte parentStoragePolicyId) {
   public byte getStoragePolicyIDForQuota(byte parentStoragePolicyId) {
     byte localId = isSymlink() ?
     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;
         localId : parentStoragePolicyId;
   }
   }
 
 

+ 1 - 1
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.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 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.
  * Directory INode class.

+ 1 - 1
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;
 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.CURRENT_STATE_ID;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.NO_SNAPSHOT_ID;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.NO_SNAPSHOT_ID;
 
 

+ 2 - 2
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 java.io.FileNotFoundException;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 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;
 import org.apache.hadoop.util.SequentialNumber;
 
 
 /**
 /**
@@ -44,7 +44,7 @@ public class INodeId extends SequentialNumber {
    */
    */
   public static void checkId(long requestId, INode inode)
   public static void checkId(long requestId, INode inode)
       throws FileNotFoundException {
       throws FileNotFoundException {
-    if (requestId != HdfsConstantsClient.GRANDFATHER_INODE_ID && requestId != inode.getId()) {
+    if (requestId != HdfsConstants.GRANDFATHER_INODE_ID && requestId != inode.getId()) {
       throw new FileNotFoundException(
       throw new FileNotFoundException(
           "ID mismatch. Request id and saved id: " + requestId + " , "
           "ID mismatch. Request id and saved id: " + requestId + " , "
               + inode.getId() + " for file " + inode.getFullPathName());
               + inode.getId() + " for file " + inode.getFullPathName());

+ 3 - 3
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.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 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.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.util.GSet;
 import org.apache.hadoop.util.GSet;
 import org.apache.hadoop.util.LightWeightGSet;
 import org.apache.hadoop.util.LightWeightGSet;
@@ -124,12 +124,12 @@ public class INodeMap {
 
 
       @Override
       @Override
       public byte getStoragePolicyID(){
       public byte getStoragePolicyID(){
-        return HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
+        return HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
       }
       }
 
 
       @Override
       @Override
       public byte getLocalStoragePolicyID() {
       public byte getLocalStoragePolicyID() {
-        return HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
+        return HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
       }
       }
     };
     };
       
       

+ 2 - 1
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.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 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.DirectoryWithSnapshotFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
 
@@ -49,7 +50,7 @@ public class INodesInPath {
    */
    */
   private static boolean isDotSnapshotDir(byte[] pathComponent) {
   private static boolean isDotSnapshotDir(byte[] pathComponent) {
     return pathComponent != null &&
     return pathComponent != null &&
-        Arrays.equals(HdfsConstants.DOT_SNAPSHOT_DIR_BYTES, pathComponent);
+        Arrays.equals(HdfsServerConstants.DOT_SNAPSHOT_DIR_BYTES, pathComponent);
   }
   }
 
 
   static INodesInPath fromINode(INode inode) {
   static INodesInPath fromINode(INode inode) {

+ 2 - 3
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.classification.InterfaceAudience;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 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.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.Daemon;
@@ -72,8 +71,8 @@ public class LeaseManager {
 
 
   private final FSNamesystem fsnamesystem;
   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
   // Used for handling lock-leases

+ 5 - 6
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.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 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.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 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.NodeType;
@@ -129,7 +128,7 @@ public class NNStorage extends Storage implements Closeable,
    * recent fsimage file. This does not include any transactions
    * recent fsimage file. This does not include any transactions
    * that have since been written to the edit log.
    * 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.
    * 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 {
   public void format(NamespaceInfo nsInfo) throws IOException {
     Preconditions.checkArgument(nsInfo.getLayoutVersion() == 0 ||
     Preconditions.checkArgument(nsInfo.getLayoutVersion() == 0 ||
-        nsInfo.getLayoutVersion() == HdfsConstants.NAMENODE_LAYOUT_VERSION,
+        nsInfo.getLayoutVersion() == HdfsServerConstants.NAMENODE_LAYOUT_VERSION,
         "Bad layout version: %s", nsInfo.getLayoutVersion());
         "Bad layout version: %s", nsInfo.getLayoutVersion());
     
     
     this.setStorageInfo(nsInfo);
     this.setStorageInfo(nsInfo);
@@ -577,7 +576,7 @@ public class NNStorage extends Storage implements Closeable,
   }
   }
   
   
   public void format() throws IOException {
   public void format() throws IOException {
-    this.layoutVersion = HdfsConstants.NAMENODE_LAYOUT_VERSION;
+    this.layoutVersion = HdfsServerConstants.NAMENODE_LAYOUT_VERSION;
     for (Iterator<StorageDirectory> it =
     for (Iterator<StorageDirectory> it =
                            dirIterator(); it.hasNext();) {
                            dirIterator(); it.hasNext();) {
       StorageDirectory sd = it.next();
       StorageDirectory sd = it.next();
@@ -634,7 +633,7 @@ public class NNStorage extends Storage implements Closeable,
             "storage directory " + sd.getRoot().getAbsolutePath());
             "storage directory " + sd.getRoot().getAbsolutePath());
       }
       }
       props.setProperty("layoutVersion",
       props.setProperty("layoutVersion",
-          Integer.toString(HdfsConstants.NAMENODE_LAYOUT_VERSION));
+          Integer.toString(HdfsServerConstants.NAMENODE_LAYOUT_VERSION));
     }
     }
     setFieldsFromProperties(props, sd);
     setFieldsFromProperties(props, sd);
   }
   }
@@ -657,7 +656,7 @@ public class NNStorage extends Storage implements Closeable,
    * This should only be used during upgrades.
    * This should only be used during upgrades.
    */
    */
   String getDeprecatedProperty(String prop) {
   String getDeprecatedProperty(String prop) {
-    assert getLayoutVersion() > HdfsConstants.NAMENODE_LAYOUT_VERSION :
+    assert getLayoutVersion() > HdfsServerConstants.NAMENODE_LAYOUT_VERSION :
       "getDeprecatedProperty should only be done when loading " +
       "getDeprecatedProperty should only be done when loading " +
       "storage from past versions during upgrade.";
       "storage from past versions during upgrade.";
     return deprecatedProperties.get(prop);
     return deprecatedProperties.get(prop);

+ 1 - 1
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) {
   public static URI getUri(InetSocketAddress namenode) {
     int port = namenode.getPort();
     int port = namenode.getPort();
     String portString = port == DEFAULT_PORT ? "" : (":"+port);
     String portString = port == DEFAULT_PORT ? "" : (":"+port);
-    return URI.create(HdfsConstants.HDFS_URI_SCHEME + "://" 
+    return URI.create(HdfsConstants.HDFS_URI_SCHEME + "://"
         + namenode.getHostName()+portString);
         + namenode.getHostName()+portString);
   }
   }
 
 

+ 5 - 5
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_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_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_HANDLER_COUNT_KEY;
 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 static org.apache.hadoop.util.Time.now;
 
 
 import java.io.FileNotFoundException;
 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.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FSLimitException;
 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.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 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.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 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.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
@@ -1500,9 +1500,9 @@ class NameNodeRpcServer implements NamenodeProtocols {
    * @throws IOException on layout version mismatch
    * @throws IOException on layout version mismatch
    */
    */
   void verifyLayoutVersion(int version) throws IOException {
   void verifyLayoutVersion(int version) throws IOException {
-    if (version != HdfsConstants.NAMENODE_LAYOUT_VERSION)
+    if (version != HdfsServerConstants.NAMENODE_LAYOUT_VERSION)
       throw new IncorrectVersionException(
       throw new IncorrectVersionException(
-          HdfsConstants.NAMENODE_LAYOUT_VERSION, version, "data node");
+          HdfsServerConstants.NAMENODE_LAYOUT_VERSION, version, "data node");
   }
   }
   
   
   private void verifySoftwareVersion(DatanodeRegistration dnReg)
   private void verifySoftwareVersion(DatanodeRegistration dnReg)

+ 6 - 6
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 java.util.Comparator;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 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 org.apache.hadoop.io.IOUtils;
 
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
@@ -88,8 +88,8 @@ class RedundantEditLogInputStream extends EditLogInputStream {
   RedundantEditLogInputStream(Collection<EditLogInputStream> streams,
   RedundantEditLogInputStream(Collection<EditLogInputStream> streams,
       long startTxId) {
       long startTxId) {
     this.curIdx = 0;
     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.state = (streams.isEmpty()) ? State.EOF : State.SKIP_UNTIL;
     this.prevException = null;
     this.prevException = null;
     // EditLogInputStreams in a RedundantEditLogInputStream must be finalized,
     // EditLogInputStreams in a RedundantEditLogInputStream must be finalized,
@@ -97,9 +97,9 @@ class RedundantEditLogInputStream extends EditLogInputStream {
     EditLogInputStream first = null;
     EditLogInputStream first = null;
     for (EditLogInputStream s : streams) {
     for (EditLogInputStream s : streams) {
       Preconditions.checkArgument(s.getFirstTxId() !=
       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() !=
       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) {
       if (first == null) {
         first = s;
         first = s;
       } else {
       } else {
@@ -172,7 +172,7 @@ class RedundantEditLogInputStream extends EditLogInputStream {
       switch (state) {
       switch (state) {
       case SKIP_UNTIL:
       case SKIP_UNTIL:
        try {
        try {
-          if (prevTxId != HdfsConstants.INVALID_TXID) {
+          if (prevTxId != HdfsServerConstants.INVALID_TXID) {
             LOG.info("Fast-forwarding stream '" + streams[curIdx].getName() +
             LOG.info("Fast-forwarding stream '" + streams[curIdx].getName() +
                 "' to transaction ID " + (prevTxId + 1));
                 "' to transaction ID " + (prevTxId + 1));
             streams[curIdx].skipUntil(prevTxId + 1);
             streams[curIdx].skipUntil(prevTxId + 1);

+ 4 - 4
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.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 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;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.StorageErrorReporter;
 import org.apache.hadoop.hdfs.server.common.StorageErrorReporter;
@@ -336,7 +336,7 @@ public class TransferFsImage {
   private static void copyFileToStream(OutputStream out, File localfile,
   private static void copyFileToStream(OutputStream out, File localfile,
       FileInputStream infile, DataTransferThrottler throttler,
       FileInputStream infile, DataTransferThrottler throttler,
       Canceler canceler) throws IOException {
       Canceler canceler) throws IOException {
-    byte buf[] = new byte[HdfsConstants.IO_FILE_BUFFER_SIZE];
+    byte buf[] = new byte[HdfsServerConstants.IO_FILE_BUFFER_SIZE];
     try {
     try {
       CheckpointFaultInjector.getInstance()
       CheckpointFaultInjector.getInstance()
           .aboutToSendFile(localfile);
           .aboutToSendFile(localfile);
@@ -345,7 +345,7 @@ public class TransferFsImage {
             shouldSendShortFile(localfile)) {
             shouldSendShortFile(localfile)) {
           // Test sending image shorter than localfile
           // Test sending image shorter than localfile
           long len = localfile.length();
           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
           // This will read at most half of the image
           // and the rest of the image will be sent over the wire
           // and the rest of the image will be sent over the wire
           infile.read(buf);
           infile.read(buf);
@@ -510,7 +510,7 @@ public class TransferFsImage {
       }
       }
       
       
       int num = 1;
       int num = 1;
-      byte[] buf = new byte[HdfsConstants.IO_FILE_BUFFER_SIZE];
+      byte[] buf = new byte[HdfsServerConstants.IO_FILE_BUFFER_SIZE];
       while (num > 0) {
       while (num > 0) {
         num = stream.read(buf);
         num = stream.read(buf);
         if (num > 0) {
         if (num > 0) {

+ 3 - 3
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.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.NameNodeProxies;
 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.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@@ -167,7 +167,7 @@ public class BootstrapStandby implements Tool, Configurable {
     if (!checkLayoutVersion(nsInfo)) {
     if (!checkLayoutVersion(nsInfo)) {
       LOG.fatal("Layout version on remote node (" + nsInfo.getLayoutVersion()
       LOG.fatal("Layout version on remote node (" + nsInfo.getLayoutVersion()
           + ") does not match " + "this node's layout version ("
           + ") does not match " + "this node's layout version ("
-          + HdfsConstants.NAMENODE_LAYOUT_VERSION + ")");
+          + HdfsServerConstants.NAMENODE_LAYOUT_VERSION + ")");
       return ERR_CODE_INVALID_VERSION;
       return ERR_CODE_INVALID_VERSION;
     }
     }
 
 
@@ -366,7 +366,7 @@ public class BootstrapStandby implements Tool, Configurable {
   }
   }
 
 
   private boolean checkLayoutVersion(NamespaceInfo nsInfo) throws IOException {
   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 {
   private void parseConfAndFindOtherNN() throws IOException {

+ 3 - 3
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.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HAUtil;
 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.NamenodeProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB;
 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.EditLogInputException;
 import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
 import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
 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.
    * 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.
    * 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
    * The last time we successfully loaded a non-zero number of edits from the

+ 2 - 2
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.fs.StorageType;
 import org.apache.hadoop.classification.InterfaceAudience;
 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.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.AclFeature;
 import org.apache.hadoop.hdfs.server.namenode.AclFeature;
@@ -151,7 +151,7 @@ public class FileWithSnapshotFeature implements INode.Feature {
     BlockStoragePolicy bsp = null;
     BlockStoragePolicy bsp = null;
     EnumCounters<StorageType> typeSpaces =
     EnumCounters<StorageType> typeSpaces =
         new EnumCounters<StorageType>(StorageType.class);
         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());
       bsp = bsps.getPolicy(file.getStoragePolicyID());
     }
     }
 
 

+ 2 - 2
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.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 
 
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
 public class NNHAStatusHeartbeat {
 public class NNHAStatusHeartbeat {
 
 
   private final HAServiceState state;
   private final HAServiceState state;
-  private long txid = HdfsConstants.INVALID_TXID;
+  private long txid = HdfsServerConstants.INVALID_TXID;
   
   
   public NNHAStatusHeartbeat(HAServiceState state, long txid) {
   public NNHAStatusHeartbeat(HAServiceState state, long txid) {
     this.state = state;
     this.state = state;

+ 2 - 2
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.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 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.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 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,
   public NamespaceInfo(int nsID, String clusterID, String bpID,
       long cT, String buildVersion, String softwareVersion,
       long cT, String buildVersion, String softwareVersion,
       long capabilities) {
       long capabilities) {
-    super(HdfsConstants.NAMENODE_LAYOUT_VERSION, nsID, clusterID, cT,
+    super(HdfsServerConstants.NAMENODE_LAYOUT_VERSION, nsID, clusterID, cT,
         NodeType.NAME_NODE);
         NodeType.NAME_NODE);
     blockPoolID = bpID;
     blockPoolID = bpID;
     this.buildVersion = buildVersion;
     this.buildVersion = buildVersion;

+ 5 - 6
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;
 package org.apache.hadoop.hdfs.server.protocol;
 
 
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-
 import com.google.common.base.Function;
 import com.google.common.base.Function;
 import com.google.common.collect.ComparisonChain;
 import com.google.common.collect.ComparisonChain;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 
 
 public class RemoteEditLog implements Comparable<RemoteEditLog> {
 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;
   private boolean isInProgress = false;
   
   
   public RemoteEditLog() {
   public RemoteEditLog() {
@@ -33,7 +32,7 @@ public class RemoteEditLog implements Comparable<RemoteEditLog> {
   public RemoteEditLog(long startTxId, long endTxId) {
   public RemoteEditLog(long startTxId, long endTxId) {
     this.startTxId = startTxId;
     this.startTxId = startTxId;
     this.endTxId = endTxId;
     this.endTxId = endTxId;
-    this.isInProgress = (endTxId == HdfsConstants.INVALID_TXID);
+    this.isInProgress = (endTxId == HdfsServerConstants.INVALID_TXID);
   }
   }
   
   
   public RemoteEditLog(long startTxId, long endTxId, boolean inProgress) {
   public RemoteEditLog(long startTxId, long endTxId, boolean inProgress) {
@@ -90,7 +89,7 @@ public class RemoteEditLog implements Comparable<RemoteEditLog> {
       @Override
       @Override
       public Long apply(RemoteEditLog log) {
       public Long apply(RemoteEditLog log) {
         if (null == log) {
         if (null == log) {
-          return HdfsConstants.INVALID_TXID;
+          return HdfsServerConstants.INVALID_TXID;
         }
         }
         return log.getStartTxId();
         return log.getStartTxId();
       }
       }

+ 2 - 2
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.fs.Path;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 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.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.tools.TableListing;
 import org.apache.hadoop.tools.TableListing;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
@@ -151,7 +151,7 @@ public class StoragePolicyAdmin extends Configured implements Tool {
           return 2;
           return 2;
         }
         }
         byte storagePolicyId = status.getStoragePolicy();
         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");
           System.out.println("The storage policy of " + path + " is unspecified");
           return 0;
           return 0;
         }
         }

+ 3 - 4
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.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 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.EditLogFileInputStream;
 
 
 import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
 import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
@@ -49,8 +48,8 @@ interface OfflineEditsLoader {
         OfflineEditsLoader loader = null;
         OfflineEditsLoader loader = null;
         try {
         try {
           file = new File(inputFileName);
           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);
           loader = new OfflineEditsBinaryLoader(visitor, elis, flags);
         } finally {
         } finally {
           if ((loader == null) && (elis != null)) {
           if ((loader == null) && (elis != null)) {

+ 2 - 2
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.conf.Configuration;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 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.LayoutFlags;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
@@ -687,7 +687,7 @@ class ImageLoaderCurrent implements ImageLoader {
     final String pathName = readINodePath(in, parentName);
     final String pathName = readINodePath(in, parentName);
     v.visit(ImageElement.INODE_PATH, pathName);
     v.visit(ImageElement.INODE_PATH, pathName);
 
 
-    long inodeId = HdfsConstantsClient.GRANDFATHER_INODE_ID;
+    long inodeId = HdfsConstants.GRANDFATHER_INODE_ID;
     if (supportInodeId) {
     if (supportInodeId) {
       inodeId = in.readLong();
       inodeId = in.readLong();
       v.visit(ImageElement.INODE_ID, inodeId);
       v.visit(ImageElement.INODE_ID, inodeId);

+ 2 - 1
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.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 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.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.web.WebHdfsConstants;
 import org.apache.hadoop.hdfs.web.WebHdfsConstants;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
@@ -259,7 +260,7 @@ abstract public class TestSymlinkHdfs extends SymlinkBaseTest {
   public void testCreateLinkMaxPathLink() throws IOException {
   public void testCreateLinkMaxPathLink() throws IOException {
     Path dir  = new Path(testBaseDir1());
     Path dir  = new Path(testBaseDir1());
     Path file = new Path(testBaseDir1(), "file");
     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;
     final int dirLen     = dir.toString().length() + 1;
     int   len            = maxPathLen - dirLen;
     int   len            = maxPathLen - dirLen;
     
     

+ 4 - 4
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.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 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.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
@@ -950,7 +951,7 @@ public class DFSTestUtil {
     final long writeTimeout = dfsClient.getDatanodeWriteTimeout(datanodes.length);
     final long writeTimeout = dfsClient.getDatanodeWriteTimeout(datanodes.length);
     final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
     final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
         NetUtils.getOutputStream(s, writeTimeout),
         NetUtils.getOutputStream(s, writeTimeout),
-        HdfsConstants.SMALL_BUFFER_SIZE));
+        HdfsServerConstants.SMALL_BUFFER_SIZE));
     final DataInputStream in = new DataInputStream(NetUtils.getInputStream(s));
     final DataInputStream in = new DataInputStream(NetUtils.getInputStream(s));
 
 
     // send the request
     // send the request
@@ -1222,7 +1223,7 @@ public class DFSTestUtil {
     s2.close();
     s2.close();
     // OP_SET_STORAGE_POLICY 45
     // OP_SET_STORAGE_POLICY 45
     filesystem.setStoragePolicy(pathFileCreate,
     filesystem.setStoragePolicy(pathFileCreate,
-        HdfsConstants.HOT_STORAGE_POLICY_NAME);
+        HdfsServerConstants.HOT_STORAGE_POLICY_NAME);
     // OP_RENAME_OLD 1
     // OP_RENAME_OLD 1
     final Path pathFileMoved = new Path("/file_moved");
     final Path pathFileMoved = new Path("/file_moved");
     filesystem.rename(pathFileCreate, pathFileMoved);
     filesystem.rename(pathFileCreate, pathFileMoved);
@@ -1689,8 +1690,7 @@ public class DFSTestUtil {
     modifiersField.setInt(field, field.getModifiers() & ~Modifier.FINAL);
     modifiersField.setInt(field, field.getModifiers() & ~Modifier.FINAL);
     field.setInt(null, lv);
     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);
     field.setAccessible(true);
     modifiersField.setInt(field, field.getModifiers() & ~Modifier.FINAL);
     modifiersField.setInt(field, field.getModifiers() & ~Modifier.FINAL);
     field.setInt(null, lv);
     field.setInt(null, lv);

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

@@ -17,7 +17,7 @@
  */
  */
 package org.apache.hadoop.hdfs;
 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.File;
 import java.io.FileNotFoundException;
 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.*;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.blockmanagement.*;
 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.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -67,12 +68,12 @@ public class TestBlockStoragePolicy {
   static final long FILE_LEN = 1024;
   static final long FILE_LEN = 1024;
   static final short REPLICATION = 3;
   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)
   @Test (timeout=300000)
   public void testConfigKeyEnabled() throws IOException {
   public void testConfigKeyEnabled() throws IOException {
@@ -83,7 +84,7 @@ public class TestBlockStoragePolicy {
     try {
     try {
       cluster.waitActive();
       cluster.waitActive();
       cluster.getFileSystem().setStoragePolicy(new Path("/"),
       cluster.getFileSystem().setStoragePolicy(new Path("/"),
-          HdfsConstants.COLD_STORAGE_POLICY_NAME);
+          HdfsServerConstants.COLD_STORAGE_POLICY_NAME);
     } finally {
     } finally {
       cluster.shutdown();
       cluster.shutdown();
     }
     }
@@ -103,7 +104,7 @@ public class TestBlockStoragePolicy {
     try {
     try {
       cluster.waitActive();
       cluster.waitActive();
       cluster.getFileSystem().setStoragePolicy(new Path("/"),
       cluster.getFileSystem().setStoragePolicy(new Path("/"),
-          HdfsConstants.COLD_STORAGE_POLICY_NAME);
+          HdfsServerConstants.COLD_STORAGE_POLICY_NAME);
     } finally {
     } finally {
       cluster.shutdown();
       cluster.shutdown();
     }
     }
@@ -860,15 +861,15 @@ public class TestBlockStoragePolicy {
 
 
       final Path invalidPath = new Path("/invalidPath");
       final Path invalidPath = new Path("/invalidPath");
       try {
       try {
-        fs.setStoragePolicy(invalidPath, HdfsConstants.WARM_STORAGE_POLICY_NAME);
+        fs.setStoragePolicy(invalidPath, HdfsServerConstants.WARM_STORAGE_POLICY_NAME);
         Assert.fail("Should throw a FileNotFoundException");
         Assert.fail("Should throw a FileNotFoundException");
       } catch (FileNotFoundException e) {
       } catch (FileNotFoundException e) {
         GenericTestUtils.assertExceptionContains(invalidPath.toString(), 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(),
       dirList = fs.getClient().listPaths(dir.toString(),
           HdfsFileStatus.EMPTY_NAME).getPartialListing();
           HdfsFileStatus.EMPTY_NAME).getPartialListing();
@@ -916,7 +917,7 @@ public class TestBlockStoragePolicy {
       DFSTestUtil.createFile(fs, fooFile1, FILE_LEN, REPLICATION, 0L);
       DFSTestUtil.createFile(fs, fooFile1, FILE_LEN, REPLICATION, 0L);
       DFSTestUtil.createFile(fs, fooFile2, 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[] dirList = fs.getClient().listPaths(dir.toString(),
           HdfsFileStatus.EMPTY_NAME, true).getPartialListing();
           HdfsFileStatus.EMPTY_NAME, true).getPartialListing();
@@ -928,7 +929,7 @@ public class TestBlockStoragePolicy {
       // take snapshot
       // take snapshot
       SnapshotTestHelper.createSnapshot(fs, dir, "s1");
       SnapshotTestHelper.createSnapshot(fs, dir, "s1");
       // change the storage policy of fooFile1
       // 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(),
       fooList = fs.getClient().listPaths(fooDir.toString(),
           HdfsFileStatus.EMPTY_NAME).getPartialListing();
           HdfsFileStatus.EMPTY_NAME).getPartialListing();
@@ -951,7 +952,7 @@ public class TestBlockStoragePolicy {
           HdfsFileStatus.EMPTY_NAME).getPartialListing(), COLD);
           HdfsFileStatus.EMPTY_NAME).getPartialListing(), COLD);
 
 
       // change the storage policy of foo dir
       // 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
       // /dir/foo is now hot
       dirList = fs.getClient().listPaths(dir.toString(),
       dirList = fs.getClient().listPaths(dir.toString(),
           HdfsFileStatus.EMPTY_NAME, true).getPartialListing();
           HdfsFileStatus.EMPTY_NAME, true).getPartialListing();
@@ -1068,7 +1069,7 @@ public class TestBlockStoragePolicy {
    */
    */
   @Test
   @Test
   public void testChangeHotFileRep() throws Exception {
   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,
         new StorageType[]{StorageType.DISK, StorageType.DISK,
             StorageType.DISK},
             StorageType.DISK},
         new StorageType[]{StorageType.DISK, StorageType.DISK, StorageType.DISK,
         new StorageType[]{StorageType.DISK, StorageType.DISK, StorageType.DISK,
@@ -1082,7 +1083,7 @@ public class TestBlockStoragePolicy {
    */
    */
   @Test
   @Test
   public void testChangeWarmRep() throws Exception {
   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,
         new StorageType[]{StorageType.DISK, StorageType.ARCHIVE,
             StorageType.ARCHIVE},
             StorageType.ARCHIVE},
         new StorageType[]{StorageType.DISK, StorageType.ARCHIVE,
         new StorageType[]{StorageType.DISK, StorageType.ARCHIVE,
@@ -1095,7 +1096,7 @@ public class TestBlockStoragePolicy {
    */
    */
   @Test
   @Test
   public void testChangeColdRep() throws Exception {
   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,
         new StorageType[]{StorageType.ARCHIVE, StorageType.ARCHIVE,
             StorageType.ARCHIVE},
             StorageType.ARCHIVE},
         new StorageType[]{StorageType.ARCHIVE, StorageType.ARCHIVE,
         new StorageType[]{StorageType.ARCHIVE, StorageType.ARCHIVE,

+ 3 - 3
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.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 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.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
@@ -191,7 +191,7 @@ public class TestDFSRollback {
           UpgradeUtilities.getCurrentBlockPoolID(cluster));
           UpgradeUtilities.getCurrentBlockPoolID(cluster));
       // Put newer layout version in current.
       // Put newer layout version in current.
       storageInfo = new StorageInfo(
       storageInfo = new StorageInfo(
-          HdfsConstants.DATANODE_LAYOUT_VERSION - 1,
+          HdfsServerConstants.DATANODE_LAYOUT_VERSION - 1,
           UpgradeUtilities.getCurrentNamespaceID(cluster),
           UpgradeUtilities.getCurrentNamespaceID(cluster),
           UpgradeUtilities.getCurrentClusterID(cluster),
           UpgradeUtilities.getCurrentClusterID(cluster),
           UpgradeUtilities.getCurrentFsscTime(cluster),
           UpgradeUtilities.getCurrentFsscTime(cluster),
@@ -277,7 +277,7 @@ public class TestDFSRollback {
       
       
       UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "current");
       UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "current");
       baseDirs = UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "previous");
       baseDirs = UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "previous");
-      storageInfo = new StorageInfo(HdfsConstants.DATANODE_LAYOUT_VERSION,
+      storageInfo = new StorageInfo(HdfsServerConstants.DATANODE_LAYOUT_VERSION,
           UpgradeUtilities.getCurrentNamespaceID(cluster),
           UpgradeUtilities.getCurrentNamespaceID(cluster),
           UpgradeUtilities.getCurrentClusterID(cluster), Long.MAX_VALUE,
           UpgradeUtilities.getCurrentClusterID(cluster), Long.MAX_VALUE,
           NodeType.DATA_NODE);
           NodeType.DATA_NODE);

+ 4 - 4
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.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 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.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
@@ -89,7 +89,7 @@ public class TestDFSStartupVersions {
    */
    */
   private StorageData[] initializeVersions() throws Exception {
   private StorageData[] initializeVersions() throws Exception {
     int layoutVersionOld = Storage.LAST_UPGRADABLE_LAYOUT_VERSION;
     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 layoutVersionNew = Integer.MIN_VALUE;
     int namespaceIdCur = UpgradeUtilities.getCurrentNamespaceID(null);
     int namespaceIdCur = UpgradeUtilities.getCurrentNamespaceID(null);
     int namespaceIdOld = Integer.MIN_VALUE;
     int namespaceIdOld = Integer.MIN_VALUE;
@@ -200,7 +200,7 @@ public class TestDFSStartupVersions {
       return false;
       return false;
     }
     }
     // check #3
     // check #3
-    int softwareLV = HdfsConstants.DATANODE_LAYOUT_VERSION;
+    int softwareLV = HdfsServerConstants.DATANODE_LAYOUT_VERSION;
     int storedLV = datanodeVer.getLayoutVersion();
     int storedLV = datanodeVer.getLayoutVersion();
     if (softwareLV == storedLV &&  
     if (softwareLV == storedLV &&  
         datanodeVer.getCTime() == namenodeVer.getCTime()) 
         datanodeVer.getCTime() == namenodeVer.getCTime()) 
@@ -252,7 +252,7 @@ public class TestDFSStartupVersions {
                                               .startupOption(StartupOption.REGULAR)
                                               .startupOption(StartupOption.REGULAR)
                                               .build();
                                               .build();
     StorageData nameNodeVersion = new StorageData(
     StorageData nameNodeVersion = new StorageData(
-        HdfsConstants.NAMENODE_LAYOUT_VERSION,
+        HdfsServerConstants.NAMENODE_LAYOUT_VERSION,
         UpgradeUtilities.getCurrentNamespaceID(cluster),
         UpgradeUtilities.getCurrentNamespaceID(cluster),
         UpgradeUtilities.getCurrentClusterID(cluster),
         UpgradeUtilities.getCurrentClusterID(cluster),
         UpgradeUtilities.getCurrentFsscTime(cluster),
         UpgradeUtilities.getCurrentFsscTime(cluster),

+ 2 - 2
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.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 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.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 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.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
@@ -307,7 +307,7 @@ public class TestDFSUpgrade {
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       cluster = createCluster();
       cluster = createCluster();
       baseDirs = UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "current");
       baseDirs = UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "current");
-      storageInfo = new StorageInfo(HdfsConstants.DATANODE_LAYOUT_VERSION,
+      storageInfo = new StorageInfo(HdfsServerConstants.DATANODE_LAYOUT_VERSION,
           UpgradeUtilities.getCurrentNamespaceID(cluster),
           UpgradeUtilities.getCurrentNamespaceID(cluster),
           UpgradeUtilities.getCurrentClusterID(cluster), Long.MAX_VALUE,
           UpgradeUtilities.getCurrentClusterID(cluster), Long.MAX_VALUE,
           NodeType.DATA_NODE);
           NodeType.DATA_NODE);

+ 4 - 5
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.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 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.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 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.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 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.apache.hadoop.util.VersionInfo;
 import org.junit.Test;
 import org.junit.Test;
 
 
-import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.security.Permission;
 import java.security.Permission;
 
 
@@ -180,7 +179,7 @@ public class TestDatanodeRegistration {
           .getCTime();
           .getCTime();
       StorageInfo mockStorageInfo = mock(StorageInfo.class);
       StorageInfo mockStorageInfo = mock(StorageInfo.class);
       doReturn(nnCTime).when(mockStorageInfo).getCTime();
       doReturn(nnCTime).when(mockStorageInfo).getCTime();
-      doReturn(HdfsConstants.DATANODE_LAYOUT_VERSION).when(mockStorageInfo)
+      doReturn(HdfsServerConstants.DATANODE_LAYOUT_VERSION).when(mockStorageInfo)
           .getLayoutVersion();
           .getLayoutVersion();
       DatanodeRegistration dnReg = new DatanodeRegistration(dnId,
       DatanodeRegistration dnReg = new DatanodeRegistration(dnId,
           mockStorageInfo, null, VersionInfo.getVersion());
           mockStorageInfo, null, VersionInfo.getVersion());
@@ -225,7 +224,7 @@ public class TestDatanodeRegistration {
       doReturn(nnCTime).when(mockStorageInfo).getCTime();
       doReturn(nnCTime).when(mockStorageInfo).getCTime();
       
       
       DatanodeRegistration mockDnReg = mock(DatanodeRegistration.class);
       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("127.0.0.1").when(mockDnReg).getIpAddr();
       doReturn(123).when(mockDnReg).getXferPort();
       doReturn(123).when(mockDnReg).getXferPort();
       doReturn("fake-storage-id").when(mockDnReg).getDatanodeUuid();
       doReturn("fake-storage-id").when(mockDnReg).getDatanodeUuid();
@@ -274,7 +273,7 @@ public class TestDatanodeRegistration {
       doReturn(nnCTime).when(mockStorageInfo).getCTime();
       doReturn(nnCTime).when(mockStorageInfo).getCTime();
       
       
       DatanodeRegistration mockDnReg = mock(DatanodeRegistration.class);
       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("fake-storage-id").when(mockDnReg).getDatanodeUuid();
       doReturn(mockStorageInfo).when(mockDnReg).getStorageInfo();
       doReturn(mockStorageInfo).when(mockDnReg).getStorageInfo();
       
       

+ 2 - 3
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.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
-import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 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.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.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 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
     // set the soft limit to be 1 second so that the
     // namenode triggers lease recovery upon append request
     // namenode triggers lease recovery upon append request
-    cluster.setLeasePeriod(1000, HdfsConstants.LEASE_HARDLIMIT_PERIOD);
+    cluster.setLeasePeriod(1000, HdfsServerConstants.LEASE_HARDLIMIT_PERIOD);
 
 
     // Trying recovery
     // Trying recovery
     int tries = 60;
     int tries = 60;

+ 2 - 3
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.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 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.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@@ -537,7 +536,7 @@ public class TestFileCreation {
 
 
       // add one block to the file
       // add one block to the file
       LocatedBlock location = client.getNamenode().addBlock(file1.toString(),
       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: "
       System.out.println("testFileCreationError2: "
           + "Added block " + location.getBlock());
           + "Added block " + location.getBlock());
 
 
@@ -588,7 +587,7 @@ public class TestFileCreation {
       createFile(dfs, f, 3);
       createFile(dfs, f, 3);
       try {
       try {
         cluster.getNameNodeRpc().addBlock(f.toString(), client.clientName,
         cluster.getNameNodeRpc().addBlock(f.toString(), client.clientName,
-            null, null, HdfsConstantsClient.GRANDFATHER_INODE_ID, null);
+            null, null, HdfsConstants.GRANDFATHER_INODE_ID, null);
         fail();
         fail();
       } catch(IOException ioe) {
       } catch(IOException ioe) {
         FileSystem.LOG.info("GOOD!", ioe);
         FileSystem.LOG.info("GOOD!", ioe);

+ 2 - 2
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.Block;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 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.HdfsConstants.DatanodeReportType;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
@@ -278,7 +278,7 @@ public class TestGetBlocks {
 
 
     for (int i = 0; i < blkids.length; i++) {
     for (int i = 0; i < blkids.length; i++) {
       Block b = new Block(blkids[i], 0,
       Block b = new Block(blkids[i], 0,
-          HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP);
+          HdfsConstants.GRANDFATHER_GENERATION_STAMP);
       Long v = map.get(b);
       Long v = map.get(b);
       System.out.println(b + " => " + v);
       System.out.println(b + " => " + v);
       assertEquals(blkids[i], v.longValue());
       assertEquals(blkids[i], v.longValue());

+ 3 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java

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

+ 2 - 3
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.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 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.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -333,8 +332,8 @@ public class TestLeaseRecovery2 {
     DFSTestUtil.updateConfWithFakeGroupMapping(conf, u2g_map);
     DFSTestUtil.updateConfWithFakeGroupMapping(conf, u2g_map);
 
 
     // Reset default lease periods
     // 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 file
     // create a random file name
     // create a random file name
     String filestr = "/foo" + AppendTestUtil.nextInt();
     String filestr = "/foo" + AppendTestUtil.nextInt();

+ 2 - 2
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.FileUtil;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
 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.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 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.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
@@ -533,7 +533,7 @@ public class UpgradeUtilities {
    * of the Namenode, whether it is running or not.
    * of the Namenode, whether it is running or not.
    */
    */
   public static int getCurrentNameNodeLayoutVersion() {
   public static int getCurrentNameNodeLayoutVersion() {
-    return HdfsConstants.NAMENODE_LAYOUT_VERSION;
+    return HdfsServerConstants.NAMENODE_LAYOUT_VERSION;
   }
   }
   
   
   /**
   /**

+ 2 - 2
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.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 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.QJMTestUtil;
 import org.apache.hadoop.hdfs.qjournal.client.IPCLoggerChannel;
 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.NewEpochResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
 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.namenode.NameNodeLayoutVersion;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
@@ -185,7 +185,7 @@ public class TestJournalNode {
     byte[] retrievedViaHttp = DFSTestUtil.urlGetBytes(new URL(urlRoot +
     byte[] retrievedViaHttp = DFSTestUtil.urlGetBytes(new URL(urlRoot +
         "/getJournal?segmentTxId=1&jid=" + journalId));
         "/getJournal?segmentTxId=1&jid=" + journalId));
     byte[] expected = Bytes.concat(
     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
             (new byte[] { 0, 0, 0, 0 }), // layout flags section
             EDITS_DATA);
             EDITS_DATA);
 
 

+ 5 - 4
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.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 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.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.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
@@ -88,7 +88,8 @@ public class TestBlockManager {
   @Before
   @Before
   public void setupMockCluster() throws IOException {
   public void setupMockCluster() throws IOException {
     Configuration conf = new HdfsConfiguration();
     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);
     fsn = Mockito.mock(FSNamesystem.class);
     Mockito.doReturn(true).when(fsn).hasWriteLock();
     Mockito.doReturn(true).when(fsn).hasWriteLock();
     bm = new BlockManager(fsn, conf);
     bm = new BlockManager(fsn, conf);
@@ -111,8 +112,8 @@ public class TestBlockManager {
     for (DatanodeDescriptor dn : nodesToAdd) {
     for (DatanodeDescriptor dn : nodesToAdd) {
       cluster.add(dn);
       cluster.add(dn);
       dn.getStorageInfos()[0].setUtilizationForTesting(
       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(
       dn.updateHeartbeat(
           BlockManagerTestUtil.getStorageReportsForDatanode(dn), 0L, 0L, 0, 0,
           BlockManagerTestUtil.getStorageReportsForDatanode(dn), 0L, 0L, 0, 0,
           null);
           null);

+ 21 - 21
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.MiniDFSCluster;
 import org.apache.hadoop.hdfs.TestBlockStoragePolicy;
 import org.apache.hadoop.hdfs.TestBlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.Block;
 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.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.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -138,8 +138,8 @@ public class TestReplicationPolicy {
     }
     }
     for (int i=0; i < NUM_OF_DATANODES; i++) {
     for (int i=0; i < NUM_OF_DATANODES; i++) {
       updateHeartbeatWithUsage(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
   @Test
   public void testChooseTarget1() throws Exception {
   public void testChooseTarget1() throws Exception {
     updateHeartbeatWithUsage(dataNodes[0],
     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
         0L, 0L, 4, 0); // overloaded
 
 
     DatanodeStorageInfo[] targets;
     DatanodeStorageInfo[] targets;
@@ -193,8 +193,8 @@ public class TestReplicationPolicy {
     assertFalse(isOnSameRack(targets[0], targets[2]));
     assertFalse(isOnSameRack(targets[0], targets[2]));
     
     
     updateHeartbeatWithUsage(dataNodes[0],
     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) {
   private static DatanodeStorageInfo[] chooseTarget(int numOfReplicas) {
@@ -316,8 +316,8 @@ public class TestReplicationPolicy {
   public void testChooseTarget3() throws Exception {
   public void testChooseTarget3() throws Exception {
     // make data node 0 to be not qualified to choose
     // make data node 0 to be not qualified to choose
     updateHeartbeatWithUsage(dataNodes[0],
     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
         0L, 0L, 0, 0); // no space
         
         
     DatanodeStorageInfo[] targets;
     DatanodeStorageInfo[] targets;
@@ -350,8 +350,8 @@ public class TestReplicationPolicy {
     assertFalse(isOnSameRack(targets[1], targets[3]));
     assertFalse(isOnSameRack(targets[1], targets[3]));
 
 
     updateHeartbeatWithUsage(dataNodes[0],
     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
     // make data node 0 & 1 to be not qualified to choose: not enough disk space
     for(int i=0; i<2; i++) {
     for(int i=0; i<2; i++) {
       updateHeartbeatWithUsage(dataNodes[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;
     DatanodeStorageInfo[] targets;
@@ -395,8 +395,8 @@ public class TestReplicationPolicy {
     
     
     for(int i=0; i<2; i++) {
     for(int i=0; i<2; i++) {
       updateHeartbeatWithUsage(dataNodes[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().getNetworkTopology().add(newDn);
     bm.getDatanodeManager().getHeartbeatManager().addDatanode(newDn);
     bm.getDatanodeManager().getHeartbeatManager().addDatanode(newDn);
     updateHeartbeatWithUsage(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.
     // Try picking three nodes. Only two should return.
     excludedNodes.clear();
     excludedNodes.clear();
@@ -506,8 +506,8 @@ public class TestReplicationPolicy {
     // make data node 0 & 1 to be not qualified to choose: not enough disk space
     // make data node 0 & 1 to be not qualified to choose: not enough disk space
     for(int i=0; i<2; i++) {
     for(int i=0; i<2; i++) {
       updateHeartbeatWithUsage(dataNodes[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();
     final LogVerificationAppender appender = new LogVerificationAppender();
@@ -531,8 +531,8 @@ public class TestReplicationPolicy {
     
     
     for(int i=0; i<2; i++) {
     for(int i=0; i<2; i++) {
       updateHeartbeatWithUsage(dataNodes[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);
     }
     }
   }
   }
 
 

+ 3 - 3
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.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.TestBlockStoragePolicy;
 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.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.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -93,8 +93,8 @@ public class TestReplicationPolicyConsiderLoad {
       dnrList.add(dnr);
       dnrList.add(dnr);
       dnManager.registerDatanode(dnr);
       dnManager.registerDatanode(dnr);
       dataNodes[i].getStorageInfos()[0].setUtilizationForTesting(
       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(
       dataNodes[i].updateHeartbeat(
           BlockManagerTestUtil.getStorageReportsForDatanode(dataNodes[i]),
           BlockManagerTestUtil.getStorageReportsForDatanode(dataNodes[i]),
           0L, 0L, 0, 0, null);
           0L, 0L, 0, 0, null);

+ 23 - 23
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.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.TestBlockStoragePolicy;
 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.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NetworkTopologyWithNodeGroup;
 import org.apache.hadoop.net.NetworkTopologyWithNodeGroup;
@@ -191,8 +191,8 @@ public class TestReplicationPolicyWithNodeGroup {
   private static void setupDataNodeCapacity() {
   private static void setupDataNodeCapacity() {
     for(int i=0; i<NUM_OF_DATANODES; i++) {
     for(int i=0; i<NUM_OF_DATANODES; i++) {
       updateHeartbeatWithUsage(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
   @Test
   public void testChooseTarget1() throws Exception {
   public void testChooseTarget1() throws Exception {
     updateHeartbeatWithUsage(dataNodes[0],
     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
         0L, 0L, 4, 0); // overloaded
 
 
     DatanodeStorageInfo[] targets;
     DatanodeStorageInfo[] targets;
@@ -312,8 +312,8 @@ public class TestReplicationPolicyWithNodeGroup {
     verifyNoTwoTargetsOnSameNodeGroup(targets);
     verifyNoTwoTargetsOnSameNodeGroup(targets);
 
 
     updateHeartbeatWithUsage(dataNodes[0],
     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) {
   private void verifyNoTwoTargetsOnSameNodeGroup(DatanodeStorageInfo[] targets) {
@@ -380,8 +380,8 @@ public class TestReplicationPolicyWithNodeGroup {
   public void testChooseTarget3() throws Exception {
   public void testChooseTarget3() throws Exception {
     // make data node 0 to be not qualified to choose
     // make data node 0 to be not qualified to choose
     updateHeartbeatWithUsage(dataNodes[0],
     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
         0L, 0L, 0, 0); // no space
 
 
     DatanodeStorageInfo[] targets;
     DatanodeStorageInfo[] targets;
@@ -412,8 +412,8 @@ public class TestReplicationPolicyWithNodeGroup {
                isOnSameRack(targets[2], targets[3]));
                isOnSameRack(targets[2], targets[3]));
 
 
     updateHeartbeatWithUsage(dataNodes[0],
     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
     // make data node 0-2 to be not qualified to choose: not enough disk space
     for(int i=0; i<3; i++) {
     for(int i=0; i<3; i++) {
       updateHeartbeatWithUsage(dataNodes[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;
     DatanodeStorageInfo[] targets;
@@ -661,13 +661,13 @@ public class TestReplicationPolicyWithNodeGroup {
     }
     }
     for(int i=0; i<NUM_OF_DATANODES_BOUNDARY; i++) {
     for(int i=0; i<NUM_OF_DATANODES_BOUNDARY; i++) {
       updateHeartbeatWithUsage(dataNodes[0],
       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);
                 0L, 0L, 0L, 0, 0);
 
 
       updateHeartbeatWithUsage(dataNodesInBoundaryCase[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);
     }
     }
 
 
     DatanodeStorageInfo[] targets;
     DatanodeStorageInfo[] targets;
@@ -697,8 +697,8 @@ public class TestReplicationPolicyWithNodeGroup {
   public void testRereplicateOnBoundaryTopology() throws Exception {
   public void testRereplicateOnBoundaryTopology() throws Exception {
     for(int i=0; i<NUM_OF_DATANODES_BOUNDARY; i++) {
     for(int i=0; i<NUM_OF_DATANODES_BOUNDARY; i++) {
       updateHeartbeatWithUsage(dataNodesInBoundaryCase[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>();
     List<DatanodeStorageInfo> chosenNodes = new ArrayList<DatanodeStorageInfo>();
     chosenNodes.add(storagesInBoundaryCase[0]);
     chosenNodes.add(storagesInBoundaryCase[0]);
@@ -735,8 +735,8 @@ public class TestReplicationPolicyWithNodeGroup {
 
 
     for(int i=0; i<NUM_OF_DATANODES_MORE_TARGETS; i++) {
     for(int i=0; i<NUM_OF_DATANODES_MORE_TARGETS; i++) {
       updateHeartbeatWithUsage(dataNodesInMoreTargetsCase[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;
     DatanodeStorageInfo[] targets;
@@ -786,8 +786,8 @@ public class TestReplicationPolicyWithNodeGroup {
     //Update heartbeat
     //Update heartbeat
     for(int i=0; i<NUM_OF_DATANODES_FOR_DEPENDENCIES; i++) {
     for(int i=0; i<NUM_OF_DATANODES_FOR_DEPENDENCIES; i++) {
       updateHeartbeatWithUsage(dataNodesForDependencies[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>();
     List<DatanodeStorageInfo> chosenNodes = new ArrayList<DatanodeStorageInfo>();

Some files were not shown because too many files changed in this diff