소스 검색

HDFS-1245. Plugable block id generation. Contributed by Konstantin Shvachko.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1432539 13f79535-47bb-0310-9956-ffa450edef68
Konstantin Shvachko 12 년 전
부모
커밋
06406d7056

+ 31 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IdGenerator.java

@@ -0,0 +1,31 @@
+/**
+ * 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.util;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Generic ID generator
+ * used for generating various types of number sequences.
+ */
+@InterfaceAudience.Private
+public interface IdGenerator {
+
+  /** Increment and then return the next value. */
+  public long nextValue();
+}

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SequentialNumber.java

@@ -27,7 +27,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
  * This class is thread safe.
  */
 @InterfaceAudience.Private
-public abstract class SequentialNumber {
+public abstract class SequentialNumber implements IdGenerator {
   private final AtomicLong currentValue;
 
   /** Create a new instance with the given initial value. */

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

@@ -701,6 +701,8 @@ Release 2.0.3-alpha - Unreleased
     HDFS-4274. BlockPoolSliceScanner does not close verification log during
     shutdown. (Chris Nauroth via suresh)
 
+    HDFS-1245. Plugable block id generation. (shv)
+
   BREAKDOWN OF HDFS-3077 SUBTASKS
 
     HDFS-3077. Quorum-based protocol for reading and writing edit logs.

+ 17 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java

@@ -59,6 +59,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.util.Canceler;
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
 import org.apache.hadoop.io.MD5Hash;
+import org.apache.hadoop.util.IdGenerator;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
@@ -92,6 +93,7 @@ public class FSImage implements Closeable {
   final private Configuration conf;
 
   protected NNStorageRetentionManager archivalManager;
+  protected IdGenerator blockIdGenerator;
 
   /**
    * Construct an FSImage
@@ -137,6 +139,9 @@ public class FSImage implements Closeable {
     Preconditions.checkState(fileCount == 1,
         "FSImage.format should be called with an uninitialized namesystem, has " +
         fileCount + " files");
+    // BlockIdGenerator is defined during formatting
+    // currently there is only one BlockIdGenerator
+    blockIdGenerator = createBlockIdGenerator(fsn);
     NamespaceInfo ns = NNStorage.newNamespaceInfo();
     ns.clusterID = clusterId;
     
@@ -253,6 +258,7 @@ public class FSImage implements Closeable {
       doRollback();
       break;
     case REGULAR:
+    default:
       // just load the image
     }
     
@@ -737,6 +743,9 @@ public class FSImage implements Closeable {
     FSImageFormat.Loader loader = new FSImageFormat.Loader(
         conf, target);
     loader.load(curFile);
+    // BlockIdGenerator is determined after loading image
+    // currently there is only one BlockIdGenerator
+    blockIdGenerator = createBlockIdGenerator(target);
     target.setBlockPoolId(this.getBlockPoolID());
 
     // Check that the image digest we loaded matches up with what
@@ -1165,4 +1174,12 @@ public class FSImage implements Closeable {
   public synchronized long getMostRecentCheckpointTxId() {
     return storage.getMostRecentCheckpointTxId();
   }
+
+  public long getUniqueBlockId() {
+    return blockIdGenerator.nextValue();
+  }
+
+  public IdGenerator createBlockIdGenerator(FSNamesystem fsn) {
+    return new RandomBlockIdGenerator(fsn);
+  }
 }

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

@@ -79,7 +79,6 @@ import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
-import java.io.FileWriter;
 import java.io.IOException;
 import java.io.OutputStreamWriter;
 import java.io.PrintWriter;
@@ -2539,10 +2538,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   private Block allocateBlock(String src, INodesInPath inodesInPath,
       DatanodeDescriptor targets[]) throws IOException {
     assert hasWriteLock();
-    Block b = new Block(DFSUtil.getRandom().nextLong(), 0, 0); 
-    while(isValidBlock(b)) {
-      b.setBlockId(DFSUtil.getRandom().nextLong());
-    }
+    Block b = new Block(getFSImage().getUniqueBlockId(), 0, 0); 
     // Increment the generation stamp for every new block.
     b.setGenerationStamp(nextGenerationStamp());
     b = dir.addBlock(src, inodesInPath, b, targets);
@@ -4554,13 +4550,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
   }
 
-  /**
-   * Returns whether the given block is one pointed-to by a file.
-   */
-  private boolean isValidBlock(Block b) {
-    return (blockManager.getBlockCollection(b) != null);
-  }
-
   PermissionStatus createFsOwnerPermissions(FsPermission permission) {
     return new PermissionStatus(fsOwner.getShortUserName(), supergroup, permission);
   }

+ 52 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RandomBlockIdGenerator.java

@@ -0,0 +1,52 @@
+/**
+ * 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.namenode;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.util.IdGenerator;
+
+/**
+ * Generator of random block IDs.
+ */
+@InterfaceAudience.Private
+public class RandomBlockIdGenerator implements IdGenerator {
+  private final BlockManager blockManager;
+
+  RandomBlockIdGenerator(FSNamesystem namesystem) {
+    this.blockManager = namesystem.getBlockManager();
+  }
+
+  @Override // NumberGenerator
+  public long nextValue() {
+    Block b = new Block(DFSUtil.getRandom().nextLong(), 0, 0); 
+    while(isValidBlock(b)) {
+      b.setBlockId(DFSUtil.getRandom().nextLong());
+    }
+    return b.getBlockId();
+  }
+
+  /**
+   * Returns whether the given block is one pointed-to by a file.
+   */
+  private boolean isValidBlock(Block b) {
+    return (blockManager.getBlockCollection(b) != null);
+  }
+}