[ 
https://issues.apache.org/jira/browse/HDFS-14292?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16775165#comment-16775165
 ] 

BELUGA BEHR commented on HDFS-14292:
------------------------------------

Hello Watchers.

There are three 3 checkstyle warnings.  Please ignore these.  To clear one of 
them, the fix breaks unit tests.  The others are minor infractions and are 
artifacts of the existing code base. 

The unit test "TestNamenodeCapacityReport" is failing because it is getting 
incorrect XCeiver numbers.  I'm not sure where this failure is coming from, it 
passes on my local machine, but it could be addressed by other changes such as 
HDFS-14295 because the thread group that this thread pool uses is also used in 
other areas of the DataNode and therefore are not controlled or limited by this 
pool.

 

Please consider the latest patch for inclusion into the project.

> Introduce Java ExecutorService to DataXceiverServer
> ---------------------------------------------------
>
>                 Key: HDFS-14292
>                 URL: https://issues.apache.org/jira/browse/HDFS-14292
>             Project: Hadoop HDFS
>          Issue Type: Improvement
>          Components: datanode
>    Affects Versions: 3.2.0
>            Reporter: BELUGA BEHR
>            Assignee: BELUGA BEHR
>            Priority: Major
>         Attachments: HDFS-14292.1.patch, HDFS-14292.2.patch, 
> HDFS-14292.3.patch, HDFS-14292.4.patch, HDFS-14292.5.patch, 
> HDFS-14292.6.patch, HDFS-14292.6.patch, HDFS-14292.7.patch
>
>
> I wanted to investigate {{dfs.datanode.max.transfer.threads}} from 
> {{hdfs-site.xml}}.  It is described as "Specifies the maximum number of 
> threads to use for transferring data in and out of the DN."   The default 
> value is 4096.  I found it interesting because 4096 threads sounds like a lot 
> to me.  I'm not sure how a system with 8-16 cores would react to this large a 
> thread count.  Intuitively, I would say that the overhead of context 
> switching would be immense.
> During mt investigation, I discovered the 
> [following|https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java#L203-L216]
>  setup in the {{DataXceiverServer}} class:
> # A peer connects to a DataNode
> # A new thread is spun up to service this connection
> # The thread runs to completion
> # The tread dies
> It would perhaps be better if we used a thread pool to better manage the 
> lifecycle of the service threads and to allow the DataNode to re-use existing 
> threads, saving on the need to create and spin-up threads on demand.
> In this JIRA, I have added a couple of things:
> # Added a thread pool to {{DataXceiverServer}} class that, on demand, will 
> create up to {{dfs.datanode.max.transfer.threads}}.  A thread that has 
> completed its prior duties will stay idle for up to 60 seconds 
> (configurable), it will be retired if no new work has arrived.
> # Added new methods to the {{Peer}} Interface to allow for better logging and 
> less code within each Thread ({{DataXceiver}}).
> # Updated the Thread code ({{DataXceiver}}) regarding its interactions with 
> {{blockReceiver}} instance variable



--
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

Reply via email to