Ver Fonte

HDFS-7381. Decouple the management of block id and gen stamps from FSNamesystem. Contributed by Haohui Mai.

Haohui Mai há 10 anos atrás
pai
commit
14a9ff1b9c

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

@@ -95,6 +95,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7365. Remove hdfs.server.blockmanagement.MutableBlockCollection.
     (Li Lu via wheat9)
 
+    HDFS-7381. Decouple the management of block id and gen stamps from
+    FSNamesystem. (wheat9)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 208 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java

@@ -0,0 +1,208 @@
+/**
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.GenerationStamp;
+
+import java.io.IOException;
+
+/**
+ * BlockIdManager allocates the generation stamps and the block ID. The
+ * {@see FSNamesystem} is responsible for persisting the allocations in the
+ * {@see EditLog}.
+ */
+public class BlockIdManager {
+  /**
+   * The global generation stamp for legacy blocks with randomly
+   * generated block IDs.
+   */
+  private final GenerationStamp generationStampV1 = new GenerationStamp();
+  /**
+   * The global generation stamp for this file system.
+   */
+  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
+   * have randomly allocated block IDs. Blocks with generation stamps above
+   * this value had sequentially allocated block IDs. Read from the fsImage
+   * (or initialized as an offset from the V1 (legacy) generation stamp on
+   * upgrade).
+   */
+  private long generationStampV1Limit;
+  /**
+   * The global block ID space for this file system.
+   */
+  private final SequentialBlockIdGenerator blockIdGenerator;
+
+  public BlockIdManager(BlockManager blockManager) {
+    this.generationStampV1Limit = GenerationStamp.GRANDFATHER_GENERATION_STAMP;
+    this.blockIdGenerator = new SequentialBlockIdGenerator(blockManager);
+  }
+
+  /**
+   * Upgrades the generation stamp for the filesystem
+   * by reserving a sufficient range for all existing blocks.
+   * Should be invoked only during the first upgrade to
+   * sequential block IDs.
+   */
+  public long upgradeGenerationStampToV2() {
+    Preconditions.checkState(generationStampV2.getCurrentValue() ==
+      GenerationStamp.LAST_RESERVED_STAMP);
+    generationStampV2.skipTo(generationStampV1.getCurrentValue() +
+      HdfsConstants.RESERVED_GENERATION_STAMPS_V1);
+
+    generationStampV1Limit = generationStampV2.getCurrentValue();
+    return generationStampV2.getCurrentValue();
+  }
+
+  /**
+   * Sets the generation stamp that delineates random and sequentially
+   * allocated block IDs.
+   *
+   * @param stamp set generation stamp limit to this value
+   */
+  public void setGenerationStampV1Limit(long stamp) {
+    Preconditions.checkState(generationStampV1Limit == GenerationStamp
+      .GRANDFATHER_GENERATION_STAMP);
+    generationStampV1Limit = stamp;
+  }
+
+  /**
+   * Gets the value of the generation stamp that delineates sequential
+   * and random block IDs.
+   */
+  public long getGenerationStampAtblockIdSwitch() {
+    return generationStampV1Limit;
+  }
+
+  @VisibleForTesting
+  SequentialBlockIdGenerator getBlockIdGenerator() {
+    return blockIdGenerator;
+  }
+
+  /**
+   * Sets the maximum allocated block ID for this filesystem. This is
+   * the basis for allocating new block IDs.
+   */
+  public void setLastAllocatedBlockId(long blockId) {
+    blockIdGenerator.skipTo(blockId);
+  }
+
+  /**
+   * Gets the maximum sequentially allocated block ID for this filesystem
+   */
+  public long getLastAllocatedBlockId() {
+    return blockIdGenerator.getCurrentValue();
+  }
+
+  /**
+   * Sets the current generation stamp for legacy blocks
+   */
+  public void setGenerationStampV1(long stamp) {
+    generationStampV1.setCurrentValue(stamp);
+  }
+
+  /**
+   * Gets the current generation stamp for legacy blocks
+   */
+  public long getGenerationStampV1() {
+    return generationStampV1.getCurrentValue();
+  }
+
+  /**
+   * Gets the current generation stamp for this filesystem
+   */
+  public void setGenerationStampV2(long stamp) {
+    generationStampV2.setCurrentValue(stamp);
+  }
+
+  public long getGenerationStampV2() {
+    return generationStampV2.getCurrentValue();
+  }
+
+  /**
+   * Increments, logs and then returns the stamp
+   */
+  public long nextGenerationStamp(boolean legacyBlock) throws IOException {
+    return legacyBlock ? getNextGenerationStampV1() :
+      getNextGenerationStampV2();
+  }
+
+  @VisibleForTesting
+  long getNextGenerationStampV1() throws IOException {
+    long genStampV1 = generationStampV1.nextValue();
+
+    if (genStampV1 >= generationStampV1Limit) {
+      // We ran out of generation stamps for legacy blocks. In practice, it
+      // 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 OutOfV1GenerationStampsException();
+    }
+
+    return genStampV1;
+  }
+
+  @VisibleForTesting
+  long getNextGenerationStampV2() {
+    return generationStampV2.nextValue();
+  }
+
+  public long getGenerationStampV1Limit() {
+    return generationStampV1Limit;
+  }
+
+  /**
+   * Determine whether the block ID was randomly generated (legacy) or
+   * sequentially generated. The generation stamp value is used to
+   * make the distinction.
+   *
+   * @return true if the block ID was randomly generated, false otherwise.
+   */
+  public boolean isLegacyBlock(Block block) {
+    return block.getGenerationStamp() < getGenerationStampV1Limit();
+  }
+
+  /**
+   * Increments, logs and then returns the block ID
+   */
+  public long nextBlockId() {
+    return blockIdGenerator.nextValue();
+  }
+
+  public boolean isGenStampInFuture(Block block) {
+    if (isLegacyBlock(block)) {
+      return block.getGenerationStamp() > getGenerationStampV1();
+    } else {
+      return block.getGenerationStamp() > getGenerationStampV2();
+    }
+  }
+
+  public void clear() {
+    generationStampV1.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
+    generationStampV2.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
+    getBlockIdGenerator().setCurrentValue(SequentialBlockIdGenerator
+      .LAST_RESERVED_BLOCK_ID);
+    setGenerationStampV1Limit(GenerationStamp.GRANDFATHER_GENERATION_STAMP);
+  }
+}

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SequentialBlockIdGenerator.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockIdGenerator.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;

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

@@ -533,7 +533,7 @@ public class FSEditLogLoader {
     }
     case OP_SET_GENSTAMP_V1: {
       SetGenstampV1Op setGenstampV1Op = (SetGenstampV1Op)op;
-      fsNamesys.setGenerationStampV1(setGenstampV1Op.genStampV1);
+      fsNamesys.getBlockIdManager().setGenerationStampV1(setGenstampV1Op.genStampV1);
       break;
     }
     case OP_SET_PERMISSIONS: {
@@ -726,12 +726,12 @@ public class FSEditLogLoader {
     }
     case OP_SET_GENSTAMP_V2: {
       SetGenstampV2Op setGenstampV2Op = (SetGenstampV2Op) op;
-      fsNamesys.setGenerationStampV2(setGenstampV2Op.genStampV2);
+      fsNamesys.getBlockIdManager().setGenerationStampV2(setGenstampV2Op.genStampV2);
       break;
     }
     case OP_ALLOCATE_BLOCK_ID: {
       AllocateBlockIdOp allocateBlockIdOp = (AllocateBlockIdOp) op;
-      fsNamesys.setLastAllocatedBlockId(allocateBlockIdOp.blockId);
+      fsNamesys.getBlockIdManager().setLastAllocatedBlockId(allocateBlockIdOp.blockId);
       break;
     }
     case OP_ROLLING_UPGRADE_START: {

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

@@ -342,24 +342,26 @@ public class FSImageFormat {
 
         // read in the last generation stamp for legacy blocks.
         long genstamp = in.readLong();
-        namesystem.setGenerationStampV1(genstamp);
-        
+        namesystem.getBlockIdManager().setGenerationStampV1(genstamp);
+
         if (NameNodeLayoutVersion.supports(
             LayoutVersion.Feature.SEQUENTIAL_BLOCK_ID, imgVersion)) {
           // read the starting generation stamp for sequential block IDs
           genstamp = in.readLong();
-          namesystem.setGenerationStampV2(genstamp);
+          namesystem.getBlockIdManager().setGenerationStampV2(genstamp);
 
           // read the last generation stamp for blocks created after
           // the switch to sequential block IDs.
           long stampAtIdSwitch = in.readLong();
-          namesystem.setGenerationStampV1Limit(stampAtIdSwitch);
+          namesystem.getBlockIdManager().setGenerationStampV1Limit(stampAtIdSwitch);
 
           // read the max sequential block ID.
           long maxSequentialBlockId = in.readLong();
-          namesystem.setLastAllocatedBlockId(maxSequentialBlockId);
+          namesystem.getBlockIdManager().setLastAllocatedBlockId(maxSequentialBlockId);
         } else {
-          long startingGenStamp = namesystem.upgradeGenerationStampToV2();
+
+          long startingGenStamp = namesystem.getBlockIdManager()
+            .upgradeGenerationStampToV2();
           // This is an upgrade.
           LOG.info("Upgrading to sequential block IDs. Generation stamp " +
                    "for new blocks set to " + startingGenStamp);
@@ -1256,10 +1258,10 @@ public class FSImageFormat {
         out.writeInt(sourceNamesystem.unprotectedGetNamespaceInfo()
             .getNamespaceID());
         out.writeLong(numINodes);
-        out.writeLong(sourceNamesystem.getGenerationStampV1());
-        out.writeLong(sourceNamesystem.getGenerationStampV2());
-        out.writeLong(sourceNamesystem.getGenerationStampAtblockIdSwitch());
-        out.writeLong(sourceNamesystem.getLastAllocatedBlockId());
+        out.writeLong(sourceNamesystem.getBlockIdManager().getGenerationStampV1());
+        out.writeLong(sourceNamesystem.getBlockIdManager().getGenerationStampV2());
+        out.writeLong(sourceNamesystem.getBlockIdManager().getGenerationStampAtblockIdSwitch());
+        out.writeLong(sourceNamesystem.getBlockIdManager().getLastAllocatedBlockId());
         out.writeLong(context.getTxId());
         out.writeLong(sourceNamesystem.getLastInodeId());
 

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

@@ -46,8 +46,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
-import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
-import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.CacheManagerSection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.NameSystemSection;
@@ -293,10 +292,11 @@ public final class FSImageFormatProtobuf {
 
     private void loadNameSystemSection(InputStream in) throws IOException {
       NameSystemSection s = NameSystemSection.parseDelimitedFrom(in);
-      fsn.setGenerationStampV1(s.getGenstampV1());
-      fsn.setGenerationStampV2(s.getGenstampV2());
-      fsn.setGenerationStampV1Limit(s.getGenstampV1Limit());
-      fsn.setLastAllocatedBlockId(s.getLastAllocatedBlockId());
+      BlockIdManager blockIdManager = fsn.getBlockIdManager();
+      blockIdManager.setGenerationStampV1(s.getGenstampV1());
+      blockIdManager.setGenerationStampV2(s.getGenstampV2());
+      blockIdManager.setGenerationStampV1Limit(s.getGenstampV1Limit());
+      blockIdManager.setLastAllocatedBlockId(s.getLastAllocatedBlockId());
       imgTxId = s.getTransactionId();
       if (s.hasRollingUpgradeStartTime()
           && fsn.getFSImage().hasRollbackFSImage()) {
@@ -407,7 +407,7 @@ public final class FSImageFormatProtobuf {
       FileOutputStream fout = new FileOutputStream(file);
       fileChannel = fout.getChannel();
       try {
-        saveInternal(fout, compression, file.getAbsolutePath().toString());
+        saveInternal(fout, compression, file.getAbsolutePath());
       } finally {
         fout.close();
       }
@@ -531,11 +531,12 @@ public final class FSImageFormatProtobuf {
         throws IOException {
       final FSNamesystem fsn = context.getSourceNamesystem();
       OutputStream out = sectionOutputStream;
+      BlockIdManager blockIdManager = fsn.getBlockIdManager();
       NameSystemSection.Builder b = NameSystemSection.newBuilder()
-          .setGenstampV1(fsn.getGenerationStampV1())
-          .setGenstampV1Limit(fsn.getGenerationStampV1Limit())
-          .setGenstampV2(fsn.getGenerationStampV2())
-          .setLastAllocatedBlockId(fsn.getLastAllocatedBlockId())
+          .setGenstampV1(blockIdManager.getGenerationStampV1())
+          .setGenstampV1Limit(blockIdManager.getGenerationStampV1Limit())
+          .setGenstampV2(blockIdManager.getGenerationStampV2())
+          .setLastAllocatedBlockId(blockIdManager.getLastAllocatedBlockId())
           .setTransactionId(context.getTxId());
 
       // We use the non-locked version of getNamespaceInfo here since

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

@@ -210,6 +210,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifie
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager.SecretManagerState;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@@ -217,8 +218,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.OutOfV1GenerationStampsException;
-import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
@@ -322,6 +321,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       }
   };
 
+  private final BlockIdManager blockIdManager;
+
   @VisibleForTesting
   public boolean isAuditEnabled() {
     return !isDefaultAuditLogger || auditLog.isInfoEnabled();
@@ -481,34 +482,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   private final long minBlockSize;         // minimum block size
   private final long maxBlocksPerFile;     // maximum # of blocks per file
 
-  /**
-   * The global generation stamp for legacy blocks with randomly
-   * generated block IDs.
-   */
-  private final GenerationStamp generationStampV1 = new GenerationStamp();
-
-  /**
-   * The global generation stamp for this file system.
-   */
-  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
-   * have randomly allocated block IDs. Blocks with generation stamps above
-   * this value had sequentially allocated block IDs. Read from the fsImage
-   * (or initialized as an offset from the V1 (legacy) generation stamp on
-   * upgrade).
-   */
-  private long generationStampV1Limit =
-      GenerationStamp.GRANDFATHER_GENERATION_STAMP;
-
-  /**
-   * The global block ID space for this file system.
-   */
-  @VisibleForTesting
-  private final SequentialBlockIdGenerator blockIdGenerator;
-
   // precision of access times.
   private final long accessTimePrecision;
 
@@ -640,11 +613,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   void clear() {
     dir.reset();
     dtSecretManager.reset();
-    generationStampV1.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
-    generationStampV2.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
-    blockIdGenerator.setCurrentValue(
-        SequentialBlockIdGenerator.LAST_RESERVED_BLOCK_ID);
-    generationStampV1Limit = GenerationStamp.GRANDFATHER_GENERATION_STAMP;
+    blockIdManager.clear();
     leaseManager.removeAllLeases();
     inodeId.setCurrentValue(INodeId.LAST_RESERVED_ID);
     snapshotManager.clearSnapshottableDirs();
@@ -794,7 +763,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
       this.blockManager = new BlockManager(this, this, conf);
       this.datanodeStatistics = blockManager.getDatanodeManager().getDatanodeStatistics();
-      this.blockIdGenerator = new SequentialBlockIdGenerator(this.blockManager);
+      this.blockIdManager = new BlockIdManager(blockManager);
 
       this.isStoragePolicyEnabled =
           conf.getBoolean(DFS_STORAGE_POLICY_ENABLED_KEY,
@@ -1358,7 +1327,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * @throws SafeModeException
    *           Otherwise if NameNode is in SafeMode.
    */
-  private void checkNameNodeSafeMode(String errorMsg)
+  void checkNameNodeSafeMode(String errorMsg)
       throws RetriableException, SafeModeException {
     if (isInSafeMode()) {
       SafeModeException se = new SafeModeException(errorMsg, safeMode);
@@ -4598,7 +4567,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         return true;
       }
       // start recovery of the last block for this file
-      long blockRecoveryId = nextGenerationStamp(isLegacyBlock(uc));
+      long blockRecoveryId = nextGenerationStamp(blockIdManager.isLegacyBlock(uc));
       lease = reassignLease(lease, src, recoveryLeaseHolder, pendingFile);
       uc.initializeBlockRecovery(blockRecoveryId);
       leaseManager.renewLease(lease);
@@ -6734,91 +6703,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return getBlockManager().getDatanodeManager().getNumStaleStorages();
   }
 
-  /**
-   * Sets the current generation stamp for legacy blocks
-   */
-  void setGenerationStampV1(long stamp) {
-    generationStampV1.setCurrentValue(stamp);
-  }
-
-  /**
-   * Gets the current generation stamp for legacy blocks
-   */
-  long getGenerationStampV1() {
-    return generationStampV1.getCurrentValue();
-  }
-
-  /**
-   * Gets the current generation stamp for this filesystem
-   */
-  void setGenerationStampV2(long stamp) {
-    generationStampV2.setCurrentValue(stamp);
-  }
-
-  /**
-   * Gets the current generation stamp for this filesystem
-   */
-  long getGenerationStampV2() {
-    return generationStampV2.getCurrentValue();
-  }
-
-  /**
-   * Upgrades the generation stamp for the filesystem
-   * by reserving a sufficient range for all existing blocks.
-   * Should be invoked only during the first upgrade to
-   * sequential block IDs.
-   */
-  long upgradeGenerationStampToV2() {
-    Preconditions.checkState(generationStampV2.getCurrentValue() ==
-        GenerationStamp.LAST_RESERVED_STAMP);
-
-    generationStampV2.skipTo(
-        generationStampV1.getCurrentValue() +
-        HdfsConstants.RESERVED_GENERATION_STAMPS_V1);
-
-    generationStampV1Limit = generationStampV2.getCurrentValue();
-    return generationStampV2.getCurrentValue();
-  }
-
-  /**
-   * Sets the generation stamp that delineates random and sequentially
-   * allocated block IDs.
-   * @param stamp set generation stamp limit to this value
-   */
-  void setGenerationStampV1Limit(long stamp) {
-    Preconditions.checkState(generationStampV1Limit ==
-                             GenerationStamp.GRANDFATHER_GENERATION_STAMP);
-    generationStampV1Limit = stamp;
-  }
-
-  /**
-   * Gets the value of the generation stamp that delineates sequential
-   * and random block IDs.
-   */
-  long getGenerationStampAtblockIdSwitch() {
-    return generationStampV1Limit;
-  }
-
-  @VisibleForTesting
-  SequentialBlockIdGenerator getBlockIdGenerator() {
-    return blockIdGenerator;
-  }
-
-  /**
-   * Sets the maximum allocated block ID for this filesystem. This is
-   * the basis for allocating new block IDs.
-   */
-  void setLastAllocatedBlockId(long blockId) {
-    blockIdGenerator.skipTo(blockId);
-  }
-
-  /**
-   * Gets the maximum sequentially allocated block ID for this filesystem
-   */
-  long getLastAllocatedBlockId() {
-    return blockIdGenerator.getCurrentValue();
-  }
-
   /**
    * Increments, logs and then returns the stamp
    */
@@ -6827,12 +6711,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     assert hasWriteLock();
     checkNameNodeSafeMode("Cannot get next generation stamp");
 
-    long gs;
+    long gs = blockIdManager.nextGenerationStamp(legacyBlock);
     if (legacyBlock) {
-      gs = getNextGenerationStampV1();
       getEditLog().logGenerationStampV1(gs);
     } else {
-      gs = getNextGenerationStampV2();
       getEditLog().logGenerationStampV2(gs);
     }
 
@@ -6840,47 +6722,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return gs;
   }
 
-  @VisibleForTesting
-  long getNextGenerationStampV1() throws IOException {
-    long genStampV1 = generationStampV1.nextValue();
-
-    if (genStampV1 >= generationStampV1Limit) {
-      // We ran out of generation stamps for legacy blocks. In practice, it
-      // 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 OutOfV1GenerationStampsException();
-    }
-
-    return genStampV1;
-  }
-
-  @VisibleForTesting
-  long getNextGenerationStampV2() {
-    return generationStampV2.nextValue();
-  }
-
-  long getGenerationStampV1Limit() {
-    return generationStampV1Limit;
-  }
-
-  /**
-   * Determine whether the block ID was randomly generated (legacy) or
-   * sequentially generated. The generation stamp value is used to
-   * make the distinction.
-   * @return true if the block ID was randomly generated, false otherwise.
-   */
-  boolean isLegacyBlock(Block block) {
-    return block.getGenerationStamp() < getGenerationStampV1Limit();
-  }
-
   /**
    * Increments, logs and then returns the block ID
    */
   private long nextBlockId() throws IOException {
     assert hasWriteLock();
     checkNameNodeSafeMode("Cannot get next block ID");
-    final long blockId = blockIdGenerator.nextValue();
+    final long blockId = blockIdManager.nextBlockId();
     getEditLog().logAllocateBlockId(blockId);
     // NB: callers sync the log
     return blockId;
@@ -6995,8 +6843,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       checkUCBlock(block, clientName);
   
       // get a new generation stamp and an access token
-      block.setGenerationStamp(
-          nextGenerationStamp(isLegacyBlock(block.getLocalBlock())));
+      block.setGenerationStamp(nextGenerationStamp(blockIdManager.isLegacyBlock(block.getLocalBlock())));
       locatedBlock = new LocatedBlock(block, new DatanodeInfo[0]);
       blockManager.setBlockToken(locatedBlock, AccessMode.WRITE);
     } finally {
@@ -7868,6 +7715,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   public BlockManager getBlockManager() {
     return blockManager;
   }
+
+  public BlockIdManager getBlockIdManager() {
+    return blockIdManager;
+  }
+
   /** @return the FSDirectory. */
   public FSDirectory getFSDirectory() {
     return dir;
@@ -7935,11 +7787,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   
   @Override
   public boolean isGenStampInFuture(Block block) {
-    if (isLegacyBlock(block)) {
-      return block.getGenerationStamp() > getGenerationStampV1();
-    } else {
-      return block.getGenerationStamp() > getGenerationStampV2();
-    }
+    return blockIdManager.isGenStampInFuture(block);
   }
 
   @VisibleForTesting

+ 15 - 27
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSequentialBlockId.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockId.java

@@ -15,10 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import java.io.IOException;
-import java.net.InetSocketAddress;
 import java.util.List;
 
 import org.apache.commons.logging.Log;
@@ -31,9 +30,8 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.junit.Test;
 
 import static org.hamcrest.CoreMatchers.is;
@@ -45,20 +43,13 @@ import static org.mockito.Mockito.*;
  * collision handling.
  */
 public class TestSequentialBlockId {
-
   private static final Log LOG = LogFactory.getLog("TestSequentialBlockId");
 
-  private static final DataChecksum DEFAULT_CHECKSUM =
-      DataChecksum.newDataChecksum(DataChecksum.Type.CRC32C, 512);
-
   final int BLOCK_SIZE = 1024;
   final int IO_SIZE = BLOCK_SIZE;
   final short REPLICATION = 1;
   final long SEED = 0;
 
-  DatanodeID datanode;
-  InetSocketAddress dnAddr;
-
   /**
    * Test that block IDs are generated sequentially.
    *
@@ -125,7 +116,8 @@ public class TestSequentialBlockId {
 
       // Rewind the block ID counter in the name system object. This will result
       // in block ID collisions when we try to allocate new blocks.
-      SequentialBlockIdGenerator blockIdGenerator = fsn.getBlockIdGenerator();
+      SequentialBlockIdGenerator blockIdGenerator = fsn.getBlockIdManager()
+        .getBlockIdGenerator();
       blockIdGenerator.setCurrentValue(blockIdGenerator.getCurrentValue() - 5);
 
       // Trigger collisions by creating a new file.
@@ -156,10 +148,10 @@ public class TestSequentialBlockId {
 
     // Setup a mock object and stub out a few routines to
     // retrieve the generation stamp counters.
-    FSNamesystem fsn = mock(FSNamesystem.class);
+    BlockIdManager bid = mock(BlockIdManager.class);
     final long maxGenStampForLegacyBlocks = 10000;
 
-    when(fsn.getGenerationStampV1Limit())
+    when(bid.getGenerationStampV1Limit())
         .thenReturn(maxGenStampForLegacyBlocks);
 
     Block legacyBlock = spy(new Block());
@@ -172,9 +164,9 @@ public class TestSequentialBlockId {
 
     // Make sure that isLegacyBlock() can correctly detect
     // legacy and new blocks.
-    when(fsn.isLegacyBlock(any(Block.class))).thenCallRealMethod();
-    assertThat(fsn.isLegacyBlock(legacyBlock), is(true));
-    assertThat(fsn.isLegacyBlock(newBlock), is(false));
+    when(bid.isLegacyBlock(any(Block.class))).thenCallRealMethod();
+    assertThat(bid.isLegacyBlock(legacyBlock), is(true));
+    assertThat(bid.isLegacyBlock(newBlock), is(false));
   }
 
   /**
@@ -185,25 +177,21 @@ public class TestSequentialBlockId {
    */
   @Test
   public void testGenerationStampUpdate() throws IOException {
-
     // Setup a mock object and stub out a few routines to
     // retrieve the generation stamp counters.
-    FSNamesystem fsn = mock(FSNamesystem.class);
-    FSEditLog editLog = mock(FSEditLog.class);
+    BlockIdManager bid = mock(BlockIdManager.class);
     final long nextGenerationStampV1 = 5000;
     final long nextGenerationStampV2 = 20000;
 
-    when(fsn.getNextGenerationStampV1())
+    when(bid.getNextGenerationStampV1())
         .thenReturn(nextGenerationStampV1);
-    when(fsn.getNextGenerationStampV2())
+    when(bid.getNextGenerationStampV2())
         .thenReturn(nextGenerationStampV2);
 
     // Make sure that the generation stamp is set correctly for both
     // kinds of blocks.
-    when(fsn.nextGenerationStamp(anyBoolean())).thenCallRealMethod();
-    when(fsn.hasWriteLock()).thenReturn(true);
-    when(fsn.getEditLog()).thenReturn(editLog);
-    assertThat(fsn.nextGenerationStamp(true), is(nextGenerationStampV1));
-    assertThat(fsn.nextGenerationStamp(false), is(nextGenerationStampV2));
+    when(bid.nextGenerationStamp(anyBoolean())).thenCallRealMethod();
+    assertThat(bid.nextGenerationStamp(true), is(nextGenerationStampV1));
+    assertThat(bid.nextGenerationStamp(false), is(nextGenerationStampV2));
   }
 }

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

@@ -50,6 +50,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
@@ -544,8 +545,10 @@ public class TestSaveNamespace {
     FSNamesystem spyFsn = spy(fsn);
     final FSNamesystem finalFsn = spyFsn;
     DelayAnswer delayer = new GenericTestUtils.DelayAnswer(LOG);
-    doAnswer(delayer).when(spyFsn).getGenerationStampV2();
-    
+    BlockIdManager bid = spy(spyFsn.getBlockIdManager());
+    Whitebox.setInternalState(finalFsn, "blockIdManager", bid);
+    doAnswer(delayer).when(bid).getGenerationStampV2();
+
     ExecutorService pool = Executors.newFixedThreadPool(2);
     
     try {
@@ -600,9 +603,7 @@ public class TestSaveNamespace {
             NNStorage.getImageFileName(0) + MD5FileUtils.MD5_SUFFIX);
       }      
     } finally {
-      if (fsn != null) {
-        fsn.close();
-      }
+      fsn.close();
     }
   }