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 Mon, 14 Sep 2015 18:48:45 GMT

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

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

Thanks for working on this, [~kaisasak] and [~szetszwo]! The current patch looks good to me.
Some comments:
# Looks like the current wait() and notifyAll() are called by the same thread? I think the
rollBuffers and processFuture related code can be simplified by utilizing CompletionService#take
and/or CompletionService#poll. Also maybe in this way we do not need to use whether bufReady
is null to indicate whether encoding is done. The simplified logic can be:
#* Everytime we should write parity cells (already got one stripe of data or the close is
called), call CompletionService#take to wait and retrieve the result of the last time submitted
encoding task.
#* If not close, switch the buffer and submit the new encoding task.
#* If to close, use bufCurrent to encode.
# Maybe we can move CellBuffers and DoubleCellBuffer out of DFSStripedOutputStream as standalone
classes? CellBuffers can be included inside of the DoubleCellBuffer.


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