hadoop-hdfs-issues mailing list archives

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

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

Zhe Zhang commented on HDFS-8287:
---------------------------------

Great work here! The idea is very interesting and overall structure LGTM. More detailed comments:

# Is parityGenerator designed to hold more than 1 {{ParityGeneratorEntity}}? IIUC it should
only hold 1 entity right?
# If parityGenerator should hold 1 entity, should we still use a List?
# Should {{bufferQueue.wait()}} be outside {{synchronized (bufferQueue)}}?
# The added logic is quite complex and we certainly need a new test. For example, we should
verify the concurrency logic where the main writing thread is much faster than encode, and
catches up with it twice. Our logic should protect the being-encoded buffers from being overwritten.

Nits:
# Extra white space
{code}
+  private int   currentCellBuffersIndex = 0;
{code}
# Each line in the section is too long
{code}
  /**
   * Uses double buffering. In the beginning, client writes data to the first buffer.
   * When the first cell stripe is full, client continues writing to the second buffer.
   * Now, the ParityGenerator will pick the first buffer and writes parity, then releases
   * the first buffer so that when the second buffer is full, the client can continue
   * writing to the first buffer again.
   */
{code}
# {{ParityGenerator#bufferQueue}} should be final.

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