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 2c1a8cd3339e7..e4404778591cb 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 @@ -213,6 +213,7 @@ import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Timer; import org.apache.hadoop.util.VersionInfo; +import org.apache.hadoop.util.concurrent.HadoopExecutors; import org.apache.htrace.core.Tracer; import org.eclipse.jetty.util.ajax.JSON; @@ -395,6 +396,8 @@ public static InetSocketAddress createSocketAddr(String target) { private static final double CONGESTION_RATIO = 1.5; private DiskBalancer diskBalancer; + private final ExecutorService xferService; + @Nullable private final StorageLocationChecker storageLocationChecker; @@ -435,6 +438,8 @@ private static Tracer createTracer(Configuration conf) { initOOBTimeout(); storageLocationChecker = null; volumeChecker = new DatasetVolumeChecker(conf, new Timer()); + this.xferService = + HadoopExecutors.newCachedThreadPool(new Daemon.DaemonFactory()); } /** @@ -475,6 +480,8 @@ private static Tracer createTracer(Configuration conf) { conf.get("hadoop.hdfs.configuration.version", "UNSPECIFIED"); this.volumeChecker = new DatasetVolumeChecker(conf, new Timer()); + this.xferService = + HadoopExecutors.newCachedThreadPool(new Daemon.DaemonFactory()); // Determine whether we should try to pass file descriptors to clients. if (conf.getBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, @@ -2056,6 +2063,9 @@ public void shutdown() { // wait reconfiguration thread, if any, to exit shutdownReconfigurationTask(); + LOG.info("Waiting up to 30 seconds for transfer threads to complete"); + HadoopExecutors.shutdown(this.xferService, LOG, 15L, TimeUnit.SECONDS); + // wait for all data receiver threads to exit if (this.threadGroup != null) { int sleepMs = 2; @@ -2329,16 +2339,16 @@ void transferBlock(ExtendedBlock block, DatanodeInfo[] xferTargets, int numTargets = xferTargets.length; if (numTargets > 0) { - StringBuilder xfersBuilder = new StringBuilder(); - for (int i = 0; i < numTargets; i++) { - xfersBuilder.append(xferTargets[i]).append(" "); - } - LOG.info(bpReg + " Starting thread to transfer " + - block + " to " + xfersBuilder); + final String xferTargetsString = + StringUtils.join(" ", Arrays.asList(xferTargets)); + LOG.info("{} Starting thread to transfer {} to {}", bpReg, block, + xferTargetsString); + + final DataTransfer dataTransferTask = new DataTransfer(xferTargets, + xferTargetStorageTypes, xferTargetStorageIDs, block, + BlockConstructionStage.PIPELINE_SETUP_CREATE, ""); - new Daemon(new DataTransfer(xferTargets, xferTargetStorageTypes, - xferTargetStorageIDs, block, - BlockConstructionStage.PIPELINE_SETUP_CREATE, "")).start(); + this.xferService.execute(dataTransferTask); } } @@ -3016,15 +3026,22 @@ void transferReplicaForPipelineRecovery(final ExtendedBlock b, b.setNumBytes(visible); if (targets.length > 0) { - Daemon daemon = new Daemon(threadGroup, - new DataTransfer(targets, targetStorageTypes, targetStorageIds, b, - stage, client)); - daemon.start(); + if (LOG.isDebugEnabled()) { + final String xferTargetsString = + StringUtils.join(" ", Arrays.asList(targets)); + LOG.debug("Transferring a replica to {}", xferTargetsString); + } + + final DataTransfer dataTransferTask = new DataTransfer(targets, + targetStorageTypes, targetStorageIds, b, stage, client); + + @SuppressWarnings("unchecked") + Future f = (Future) this.xferService.submit(dataTransferTask); try { - daemon.join(); - } catch (InterruptedException e) { - throw new IOException( - "Pipeline recovery for " + b + " is interrupted.", e); + f.get(); + } catch (InterruptedException | ExecutionException e) { + throw new IOException("Pipeline recovery for " + b + " is interrupted.", + e); } } }