Repository: hadoop Updated Branches: refs/heads/trunk d6fa34e01 -> 715dbddf7
HDFS-9141. Thread leak in Datanode#refreshVolumes. (Uma Maheswara Rao G via yliu) Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/715dbddf Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/715dbddf Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/715dbddf Branch: refs/heads/trunk Commit: 715dbddf77866bb47a4b95421091f64a3785444f Parents: d6fa34e Author: yliu <[email protected]> Authored: Tue Sep 29 22:05:34 2015 +0800 Committer: yliu <[email protected]> Committed: Tue Sep 29 22:05:34 2015 +0800 ---------------------------------------------------------------------- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 3 +++ .../org/apache/hadoop/hdfs/server/datanode/DataNode.java | 11 ++++++----- 2 files changed, 9 insertions(+), 5 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hadoop/blob/715dbddf/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 182464b..2c90b23 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -1453,6 +1453,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 http://git-wip-us.apache.org/repos/asf/hadoop/blob/715dbddf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java index 2646089..2fe67fd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java @@ -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; @@ -604,7 +602,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(); @@ -627,8 +625,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>() { @@ -678,6 +676,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);
