|
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs;
|
|
|
import java.io.FileNotFoundException;
|
|
|
import java.io.IOException;
|
|
|
import java.io.InterruptedIOException;
|
|
|
+import java.nio.ByteBuffer;
|
|
|
import java.nio.channels.ClosedChannelException;
|
|
|
import java.util.EnumSet;
|
|
|
import java.util.concurrent.atomic.AtomicReference;
|
|
@@ -393,11 +394,47 @@ public class DFSOutputStream extends FSOutputSummer
|
|
|
@Override
|
|
|
protected synchronized void writeChunk(byte[] b, int offset, int len,
|
|
|
byte[] checksum, int ckoff, int cklen) throws IOException {
|
|
|
+ writeChunkPrepare(len, ckoff, cklen);
|
|
|
+
|
|
|
+ currentPacket.writeChecksum(checksum, ckoff, cklen);
|
|
|
+ currentPacket.writeData(b, offset, len);
|
|
|
+ currentPacket.incNumChunks();
|
|
|
+ getStreamer().incBytesCurBlock(len);
|
|
|
+
|
|
|
+ // If packet is full, enqueue it for transmission
|
|
|
+ if (currentPacket.getNumChunks() == currentPacket.getMaxChunks() ||
|
|
|
+ getStreamer().getBytesCurBlock() == blockSize) {
|
|
|
+ enqueueCurrentPacketFull();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /* write the data chunk in <code>buffer</code> staring at
|
|
|
+ * <code>buffer.position</code> with
|
|
|
+ * a length of <code>len > 0</code>, and its checksum
|
|
|
+ */
|
|
|
+ protected synchronized void writeChunk(ByteBuffer buffer, int len,
|
|
|
+ byte[] checksum, int ckoff, int cklen) throws IOException {
|
|
|
+ writeChunkPrepare(len, ckoff, cklen);
|
|
|
+
|
|
|
+ currentPacket.writeChecksum(checksum, ckoff, cklen);
|
|
|
+ currentPacket.writeData(buffer, len);
|
|
|
+ currentPacket.incNumChunks();
|
|
|
+ getStreamer().incBytesCurBlock(len);
|
|
|
+
|
|
|
+ // If packet is full, enqueue it for transmission
|
|
|
+ if (currentPacket.getNumChunks() == currentPacket.getMaxChunks() ||
|
|
|
+ getStreamer().getBytesCurBlock() == blockSize) {
|
|
|
+ enqueueCurrentPacketFull();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private synchronized void writeChunkPrepare(int buflen,
|
|
|
+ int ckoff, int cklen) throws IOException {
|
|
|
dfsClient.checkOpen();
|
|
|
checkClosed();
|
|
|
|
|
|
- if (len > bytesPerChecksum) {
|
|
|
- throw new IOException("writeChunk() buffer size is " + len +
|
|
|
+ if (buflen > bytesPerChecksum) {
|
|
|
+ throw new IOException("writeChunk() buffer size is " + buflen +
|
|
|
" is larger than supported bytesPerChecksum " +
|
|
|
bytesPerChecksum);
|
|
|
}
|
|
@@ -414,17 +451,6 @@ public class DFSOutputStream extends FSOutputSummer
|
|
|
currentPacket.getSeqno(), src, packetSize, chunksPerPacket,
|
|
|
getStreamer().getBytesCurBlock() + ", " + this);
|
|
|
}
|
|
|
-
|
|
|
- currentPacket.writeChecksum(checksum, ckoff, cklen);
|
|
|
- currentPacket.writeData(b, offset, len);
|
|
|
- currentPacket.incNumChunks();
|
|
|
- getStreamer().incBytesCurBlock(len);
|
|
|
-
|
|
|
- // If packet is full, enqueue it for transmission
|
|
|
- if (currentPacket.getNumChunks() == currentPacket.getMaxChunks() ||
|
|
|
- getStreamer().getBytesCurBlock() == blockSize) {
|
|
|
- enqueueCurrentPacketFull();
|
|
|
- }
|
|
|
}
|
|
|
|
|
|
void enqueueCurrentPacket() throws IOException {
|