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 ae...@apache.org on 2016/02/01 19:40:42 UTC

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

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

This reverts commit 8a91109d16394310f2568717f103e6fff7cbddb0.


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

Branch: refs/heads/HDFS-7240
Commit: 3a9571308e99cc374681bbc451a517d41a150aa0
Parents: 8a91109
Author: Jing Zhao <ji...@apache.org>
Authored: Wed Jan 27 16:31:19 2016 -0800
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Jan 27 16:31:19 2016 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 -
 .../server/blockmanagement/BlockIdManager.java  | 83 ++++++++++----------
 .../server/blockmanagement/BlockManager.java    |  8 +-
 .../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      |  8 +-
 .../hadoop/hdfs/server/namenode/Namesystem.java | 13 ++-
 .../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 +-
 17 files changed, 134 insertions(+), 126 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/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 7e75558..097c051 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -959,9 +959,6 @@ 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/3a957130/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 3f21d9b..9c71287 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 legacyGenerationStamp = new GenerationStamp();
+  private final GenerationStamp generationStampV1 = new GenerationStamp();
   /**
    * The global generation stamp for this file system.
    */
-  private final GenerationStamp generationStamp = new GenerationStamp();
+  private final GenerationStamp generationStampV2 = 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 legacyGenerationStampLimit;
+  private long generationStampV1Limit;
   /**
    * The global block ID space for this file system.
    */
@@ -57,8 +57,7 @@ public class BlockIdManager {
   private final SequentialBlockGroupIdGenerator blockGroupIdGenerator;
 
   public BlockIdManager(BlockManager blockManager) {
-    this.legacyGenerationStampLimit =
-        HdfsConstants.GRANDFATHER_GENERATION_STAMP;
+    this.generationStampV1Limit = HdfsConstants.GRANDFATHER_GENERATION_STAMP;
     this.blockIdGenerator = new SequentialBlockIdGenerator(blockManager);
     this.blockGroupIdGenerator = new SequentialBlockGroupIdGenerator(blockManager);
   }
@@ -69,14 +68,14 @@ public class BlockIdManager {
    * Should be invoked only during the first upgrade to
    * sequential block IDs.
    */
-  public long upgradeLegacyGenerationStamp() {
-    Preconditions.checkState(generationStamp.getCurrentValue() ==
+  public long upgradeGenerationStampToV2() {
+    Preconditions.checkState(generationStampV2.getCurrentValue() ==
       GenerationStamp.LAST_RESERVED_STAMP);
-    generationStamp.skipTo(legacyGenerationStamp.getCurrentValue() +
-      HdfsServerConstants.RESERVED_LEGACY_GENERATION_STAMPS);
+    generationStampV2.skipTo(generationStampV1.getCurrentValue() +
+      HdfsServerConstants.RESERVED_GENERATION_STAMPS_V1);
 
-    legacyGenerationStampLimit = generationStamp.getCurrentValue();
-    return generationStamp.getCurrentValue();
+    generationStampV1Limit = generationStampV2.getCurrentValue();
+    return generationStampV2.getCurrentValue();
   }
 
   /**
@@ -85,10 +84,10 @@ public class BlockIdManager {
    *
    * @param stamp set generation stamp limit to this value
    */
-  public void setLegacyGenerationStampLimit(long stamp) {
-    Preconditions.checkState(legacyGenerationStampLimit ==
-        HdfsConstants.GRANDFATHER_GENERATION_STAMP);
-    legacyGenerationStampLimit = stamp;
+  public void setGenerationStampV1Limit(long stamp) {
+    Preconditions.checkState(generationStampV1Limit == HdfsConstants
+      .GRANDFATHER_GENERATION_STAMP);
+    generationStampV1Limit = stamp;
   }
 
   /**
@@ -96,7 +95,7 @@ public class BlockIdManager {
    * and random block IDs.
    */
   public long getGenerationStampAtblockIdSwitch() {
-    return legacyGenerationStampLimit;
+    return generationStampV1Limit;
   }
 
   @VisibleForTesting
@@ -139,58 +138,58 @@ public class BlockIdManager {
   /**
    * Sets the current generation stamp for legacy blocks
    */
-  public void setLegacyGenerationStamp(long stamp) {
-    legacyGenerationStamp.setCurrentValue(stamp);
+  public void setGenerationStampV1(long stamp) {
+    generationStampV1.setCurrentValue(stamp);
   }
 
   /**
    * Gets the current generation stamp for legacy blocks
    */
-  public long getLegacyGenerationStamp() {
-    return legacyGenerationStamp.getCurrentValue();
+  public long getGenerationStampV1() {
+    return generationStampV1.getCurrentValue();
   }
 
   /**
    * Gets the current generation stamp for this filesystem
    */
-  public void setGenerationStamp(long stamp) {
-    generationStamp.setCurrentValue(stamp);
+  public void setGenerationStampV2(long stamp) {
+    generationStampV2.setCurrentValue(stamp);
   }
 
-  public long getGenerationStamp() {
-    return generationStamp.getCurrentValue();
+  public long getGenerationStampV2() {
+    return generationStampV2.getCurrentValue();
   }
 
   /**
    * Increments, logs and then returns the stamp
    */
   long nextGenerationStamp(boolean legacyBlock) throws IOException {
-    return legacyBlock ? getNextLegacyGenerationStamp() :
-        getNextGenerationStamp();
+    return legacyBlock ? getNextGenerationStampV1() :
+      getNextGenerationStampV2();
   }
 
   @VisibleForTesting
-  long getNextLegacyGenerationStamp() throws IOException {
-    long legacyGenStamp = legacyGenerationStamp.nextValue();
+  long getNextGenerationStampV1() throws IOException {
+    long genStampV1 = generationStampV1.nextValue();
 
-    if (legacyGenStamp >= legacyGenerationStampLimit) {
+    if (genStampV1 >= generationStampV1Limit) {
       // We ran out of generation stamps for legacy blocks. In practice, it
-      // is extremely unlikely as we reserved 1T legacy generation stamps. The
+      // is extremely unlikely as we reserved 1T v1 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 OutOfLegacyGenerationStampsException();
+      throw new OutOfV1GenerationStampsException();
     }
 
-    return legacyGenStamp;
+    return genStampV1;
   }
 
   @VisibleForTesting
-  long getNextGenerationStamp() {
-    return generationStamp.nextValue();
+  long getNextGenerationStampV2() {
+    return generationStampV2.nextValue();
   }
 
-  public long getLegacyGenerationStampLimit() {
-    return legacyGenerationStampLimit;
+  public long getGenerationStampV1Limit() {
+    return generationStampV1Limit;
   }
 
   /**
@@ -201,7 +200,7 @@ public class BlockIdManager {
    * @return true if the block ID was randomly generated, false otherwise.
    */
   boolean isLegacyBlock(Block block) {
-    return block.getGenerationStamp() < getLegacyGenerationStampLimit();
+    return block.getGenerationStamp() < getGenerationStampV1Limit();
   }
 
   /**
@@ -214,18 +213,18 @@ public class BlockIdManager {
 
   boolean isGenStampInFuture(Block block) {
     if (isLegacyBlock(block)) {
-      return block.getGenerationStamp() > getLegacyGenerationStamp();
+      return block.getGenerationStamp() > getGenerationStampV1();
     } else {
-      return block.getGenerationStamp() > getGenerationStamp();
+      return block.getGenerationStamp() > getGenerationStampV2();
     }
   }
 
   void clear() {
-    legacyGenerationStamp.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
-    generationStamp.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
+    generationStampV1.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
+    generationStampV2.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
     getBlockIdGenerator().setCurrentValue(SequentialBlockIdGenerator
       .LAST_RESERVED_BLOCK_ID);
-    legacyGenerationStampLimit = HdfsConstants.GRANDFATHER_GENERATION_STAMP;
+    generationStampV1Limit = HdfsConstants.GRANDFATHER_GENERATION_STAMP;
   }
 
   public static boolean isStripedBlockID(long id) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 6215437..a76429e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -927,13 +927,13 @@ public class BlockManager implements BlockStatsMXBean {
       final BlockUnderConstructionFeature uc = blk.getUnderConstructionFeature();
       if (blk.isStriped()) {
         final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
-        final ExtendedBlock eb = new ExtendedBlock(getBlockPoolId(),
+        final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
             blk);
         return newLocatedStripedBlock(eb, storages, uc.getBlockIndices(), pos,
             false);
       } else {
         final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
-        final ExtendedBlock eb = new ExtendedBlock(getBlockPoolId(),
+        final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
             blk);
         return newLocatedBlock(eb, storages, pos, false);
       }
@@ -976,7 +976,7 @@ public class BlockManager implements BlockStatsMXBean {
       " numNodes: " + numNodes +
       " numCorrupt: " + numCorruptNodes +
       " numCorruptRepls: " + numCorruptReplicas;
-    final ExtendedBlock eb = new ExtendedBlock(getBlockPoolId(), blk);
+    final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
     return blockIndices == null ?
         newLocatedBlock(eb, machines, pos, isCorrupt) :
         newLocatedStripedBlock(eb, machines, blockIndices, pos, isCorrupt);
@@ -1653,7 +1653,7 @@ public class BlockManager implements BlockStatsMXBean {
         return false;
       }
       rw.getTargets()[0].getDatanodeDescriptor().addBlockToBeErasureCoded(
-          new ExtendedBlock(getBlockPoolId(), block),
+          new ExtendedBlock(namesystem.getBlockPoolId(), block),
           rw.getSrcNodes(), rw.getTargets(),
           ((ErasureCodingWork) rw).getLiveBlockIndicies(), ecPolicy);
     } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/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
deleted file mode 100644
index f81392f..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfLegacyGenerationStampsException.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 (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/3a957130/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
new file mode 100644
index 0000000..127ad54
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/OutOfV1GenerationStampsException.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 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/3a957130/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 b2dda3c..13c9137 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,7 +61,8 @@ 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_LEGACY_GENERATION_STAMPS = 1024L * 1024 * 1024 * 1024;
+  long RESERVED_GENERATION_STAMPS_V1 =
+      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/3a957130/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 c8986dc..e255cff 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 logLegacyGenerationStamp(long genstamp) {
+  void logGenerationStampV1(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 logGenerationStamp(long genstamp) {
+  void logGenerationStampV2(long genstamp) {
     SetGenstampV2Op op = SetGenstampV2Op.getInstance(cache.get())
         .setGenerationStamp(genstamp);
     logEdit(op);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/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 25f5a4f..1a9d7a9 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().setLegacyGenerationStamp(
+      blockManager.getBlockIdManager().setGenerationStampV1(
           setGenstampV1Op.genStampV1);
       break;
     }
@@ -797,7 +797,7 @@ public class FSEditLogLoader {
     }
     case OP_SET_GENSTAMP_V2: {
       SetGenstampV2Op setGenstampV2Op = (SetGenstampV2Op) op;
-      blockManager.getBlockIdManager().setGenerationStamp(
+      blockManager.getBlockIdManager().setGenerationStampV2(
           setGenstampV2Op.genStampV2);
       break;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/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 e4263bd..a815864 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.setLegacyGenerationStamp(genstamp);
+        blockIdManager.setGenerationStampV1(genstamp);
 
         if (NameNodeLayoutVersion.supports(
             LayoutVersion.Feature.SEQUENTIAL_BLOCK_ID, imgVersion)) {
           // read the starting generation stamp for sequential block IDs
           genstamp = in.readLong();
-          blockIdManager.setGenerationStamp(genstamp);
+          blockIdManager.setGenerationStampV2(genstamp);
 
           // read the last generation stamp for blocks created after
           // the switch to sequential block IDs.
           long stampAtIdSwitch = in.readLong();
-          blockIdManager.setLegacyGenerationStampLimit(stampAtIdSwitch);
+          blockIdManager.setGenerationStampV1Limit(stampAtIdSwitch);
 
           // read the max sequential block ID.
           long maxSequentialBlockId = in.readLong();
           blockIdManager.setLastAllocatedContiguousBlockId(maxSequentialBlockId);
         } else {
-          long startingGenStamp = blockIdManager.upgradeLegacyGenerationStamp();
+          long startingGenStamp = blockIdManager.upgradeGenerationStampToV2();
           // 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.getLegacyGenerationStamp());
-        out.writeLong(blockIdManager.getGenerationStamp());
+        out.writeLong(blockIdManager.getGenerationStampV1());
+        out.writeLong(blockIdManager.getGenerationStampV2());
         out.writeLong(blockIdManager.getGenerationStampAtblockIdSwitch());
         out.writeLong(blockIdManager.getLastAllocatedContiguousBlockId());
         out.writeLong(context.getTxId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/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 e9bd8c5..635dc34 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.setLegacyGenerationStamp(s.getGenstampV1());
-      blockIdManager.setGenerationStamp(s.getGenstampV2());
-      blockIdManager.setLegacyGenerationStampLimit(s.getGenstampV1Limit());
+      blockIdManager.setGenerationStampV1(s.getGenstampV1());
+      blockIdManager.setGenerationStampV2(s.getGenstampV2());
+      blockIdManager.setGenerationStampV1Limit(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.getLegacyGenerationStamp())
-          .setGenstampV1Limit(blockIdManager.getLegacyGenerationStampLimit())
-          .setGenstampV2(blockIdManager.getGenerationStamp())
+          .setGenstampV1(blockIdManager.getGenerationStampV1())
+          .setGenstampV1Limit(blockIdManager.getGenerationStampV1Limit())
+          .setGenstampV2(blockIdManager.getGenerationStampV2())
           .setLastAllocatedBlockId(blockIdManager.getLastAllocatedContiguousBlockId())
           .setLastAllocatedStripedBlockId(blockIdManager.getLastAllocatedStripedBlockId())
           .setTransactionId(context.getTxId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/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 166526b..1693958 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
@@ -4307,7 +4307,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return new PermissionStatus(fsOwner.getShortUserName(), supergroup, permission);
   }
 
-  void checkSuperuserPrivilege() throws AccessControlException {
+  @Override
+  public void checkSuperuserPrivilege()
+      throws AccessControlException {
     if (isPermissionEnabled) {
       FSPermissionChecker pc = getPermissionChecker();
       pc.checkSuperuserPrivilege();
@@ -4573,9 +4575,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     long gs = blockManager.nextGenerationStamp(legacyBlock);
     if (legacyBlock) {
-      getEditLog().logLegacyGenerationStamp(gs);
+      getEditLog().logGenerationStampV1(gs);
     } else {
-      getEditLog().logGenerationStamp(gs);
+      getEditLog().logGenerationStampV2(gs);
     }
 
     // NB: callers sync the log

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
index 95d7c20..5a9e69b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
@@ -35,13 +35,22 @@ public interface Namesystem extends RwLock, SafeMode {
   /** Is this name system running? */
   boolean isRunning();
 
+  /** Check if the user has superuser privilege. */
+  void checkSuperuserPrivilege() throws AccessControlException;
+
+  /** @return the block pool ID */
+  String getBlockPoolId();
+
   BlockCollection getBlockCollection(long id);
 
   void startSecretManagerIfNecessary();
 
   /**
-   * @param src file/directory path
-   * @return The {@link ErasureCodingPolicy} for the given file/directory path
+   * Gets the erasure coding policy for the path
+   * @param src
+   *          - path
+   * @return {@link ErasureCodingPolicy}
+   * @throws IOException
    */
   ErasureCodingPolicy getErasureCodingPolicyForPath(String src)
       throws IOException;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/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 e57efee..43a1fc5 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; // legacy generation stamp
-  optional uint64 genstampV2 = 3; // generation stamp of latest version
+  optional uint64 genstampV1 = 2;
+  optional uint64 genstampV2 = 3;
   optional uint64 genstampV1Limit = 4;
   optional uint64 lastAllocatedBlockId = 5;
   optional uint64 transactionId = 6;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/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 bfda393..e612ea9 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.getLegacyGenerationStampLimit())
+    when(bid.getGenerationStampV1Limit())
         .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 nextLegacyGenerationStamp = 5000;
-    final long nextGenerationStamp = 20000;
+    final long nextGenerationStampV1 = 5000;
+    final long nextGenerationStampV2 = 20000;
 
-    when(bid.getNextLegacyGenerationStamp())
-        .thenReturn(nextLegacyGenerationStamp);
-    when(bid.getNextGenerationStamp())
-        .thenReturn(nextGenerationStamp);
+    when(bid.getNextGenerationStampV1())
+        .thenReturn(nextGenerationStampV1);
+    when(bid.getNextGenerationStampV2())
+        .thenReturn(nextGenerationStampV2);
 
     // Make sure that the generation stamp is set correctly for both
     // kinds of blocks.
     when(bid.nextGenerationStamp(anyBoolean())).thenCallRealMethod();
-    assertThat(bid.nextGenerationStamp(true), is(nextLegacyGenerationStamp));
-    assertThat(bid.nextGenerationStamp(false), is(nextGenerationStamp));
+    assertThat(bid.nextGenerationStamp(true), is(nextGenerationStampV1));
+    assertThat(bid.nextGenerationStamp(false), is(nextGenerationStampV2));
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/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 14240e0..4b72112 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.logGenerationStamp((long) 0);
+      editlog.logGenerationStampV2((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.logGenerationStamp((long) i);
+      editlog.logGenerationStampV2((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.logGenerationStamp((long) i);
+        editlog.logGenerationStampV2((long) i);
       }
       editlog.logSync();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/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 d723525..f2c0330 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().getGenerationStamp()));
+          is(fsn.getBlockManager().getBlockIdManager().getGenerationStampV2()));
       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().getGenerationStamp()));
+          is(fsn.getBlockManager().getBlockIdManager().getGenerationStampV2()));
       assertThat(file.getLastBlock().getBlockUCState(),
           is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
       long blockRecoveryId = file.getLastBlock().getUnderConstructionFeature()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a957130/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 a03e900..4d37d61 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).getGenerationStamp();
+    doAnswer(delayer).when(bid).getGenerationStampV2();
 
     ExecutorService pool = Executors.newFixedThreadPool(2);