瀏覽代碼

HDFS-5232. Protocol changes to transmit StorageUuid.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2832@1525153 13f79535-47bb-0310-9956-ffa450edef68
Arpit Agarwal 11 年之前
父節點
當前提交
eb9f1b6707

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES_HDFS-2832.txt

@@ -26,3 +26,5 @@ IMPROVEMENTS:
 
     HDFS-4990. Change BlockPlacementPolicy to choose storages instead of
     datanodes.  (szetszwo)
+
+    HDFS-5232. Protocol changes to transmit StorageUuid. (Arpit Agarwal)

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -405,7 +405,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       throws ServiceException {
     try {
       List<DatanodeInfoProto> existingList = req.getExistingsList();
-      List<String> existingStorageIDsList = req.getExistingStorageIDsList();
+      List<String> existingStorageIDsList = req.getExistingStorageUuidsList();
       List<DatanodeInfoProto> excludesList = req.getExcludesList();
       LocatedBlock result = server.getAdditionalDatanode(req.getSrc(),
           PBHelper.convert(req.getBlk()),

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -345,7 +345,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
         .setSrc(src)
         .setBlk(PBHelper.convert(blk))
         .addAllExistings(PBHelper.convert(existings))
-        .addAllExistingStorageIDs(Arrays.asList(existingStorageIDs))
+        .addAllExistingStorageUuids(Arrays.asList(existingStorageIDs))
         .addAllExcludes(PBHelper.convert(excludes))
         .setNumAdditionalNodes(numAdditionalNodes)
         .setClientName(clientName)

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java

@@ -213,7 +213,7 @@ public class DatanodeProtocolClientSideTranslatorPB implements
     for (StorageReceivedDeletedBlocks storageBlock : receivedAndDeletedBlocks) {
       StorageReceivedDeletedBlocksProto.Builder repBuilder = 
           StorageReceivedDeletedBlocksProto.newBuilder();
-      repBuilder.setStorageID(storageBlock.getStorageID());
+      repBuilder.setStorageUuid(storageBlock.getStorageID());
       for (ReceivedDeletedBlockInfo rdBlock : storageBlock.getBlocks()) {
         repBuilder.addBlocks(PBHelper.convert(rdBlock));
       }

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

@@ -104,7 +104,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements
       StorageReport[] report = new StorageReport[list.size()];
       int i = 0;
       for (StorageReportProto p : list) {
-        report[i++] = new StorageReport(p.getStorageID(), p.getFailed(),
+        report[i++] = new StorageReport(p.getStorageUuid(), p.getFailed(),
             p.getCapacity(), p.getDfsUsed(), p.getRemaining(),
             p.getBlockPoolUsed());
       }
@@ -174,7 +174,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements
       for (int j = 0; j < list.size(); j++) {
         rdBlocks[j] = PBHelper.convert(list.get(j));
       }
-      info[i] = new StorageReceivedDeletedBlocks(sBlock.getStorageID(), rdBlocks);
+      info[i] = new StorageReceivedDeletedBlocks(sBlock.getStorageUuid(), rdBlocks);
     }
     try {
       impl.blockReceivedAndDeleted(PBHelper.convert(request.getRegistration()),

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java

@@ -82,6 +82,6 @@ public class InterDatanodeProtocolServerSideTranslatorPB implements
       throw new ServiceException(e);
     }
     return UpdateReplicaUnderRecoveryResponseProto.newBuilder()
-        .setStorageID(storageID).build();
+        .setStorageUuid(storageID).build();
   }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolTranslatorPB.java

@@ -109,7 +109,7 @@ public class InterDatanodeProtocolTranslatorPB implements
         .setNewLength(newLength).setRecoveryId(recoveryId).build();
     try {
       return rpcProxy.updateReplicaUnderRecovery(NULL_CONTROLLER, req
-          ).getStorageID();
+          ).getStorageUuid();
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }

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

@@ -94,7 +94,7 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto.File
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto.Builder;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlocksProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageIDsProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageUuidsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto;
@@ -226,7 +226,7 @@ public class PBHelper {
 
   // DatanodeId
   public static DatanodeID convert(DatanodeIDProto dn) {
-    return new DatanodeID(dn.getIpAddr(), dn.getHostName(), dn.getStorageID(),
+    return new DatanodeID(dn.getIpAddr(), dn.getHostName(), dn.getDatanodeUuid(),
         dn.getXferPort(), dn.getInfoPort(), dn.getIpcPort());
   }
 
@@ -234,7 +234,7 @@ public class PBHelper {
     return DatanodeIDProto.newBuilder()
         .setIpAddr(dn.getIpAddr())
         .setHostName(dn.getHostName())
-        .setStorageID(dn.getStorageID())
+        .setDatanodeUuid(dn.getStorageID())
         .setXferPort(dn.getXferPort())
         .setInfoPort(dn.getInfoPort())
         .setIpcPort(dn.getIpcPort()).build();
@@ -276,11 +276,11 @@ public class PBHelper {
   public static BlockWithLocationsProto convert(BlockWithLocations blk) {
     return BlockWithLocationsProto.newBuilder()
         .setBlock(convert(blk.getBlock()))
-        .addAllStorageIDs(Arrays.asList(blk.getStorageIDs())).build();
+        .addAllStorageUuids(Arrays.asList(blk.getStorageIDs())).build();
   }
 
   public static BlockWithLocations convert(BlockWithLocationsProto b) {
-    return new BlockWithLocations(convert(b.getBlock()), b.getStorageIDsList()
+    return new BlockWithLocations(convert(b.getBlock()), b.getStorageUuidsList()
         .toArray(new String[0]));
   }
 
@@ -746,7 +746,7 @@ public class PBHelper {
       builder.addBlocks(PBHelper.convert(blocks[i]));
     }
     builder.addAllTargets(convert(cmd.getTargets()))
-           .addAllTargetStorageIDs(convert(cmd.getTargetStorageIDs()));
+           .addAllTargetStorageUuids(convert(cmd.getTargetStorageIDs()));
     return builder.build();
   }
 
@@ -759,11 +759,11 @@ public class PBHelper {
     return Arrays.asList(ret);
   }
 
-  private static List<StorageIDsProto> convert(String[][] targetStorageIDs) {
-    StorageIDsProto[] ret = new StorageIDsProto[targetStorageIDs.length];
-    for (int i = 0; i < targetStorageIDs.length; i++) {
-      ret[i] = StorageIDsProto.newBuilder()
-          .addAllStorageIDs(Arrays.asList(targetStorageIDs[i])).build();
+  private static List<StorageUuidsProto> convert(String[][] targetStorageUuids) {
+    StorageUuidsProto[] ret = new StorageUuidsProto[targetStorageUuids.length];
+    for (int i = 0; i < targetStorageUuids.length; i++) {
+      ret[i] = StorageUuidsProto.newBuilder()
+          .addAllStorageUuids(Arrays.asList(targetStorageUuids[i])).build();
     }
     return Arrays.asList(ret);
   }
@@ -843,10 +843,10 @@ public class PBHelper {
       targets[i] = PBHelper.convert(targetList.get(i));
     }
 
-    List<StorageIDsProto> targetStorageIDsList = blkCmd.getTargetStorageIDsList();
-    String[][] targetStorageIDs = new String[targetStorageIDsList.size()][];
+    List<StorageUuidsProto> targetStorageUuidsList = blkCmd.getTargetStorageUuidsList();
+    String[][] targetStorageIDs = new String[targetStorageUuidsList.size()][];
     for(int i = 0; i < targetStorageIDs.length; i++) {
-      List<String> storageIDs = targetStorageIDsList.get(i).getStorageIDsList();
+      List<String> storageIDs = targetStorageUuidsList.get(i).getStorageUuidsList();
       targetStorageIDs[i] = storageIDs.toArray(new String[storageIDs.size()]);
     }
 
@@ -1375,7 +1375,7 @@ public class PBHelper {
     return DatanodeStorageProto.newBuilder()
         .setState(PBHelper.convertState(s.getState()))
         .setStorageType(PBHelper.convertStorageType(s.getStorageType()))
-        .setStorageID(s.getStorageID()).build();
+        .setStorageUuid(s.getStorageID()).build();
   }
 
   private static StorageState convertState(State state) {
@@ -1406,11 +1406,11 @@ public class PBHelper {
 
   public static DatanodeStorage convert(DatanodeStorageProto s) {
     if (s.hasStorageType()) {
-      return new DatanodeStorage(s.getStorageID(),
+      return new DatanodeStorage(s.getStorageUuid(),
                                  PBHelper.convertState(s.getState()),
                                  PBHelper.convertType(s.getStorageType()));
     } else {
-      return new DatanodeStorage(s.getStorageID(),
+      return new DatanodeStorage(s.getStorageUuid(),
                                  PBHelper.convertState(s.getState()));
     }
   }
@@ -1440,7 +1440,7 @@ public class PBHelper {
     return StorageReportProto.newBuilder()
         .setBlockPoolUsed(r.getBlockPoolUsed()).setCapacity(r.getCapacity())
         .setDfsUsed(r.getDfsUsed()).setRemaining(r.getRemaining())
-        .setStorageID(r.getStorageID()).build();
+        .setStorageUuid(r.getStorageID()).build();
   }
 
   public static JournalInfo convert(JournalInfoProto info) {

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

@@ -141,7 +141,7 @@ message GetAdditionalDatanodeRequestProto {
   repeated DatanodeInfoProto excludes = 4;
   required uint32 numAdditionalNodes = 5;
   required string clientName = 6;
-  repeated string existingStorageIDs = 7;
+  repeated string existingStorageUuids = 7;
 }
 
 message GetAdditionalDatanodeResponseProto {

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto

@@ -53,7 +53,7 @@ message DatanodeStorageProto {
     READ_ONLY = 1;
   }
 
-  required string storageID = 1;    // Unique identifier for the storage
+  required string storageUuid = 1;
   optional StorageState state = 2 [default = NORMAL];
   optional StorageTypeProto storageType = 3;
 }
@@ -110,7 +110,7 @@ message BlockCommandProto {
   required string blockPoolId = 2;
   repeated BlockProto blocks = 3;
   repeated DatanodeInfosProto targets = 4;
-  repeated StorageIDsProto targetStorageIDs = 5;
+  repeated StorageUuidsProto targetStorageUuids = 5;
 }
 
 /**
@@ -176,7 +176,7 @@ message HeartbeatRequestProto {
 }
 
 message StorageReportProto {
-  required string storageID = 1;
+  required string storageUuid = 1;
   optional bool failed = 2 [ default = false ];
   optional uint64 capacity = 3 [ default = 0 ];
   optional uint64 dfsUsed = 4 [ default = 0 ];
@@ -250,7 +250,7 @@ message ReceivedDeletedBlockInfoProto {
  * List of blocks received and deleted for a storage.
  */
 message StorageReceivedDeletedBlocksProto {
-  required string storageID = 1;
+  required string storageUuid = 1;
   repeated ReceivedDeletedBlockInfoProto blocks = 2;
 }
 

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

@@ -65,7 +65,7 @@ message UpdateReplicaUnderRecoveryRequestProto {
  * Response returns updated block information
  */
 message UpdateReplicaUnderRecoveryResponseProto {
-  required string storageID = 1; // ID of the storage that stores replica
+  optional string storageUuid = 1; // ID of the storage that stores replica
 }
 
 /**

+ 7 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto

@@ -50,7 +50,10 @@ message ExtendedBlockProto {
 message DatanodeIDProto {
   required string ipAddr = 1;    // IP address
   required string hostName = 2;  // hostname
-  required string storageID = 3; // unique storage id
+  required string datanodeUuid = 3;     // UUID assigned to the Datanode. For
+                                        // upgraded clusters this is the same
+                                        // as the original StorageID of the
+                                        // Datanode.
   required uint32 xferPort = 4;  // data streaming port
   required uint32 infoPort = 5;  // info server port
   required uint32 ipcPort = 6;   // ipc server port
@@ -124,8 +127,8 @@ enum StorageTypeProto {
 /**
  * A list of storage IDs. 
  */
-message StorageIDsProto {
-  repeated string storageIDs = 1;
+message StorageUuidsProto {
+  repeated string storageUuids = 1;
 }
 
 /**
@@ -348,7 +351,7 @@ message BlockProto {
  */
 message BlockWithLocationsProto {
   required BlockProto block = 1;   // Block
-  repeated string storageIDs = 2;  // Datanodes with replicas of the block
+  repeated string storageUuids = 2;  // Datanodes with replicas of the block
 }
 
 /**