[ https://issues.apache.org/jira/browse/HDFS-8287?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14960884#comment-14960884 ]
Rakesh R commented on HDFS-8287: -------------------------------- Thank you [~kaisasak] for taking care this, latest patch mostly looks fine to me. There are some more comments, could you please take a look at it. # There are few minor checkstyle warnings, please fix it. # I failed to understand the purpose of synchronization here. Is this required? {code} synchronized public CellBuffers flip() {code} # During DFSStripedOutputStream#closeImpl, I could see a corner case - number of bytes reaches striped boundary. Assume writeParityCells() has submitted a parity generator task and again assume the client has invoked #close() function. Now, generateParityCellsForLastStripe() will return false and its not waiting for the parity gen task in queue of previous cell, right? IMHO, we could have a mechanism to wait for any previously submitted parity gen task before closure. {code} private boolean generateParityCellsForLastStripe(){ .... .... final long lastStripeSize = currentBlockGroupBytes % stripeDataSize(); if (lastStripeSize == 0) { return false; } {code} # I think executor service can be moved to DFSClient, rather than creating again and again for every DFSStripedOutputStream, isn't it? {code} private final ExecutorService executorService; {code} Also, I've one comment about {{Executors.newCachedThreadPool}} -> It's unbounded, which means that you're opening the door for anyone to cripple your JVM by simply injecting more work into the service (DoS attack). Any specific reason to use cachedThreadPool? If not, I prefer to use fixed Executors.newFixedThreadPool or a ThreadPoolExecutor with a set maximum number of threads; # {{public}} class DoubleCellBuffer, please make this to {{private}}. Also, you can make the methods to private visibility. > 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, 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)