hadoop-hdfs-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Jing Zhao (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HDFS-8287) DFSStripedOutputStream.writeChunk should not wait for writing parity
Date Wed, 21 Oct 2015 23:27:27 GMT

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

Jing Zhao commented on HDFS-8287:
---------------------------------

Thanks for updating the patch, [~lewuathe]. Also thanks for the review, [~rakeshr]. 

For the current patch, looks like we're not currently using the two buffers included in {{DoubleCellBuffer}}.
Specifically, {{writeParityCells}} does the following in a serial manner:
# wait for the encoding work (against the ready buffer) to finish
# write encoded data in the ready buffer to all the data streamers
# flip buffers
# start encoding in the new ready buffer

Since #3 and #4 happen only after #2 finishes, the above steps actually only need/utilize
one buffer. Maybe what we can do is:
# move the above #2 to the end
# the checksum calculation can be done along with encoding

Besides, it will be nice to run some simple performance tests to see what improvement the
changes can achieve.



> DFSStripedOutputStream.writeChunk should not wait for writing parity 
> ---------------------------------------------------------------------
>
>                 Key: HDFS-8287
>                 URL: https://issues.apache.org/jira/browse/HDFS-8287
>             Project: Hadoop HDFS
>          Issue Type: Sub-task
>          Components: hdfs-client
>            Reporter: Tsz Wo Nicholas Sze
>            Assignee: Kai Sasaki
>         Attachments: HDFS-8287-HDFS-7285.00.patch, HDFS-8287-HDFS-7285.01.patch, HDFS-8287-HDFS-7285.02.patch,
HDFS-8287-HDFS-7285.03.patch, HDFS-8287-HDFS-7285.04.patch, HDFS-8287-HDFS-7285.05.patch,
HDFS-8287-HDFS-7285.06.patch, HDFS-8287-HDFS-7285.07.patch, HDFS-8287-HDFS-7285.08.patch,
HDFS-8287-HDFS-7285.09.patch, HDFS-8287-HDFS-7285.10.patch, HDFS-8287-HDFS-7285.11.patch,
HDFS-8287-HDFS-7285.WIP.patch, HDFS-8287-performance-report.pdf, HDFS-8287.12.patch, HDFS-8287.13.patch,
HDFS-8287.14.patch, HDFS-8287.15.patch, h8287_20150911.patch, jstack-dump.txt
>
>
> When a stripping cell is full, writeChunk computes and generates parity packets.  It
sequentially calls waitAndQueuePacket so that user client cannot continue to write data until
it finishes.
> We should allow user client to continue writing instead but not blocking it when writing
parity.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message