[ https://issues.apache.org/jira/browse/HDFS-8287?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14733651#comment-14733651 ]
Tsz Wo Nicholas Sze commented on HDFS-8287: ------------------------------------------- Some minor comments: - We don't really need to declare the new class ParityGenerateTask. Just use an anonymous class. {code} final byte[][] checkSumArrays = doubleCellBuffer.getCheckSumArrays().clone(); completionService.submit(new Callable<Void>() { @Override public Void call() throws Exception { encode(encoder, numDataBlocks, copiedBuffers); synchronized (copiedBuffers) { for (int i = numDataBlocks; i < numAllBlocks; i++) { try { writeParity(i, copiedBuffers[i], checkSumArrays[i]); } catch (IOException e) { LOG.warn("Caught exception ", e); } } } return null; } }); {code} - Do we need to synchronize copiedBuffers? It is just a local variable. - We should avoid copying arrays/buffers. Let do this improvement separately. > 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 > > > 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)