Kaynağa Gözat

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

Haohui Mai 10 yıl önce
ebeveyn
işleme
6ae2a0d048
100 değiştirilmiş dosya ile 610 ekleme ve 671 silme
  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 static org.apache.hadoop.hdfs.protocol.HdfsConstantsClient.HA_DT_SERVICE_PREFIX;
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.HA_DT_SERVICE_PREFIX;
 
 @InterfaceAudience.Private
 public class HAUtilClient {

+ 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) {
     Matcher m = metaFilePattern.matcher(metaFile);
     return m.matches() ? Long.parseLong(m.group(2))
-        : HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP;
+        : HdfsConstants.GRANDFATHER_GENERATION_STAMP;
   }
 
   /**
@@ -100,7 +100,7 @@ public class Block implements Writable, Comparable<Block> {
   }
 
   public Block(final long blkid) {
-    this(blkid, 0, HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP);
+    this(blkid, 0, HdfsConstants.GRANDFATHER_GENERATION_STAMP);
   }
 
   public Block(Block blk) {

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

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

+ 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.NfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.net.DNS;
@@ -2031,7 +2031,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       }
 
       return new PATHCONF3Response(Nfs3Status.NFS3_OK, attrs, 0,
-          HdfsConstants.MAX_PATH_LENGTH, true, false, false, true);
+          HdfsServerConstants.MAX_PATH_LENGTH, true, false, false, true);
     } catch (IOException e) {
       LOG.warn("Exception ", e);
       int status = mapErrorStatus(e);

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

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

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

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

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

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

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

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

+ 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>
    *                       
    * The quota can have three types of values : (1) 0 or more will set 
-   * the quota to that value, (2) {@link HdfsConstants#QUOTA_DONT_SET}  implies 
-   * the quota will not be changed, and (3) {@link HdfsConstants#QUOTA_RESET} 
+   * the quota to that value, (2) {@link HdfsConstants#QUOTA_DONT_SET}  implies
+   * the quota will not be changed, and (3) {@link HdfsConstants#QUOTA_RESET}
    * implies the quota will be reset. Any other value is a runtime error.
    * 
    * @throws AccessControlException permission denied

+ 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) {
     this.dirStatus = new HdfsFileStatus(0, true, 0, 0, modification_time,
         access_time, permission, owner, group, null, localName, inodeId,
-        childrenNum, null, HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
+        childrenNum, null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
     this.snapshotNumber = snapshotNumber;
     this.snapshotQuota = snapshotQuota;
     this.parentFullPath = parentFullPath;

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

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

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

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

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

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

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

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

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

+ 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.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.Node;
@@ -792,7 +792,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       }
     }
     
-    final long requiredSize = blockSize * HdfsConstants.MIN_BLOCKS_FOR_WRITE;
+    final long requiredSize = blockSize * HdfsServerConstants.MIN_BLOCKS_FOR_WRITE;
     final long scheduledSize = blockSize * node.getBlocksScheduled(storage.getStorageType());
     final long remaining = node.getRemaining(storage.getStorageType());
     if (requiredSize > remaining - scheduledSize) {

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

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

+ 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 org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.LayoutFeature;
@@ -222,8 +221,8 @@ public class StorageInfo {
   }
 
   public int getServiceLayoutVersion() {
-    return storageType == NodeType.DATA_NODE ? HdfsConstants.DATANODE_LAYOUT_VERSION
-        : HdfsConstants.NAMENODE_LAYOUT_VERSION;
+    return storageType == NodeType.DATA_NODE ? HdfsServerConstants.DATANODE_LAYOUT_VERSION
+        : HdfsServerConstants.NAMENODE_LAYOUT_VERSION;
   }
 
   public Map<Integer, SortedSet<LayoutFeature>> getServiceLayoutFeatureMap() {

+ 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.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.DataChecksum;
 
@@ -85,7 +85,7 @@ public class BlockMetadataHeader {
     DataInputStream in = null;
     try {
       in = new DataInputStream(new BufferedInputStream(
-        new FileInputStream(metaFile), HdfsConstants.IO_FILE_BUFFER_SIZE));
+        new FileInputStream(metaFile), HdfsServerConstants.IO_FILE_BUFFER_SIZE));
       return readDataChecksum(in, metaFile);
     } finally {
       IOUtils.closeStream(in);

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

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

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

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

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

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

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

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

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

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

+ 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) {
       final byte policyId = status.getStoragePolicy();
       // currently we ignore files with unspecified storage policy
-      if (policyId == HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED) {
+      if (policyId == HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED) {
         return false;
       }
       final BlockStoragePolicy policy = blockStoragePolicies[policyId];

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

+ 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.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -511,7 +510,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    */
   public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps) {
     final byte storagePolicyId = isSymlink() ?
-        HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED : getStoragePolicyID();
+        HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED : getStoragePolicyID();
     return computeQuotaUsage(bsps, storagePolicyId,
         new QuotaCounts.Builder().build(), true, Snapshot.CURRENT_STATE_ID);
   }
@@ -555,7 +554,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   public final QuotaCounts computeQuotaUsage(
     BlockStoragePolicySuite bsps, QuotaCounts counts, boolean useCache) {
     final byte storagePolicyId = isSymlink() ?
-        HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED : getStoragePolicyID();
+        HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED : getStoragePolicyID();
     return computeQuotaUsage(bsps, storagePolicyId, counts,
         useCache, Snapshot.CURRENT_STATE_ID);
   }
@@ -712,7 +711,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
 
   /**
    * @return the storage policy directly specified on the INode. Return
-   * {@link HdfsConstantsClient#BLOCK_STORAGE_POLICY_ID_UNSPECIFIED} if no policy has
+   * {@link HdfsConstants#BLOCK_STORAGE_POLICY_ID_UNSPECIFIED} if no policy has
    * been specified.
    */
   public abstract byte getLocalStoragePolicyID();
@@ -721,13 +720,13 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    * Get the storage policy ID while computing quota usage
    * @param parentStoragePolicyId the storage policy ID of the parent directory
    * @return the storage policy ID of this INode. Note that for an
-   * {@link INodeSymlink} we return {@link HdfsConstantsClient#BLOCK_STORAGE_POLICY_ID_UNSPECIFIED}
+   * {@link INodeSymlink} we return {@link HdfsConstants#BLOCK_STORAGE_POLICY_ID_UNSPECIFIED}
    * instead of throwing Exception
    */
   public byte getStoragePolicyIDForQuota(byte parentStoragePolicyId) {
     byte localId = isSymlink() ?
-        HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED : getLocalStoragePolicyID();
-    return localId != HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED ?
+        HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED : getLocalStoragePolicyID();
+    return localId != HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED ?
         localId : parentStoragePolicyId;
   }
 

+ 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.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.

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

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

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

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

+ 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.fs.Path;
 import org.apache.hadoop.fs.UnresolvedLinkException;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.util.Daemon;
@@ -72,8 +71,8 @@ public class LeaseManager {
 
   private final FSNamesystem fsnamesystem;
 
-  private long softLimit = HdfsConstants.LEASE_SOFTLIMIT_PERIOD;
-  private long hardLimit = HdfsConstants.LEASE_HARDLIMIT_PERIOD;
+  private long softLimit = HdfsServerConstants.LEASE_SOFTLIMIT_PERIOD;
+  private long hardLimit = HdfsServerConstants.LEASE_HARDLIMIT_PERIOD;
 
   //
   // Used for handling lock-leases

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

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

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

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

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

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

+ 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.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HAUtil;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.EditLogInputException;
 import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
@@ -75,12 +75,12 @@ public class EditLogTailer {
   /**
    * The last transaction ID at which an edit log roll was initiated.
    */
-  private long lastRollTriggerTxId = HdfsConstants.INVALID_TXID;
+  private long lastRollTriggerTxId = HdfsServerConstants.INVALID_TXID;
   
   /**
    * The highest transaction ID loaded by the Standby.
    */
-  private long lastLoadedTxnId = HdfsConstants.INVALID_TXID;
+  private long lastLoadedTxnId = HdfsServerConstants.INVALID_TXID;
 
   /**
    * The last time we successfully loaded a non-zero number of edits from the

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

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

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

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

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

+ 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.InterfaceStability;
 
-import org.apache.hadoop.hdfs.tools.offlineEditsViewer.OfflineEditsViewer;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream;
 
 import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
@@ -49,8 +48,8 @@ interface OfflineEditsLoader {
         OfflineEditsLoader loader = null;
         try {
           file = new File(inputFileName);
-          elis = new EditLogFileInputStream(file, HdfsConstants.INVALID_TXID,
-              HdfsConstants.INVALID_TXID, false);
+          elis = new EditLogFileInputStream(file, HdfsServerConstants.INVALID_TXID,
+              HdfsServerConstants.INVALID_TXID, false);
           loader = new OfflineEditsBinaryLoader(visitor, elis, flags);
         } finally {
           if ((loader == null) && (elis != null)) {

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

Bu fark içinde çok fazla dosya değişikliği olduğu için bazı dosyalar gösterilmiyor