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

Chris Nauroth commented on HDFS-4049:
-------------------------------------

Looks good.  In the following lines, there is some repetition of the length 
calculations:

{code}
+    curPacketBuf.position(PacketHeader.PKT_LENGTHS_LEN + headerLen);
+    curPacketBuf.limit(PacketHeader.PKT_LENGTHS_LEN + headerLen + 
checksumsLen);

+    curPacketBuf.position(
+        PacketHeader.PKT_LENGTHS_LEN + headerLen + checksumsLen);
+    curPacketBuf.limit(
+        PacketHeader.PKT_LENGTHS_LEN + headerLen + checksumsLen + dataLen);

+    curPacketBuf.limit(
+        PacketHeader.PKT_LENGTHS_LEN + headerLen + checksumsLen + dataLen);
{code}

The method might read/maintain better if those calculations were captured in 
variables, like lenThroughHeader, lenThroughChecksums, and lenThroughData.

                
> hflush performance regression due to nagling delays
> ---------------------------------------------------
>
>                 Key: HDFS-4049
>                 URL: https://issues.apache.org/jira/browse/HDFS-4049
>             Project: Hadoop HDFS
>          Issue Type: Bug
>          Components: data-node, performance
>    Affects Versions: 3.0.0, 2.0.2-alpha
>            Reporter: Todd Lipcon
>            Assignee: Todd Lipcon
>            Priority: Critical
>         Attachments: hdfs-4049.txt
>
>
> HDFS-3721 reworked the way that packets are mirrored through the pipeline in 
> the datanode. This caused two write() calls where there used to be one, which 
> interacts badly with nagling so that there are 40ms bubbles on hflush() 
> calls. We didn't notice this in the tests because the hflush perf test only 
> uses a single datanode.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

Reply via email to