Jelajahi Sumber

HDFS-8165. Move GRANDFATHER_GENERATION_STAMP and GRANDFATER_INODE_ID to hdfs-client. Contributed by Haohui Mai.

Haohui Mai 10 tahun lalu
induk
melakukan
76e7264e8d
25 mengubah file dengan 106 tambahan dan 75 penghapusan
  1. 34 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstantsClient.java
  2. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  3. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/Block.java
  4. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  5. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  6. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
  7. 0 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/GenerationStamp.java
  8. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
  9. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
  10. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java
  11. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  12. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
  13. 6 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  14. 2 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeId.java
  15. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
  16. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
  17. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
  18. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
  19. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
  20. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
  21. 6 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java
  22. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java
  23. 8 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
  24. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java
  25. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java

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

@@ -0,0 +1,34 @@
+/**
+ * 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;
+}

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -445,6 +445,9 @@ Release 2.8.0 - UNRELEASED
     dfs.client.mmap.* and dfs.client.hedged.read.* conf from DFSConfigKeys
     to HdfsClientConfigKeys.  (szetszwo)
 
+    HDFS-8165. Move GRANDFATHER_GENERATION_STAMP and GRANDFATER_INODE_ID to
+    hdfs-client. (wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

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

@@ -23,7 +23,6 @@ import java.util.regex.Pattern;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.io.*;
 
 /**************************************************
@@ -79,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))
-        : GenerationStamp.GRANDFATHER_GENERATION_STAMP;
+        : HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP;
   }
 
   /**
@@ -101,7 +100,7 @@ public class Block implements Writable, Comparable<Block> {
   }
 
   public Block(final long blkid) {
-    this(blkid, 0, GenerationStamp.GRANDFATHER_GENERATION_STAMP);
+    this(blkid, 0, HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP);
   }
 
   public Block(Block blk) {

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -36,6 +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.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -210,7 +211,6 @@ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.SetXAttrRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.SetXAttrResponseProto;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto;
@@ -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() : INodeId.GRANDFATHER_INODE_ID);
+          req.hasFileId() ? req.getFileId() : HdfsConstantsClient.GRANDFATHER_INODE_ID);
       return CompleteResponseProto.newBuilder().setResult(result).build();
     } catch (IOException e) {
       throw new ServiceException(e);

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -76,6 +76,7 @@ import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 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;
@@ -111,7 +112,6 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rollin
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SafeModeActionProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmIdProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmSlotProto;
-import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockIdCommandProto;
@@ -190,7 +190,6 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
-import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockIdCommand;
@@ -1438,7 +1437,7 @@ public class PBHelper {
         fs.getFileType().equals(FileType.IS_SYMLINK) ? 
             fs.getSymlink().toByteArray() : null,
         fs.getPath().toByteArray(),
-        fs.hasFileId()? fs.getFileId(): INodeId.GRANDFATHER_INODE_ID,
+        fs.hasFileId()? fs.getFileId(): HdfsConstantsClient.GRANDFATHER_INODE_ID,
         fs.hasLocations() ? PBHelper.convert(fs.getLocations()) : null,
         fs.hasChildrenNum() ? fs.getChildrenNum() : -1,
         fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) : null,

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java

@@ -21,6 +21,7 @@ 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 java.io.IOException;
@@ -55,7 +56,7 @@ public class BlockIdManager {
   private final SequentialBlockIdGenerator blockIdGenerator;
 
   public BlockIdManager(BlockManager blockManager) {
-    this.generationStampV1Limit = GenerationStamp.GRANDFATHER_GENERATION_STAMP;
+    this.generationStampV1Limit = HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP;
     this.blockIdGenerator = new SequentialBlockIdGenerator(blockManager);
   }
 
@@ -82,7 +83,7 @@ public class BlockIdManager {
    * @param stamp set generation stamp limit to this value
    */
   public void setGenerationStampV1Limit(long stamp) {
-    Preconditions.checkState(generationStampV1Limit == GenerationStamp
+    Preconditions.checkState(generationStampV1Limit == HdfsConstantsClient
       .GRANDFATHER_GENERATION_STAMP);
     generationStampV1Limit = stamp;
   }
@@ -203,6 +204,6 @@ public class BlockIdManager {
     generationStampV2.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
     getBlockIdGenerator().setCurrentValue(SequentialBlockIdGenerator
       .LAST_RESERVED_BLOCK_ID);
-    generationStampV1Limit = GenerationStamp.GRANDFATHER_GENERATION_STAMP;
+    generationStampV1Limit = HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP;
   }
 }

+ 0 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/GenerationStamp.java

@@ -30,12 +30,6 @@ public class GenerationStamp extends SequentialNumber {
    */
   public static final long LAST_RESERVED_STAMP = 1000L;
 
-  /**
-   * Generation stamp of blocks that pre-date the introduction
-   * of a generation stamp.
-   */
-  public static final long GRANDFATHER_GENERATION_STAMP = 0;
-
   /**
    * Create a new instance, initialized to {@link #LAST_RESERVED_STAMP}.
    */

+ 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.server.common.GenerationStamp;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 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()) : 
-            GenerationStamp.GRANDFATHER_GENERATION_STAMP;
+            HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP;
     }
   }
 

+ 3 - 3
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.GenerationStamp;
 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;
@@ -2101,7 +2101,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
 
       final long diskGS = diskMetaFile != null && diskMetaFile.exists() ?
           Block.getGenerationStamp(diskMetaFile.getName()) :
-            GenerationStamp.GRANDFATHER_GENERATION_STAMP;
+            HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP;
 
       if (diskFile == null || !diskFile.exists()) {
         if (memBlockInfo == null) {
@@ -2202,7 +2202,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
-              : GenerationStamp.GRANDFATHER_GENERATION_STAMP;
+              : HdfsConstantsClient.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.server.common.GenerationStamp;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 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 GenerationStamp.GRANDFATHER_GENERATION_STAMP;
+    return HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP;
   }
 
   /** Find the corresponding meta data file from a given block file */

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java

@@ -34,6 +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.XAttrSetFlag;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -301,7 +302,7 @@ public class FSEditLogLoader {
       long lastInodeId) throws IOException {
     long inodeId = inodeIdFromOp;
 
-    if (inodeId == INodeId.GRANDFATHER_INODE_ID) {
+    if (inodeId == HdfsConstantsClient.GRANDFATHER_INODE_ID) {
       if (NameNodeLayoutVersion.supports(
           LayoutVersion.Feature.ADD_INODE_ID, logVersion)) {
         throw new IOException("The layout version " + logVersion
@@ -321,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 = INodeId.GRANDFATHER_INODE_ID;
+    long inodeId = HdfsConstantsClient.GRANDFATHER_INODE_ID;
     if (LOG.isTraceEnabled()) {
       LOG.trace("replaying edit log: " + op);
     }

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java

@@ -101,6 +101,7 @@ 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;
@@ -577,7 +578,7 @@ public abstract class FSEditLogOp {
         this.inodeId = in.readLong();
       } else {
         // The inodeId should be updated when this editLogOp is applied
-        this.inodeId = INodeId.GRANDFATHER_INODE_ID;
+        this.inodeId = HdfsConstantsClient.GRANDFATHER_INODE_ID;
       }
       if ((-17 < logVersion && length != 4) ||
           (logVersion <= -17 && length != 5 && !NameNodeLayoutVersion.supports(
@@ -1652,7 +1653,7 @@ public abstract class FSEditLogOp {
         this.inodeId = FSImageSerialization.readLong(in);
       } else {
         // This id should be updated when this editLogOp is applied
-        this.inodeId = INodeId.GRANDFATHER_INODE_ID;
+        this.inodeId = HdfsConstantsClient.GRANDFATHER_INODE_ID;
       }
       this.path = FSImageSerialization.readString(in);
       if (NameNodeLayoutVersion.supports(
@@ -2545,7 +2546,7 @@ public abstract class FSEditLogOp {
         this.inodeId = FSImageSerialization.readLong(in);
       } else {
         // This id should be updated when the editLogOp is applied
-        this.inodeId = INodeId.GRANDFATHER_INODE_ID;
+        this.inodeId = HdfsConstantsClient.GRANDFATHER_INODE_ID;
       }
       this.path = FSImageSerialization.readString(in);
       this.value = FSImageSerialization.readString(in);

+ 6 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -183,6 +183,7 @@ 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;
@@ -3200,7 +3201,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     Block previousBlock = ExtendedBlock.getLocalBlock(previous);
     final INode inode;
     final INodesInPath iip;
-    if (fileId == INodeId.GRANDFATHER_INODE_ID) {
+    if (fileId == HdfsConstantsClient.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.
@@ -3320,7 +3321,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
       //check lease
       final INode inode;
-      if (fileId == INodeId.GRANDFATHER_INODE_ID) {
+      if (fileId == HdfsConstantsClient.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.
@@ -3376,7 +3377,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
       final INode inode;
       final INodesInPath iip;
-      if (fileId == INodeId.GRANDFATHER_INODE_ID) {
+      if (fileId == HdfsConstantsClient.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.
@@ -3495,7 +3496,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final INodesInPath iip;
     INode inode = null;
     try {
-      if (fileId == INodeId.GRANDFATHER_INODE_ID) {
+      if (fileId == HdfsConstantsClient.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.
@@ -3945,7 +3946,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkNameNodeSafeMode("Cannot fsync file " + src);
       src = dir.resolvePath(pc, src, pathComponents);
       final INode inode;
-      if (fileId == INodeId.GRANDFATHER_INODE_ID) {
+      if (fileId == HdfsConstantsClient.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.

+ 2 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeId.java

@@ -20,6 +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.util.SequentialNumber;
 
 /**
@@ -37,19 +38,13 @@ public class INodeId extends SequentialNumber {
   public static final long LAST_RESERVED_ID = 2 << 14 - 1;
   public static final long ROOT_INODE_ID = LAST_RESERVED_ID + 1;
 
-  /**
-   * 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;
-
   /**
    * To check if the request id is the same as saved id. Don't check fileId
    * with GRANDFATHER_INODE_ID for backward compatibility.
    */
   public static void checkId(long requestId, INode inode)
       throws FileNotFoundException {
-    if (requestId != GRANDFATHER_INODE_ID && requestId != inode.getId()) {
+    if (requestId != HdfsConstantsClient.GRANDFATHER_INODE_ID && requestId != inode.getId()) {
       throw new FileNotFoundException(
           "ID mismatch. Request id and saved id: " + requestId + " , "
               + inode.getId() + " for file " + inode.getFullPathName());

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java

@@ -28,11 +28,11 @@ 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.LayoutFlags;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
-import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
 import org.apache.hadoop.hdfs.tools.offlineImageViewer.ImageVisitor.ImageElement;
 import org.apache.hadoop.io.Text;
@@ -687,7 +687,7 @@ class ImageLoaderCurrent implements ImageLoader {
     final String pathName = readINodePath(in, parentName);
     v.visit(ImageElement.INODE_PATH, pathName);
 
-    long inodeId = INodeId.GRANDFATHER_INODE_ID;
+    long inodeId = HdfsConstantsClient.GRANDFATHER_INODE_ID;
     if (supportInodeId) {
       inodeId = in.readLong();
       v.visit(ImageElement.INODE_ID, inodeId);

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java

@@ -33,13 +33,13 @@ import org.apache.hadoop.hdfs.DFSUtil;
 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.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
-import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
@@ -124,7 +124,7 @@ 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() : INodeId.GRANDFATHER_INODE_ID;
+        ((Number) m.get("fileId")).longValue() : HdfsConstantsClient.GRANDFATHER_INODE_ID;
     final int childrenNum = getInt(m, "childrenNum", -1);
     final byte storagePolicy = m.containsKey("storagePolicy") ?
         (byte) ((Number) m.get("storagePolicy")).longValue() :

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

@@ -70,6 +70,7 @@ 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;
@@ -78,7 +79,6 @@ import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -537,7 +537,7 @@ public class TestFileCreation {
 
       // add one block to the file
       LocatedBlock location = client.getNamenode().addBlock(file1.toString(),
-          client.clientName, null, null, INodeId.GRANDFATHER_INODE_ID, null);
+          client.clientName, null, null, HdfsConstantsClient.GRANDFATHER_INODE_ID, null);
       System.out.println("testFileCreationError2: "
           + "Added block " + location.getBlock());
 
@@ -588,7 +588,7 @@ public class TestFileCreation {
       createFile(dfs, f, 3);
       try {
         cluster.getNameNodeRpc().addBlock(f.toString(), client.clientName,
-            null, null, INodeId.GRANDFATHER_INODE_ID, null);
+            null, null, HdfsConstantsClient.GRANDFATHER_INODE_ID, null);
         fail();
       } catch(IOException ioe) {
         FileSystem.LOG.info("GOOD!", ioe);

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java

@@ -35,17 +35,16 @@ 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.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;
-import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.util.Time;
 import org.junit.Test;
 
 /**
@@ -279,7 +278,7 @@ public class TestGetBlocks {
 
     for (int i = 0; i < blkids.length; i++) {
       Block b = new Block(blkids[i], 0,
-          GenerationStamp.GRANDFATHER_GENERATION_STAMP);
+          HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP);
       Long v = map.get(b);
       System.out.println(b + " => " + v);
       assertEquals(blkids[i], v.longValue());

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java

@@ -47,8 +47,8 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
@@ -405,7 +405,7 @@ public class TestDirectoryScanner {
       // Test2: block metafile is missing
       long blockId = deleteMetaFile();
       scan(totalBlocks, 1, 1, 0, 0, 1);
-      verifyGenStamp(blockId, GenerationStamp.GRANDFATHER_GENERATION_STAMP);
+      verifyGenStamp(blockId, HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP);
       scan(totalBlocks, 0, 0, 0, 0, 0);
 
       // Test3: block file is missing
@@ -420,7 +420,7 @@ public class TestDirectoryScanner {
       blockId = createBlockFile();
       totalBlocks++;
       scan(totalBlocks, 1, 1, 0, 1, 0);
-      verifyAddition(blockId, GenerationStamp.GRANDFATHER_GENERATION_STAMP, 0);
+      verifyAddition(blockId, HdfsConstantsClient.GRANDFATHER_GENERATION_STAMP, 0);
       scan(totalBlocks, 0, 0, 0, 0, 0);
 
       // Test5: A metafile exists for which there is no block file and

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java

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

+ 6 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java

@@ -30,6 +30,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
@@ -88,14 +89,14 @@ public class TestAddBlockRetry {
     LOG.info("Starting first addBlock for " + src);
     LocatedBlock[] onRetryBlock = new LocatedBlock[1];
     DatanodeStorageInfo targets[] = ns.getNewBlockTargets(
-        src, INodeId.GRANDFATHER_INODE_ID, "clientName",
+        src, HdfsConstantsClient.GRANDFATHER_INODE_ID, "clientName",
         null, null, null, onRetryBlock);
     assertNotNull("Targets must be generated", targets);
 
     // run second addBlock()
     LOG.info("Starting second addBlock for " + src);
     nn.addBlock(src, "clientName", null, null,
-        INodeId.GRANDFATHER_INODE_ID, null);
+        HdfsConstantsClient.GRANDFATHER_INODE_ID, null);
     assertTrue("Penultimate block must be complete",
         checkFileProgress(src, false));
     LocatedBlocks lbs = nn.getBlockLocations(src, 0, Long.MAX_VALUE);
@@ -105,7 +106,7 @@ public class TestAddBlockRetry {
 
     // continue first addBlock()
     LocatedBlock newBlock = ns.storeAllocatedBlock(
-        src, INodeId.GRANDFATHER_INODE_ID, "clientName", null, targets);
+        src, HdfsConstantsClient.GRANDFATHER_INODE_ID, "clientName", null, targets);
     assertEquals("Blocks are not equal", lb2.getBlock(), newBlock.getBlock());
 
     // check locations
@@ -143,14 +144,14 @@ public class TestAddBlockRetry {
     // start first addBlock()
     LOG.info("Starting first addBlock for " + src);
     LocatedBlock lb1 = nameNodeRpc.addBlock(src, "clientName", null, null,
-        INodeId.GRANDFATHER_INODE_ID, null);
+        HdfsConstantsClient.GRANDFATHER_INODE_ID, null);
     assertTrue("Block locations should be present",
         lb1.getLocations().length > 0);
 
     cluster.restartNameNode();
     nameNodeRpc = cluster.getNameNodeRpc();
     LocatedBlock lb2 = nameNodeRpc.addBlock(src, "clientName", null, null,
-        INodeId.GRANDFATHER_INODE_ID, null);
+        HdfsConstantsClient.GRANDFATHER_INODE_ID, null);
     assertEquals("Blocks are not equal", lb1.getBlock(), lb2.getBlock());
     assertTrue("Wrong locations with retry", lb2.getLocations().length > 0);
   }

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java

@@ -47,6 +47,7 @@ import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -429,7 +430,7 @@ public class TestFSPermissionChecker {
     PermissionStatus permStatus = PermissionStatus.createImmutable(owner, group,
       FsPermission.createImmutable(perm));
     INodeDirectory inodeDirectory = new INodeDirectory(
-      INodeId.GRANDFATHER_INODE_ID, name.getBytes("UTF-8"), permStatus, 0L);
+      HdfsConstantsClient.GRANDFATHER_INODE_ID, name.getBytes("UTF-8"), permStatus, 0L);
     parent.addChild(inodeDirectory);
     return inodeDirectory;
   }
@@ -438,7 +439,7 @@ public class TestFSPermissionChecker {
       String owner, String group, short perm) throws IOException {
     PermissionStatus permStatus = PermissionStatus.createImmutable(owner, group,
       FsPermission.createImmutable(perm));
-    INodeFile inodeFile = new INodeFile(INodeId.GRANDFATHER_INODE_ID,
+    INodeFile inodeFile = new INodeFile(HdfsConstantsClient.GRANDFATHER_INODE_ID,
       name.getBytes("UTF-8"), permStatus, 0L, 0L, null, REPLICATION,
       PREFERRED_BLOCK_SIZE);
     parent.addChild(inodeFile);

+ 8 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java

@@ -55,6 +55,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -81,12 +82,12 @@ public class TestINodeFile {
   private long preferredBlockSize = 1024;
 
   INodeFile createINodeFile(short replication, long preferredBlockSize) {
-    return new INodeFile(INodeId.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
+    return new INodeFile(HdfsConstantsClient.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
         null, replication, preferredBlockSize);
   }
 
   private static INodeFile createINodeFile(byte storagePolicyID) {
-    return new INodeFile(INodeId.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
+    return new INodeFile(HdfsConstantsClient.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
         null, (short)3, 1024L, storagePolicyID);
   }
 
@@ -189,9 +190,9 @@ public class TestINodeFile {
     INodeFile inf = createINodeFile(replication, preferredBlockSize);
     inf.setLocalName(DFSUtil.string2Bytes("f"));
 
-    INodeDirectory root = new INodeDirectory(INodeId.GRANDFATHER_INODE_ID,
+    INodeDirectory root = new INodeDirectory(HdfsConstantsClient.GRANDFATHER_INODE_ID,
         INodeDirectory.ROOT_NAME, perm, 0L);
-    INodeDirectory dir = new INodeDirectory(INodeId.GRANDFATHER_INODE_ID,
+    INodeDirectory dir = new INodeDirectory(HdfsConstantsClient.GRANDFATHER_INODE_ID,
         DFSUtil.string2Bytes("d"), perm, 0L);
 
     assertEquals("f", inf.getFullPathName());
@@ -340,7 +341,7 @@ public class TestINodeFile {
 
     {//cast from INodeFileUnderConstruction
       final INode from = new INodeFile(
-          INodeId.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, null, replication,
+          HdfsConstantsClient.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, null, replication,
           1024L);
       from.asFile().toUnderConstruction("client", "machine");
     
@@ -358,7 +359,7 @@ public class TestINodeFile {
     }
 
     {//cast from INodeDirectory
-      final INode from = new INodeDirectory(INodeId.GRANDFATHER_INODE_ID, null,
+      final INode from = new INodeDirectory(HdfsConstantsClient.GRANDFATHER_INODE_ID, null,
           perm, 0L);
 
       //cast to INodeFile, should fail
@@ -1103,7 +1104,7 @@ public class TestINodeFile {
   @Test
   public void testFileUnderConstruction() {
     replication = 3;
-    final INodeFile file = new INodeFile(INodeId.GRANDFATHER_INODE_ID, null,
+    final INodeFile file = new INodeFile(HdfsConstantsClient.GRANDFATHER_INODE_ID, null,
         perm, 0L, 0L, null, replication, 1024L);
     assertFalse(file.isUnderConstruction());
 

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java

@@ -28,7 +28,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
-import org.apache.hadoop.hdfs.server.namenode.INodeId;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
@@ -196,7 +196,7 @@ public class TestOpenFilesWithSnapshot {
     String clientName = fs.getClient().getClientName();
     // create one empty block
     nameNodeRpc.addBlock(fileWithEmptyBlock.toString(), clientName, null, null,
-        INodeId.GRANDFATHER_INODE_ID, null);
+        HdfsConstantsClient.GRANDFATHER_INODE_ID, null);
     fs.createSnapshot(path, "s2");
 
     fs.rename(new Path("/test/test"), new Path("/test/test-renamed"));

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java

@@ -39,8 +39,8 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.util.Time;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.map.ObjectReader;
@@ -65,7 +65,7 @@ public class TestJsonUtil {
     final HdfsFileStatus status = new HdfsFileStatus(1001L, false, 3, 1L << 26,
         now, now + 10, new FsPermission((short) 0644), "user", "group",
         DFSUtil.string2Bytes("bar"), DFSUtil.string2Bytes("foo"),
-        INodeId.GRANDFATHER_INODE_ID, 0, null, (byte) 0);
+        HdfsConstantsClient.GRANDFATHER_INODE_ID, 0, null, (byte) 0);
     final FileStatus fstatus = toFileStatus(status, parent);
     System.out.println("status  = " + status);
     System.out.println("fstatus = " + fstatus);