HDFS-9141. Thread leak in Datanode#refreshVolumes. (Uma Maheswara Rao G via yliu)

This commit is contained in:
yliu 2015-09-29 22:06:20 +08:00
parent 9c47ab32b1
commit 28805cb8d7
2 changed files with 9 additions and 5 deletions

View File

@ -1106,6 +1106,9 @@ Release 2.8.0 - UNRELEASED
HDFS-9092. Nfs silently drops overlapping write requests and causes data
copying to fail. (Yongjun Zhang)
HDFS-9141. Thread leak in Datanode#refreshVolumes. (Uma Maheswara Rao G
via yliu)
Release 2.7.2 - UNRELEASED
INCOMPATIBLE CHANGES

View File

@ -25,9 +25,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_PERMISS
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_PERMISSION_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_INTERFACE_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_INTERFACE_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_NAMESERVER_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_NAMESERVER_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_KEY;
@ -611,7 +609,7 @@ public class DataNode extends ReconfigurableBase
private synchronized void refreshVolumes(String newVolumes) throws IOException {
Configuration conf = getConf();
conf.set(DFS_DATANODE_DATA_DIR_KEY, newVolumes);
ExecutorService service = null;
int numOldDataDirs = dataDirs.size();
ChangedVolumes changedVolumes = parseChangedVolumes(newVolumes);
StringBuilder errorMessageBuilder = new StringBuilder();
@ -634,8 +632,8 @@ public class DataNode extends ReconfigurableBase
for (BPOfferService bpos : blockPoolManager.getAllNamenodeThreads()) {
nsInfos.add(bpos.getNamespaceInfo());
}
ExecutorService service = Executors.newFixedThreadPool(
changedVolumes.newLocations.size());
service = Executors
.newFixedThreadPool(changedVolumes.newLocations.size());
List<Future<IOException>> exceptions = Lists.newArrayList();
for (final StorageLocation location : changedVolumes.newLocations) {
exceptions.add(service.submit(new Callable<IOException>() {
@ -685,6 +683,9 @@ public class DataNode extends ReconfigurableBase
throw new IOException(errorMessageBuilder.toString());
}
} finally {
if (service != null) {
service.shutdown();
}
conf.set(DFS_DATANODE_DATA_DIR_KEY,
Joiner.on(",").join(effectiveVolumes));
dataDirs = getStorageLocations(conf);