[ https://issues.apache.org/jira/browse/HDFS-14295?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
BELUGA BEHR updated HDFS-14295: ------------------------------- Description: When a DataNode data transfers a block, is spins up a new thread for each transfer. [Here|https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java#L2339] and [Here|https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java#L3019-L3022]. Instead, add the threads to a {{CachedThreadPool}} so that when their threads complete the transfer, they can be re-used for another transfer. This should save resources spent on creating and spinning up transfer threads. One thing I'll point out that's a bit off, which I address in this patch, ... There are two places in the code where a {{DataTransfer}} thread is started. In [one place|https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java#L2339-L2341], it's started in a default thread group. In [another place|https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java#L3019-L3022], it's started in the [dataXceiverServer|https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java#L1164] thread group. I do not think it's correct to include any of these threads in the {{dataXceiverServer}} thread group. Anything submitted to the {{dataXceiverServer}} should probably be tied to the {{dfs.datanode.max.transfer.threads}} configurations, and neither of these methods are. Instead, they should be submitted into the same thread pool with its own thread group (probably the default thread group, unless someone suggests otherwise) and is what I have included in this patch. was:When a DataNode data transfers a block, is spins up a new thread for each transfer. [Here|https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java#L2339] and [Here|https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java#L3019-L3022]. Instead, add the threads to a {{CachedThreadPool}} so that when their threads complete the transfer, they can be re-used for another transfer. This should save resources spent on creating and spinning up transfer threads. > Add Threadpool for DataTransfers > -------------------------------- > > Key: HDFS-14295 > URL: https://issues.apache.org/jira/browse/HDFS-14295 > Project: Hadoop HDFS > Issue Type: Improvement > Components: datanode > Affects Versions: 3.2.0 > Reporter: BELUGA BEHR > Assignee: BELUGA BEHR > Priority: Major > Attachments: HDFS-14295.1.patch, HDFS-14295.2.patch, > HDFS-14295.3.patch, HDFS-14295.3.patch > > > When a DataNode data transfers a block, is spins up a new thread for each > transfer. > [Here|https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java#L2339] > and > [Here|https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java#L3019-L3022]. > Instead, add the threads to a {{CachedThreadPool}} so that when their > threads complete the transfer, they can be re-used for another transfer. This > should save resources spent on creating and spinning up transfer threads. > One thing I'll point out that's a bit off, which I address in this patch, ... > There are two places in the code where a {{DataTransfer}} thread is started. > In [one > place|https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java#L2339-L2341], > it's started in a default thread group. In [another > place|https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java#L3019-L3022], > it's started in the > [dataXceiverServer|https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java#L1164] > thread group. > I do not think it's correct to include any of these threads in the > {{dataXceiverServer}} thread group. Anything submitted to the > {{dataXceiverServer}} should probably be tied to the > {{dfs.datanode.max.transfer.threads}} configurations, and neither of these > methods are. Instead, they should be submitted into the same thread pool with > its own thread group (probably the default thread group, unless someone > suggests otherwise) and is what I have included in this patch. -- This message was sent by Atlassian JIRA (v7.6.3#76005) --------------------------------------------------------------------- To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org