فهرست منبع

HDFS-5494. Fix findbugs warnings for HDFS-2832.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2832@1540924 13f79535-47bb-0310-9956-ffa450edef68
Arpit Agarwal 11 سال پیش
والد
کامیت
e5b3171c3b

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

@@ -97,3 +97,5 @@ IMPROVEMENTS:
 
     HDFS-5491. Update editsStored for HDFS-2832. (Arpit Agarwal)
 
+    HDFS-5494. Fix findbugs warnings for HDFS-2832. (Arpit Agarwal)
+

+ 38 - 27
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java

@@ -98,7 +98,7 @@ class BPServiceActor implements Runnable {
    * reported to the NN. Access should be synchronized on this object.
    */
   private final Map<String, PerStoragePendingIncrementalBR>
-      pendingIncrementalBRperStorage = Maps.newConcurrentMap();
+      pendingIncrementalBRperStorage = Maps.newHashMap();
 
   private volatile int pendingReceivedRequests = 0;
   private volatile boolean shouldServiceRun = true;
@@ -266,43 +266,54 @@ class BPServiceActor implements Runnable {
   }
   
   /**
-   * Report received blocks and delete hints to the Namenode
+   * Report received blocks and delete hints to the Namenode for each
+   * storage.
+   *
    * @throws IOException
    */
   private void reportReceivedDeletedBlocks() throws IOException {
-    // For each storage, check if there are newly received blocks and if
-    // so then send an incremental report to the NameNode.
-    for (Map.Entry<String, PerStoragePendingIncrementalBR> entry :
-        pendingIncrementalBRperStorage.entrySet()) {
-      final String storageUuid = entry.getKey();
-      final PerStoragePendingIncrementalBR perStorageMap = entry.getValue();
-      ReceivedDeletedBlockInfo[] receivedAndDeletedBlockArray = null;
-      // TODO: We can probably use finer-grained synchronization now.
-      synchronized (pendingIncrementalBRperStorage) {
+
+    // Generate a list of the pending reports for each storage under the lock
+    Map<String, ReceivedDeletedBlockInfo[]> blockArrays = Maps.newHashMap();
+    synchronized (pendingIncrementalBRperStorage) {
+      for (Map.Entry<String, PerStoragePendingIncrementalBR> entry :
+           pendingIncrementalBRperStorage.entrySet()) {
+        final String storageUuid = entry.getKey();
+        final PerStoragePendingIncrementalBR perStorageMap = entry.getValue();
+        ReceivedDeletedBlockInfo[] receivedAndDeletedBlockArray = null;
+
         if (perStorageMap.getBlockInfoCount() > 0) {
           // Send newly-received and deleted blockids to namenode
           receivedAndDeletedBlockArray = perStorageMap.dequeueBlockInfos();
           pendingReceivedRequests -= receivedAndDeletedBlockArray.length;
+          blockArrays.put(storageUuid, receivedAndDeletedBlockArray);
         }
       }
+    }
 
-      if (receivedAndDeletedBlockArray != null) {
-        StorageReceivedDeletedBlocks[] report = { new StorageReceivedDeletedBlocks(
-            storageUuid, receivedAndDeletedBlockArray) };
-        boolean success = false;
-        try {
-          bpNamenode.blockReceivedAndDeleted(bpRegistration, bpos.getBlockPoolId(),
-              report);
-          success = true;
-        } finally {
+    // Send incremental block reports to the Namenode outside the lock
+    for (Map.Entry<String, ReceivedDeletedBlockInfo[]> entry :
+         blockArrays.entrySet()) {
+      final String storageUuid = entry.getKey();
+      final ReceivedDeletedBlockInfo[] rdbi = entry.getValue();
+
+      StorageReceivedDeletedBlocks[] report = { new StorageReceivedDeletedBlocks(
+          storageUuid, rdbi) };
+      boolean success = false;
+      try {
+        bpNamenode.blockReceivedAndDeleted(bpRegistration,
+            bpos.getBlockPoolId(), report);
+        success = true;
+      } finally {
+        if (!success) {
           synchronized (pendingIncrementalBRperStorage) {
-            if (!success) {
-              // If we didn't succeed in sending the report, put all of the
-              // blocks back onto our queue, but only in the case where we
-              // didn't put something newer in the meantime.
-              perStorageMap.putMissingBlockInfos(receivedAndDeletedBlockArray);
-              pendingReceivedRequests += perStorageMap.getBlockInfoCount();
-            }
+            // If we didn't succeed in sending the report, put all of the
+            // blocks back onto our queue, but only in the case where we
+            // didn't put something newer in the meantime.
+            PerStoragePendingIncrementalBR perStorageMap =
+                pendingIncrementalBRperStorage.get(storageUuid);
+            perStorageMap.putMissingBlockInfos(rdbi);
+            pendingReceivedRequests += perStorageMap.getBlockInfoCount();
           }
         }
       }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java

@@ -293,7 +293,7 @@ public class DataStorage extends Storage {
 
     if (LayoutVersion.supports(Feature.ADD_DATANODE_AND_STORAGE_UUIDS,
           layoutVersion) && datanodeUuid != null) {
-      props.setProperty("datanodeUuid", datanodeUuid);
+      props.setProperty("datanodeUuid", getDatanodeUuid());
     }
 
     // Set NamespaceID in version before federation
@@ -348,7 +348,7 @@ public class DataStorage extends Storage {
       } else if (getDatanodeUuid().compareTo(dnUuid) != 0) {
         throw new InconsistentFSStateException(sd.getRoot(),
             "Root " + sd.getRoot() + ": DatanodeUuid=" + dnUuid +
-            ", does not match " + datanodeUuid + " from other" +
+            ", does not match " + getDatanodeUuid() + " from other" +
             " StorageDirectory.");
       }
     }