hadoop-hdfs-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Tsz Wo Nicholas Sze (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HDFS-8287) DFSStripedOutputStream.writeChunk should not wait for writing parity
Date Sun, 13 Sep 2015 21:48:45 GMT

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

Tsz Wo Nicholas Sze commented on HDFS-8287:
-------------------------------------------

> I checked patch and passed our previous test case and added some comments to this patch.
...

That's a good news.

- Let's put future and processFuture() in a class and add a setFuture method so that future
can be synchronized.
{code}
  class ParityTask {
    private Future<CellBuffers> future = null;
    
    synchronized void setFuture(Future<CellBuffers> f) {
      Preconditions.checkNotNull(f);
      Preconditions.checkState(future == null);
      future = f;
    }

    synchronized void processFuture() throws IOException {
      ...
    }
  }
{code}
- "flip interrupted" should be updated "waitForBufReady interrupted".
- "checkParityTasks interrupted" should be updated "processFuture interrupted".

Please see if you can test the patch by writing large files to a real cluster and compare
the performance before and after the patch.  

> 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