You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ar...@apache.org on 2016/01/29 20:07:23 UTC

[32/50] hadoop git commit: HDFS-9677. Rename generationStampV1/generationStampV2 to legacyGenerationStamp/generationStamp. Contributed by Mingliang Liu.

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


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

Branch: refs/heads/HDFS-1312
Commit: ec25c7f9c7e60c077d8c4143253c20445fcdaecf
Parents: 3a95713
Author: Jing Zhao <ji...@apache.org>
Authored: Wed Jan 27 16:34:40 2016 -0800
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Jan 27 16:34:40 2016 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../server/blockmanagement/BlockIdManager.java  | 83 ++++++++++----------
 .../OutOfLegacyGenerationStampsException.java   | 38 +++++++++
 .../OutOfV1GenerationStampsException.java       | 38 ---------
 .../hdfs/server/common/HdfsServerConstants.java |  3 +-
 .../hadoop/hdfs/server/namenode/FSEditLog.java  |  4 +-
 .../hdfs/server/namenode/FSEditLogLoader.java   |  4 +-
 .../hdfs/server/namenode/FSImageFormat.java     | 12 +--
 .../server/namenode/FSImageFormatProtobuf.java  | 12 +--
 .../hdfs/server/namenode/FSNamesystem.java      |  4 +-
 .../hadoop-hdfs/src/main/proto/fsimage.proto    |  4 +-
 .../blockmanagement/TestSequentialBlockId.java  | 18 ++---
 .../hdfs/server/namenode/TestEditLog.java       |  6 +-
 .../hdfs/server/namenode/TestFileTruncate.java  |  4 +-
 .../hdfs/server/namenode/TestSaveNamespace.java |  2 +-
 15 files changed, 119 insertions(+), 116 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 097c051..7e75558 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/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
     greater than 2 GB. (cmccabe via zhz)
 
+    HDFS-9677. Rename generationStampV1/generationStampV2 to
+    legacyGenerationStamp/generationStamp. (Mingliang Liu via jing9)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
index 9c71287..3f21d9b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
@@ -36,11 +36,11 @@ public class BlockIdManager {
    * The global generation stamp for legacy blocks with randomly
    * generated block IDs.
    */
-  private final GenerationStamp generationStampV1 = new GenerationStamp();
+  private final GenerationStamp legacyGenerationStamp = new GenerationStamp();
   /**
    * 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
    * 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
    * upgrade).
    */
-  private long generationStampV1Limit;
+  private long legacyGenerationStampLimit;
   /**
    * The global block ID space for this file system.
    */
@@ -57,7 +57,8 @@ public class BlockIdManager {
   private final SequentialBlockGroupIdGenerator blockGroupIdGenerator;
 
   public BlockIdManager(BlockManager blockManager) {
-    this.generationStampV1Limit = HdfsConstants.GRANDFATHER_GENERATION_STAMP;
+    this.legacyGenerationStampLimit =
+        HdfsConstants.GRANDFATHER_GENERATION_STAMP;
     this.blockIdGenerator = new SequentialBlockIdGenerator(blockManager);
     this.blockGroupIdGenerator = new SequentialBlockGroupIdGenerator(blockManager);
   }
@@ -68,14 +69,14 @@ public class BlockIdManager {
    * Should be invoked only during the first upgrade to
    * sequential block IDs.
    */
-  public long upgradeGenerationStampToV2() {
-    Preconditions.checkState(generationStampV2.getCurrentValue() ==
+  public long upgradeLegacyGenerationStamp() {
+    Preconditions.checkState(generationStamp.getCurrentValue() ==
       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
    */
-  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.
    */
   public long getGenerationStampAtblockIdSwitch() {
-    return generationStampV1Limit;
+    return legacyGenerationStampLimit;
   }
 
   @VisibleForTesting
@@ -138,58 +139,58 @@ public class BlockIdManager {
   /**
    * 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
    */
-  public long getGenerationStampV1() {
-    return generationStampV1.getCurrentValue();
+  public long getLegacyGenerationStamp() {
+    return legacyGenerationStamp.getCurrentValue();
   }
 
   /**
    * 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
    */
   long nextGenerationStamp(boolean legacyBlock) throws IOException {
-    return legacyBlock ? getNextGenerationStampV1() :
-      getNextGenerationStampV2();
+    return legacyBlock ? getNextLegacyGenerationStamp() :
+        getNextGenerationStamp();
   }
 
   @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
-      // 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
       // were created before the upgrade to sequential block IDs.
-      throw new OutOfV1GenerationStampsException();
+      throw new OutOfLegacyGenerationStampsException();
     }
 
-    return genStampV1;
+    return legacyGenStamp;
   }
 
   @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.
    */
   boolean isLegacyBlock(Block block) {
-    return block.getGenerationStamp() < getGenerationStampV1Limit();
+    return block.getGenerationStamp() < getLegacyGenerationStampLimit();
   }
 
   /**
@@ -213,18 +214,18 @@ public class BlockIdManager {
 
   boolean isGenStampInFuture(Block block) {
     if (isLegacyBlock(block)) {
-      return block.getGenerationStamp() > getGenerationStampV1();
+      return block.getGenerationStamp() > getLegacyGenerationStamp();
     } else {
-      return block.getGenerationStamp() > getGenerationStampV2();
+      return block.getGenerationStamp() > getGenerationStamp();
     }
   }
 
   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
       .LAST_RESERVED_BLOCK_ID);
-    generationStampV1Limit = HdfsConstants.GRANDFATHER_GENERATION_STAMP;
+    legacyGenerationStampLimit = HdfsConstants.GRANDFATHER_GENERATION_STAMP;
   }
 
   public static boolean isStripedBlockID(long id) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfLegacyGenerationStampsException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfLegacyGenerationStampsException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfLegacyGenerationStampsException.java
new file mode 100644
index 0000000..f81392f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfLegacyGenerationStampsException.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import java.io.IOException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * This exception is thrown when the name node runs out of V1 (legacy)
+ * generation stamps.
+ *
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class OutOfLegacyGenerationStampsException extends IOException {
+  private static final long serialVersionUID = 1L;
+
+  public OutOfLegacyGenerationStampsException() {
+    super("Out of V1 (legacy) generation stamps\n");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfV1GenerationStampsException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfV1GenerationStampsException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfV1GenerationStampsException.java
deleted file mode 100644
index 127ad54..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfV1GenerationStampsException.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs.server.blockmanagement;
-
-import java.io.IOException;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * This exception is thrown when the name node runs out of V1 generation
- * stamps.
- *
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class OutOfV1GenerationStampsException extends IOException {
-  private static final long serialVersionUID = 1L;
-
-  public OutOfV1GenerationStampsException() {
-    super("Out of V1 (legacy) generation stamps\n");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
index 13c9137..b2dda3c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
@@ -61,8 +61,7 @@ public interface HdfsServerConstants {
   // 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;
+  long RESERVED_LEGACY_GENERATION_STAMPS = 1024L * 1024 * 1024 * 1024;
   /**
    * Current layout version for NameNode.
    * Please see {@link NameNodeLayoutVersion.Feature} on adding new layout version.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
index e255cff..c8986dc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
@@ -960,7 +960,7 @@ public class FSEditLog implements LogsPurgeable {
   /**
    * Add legacy block generation stamp record to edit log
    */
-  void logGenerationStampV1(long genstamp) {
+  void logLegacyGenerationStamp(long genstamp) {
     SetGenstampV1Op op = SetGenstampV1Op.getInstance(cache.get())
         .setGenerationStamp(genstamp);
     logEdit(op);
@@ -969,7 +969,7 @@ public class FSEditLog implements LogsPurgeable {
   /**
    * Add generation stamp record to edit log
    */
-  void logGenerationStampV2(long genstamp) {
+  void logGenerationStamp(long genstamp) {
     SetGenstampV2Op op = SetGenstampV2Op.getInstance(cache.get())
         .setGenerationStamp(genstamp);
     logEdit(op);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index 1a9d7a9..25f5a4f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -589,7 +589,7 @@ public class FSEditLogLoader {
     }
     case OP_SET_GENSTAMP_V1: {
       SetGenstampV1Op setGenstampV1Op = (SetGenstampV1Op)op;
-      blockManager.getBlockIdManager().setGenerationStampV1(
+      blockManager.getBlockIdManager().setLegacyGenerationStamp(
           setGenstampV1Op.genStampV1);
       break;
     }
@@ -797,7 +797,7 @@ public class FSEditLogLoader {
     }
     case OP_SET_GENSTAMP_V2: {
       SetGenstampV2Op setGenstampV2Op = (SetGenstampV2Op) op;
-      blockManager.getBlockIdManager().setGenerationStampV2(
+      blockManager.getBlockIdManager().setGenerationStamp(
           setGenstampV2Op.genStampV2);
       break;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
index a815864..e4263bd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
@@ -346,24 +346,24 @@ public class FSImageFormat {
         long genstamp = in.readLong();
         final BlockIdManager blockIdManager = namesystem.getBlockManager()
             .getBlockIdManager();
-        blockIdManager.setGenerationStampV1(genstamp);
+        blockIdManager.setLegacyGenerationStamp(genstamp);
 
         if (NameNodeLayoutVersion.supports(
             LayoutVersion.Feature.SEQUENTIAL_BLOCK_ID, imgVersion)) {
           // read the starting generation stamp for sequential block IDs
           genstamp = in.readLong();
-          blockIdManager.setGenerationStampV2(genstamp);
+          blockIdManager.setGenerationStamp(genstamp);
 
           // read the last generation stamp for blocks created after
           // the switch to sequential block IDs.
           long stampAtIdSwitch = in.readLong();
-          blockIdManager.setGenerationStampV1Limit(stampAtIdSwitch);
+          blockIdManager.setLegacyGenerationStampLimit(stampAtIdSwitch);
 
           // read the max sequential block ID.
           long maxSequentialBlockId = in.readLong();
           blockIdManager.setLastAllocatedContiguousBlockId(maxSequentialBlockId);
         } else {
-          long startingGenStamp = blockIdManager.upgradeGenerationStampToV2();
+          long startingGenStamp = blockIdManager.upgradeLegacyGenerationStamp();
           // This is an upgrade.
           LOG.info("Upgrading to sequential block IDs. Generation stamp " +
                    "for new blocks set to " + startingGenStamp);
@@ -1270,8 +1270,8 @@ public class FSImageFormat {
         out.writeLong(numINodes);
         final BlockIdManager blockIdManager = sourceNamesystem.getBlockManager()
             .getBlockIdManager();
-        out.writeLong(blockIdManager.getGenerationStampV1());
-        out.writeLong(blockIdManager.getGenerationStampV2());
+        out.writeLong(blockIdManager.getLegacyGenerationStamp());
+        out.writeLong(blockIdManager.getGenerationStamp());
         out.writeLong(blockIdManager.getGenerationStampAtblockIdSwitch());
         out.writeLong(blockIdManager.getLastAllocatedContiguousBlockId());
         out.writeLong(context.getTxId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
index 635dc34..e9bd8c5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
@@ -294,9 +294,9 @@ public final class FSImageFormatProtobuf {
     private void loadNameSystemSection(InputStream in) throws IOException {
       NameSystemSection s = NameSystemSection.parseDelimitedFrom(in);
       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());
       if (s.hasLastAllocatedStripedBlockId()) {
         blockIdManager.setLastAllocatedStripedBlockId(
@@ -550,9 +550,9 @@ public final class FSImageFormatProtobuf {
       OutputStream out = sectionOutputStream;
       BlockIdManager blockIdManager = fsn.getBlockManager().getBlockIdManager();
       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())
           .setLastAllocatedStripedBlockId(blockIdManager.getLastAllocatedStripedBlockId())
           .setTransactionId(context.getTxId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 1693958..5e2e975 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -4575,9 +4575,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     long gs = blockManager.nextGenerationStamp(legacyBlock);
     if (legacyBlock) {
-      getEditLog().logGenerationStampV1(gs);
+      getEditLog().logLegacyGenerationStamp(gs);
     } else {
-      getEditLog().logGenerationStampV2(gs);
+      getEditLog().logGenerationStamp(gs);
     }
 
     // NB: callers sync the log

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
index 43a1fc5..e57efee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
@@ -67,8 +67,8 @@ message FileSummary {
  */
 message NameSystemSection {
   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 lastAllocatedBlockId = 5;
   optional uint64 transactionId = 6;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockId.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockId.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockId.java
index e612ea9..bfda393 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockId.java
+++ b/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);
     final long maxGenStampForLegacyBlocks = 10000;
 
-    when(bid.getGenerationStampV1Limit())
+    when(bid.getLegacyGenerationStampLimit())
         .thenReturn(maxGenStampForLegacyBlocks);
 
     Block legacyBlock = spy(new Block());
@@ -180,18 +180,18 @@ public class TestSequentialBlockId {
     // Setup a mock object and stub out a few routines to
     // retrieve the generation stamp counters.
     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
     // kinds of blocks.
     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));
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
index 4b72112..14240e0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
@@ -1145,7 +1145,7 @@ public class TestEditLog {
     editlog.initJournalsForWrite();
     editlog.openForWrite(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
     for (int i = 2; i < TXNS_PER_ROLL; i++) {
-      editlog.logGenerationStampV2((long) 0);
+      editlog.logGenerationStamp((long) 0);
     }
     editlog.logSync();
     
@@ -1157,7 +1157,7 @@ public class TestEditLog {
     for (int i = 0; i < numrolls; i++) {
       editlog.rollEditLog(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
       
-      editlog.logGenerationStampV2((long) i);
+      editlog.logGenerationStamp((long) i);
       editlog.logSync();
 
       while (aborts.size() > 0 
@@ -1167,7 +1167,7 @@ public class TestEditLog {
       } 
       
       for (int j = 3; j < TXNS_PER_ROLL; j++) {
-        editlog.logGenerationStampV2((long) i);
+        editlog.logGenerationStamp((long) i);
       }
       editlog.logSync();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
index f2c0330..d723525 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
@@ -1020,7 +1020,7 @@ public class TestFileTruncate {
       assertThat(truncateBlock.getNumBytes(),
           is(oldBlock.getNumBytes()));
       assertThat(truncateBlock.getGenerationStamp(),
-          is(fsn.getBlockManager().getBlockIdManager().getGenerationStampV2()));
+          is(fsn.getBlockManager().getBlockIdManager().getGenerationStamp()));
       assertThat(file.getLastBlock().getBlockUCState(),
           is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
       long blockRecoveryId = file.getLastBlock().getUnderConstructionFeature()
@@ -1054,7 +1054,7 @@ public class TestFileTruncate {
       assertThat(truncateBlock.getNumBytes() < oldBlock.getNumBytes(),
           is(true));
       assertThat(truncateBlock.getGenerationStamp(),
-          is(fsn.getBlockManager().getBlockIdManager().getGenerationStampV2()));
+          is(fsn.getBlockManager().getBlockIdManager().getGenerationStamp()));
       assertThat(file.getLastBlock().getBlockUCState(),
           is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
       long blockRecoveryId = file.getLastBlock().getUnderConstructionFeature()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec25c7f9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
index 4d37d61..a03e900 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
+++ b/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);
     BlockIdManager bid = spy(spyFsn.getBlockManager().getBlockIdManager());
     Whitebox.setInternalState(finalFsn.getBlockManager(), "blockIdManager", bid);
-    doAnswer(delayer).when(bid).getGenerationStampV2();
+    doAnswer(delayer).when(bid).getGenerationStamp();
 
     ExecutorService pool = Executors.newFixedThreadPool(2);