Browse Source

HDFS-9677. Rename generationStampV1/generationStampV2 to legacyGenerationStamp/generationStamp. Contributed by Mingliang Liu.

Jing Zhao 9 years ago
parent
commit
ec25c7f9c7
14 changed files with 85 additions and 82 deletions
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 42 41
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
  3. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfLegacyGenerationStampsException.java
  4. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
  5. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  6. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  7. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  8. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
  9. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  10. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
  11. 9 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockId.java
  12. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
  13. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
  14. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java

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

@@ -959,6 +959,9 @@ Release 2.9.0 - UNRELEASED
     HDFS-9541. Add hdfsStreamBuilder API to libhdfs to support defaultBlockSizes
     HDFS-9541. Add hdfsStreamBuilder API to libhdfs to support defaultBlockSizes
     greater than 2 GB. (cmccabe via zhz)
     greater than 2 GB. (cmccabe via zhz)
 
 
+    HDFS-9677. Rename generationStampV1/generationStampV2 to
+    legacyGenerationStamp/generationStamp. (Mingliang Liu via jing9)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES

+ 42 - 41
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java

@@ -36,11 +36,11 @@ public class BlockIdManager {
    * The global generation stamp for legacy blocks with randomly
    * The global generation stamp for legacy blocks with randomly
    * generated block IDs.
    * generated block IDs.
    */
    */
-  private final GenerationStamp generationStampV1 = new GenerationStamp();
+  private final GenerationStamp legacyGenerationStamp = new GenerationStamp();
   /**
   /**
    * The global generation stamp for this file system.
    * The global generation stamp for this file system.
    */
    */
-  private final GenerationStamp generationStampV2 = new GenerationStamp();
+  private final GenerationStamp generationStamp = new GenerationStamp();
   /**
   /**
    * The value of the generation stamp when the first switch to sequential
    * The value of the generation stamp when the first switch to sequential
    * block IDs was made. Blocks with generation stamps below this value
    * block IDs was made. Blocks with generation stamps below this value
@@ -49,7 +49,7 @@ public class BlockIdManager {
    * (or initialized as an offset from the V1 (legacy) generation stamp on
    * (or initialized as an offset from the V1 (legacy) generation stamp on
    * upgrade).
    * upgrade).
    */
    */
-  private long generationStampV1Limit;
+  private long legacyGenerationStampLimit;
   /**
   /**
    * The global block ID space for this file system.
    * The global block ID space for this file system.
    */
    */
@@ -57,7 +57,8 @@ public class BlockIdManager {
   private final SequentialBlockGroupIdGenerator blockGroupIdGenerator;
   private final SequentialBlockGroupIdGenerator blockGroupIdGenerator;
 
 
   public BlockIdManager(BlockManager blockManager) {
   public BlockIdManager(BlockManager blockManager) {
-    this.generationStampV1Limit = HdfsConstants.GRANDFATHER_GENERATION_STAMP;
+    this.legacyGenerationStampLimit =
+        HdfsConstants.GRANDFATHER_GENERATION_STAMP;
     this.blockIdGenerator = new SequentialBlockIdGenerator(blockManager);
     this.blockIdGenerator = new SequentialBlockIdGenerator(blockManager);
     this.blockGroupIdGenerator = new SequentialBlockGroupIdGenerator(blockManager);
     this.blockGroupIdGenerator = new SequentialBlockGroupIdGenerator(blockManager);
   }
   }
@@ -68,14 +69,14 @@ public class BlockIdManager {
    * Should be invoked only during the first upgrade to
    * Should be invoked only during the first upgrade to
    * sequential block IDs.
    * sequential block IDs.
    */
    */
-  public long upgradeGenerationStampToV2() {
-    Preconditions.checkState(generationStampV2.getCurrentValue() ==
+  public long upgradeLegacyGenerationStamp() {
+    Preconditions.checkState(generationStamp.getCurrentValue() ==
       GenerationStamp.LAST_RESERVED_STAMP);
       GenerationStamp.LAST_RESERVED_STAMP);
-    generationStampV2.skipTo(generationStampV1.getCurrentValue() +
-      HdfsServerConstants.RESERVED_GENERATION_STAMPS_V1);
+    generationStamp.skipTo(legacyGenerationStamp.getCurrentValue() +
+      HdfsServerConstants.RESERVED_LEGACY_GENERATION_STAMPS);
 
 
-    generationStampV1Limit = generationStampV2.getCurrentValue();
-    return generationStampV2.getCurrentValue();
+    legacyGenerationStampLimit = generationStamp.getCurrentValue();
+    return generationStamp.getCurrentValue();
   }
   }
 
 
   /**
   /**
@@ -84,10 +85,10 @@ public class BlockIdManager {
    *
    *
    * @param stamp set generation stamp limit to this value
    * @param stamp set generation stamp limit to this value
    */
    */
-  public void setGenerationStampV1Limit(long stamp) {
-    Preconditions.checkState(generationStampV1Limit == HdfsConstants
-      .GRANDFATHER_GENERATION_STAMP);
-    generationStampV1Limit = stamp;
+  public void setLegacyGenerationStampLimit(long stamp) {
+    Preconditions.checkState(legacyGenerationStampLimit ==
+        HdfsConstants.GRANDFATHER_GENERATION_STAMP);
+    legacyGenerationStampLimit = stamp;
   }
   }
 
 
   /**
   /**
@@ -95,7 +96,7 @@ public class BlockIdManager {
    * and random block IDs.
    * and random block IDs.
    */
    */
   public long getGenerationStampAtblockIdSwitch() {
   public long getGenerationStampAtblockIdSwitch() {
-    return generationStampV1Limit;
+    return legacyGenerationStampLimit;
   }
   }
 
 
   @VisibleForTesting
   @VisibleForTesting
@@ -138,58 +139,58 @@ public class BlockIdManager {
   /**
   /**
    * Sets the current generation stamp for legacy blocks
    * Sets the current generation stamp for legacy blocks
    */
    */
-  public void setGenerationStampV1(long stamp) {
-    generationStampV1.setCurrentValue(stamp);
+  public void setLegacyGenerationStamp(long stamp) {
+    legacyGenerationStamp.setCurrentValue(stamp);
   }
   }
 
 
   /**
   /**
    * Gets the current generation stamp for legacy blocks
    * Gets the current generation stamp for legacy blocks
    */
    */
-  public long getGenerationStampV1() {
-    return generationStampV1.getCurrentValue();
+  public long getLegacyGenerationStamp() {
+    return legacyGenerationStamp.getCurrentValue();
   }
   }
 
 
   /**
   /**
    * Gets the current generation stamp for this filesystem
    * Gets the current generation stamp for this filesystem
    */
    */
-  public void setGenerationStampV2(long stamp) {
-    generationStampV2.setCurrentValue(stamp);
+  public void setGenerationStamp(long stamp) {
+    generationStamp.setCurrentValue(stamp);
   }
   }
 
 
-  public long getGenerationStampV2() {
-    return generationStampV2.getCurrentValue();
+  public long getGenerationStamp() {
+    return generationStamp.getCurrentValue();
   }
   }
 
 
   /**
   /**
    * Increments, logs and then returns the stamp
    * Increments, logs and then returns the stamp
    */
    */
   long nextGenerationStamp(boolean legacyBlock) throws IOException {
   long nextGenerationStamp(boolean legacyBlock) throws IOException {
-    return legacyBlock ? getNextGenerationStampV1() :
-      getNextGenerationStampV2();
+    return legacyBlock ? getNextLegacyGenerationStamp() :
+        getNextGenerationStamp();
   }
   }
 
 
   @VisibleForTesting
   @VisibleForTesting
-  long getNextGenerationStampV1() throws IOException {
-    long genStampV1 = generationStampV1.nextValue();
+  long getNextLegacyGenerationStamp() throws IOException {
+    long legacyGenStamp = legacyGenerationStamp.nextValue();
 
 
-    if (genStampV1 >= generationStampV1Limit) {
+    if (legacyGenStamp >= legacyGenerationStampLimit) {
       // We ran out of generation stamps for legacy blocks. In practice, it
       // We ran out of generation stamps for legacy blocks. In practice, it
-      // is extremely unlikely as we reserved 1T v1 generation stamps. The
+      // is extremely unlikely as we reserved 1T legacy generation stamps. The
       // result is that we can no longer append to the legacy blocks that
       // result is that we can no longer append to the legacy blocks that
       // were created before the upgrade to sequential block IDs.
       // were created before the upgrade to sequential block IDs.
-      throw new OutOfV1GenerationStampsException();
+      throw new OutOfLegacyGenerationStampsException();
     }
     }
 
 
-    return genStampV1;
+    return legacyGenStamp;
   }
   }
 
 
   @VisibleForTesting
   @VisibleForTesting
-  long getNextGenerationStampV2() {
-    return generationStampV2.nextValue();
+  long getNextGenerationStamp() {
+    return generationStamp.nextValue();
   }
   }
 
 
-  public long getGenerationStampV1Limit() {
-    return generationStampV1Limit;
+  public long getLegacyGenerationStampLimit() {
+    return legacyGenerationStampLimit;
   }
   }
 
 
   /**
   /**
@@ -200,7 +201,7 @@ public class BlockIdManager {
    * @return true if the block ID was randomly generated, false otherwise.
    * @return true if the block ID was randomly generated, false otherwise.
    */
    */
   boolean isLegacyBlock(Block block) {
   boolean isLegacyBlock(Block block) {
-    return block.getGenerationStamp() < getGenerationStampV1Limit();
+    return block.getGenerationStamp() < getLegacyGenerationStampLimit();
   }
   }
 
 
   /**
   /**
@@ -213,18 +214,18 @@ public class BlockIdManager {
 
 
   boolean isGenStampInFuture(Block block) {
   boolean isGenStampInFuture(Block block) {
     if (isLegacyBlock(block)) {
     if (isLegacyBlock(block)) {
-      return block.getGenerationStamp() > getGenerationStampV1();
+      return block.getGenerationStamp() > getLegacyGenerationStamp();
     } else {
     } else {
-      return block.getGenerationStamp() > getGenerationStampV2();
+      return block.getGenerationStamp() > getGenerationStamp();
     }
     }
   }
   }
 
 
   void clear() {
   void clear() {
-    generationStampV1.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
-    generationStampV2.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
+    legacyGenerationStamp.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
+    generationStamp.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
     getBlockIdGenerator().setCurrentValue(SequentialBlockIdGenerator
     getBlockIdGenerator().setCurrentValue(SequentialBlockIdGenerator
       .LAST_RESERVED_BLOCK_ID);
       .LAST_RESERVED_BLOCK_ID);
-    generationStampV1Limit = HdfsConstants.GRANDFATHER_GENERATION_STAMP;
+    legacyGenerationStampLimit = HdfsConstants.GRANDFATHER_GENERATION_STAMP;
   }
   }
 
 
   public static boolean isStripedBlockID(long id) {
   public static boolean isStripedBlockID(long id) {

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfV1GenerationStampsException.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfLegacyGenerationStampsException.java

@@ -23,16 +23,16 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 
 
 /**
 /**
- * This exception is thrown when the name node runs out of V1 generation
- * stamps.
+ * This exception is thrown when the name node runs out of V1 (legacy)
+ * generation stamps.
  *
  *
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
-public class OutOfV1GenerationStampsException extends IOException {
+public class OutOfLegacyGenerationStampsException extends IOException {
   private static final long serialVersionUID = 1L;
   private static final long serialVersionUID = 1L;
 
 
-  public OutOfV1GenerationStampsException() {
+  public OutOfLegacyGenerationStampsException() {
     super("Out of V1 (legacy) generation stamps\n");
     super("Out of V1 (legacy) generation stamps\n");
   }
   }
 }
 }

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java

@@ -61,8 +61,7 @@ public interface HdfsServerConstants {
   // An invalid transaction ID that will never be seen in a real namesystem.
   // An invalid transaction ID that will never be seen in a real namesystem.
   long INVALID_TXID = -12345;
   long INVALID_TXID = -12345;
   // Number of generation stamps reserved for legacy blocks.
   // Number of generation stamps reserved for legacy blocks.
-  long RESERVED_GENERATION_STAMPS_V1 =
-      1024L * 1024 * 1024 * 1024;
+  long RESERVED_LEGACY_GENERATION_STAMPS = 1024L * 1024 * 1024 * 1024;
   /**
   /**
    * Current layout version for NameNode.
    * Current layout version for NameNode.
    * Please see {@link NameNodeLayoutVersion.Feature} on adding new layout version.
    * Please see {@link NameNodeLayoutVersion.Feature} on adding new layout version.

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

@@ -960,7 +960,7 @@ public class FSEditLog implements LogsPurgeable {
   /**
   /**
    * Add legacy block generation stamp record to edit log
    * Add legacy block generation stamp record to edit log
    */
    */
-  void logGenerationStampV1(long genstamp) {
+  void logLegacyGenerationStamp(long genstamp) {
     SetGenstampV1Op op = SetGenstampV1Op.getInstance(cache.get())
     SetGenstampV1Op op = SetGenstampV1Op.getInstance(cache.get())
         .setGenerationStamp(genstamp);
         .setGenerationStamp(genstamp);
     logEdit(op);
     logEdit(op);
@@ -969,7 +969,7 @@ public class FSEditLog implements LogsPurgeable {
   /**
   /**
    * Add generation stamp record to edit log
    * Add generation stamp record to edit log
    */
    */
-  void logGenerationStampV2(long genstamp) {
+  void logGenerationStamp(long genstamp) {
     SetGenstampV2Op op = SetGenstampV2Op.getInstance(cache.get())
     SetGenstampV2Op op = SetGenstampV2Op.getInstance(cache.get())
         .setGenerationStamp(genstamp);
         .setGenerationStamp(genstamp);
     logEdit(op);
     logEdit(op);

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

@@ -589,7 +589,7 @@ public class FSEditLogLoader {
     }
     }
     case OP_SET_GENSTAMP_V1: {
     case OP_SET_GENSTAMP_V1: {
       SetGenstampV1Op setGenstampV1Op = (SetGenstampV1Op)op;
       SetGenstampV1Op setGenstampV1Op = (SetGenstampV1Op)op;
-      blockManager.getBlockIdManager().setGenerationStampV1(
+      blockManager.getBlockIdManager().setLegacyGenerationStamp(
           setGenstampV1Op.genStampV1);
           setGenstampV1Op.genStampV1);
       break;
       break;
     }
     }
@@ -797,7 +797,7 @@ public class FSEditLogLoader {
     }
     }
     case OP_SET_GENSTAMP_V2: {
     case OP_SET_GENSTAMP_V2: {
       SetGenstampV2Op setGenstampV2Op = (SetGenstampV2Op) op;
       SetGenstampV2Op setGenstampV2Op = (SetGenstampV2Op) op;
-      blockManager.getBlockIdManager().setGenerationStampV2(
+      blockManager.getBlockIdManager().setGenerationStamp(
           setGenstampV2Op.genStampV2);
           setGenstampV2Op.genStampV2);
       break;
       break;
     }
     }

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

@@ -346,24 +346,24 @@ public class FSImageFormat {
         long genstamp = in.readLong();
         long genstamp = in.readLong();
         final BlockIdManager blockIdManager = namesystem.getBlockManager()
         final BlockIdManager blockIdManager = namesystem.getBlockManager()
             .getBlockIdManager();
             .getBlockIdManager();
-        blockIdManager.setGenerationStampV1(genstamp);
+        blockIdManager.setLegacyGenerationStamp(genstamp);
 
 
         if (NameNodeLayoutVersion.supports(
         if (NameNodeLayoutVersion.supports(
             LayoutVersion.Feature.SEQUENTIAL_BLOCK_ID, imgVersion)) {
             LayoutVersion.Feature.SEQUENTIAL_BLOCK_ID, imgVersion)) {
           // read the starting generation stamp for sequential block IDs
           // read the starting generation stamp for sequential block IDs
           genstamp = in.readLong();
           genstamp = in.readLong();
-          blockIdManager.setGenerationStampV2(genstamp);
+          blockIdManager.setGenerationStamp(genstamp);
 
 
           // read the last generation stamp for blocks created after
           // read the last generation stamp for blocks created after
           // the switch to sequential block IDs.
           // the switch to sequential block IDs.
           long stampAtIdSwitch = in.readLong();
           long stampAtIdSwitch = in.readLong();
-          blockIdManager.setGenerationStampV1Limit(stampAtIdSwitch);
+          blockIdManager.setLegacyGenerationStampLimit(stampAtIdSwitch);
 
 
           // read the max sequential block ID.
           // read the max sequential block ID.
           long maxSequentialBlockId = in.readLong();
           long maxSequentialBlockId = in.readLong();
           blockIdManager.setLastAllocatedContiguousBlockId(maxSequentialBlockId);
           blockIdManager.setLastAllocatedContiguousBlockId(maxSequentialBlockId);
         } else {
         } else {
-          long startingGenStamp = blockIdManager.upgradeGenerationStampToV2();
+          long startingGenStamp = blockIdManager.upgradeLegacyGenerationStamp();
           // This is an upgrade.
           // This is an upgrade.
           LOG.info("Upgrading to sequential block IDs. Generation stamp " +
           LOG.info("Upgrading to sequential block IDs. Generation stamp " +
                    "for new blocks set to " + startingGenStamp);
                    "for new blocks set to " + startingGenStamp);
@@ -1270,8 +1270,8 @@ public class FSImageFormat {
         out.writeLong(numINodes);
         out.writeLong(numINodes);
         final BlockIdManager blockIdManager = sourceNamesystem.getBlockManager()
         final BlockIdManager blockIdManager = sourceNamesystem.getBlockManager()
             .getBlockIdManager();
             .getBlockIdManager();
-        out.writeLong(blockIdManager.getGenerationStampV1());
-        out.writeLong(blockIdManager.getGenerationStampV2());
+        out.writeLong(blockIdManager.getLegacyGenerationStamp());
+        out.writeLong(blockIdManager.getGenerationStamp());
         out.writeLong(blockIdManager.getGenerationStampAtblockIdSwitch());
         out.writeLong(blockIdManager.getGenerationStampAtblockIdSwitch());
         out.writeLong(blockIdManager.getLastAllocatedContiguousBlockId());
         out.writeLong(blockIdManager.getLastAllocatedContiguousBlockId());
         out.writeLong(context.getTxId());
         out.writeLong(context.getTxId());

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

@@ -294,9 +294,9 @@ public final class FSImageFormatProtobuf {
     private void loadNameSystemSection(InputStream in) throws IOException {
     private void loadNameSystemSection(InputStream in) throws IOException {
       NameSystemSection s = NameSystemSection.parseDelimitedFrom(in);
       NameSystemSection s = NameSystemSection.parseDelimitedFrom(in);
       BlockIdManager blockIdManager = fsn.getBlockManager().getBlockIdManager();
       BlockIdManager blockIdManager = fsn.getBlockManager().getBlockIdManager();
-      blockIdManager.setGenerationStampV1(s.getGenstampV1());
-      blockIdManager.setGenerationStampV2(s.getGenstampV2());
-      blockIdManager.setGenerationStampV1Limit(s.getGenstampV1Limit());
+      blockIdManager.setLegacyGenerationStamp(s.getGenstampV1());
+      blockIdManager.setGenerationStamp(s.getGenstampV2());
+      blockIdManager.setLegacyGenerationStampLimit(s.getGenstampV1Limit());
       blockIdManager.setLastAllocatedContiguousBlockId(s.getLastAllocatedBlockId());
       blockIdManager.setLastAllocatedContiguousBlockId(s.getLastAllocatedBlockId());
       if (s.hasLastAllocatedStripedBlockId()) {
       if (s.hasLastAllocatedStripedBlockId()) {
         blockIdManager.setLastAllocatedStripedBlockId(
         blockIdManager.setLastAllocatedStripedBlockId(
@@ -550,9 +550,9 @@ public final class FSImageFormatProtobuf {
       OutputStream out = sectionOutputStream;
       OutputStream out = sectionOutputStream;
       BlockIdManager blockIdManager = fsn.getBlockManager().getBlockIdManager();
       BlockIdManager blockIdManager = fsn.getBlockManager().getBlockIdManager();
       NameSystemSection.Builder b = NameSystemSection.newBuilder()
       NameSystemSection.Builder b = NameSystemSection.newBuilder()
-          .setGenstampV1(blockIdManager.getGenerationStampV1())
-          .setGenstampV1Limit(blockIdManager.getGenerationStampV1Limit())
-          .setGenstampV2(blockIdManager.getGenerationStampV2())
+          .setGenstampV1(blockIdManager.getLegacyGenerationStamp())
+          .setGenstampV1Limit(blockIdManager.getLegacyGenerationStampLimit())
+          .setGenstampV2(blockIdManager.getGenerationStamp())
           .setLastAllocatedBlockId(blockIdManager.getLastAllocatedContiguousBlockId())
           .setLastAllocatedBlockId(blockIdManager.getLastAllocatedContiguousBlockId())
           .setLastAllocatedStripedBlockId(blockIdManager.getLastAllocatedStripedBlockId())
           .setLastAllocatedStripedBlockId(blockIdManager.getLastAllocatedStripedBlockId())
           .setTransactionId(context.getTxId());
           .setTransactionId(context.getTxId());

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

@@ -4575,9 +4575,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
 
     long gs = blockManager.nextGenerationStamp(legacyBlock);
     long gs = blockManager.nextGenerationStamp(legacyBlock);
     if (legacyBlock) {
     if (legacyBlock) {
-      getEditLog().logGenerationStampV1(gs);
+      getEditLog().logLegacyGenerationStamp(gs);
     } else {
     } else {
-      getEditLog().logGenerationStampV2(gs);
+      getEditLog().logGenerationStamp(gs);
     }
     }
 
 
     // NB: callers sync the log
     // NB: callers sync the log

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto

@@ -67,8 +67,8 @@ message FileSummary {
  */
  */
 message NameSystemSection {
 message NameSystemSection {
   optional uint32 namespaceId = 1;
   optional uint32 namespaceId = 1;
-  optional uint64 genstampV1 = 2;
-  optional uint64 genstampV2 = 3;
+  optional uint64 genstampV1 = 2; // legacy generation stamp
+  optional uint64 genstampV2 = 3; // generation stamp of latest version
   optional uint64 genstampV1Limit = 4;
   optional uint64 genstampV1Limit = 4;
   optional uint64 lastAllocatedBlockId = 5;
   optional uint64 lastAllocatedBlockId = 5;
   optional uint64 transactionId = 6;
   optional uint64 transactionId = 6;

+ 9 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockId.java

@@ -151,7 +151,7 @@ public class TestSequentialBlockId {
     BlockIdManager bid = mock(BlockIdManager.class);
     BlockIdManager bid = mock(BlockIdManager.class);
     final long maxGenStampForLegacyBlocks = 10000;
     final long maxGenStampForLegacyBlocks = 10000;
 
 
-    when(bid.getGenerationStampV1Limit())
+    when(bid.getLegacyGenerationStampLimit())
         .thenReturn(maxGenStampForLegacyBlocks);
         .thenReturn(maxGenStampForLegacyBlocks);
 
 
     Block legacyBlock = spy(new Block());
     Block legacyBlock = spy(new Block());
@@ -180,18 +180,18 @@ public class TestSequentialBlockId {
     // Setup a mock object and stub out a few routines to
     // Setup a mock object and stub out a few routines to
     // retrieve the generation stamp counters.
     // retrieve the generation stamp counters.
     BlockIdManager bid = mock(BlockIdManager.class);
     BlockIdManager bid = mock(BlockIdManager.class);
-    final long nextGenerationStampV1 = 5000;
-    final long nextGenerationStampV2 = 20000;
+    final long nextLegacyGenerationStamp = 5000;
+    final long nextGenerationStamp = 20000;
 
 
-    when(bid.getNextGenerationStampV1())
-        .thenReturn(nextGenerationStampV1);
-    when(bid.getNextGenerationStampV2())
-        .thenReturn(nextGenerationStampV2);
+    when(bid.getNextLegacyGenerationStamp())
+        .thenReturn(nextLegacyGenerationStamp);
+    when(bid.getNextGenerationStamp())
+        .thenReturn(nextGenerationStamp);
 
 
     // Make sure that the generation stamp is set correctly for both
     // Make sure that the generation stamp is set correctly for both
     // kinds of blocks.
     // kinds of blocks.
     when(bid.nextGenerationStamp(anyBoolean())).thenCallRealMethod();
     when(bid.nextGenerationStamp(anyBoolean())).thenCallRealMethod();
-    assertThat(bid.nextGenerationStamp(true), is(nextGenerationStampV1));
-    assertThat(bid.nextGenerationStamp(false), is(nextGenerationStampV2));
+    assertThat(bid.nextGenerationStamp(true), is(nextLegacyGenerationStamp));
+    assertThat(bid.nextGenerationStamp(false), is(nextGenerationStamp));
   }
   }
 }
 }

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

@@ -1145,7 +1145,7 @@ public class TestEditLog {
     editlog.initJournalsForWrite();
     editlog.initJournalsForWrite();
     editlog.openForWrite(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
     editlog.openForWrite(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
     for (int i = 2; i < TXNS_PER_ROLL; i++) {
     for (int i = 2; i < TXNS_PER_ROLL; i++) {
-      editlog.logGenerationStampV2((long) 0);
+      editlog.logGenerationStamp((long) 0);
     }
     }
     editlog.logSync();
     editlog.logSync();
     
     
@@ -1157,7 +1157,7 @@ public class TestEditLog {
     for (int i = 0; i < numrolls; i++) {
     for (int i = 0; i < numrolls; i++) {
       editlog.rollEditLog(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
       editlog.rollEditLog(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
       
       
-      editlog.logGenerationStampV2((long) i);
+      editlog.logGenerationStamp((long) i);
       editlog.logSync();
       editlog.logSync();
 
 
       while (aborts.size() > 0 
       while (aborts.size() > 0 
@@ -1167,7 +1167,7 @@ public class TestEditLog {
       } 
       } 
       
       
       for (int j = 3; j < TXNS_PER_ROLL; j++) {
       for (int j = 3; j < TXNS_PER_ROLL; j++) {
-        editlog.logGenerationStampV2((long) i);
+        editlog.logGenerationStamp((long) i);
       }
       }
       editlog.logSync();
       editlog.logSync();
     }
     }

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

@@ -1020,7 +1020,7 @@ public class TestFileTruncate {
       assertThat(truncateBlock.getNumBytes(),
       assertThat(truncateBlock.getNumBytes(),
           is(oldBlock.getNumBytes()));
           is(oldBlock.getNumBytes()));
       assertThat(truncateBlock.getGenerationStamp(),
       assertThat(truncateBlock.getGenerationStamp(),
-          is(fsn.getBlockManager().getBlockIdManager().getGenerationStampV2()));
+          is(fsn.getBlockManager().getBlockIdManager().getGenerationStamp()));
       assertThat(file.getLastBlock().getBlockUCState(),
       assertThat(file.getLastBlock().getBlockUCState(),
           is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
           is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
       long blockRecoveryId = file.getLastBlock().getUnderConstructionFeature()
       long blockRecoveryId = file.getLastBlock().getUnderConstructionFeature()
@@ -1054,7 +1054,7 @@ public class TestFileTruncate {
       assertThat(truncateBlock.getNumBytes() < oldBlock.getNumBytes(),
       assertThat(truncateBlock.getNumBytes() < oldBlock.getNumBytes(),
           is(true));
           is(true));
       assertThat(truncateBlock.getGenerationStamp(),
       assertThat(truncateBlock.getGenerationStamp(),
-          is(fsn.getBlockManager().getBlockIdManager().getGenerationStampV2()));
+          is(fsn.getBlockManager().getBlockIdManager().getGenerationStamp()));
       assertThat(file.getLastBlock().getBlockUCState(),
       assertThat(file.getLastBlock().getBlockUCState(),
           is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
           is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
       long blockRecoveryId = file.getLastBlock().getUnderConstructionFeature()
       long blockRecoveryId = file.getLastBlock().getUnderConstructionFeature()

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java

@@ -519,7 +519,7 @@ public class TestSaveNamespace {
     DelayAnswer delayer = new GenericTestUtils.DelayAnswer(LOG);
     DelayAnswer delayer = new GenericTestUtils.DelayAnswer(LOG);
     BlockIdManager bid = spy(spyFsn.getBlockManager().getBlockIdManager());
     BlockIdManager bid = spy(spyFsn.getBlockManager().getBlockIdManager());
     Whitebox.setInternalState(finalFsn.getBlockManager(), "blockIdManager", bid);
     Whitebox.setInternalState(finalFsn.getBlockManager(), "blockIdManager", bid);
-    doAnswer(delayer).when(bid).getGenerationStampV2();
+    doAnswer(delayer).when(bid).getGenerationStamp();
 
 
     ExecutorService pool = Executors.newFixedThreadPool(2);
     ExecutorService pool = Executors.newFixedThreadPool(2);