Переглянути джерело

HDFS-3418. svn merge -c 1338830 from trunk

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2.0.0-alpha@1338832 13f79535-47bb-0310-9956-ffa450edef68
Eli Collins 13 роки тому
батько
коміт
28a1b3500a

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

@@ -300,6 +300,9 @@ Release 2.0.0 - UNRELEASED
 
     HADOOP-8285 HDFS changes for Use ProtoBuf for RpcPayLoadHeader (sanjay radia)
 
+    HDFS-3418. Rename BlockWithLocationsProto datanodeIDs field to storageIDs.
+    (eli)
+
   OPTIMIZATIONS
 
     HDFS-2477. Optimize computing the diff between a block report and the

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -254,11 +254,11 @@ public class PBHelper {
   public static BlockWithLocationsProto convert(BlockWithLocations blk) {
     return BlockWithLocationsProto.newBuilder()
         .setBlock(convert(blk.getBlock()))
-        .addAllDatanodeIDs(Arrays.asList(blk.getDatanodes())).build();
+        .addAllStorageIDs(Arrays.asList(blk.getStorageIDs())).build();
   }
 
   public static BlockWithLocations convert(BlockWithLocationsProto b) {
-    return new BlockWithLocations(convert(b.getBlock()), b.getDatanodeIDsList()
+    return new BlockWithLocations(convert(b.getBlock()), b.getStorageIDsList()
         .toArray(new String[0]));
   }
 

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java

@@ -205,6 +205,7 @@ public class Balancer {
   private Map<Block, BalancerBlock> globalBlockList
                  = new HashMap<Block, BalancerBlock>();
   private MovedBlocks movedBlocks = new MovedBlocks();
+  // Map storage IDs to BalancerDatanodes
   private Map<String, BalancerDatanode> datanodes
                  = new HashMap<String, BalancerDatanode>();
   
@@ -620,8 +621,8 @@ public class Balancer {
         
           synchronized (block) {
             // update locations
-            for ( String location : blk.getDatanodes() ) {
-              BalancerDatanode datanode = datanodes.get(location);
+            for ( String storageID : blk.getStorageIDs() ) {
+              BalancerDatanode datanode = datanodes.get(storageID);
               if (datanode != null) { // not an unknown datanode
                 block.addLocation(datanode);
               }

+ 9 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksWithLocations.java

@@ -21,9 +21,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.Block;
 
-/** A class to implement an array of BlockLocations
- *  It provide efficient customized serialization/deserialization methods
- *  in stead of using the default array (de)serialization provided by RPC
+/**
+ * Maintains an array of blocks and their corresponding storage IDs.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
@@ -36,12 +35,12 @@ public class BlocksWithLocations {
   @InterfaceStability.Evolving
   public static class BlockWithLocations {
     Block block;
-    String datanodeIDs[];
+    String storageIDs[];
     
     /** constructor */
-    public BlockWithLocations(Block b, String[] datanodes) {
-      block = b;
-      datanodeIDs = datanodes;
+    public BlockWithLocations(Block block, String[] storageIDs) {
+      this.block = block;
+      this.storageIDs = storageIDs;
     }
     
     /** get the block */
@@ -50,15 +49,15 @@ public class BlocksWithLocations {
     }
     
     /** get the block's locations */
-    public String[] getDatanodes() {
-      return datanodeIDs;
+    public String[] getStorageIDs() {
+      return storageIDs;
     }
   }
 
   private BlockWithLocations[] blocks;
 
   /** Constructor with one parameter */
-  public BlocksWithLocations( BlockWithLocations[] blocks ) {
+  public BlocksWithLocations(BlockWithLocations[] blocks) {
     this.blocks = blocks;
   }
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto

@@ -274,7 +274,7 @@ message BlockProto {
  */
 message BlockWithLocationsProto {
   required BlockProto block = 1;   // Block
-  repeated string datanodeIDs = 2; // Datanodes with replicas of the block
+  repeated string storageIDs = 2;  // Datanodes with replicas of the block
 }
 
 /**

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java

@@ -101,18 +101,18 @@ public class TestGetBlocks extends TestCase {
       BlockWithLocations[] locs;
       locs = namenode.getBlocks(dataNodes[0], fileLen).getBlocks();
       assertEquals(locs.length, 2);
-      assertEquals(locs[0].getDatanodes().length, 2);
-      assertEquals(locs[1].getDatanodes().length, 2);
+      assertEquals(locs[0].getStorageIDs().length, 2);
+      assertEquals(locs[1].getStorageIDs().length, 2);
 
       // get blocks of size BlockSize from dataNodes[0]
       locs = namenode.getBlocks(dataNodes[0], DEFAULT_BLOCK_SIZE).getBlocks();
       assertEquals(locs.length, 1);
-      assertEquals(locs[0].getDatanodes().length, 2);
+      assertEquals(locs[0].getStorageIDs().length, 2);
 
       // get blocks of size 1 from dataNodes[0]
       locs = namenode.getBlocks(dataNodes[0], 1).getBlocks();
       assertEquals(locs.length, 1);
-      assertEquals(locs[0].getDatanodes().length, 2);
+      assertEquals(locs[0].getStorageIDs().length, 2);
 
       // get blocks of size 0 from dataNodes[0]
       getBlocksWithException(namenode, dataNodes[0], 0);     

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java

@@ -161,7 +161,7 @@ public class TestPBHelper {
 
   private void compare(BlockWithLocations locs1, BlockWithLocations locs2) {
     assertEquals(locs1.getBlock(), locs2.getBlock());
-    assertTrue(Arrays.equals(locs1.getDatanodes(), locs2.getDatanodes()));
+    assertTrue(Arrays.equals(locs1.getStorageIDs(), locs2.getStorageIDs()));
   }
 
   @Test