HDFS-11805. Ensure LevelDB DBIterator is closed. Contributed by Chen Liang.
This commit is contained in:
parent
7e8b3e2540
commit
b8e063ebcb
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
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);
|
||||
}
|
||||
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();
|
||||
|
||||
// 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);
|
||||
|
||||
byte[] volumeList = userDB.get(userName.getBytes(encoding));
|
||||
if (volumeList == null) {
|
||||
if (userName == null || userName.isEmpty()) {
|
||||
throw ErrorTable.newError(ErrorTable.USER_NOT_FOUND, args.getArgs());
|
||||
}
|
||||
|
||||
returnSet.getVolumes().addAll(
|
||||
getFilteredVolumes(volumeList, prefix, prevKey, count).getVolumes());
|
||||
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) {
|
||||
|
||||
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());
|
||||
}
|
||||
|
||||
returnSet.getVolumes().addAll(getFilteredVolumes(
|
||||
volumeList, prefix, prevKey, count).getVolumes());
|
||||
count = maxCount - returnSet.getVolumes().size();
|
||||
}
|
||||
return returnSet;
|
||||
}
|
||||
return returnSet;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
Loading…
Reference in New Issue