瀏覽代碼

HDFS-11805. Ensure LevelDB DBIterator is closed. Contributed by Chen Liang.

Xiaoyu Yao 8 年之前
父節點
當前提交
b8e063ebcb

+ 13 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/cblock/CBlockManager.java

@@ -74,7 +74,6 @@ import static org.apache.hadoop.cblock.CBlockConfigKeys.DFS_CBLOCK_SERVICERPC_HA
 import static org.apache.hadoop.cblock.CBlockConfigKeys.DFS_CBLOCK_SERVICERPC_HANDLER_COUNT_KEY;
 import static org.apache.hadoop.cblock.CBlockConfigKeys.DFS_CBLOCK_SERVICE_LEVELDB_PATH_DEFAULT;
 import static org.apache.hadoop.cblock.CBlockConfigKeys.DFS_CBLOCK_SERVICE_LEVELDB_PATH_KEY;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT;
 
 /**
  * The main entry point of CBlock operations, ALL the CBlock operations
@@ -269,18 +268,19 @@ public class CBlockManager implements CBlockServiceProtocol,
     levelDBStore.delete(key);
   }
 
-  public void readFromPersistentStore() {
-    DBIterator iter = levelDBStore.getIterator();
-    iter.seekToFirst();
-    while (iter.hasNext()) {
-      Map.Entry<byte[], byte[]> entry = iter.next();
-      String volumeKey = new String(entry.getKey(), encoding);
-      try {
-        VolumeDescriptor volumeDescriptor =
-            VolumeDescriptor.fromProtobuf(entry.getValue());
-        storageManager.addVolume(volumeDescriptor);
-      } catch (IOException e) {
-        LOG.error("Loading volume " + volumeKey + " error " + e);
+  public void readFromPersistentStore() throws IOException {
+    try (DBIterator iter = levelDBStore.getIterator()) {
+      iter.seekToFirst();
+      while (iter.hasNext()) {
+        Map.Entry<byte[], byte[]> entry = iter.next();
+        String volumeKey = new String(entry.getKey(), encoding);
+        try {
+          VolumeDescriptor volumeDescriptor =
+              VolumeDescriptor.fromProtobuf(entry.getValue());
+          storageManager.addVolume(volumeDescriptor);
+        } catch (IOException e) {
+          LOG.error("Loading volume " + volumeKey + " error " + e);
+        }
       }
     }
   }

+ 29 - 28
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/localstorage/OzoneMetadataManager.java

@@ -452,45 +452,46 @@ public final class OzoneMetadataManager {
     String prevKey = args.getPrevKey();
     int maxCount = args.getMaxKeys();
     String userName = null;
-    DBIterator iterator = this.userDB.getDB().iterator();
+    try (DBIterator iterator = this.userDB.getDB().iterator()) {
 
-    if (prevKey != null) {
-      // Format is username/volumeName
+      if (prevKey != null) {
+        // Format is username/volumeName
+
+        String[] volName = args.getPrevKey().split("/");
+        if (volName.length < 2) {
+          throw ErrorTable.newError(ErrorTable.USER_NOT_FOUND, args.getArgs());
+        }
+        seekToUser(iterator, volName[0]);
+        userName = new String(iterator.peekNext().getKey(), encoding);
+        prevKey = volName[1];
+      } else {
+        userName = getFirstUser(iterator);
+      }
 
-      String[] volName = args.getPrevKey().split("/");
-      if (volName.length < 2) {
+      if (userName == null || userName.isEmpty()) {
         throw ErrorTable.newError(ErrorTable.USER_NOT_FOUND, args.getArgs());
       }
-      seekToUser(iterator, volName[0]);
-      userName = new String(iterator.peekNext().getKey(), encoding);
-      prevKey = volName[1];
-    } else {
-      userName = getFirstUser(iterator);
-    }
 
-    if (userName == null || userName.isEmpty()) {
-      throw ErrorTable.newError(ErrorTable.USER_NOT_FOUND, args.getArgs());
-    }
+      ListVolumes returnSet = new ListVolumes();
+      int count = maxCount - returnSet.getVolumes().size();
 
-    ListVolumes returnSet = new ListVolumes();
-    int count = maxCount - returnSet.getVolumes().size();
+      // we need to iterate through users until we get maxcount volumes
+      // or no more volumes are left.
+      while (iterator.hasNext() && count > 0) {
 
-    // we need to iterate through users until we get maxcount volumes
-    // or no more volumes are left.
-    while (iterator.hasNext() && count > 0) {
+        userName = new String(iterator.next().getKey(), encoding);
 
-      userName = new String(iterator.next().getKey(), encoding);
+        byte[] volumeList = userDB.get(userName.getBytes(encoding));
+        if (volumeList == null) {
+          throw ErrorTable.newError(ErrorTable.USER_NOT_FOUND, args.getArgs());
+        }
 
-      byte[] volumeList = userDB.get(userName.getBytes(encoding));
-      if (volumeList == null) {
-        throw ErrorTable.newError(ErrorTable.USER_NOT_FOUND, args.getArgs());
+        returnSet.getVolumes().addAll(getFilteredVolumes(
+            volumeList, prefix, prevKey, count).getVolumes());
+        count = maxCount - returnSet.getVolumes().size();
       }
-
-      returnSet.getVolumes().addAll(
-          getFilteredVolumes(volumeList, prefix, prevKey, count).getVolumes());
-      count = maxCount - returnSet.getVolumes().size();
+      return returnSet;
     }
-    return returnSet;
   }
 
   /**