diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java
index 511ca7f7549..c8d1b69ddaf 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java
@@ -381,7 +381,8 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
long blockSize,
Progressable progress)
throws IOException {
- super(DataChecksum.newCrc32(), fs.getBytesPerSum(), 4);
+ super(DataChecksum.newDataChecksum(DataChecksum.Type.CRC32,
+ fs.getBytesPerSum()));
int bytesPerSum = fs.getBytesPerSum();
this.datas = fs.getRawFileSystem().create(file, overwrite, bufferSize,
replication, blockSize, progress);
@@ -405,10 +406,11 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
}
@Override
- protected void writeChunk(byte[] b, int offset, int len, byte[] checksum)
+ protected void writeChunk(byte[] b, int offset, int len, byte[] checksum,
+ int ckoff, int cklen)
throws IOException {
datas.write(b, offset, len);
- sums.write(checksum);
+ sums.write(checksum, ckoff, cklen);
}
@Override
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFs.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFs.java
index 4be3b291190..ab5cd13e0c3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFs.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFs.java
@@ -337,7 +337,8 @@ public abstract class ChecksumFs extends FilterFs {
final short replication, final long blockSize,
final Progressable progress, final ChecksumOpt checksumOpt,
final boolean createParent) throws IOException {
- super(DataChecksum.newCrc32(), fs.getBytesPerSum(), 4);
+ super(DataChecksum.newDataChecksum(DataChecksum.Type.CRC32,
+ fs.getBytesPerSum()));
// checksumOpt is passed down to the raw fs. Unless it implements
// checksum impelemts internally, checksumOpt will be ignored.
@@ -370,10 +371,11 @@ public abstract class ChecksumFs extends FilterFs {
}
@Override
- protected void writeChunk(byte[] b, int offset, int len, byte[] checksum)
+ protected void writeChunk(byte[] b, int offset, int len, byte[] checksum,
+ int ckoff, int cklen)
throws IOException {
datas.write(b, offset, len);
- sums.write(checksum);
+ sums.write(checksum, ckoff, cklen);
}
@Override
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
index 49c919af196..19cbb6f9354 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
@@ -18,13 +18,14 @@
package org.apache.hadoop.fs;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.util.DataChecksum;
+
import java.io.IOException;
import java.io.OutputStream;
import java.util.zip.Checksum;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
/**
* This is a generic output stream for generating checksums for
* data before it is written to the underlying stream
@@ -33,7 +34,7 @@ import org.apache.hadoop.classification.InterfaceStability;
@InterfaceStability.Unstable
abstract public class FSOutputSummer extends OutputStream {
// data checksum
- private Checksum sum;
+ private final DataChecksum sum;
// internal buffer for storing data before it is checksumed
private byte buf[];
// internal buffer for storing checksum
@@ -41,18 +42,24 @@ abstract public class FSOutputSummer extends OutputStream {
// The number of valid bytes in the buffer.
private int count;
- protected FSOutputSummer(Checksum sum, int maxChunkSize, int checksumSize) {
+ // We want this value to be a multiple of 3 because the native code checksums
+ // 3 chunks simultaneously. The chosen value of 9 strikes a balance between
+ // limiting the number of JNI calls and flushing to the underlying stream
+ // relatively frequently.
+ private static final int BUFFER_NUM_CHUNKS = 9;
+
+ protected FSOutputSummer(DataChecksum sum) {
this.sum = sum;
- this.buf = new byte[maxChunkSize];
- this.checksum = new byte[checksumSize];
+ this.buf = new byte[sum.getBytesPerChecksum() * BUFFER_NUM_CHUNKS];
+ this.checksum = new byte[sum.getChecksumSize() * BUFFER_NUM_CHUNKS];
this.count = 0;
}
/* write the data chunk in b
staring at offset
with
- * a length of len
, and its checksum
+ * a length of len > 0
, and its checksum
*/
- protected abstract void writeChunk(byte[] b, int offset, int len, byte[] checksum)
- throws IOException;
+ protected abstract void writeChunk(byte[] b, int bOffset, int bLen,
+ byte[] checksum, int checksumOffset, int checksumLen) throws IOException;
/**
* Check if the implementing OutputStream is closed and should no longer
@@ -66,7 +73,6 @@ abstract public class FSOutputSummer extends OutputStream {
/** Write one byte */
@Override
public synchronized void write(int b) throws IOException {
- sum.update(b);
buf[count++] = (byte)b;
if(count == buf.length) {
flushBuffer();
@@ -111,18 +117,17 @@ abstract public class FSOutputSummer extends OutputStream {
*/
private int write1(byte b[], int off, int len) throws IOException {
if(count==0 && len>=buf.length) {
- // local buffer is empty and user data has one chunk
- // checksum and output data
+ // local buffer is empty and user buffer size >= local buffer size, so
+ // simply checksum the user buffer and send it directly to the underlying
+ // stream
final int length = buf.length;
- sum.update(b, off, length);
- writeChecksumChunk(b, off, length, false);
+ writeChecksumChunks(b, off, length);
return length;
}
// copy user data to local buffer
int bytesToCopy = buf.length-count;
bytesToCopy = (len bytesPerChecksum) {
throw new IOException("writeChunk() buffer size is " + len +
" is larger than supported bytesPerChecksum " +
bytesPerChecksum);
}
- if (checksum.length != this.checksum.getChecksumSize()) {
+ if (cklen != this.checksum.getChecksumSize()) {
throw new IOException("writeChunk() checksum size is supposed to be " +
this.checksum.getChecksumSize() +
- " but found to be " + checksum.length);
+ " but found to be " + cklen);
}
if (currentPacket == null) {
@@ -1748,7 +1747,7 @@ public class DFSOutputStream extends FSOutputSummer
}
}
- currentPacket.writeChecksum(checksum, 0, cklen);
+ currentPacket.writeChecksum(checksum, ckoff, cklen);
currentPacket.writeData(b, offset, len);
currentPacket.numChunks++;
bytesCurBlock += len;
@@ -1772,7 +1771,7 @@ public class DFSOutputStream extends FSOutputSummer
// crc chunks from now on.
if (appendChunk && bytesCurBlock%bytesPerChecksum == 0) {
appendChunk = false;
- resetChecksumChunk(bytesPerChecksum);
+ resetChecksumBufSize();
}
if (!appendChunk) {
@@ -1859,20 +1858,13 @@ public class DFSOutputStream extends FSOutputSummer
long lastBlockLength = -1L;
boolean updateLength = syncFlags.contains(SyncFlag.UPDATE_LENGTH);
synchronized (this) {
- /* Record current blockOffset. This might be changed inside
- * flushBuffer() where a partial checksum chunk might be flushed.
- * After the flush, reset the bytesCurBlock back to its previous value,
- * any partial checksum chunk will be sent now and in next packet.
- */
- long saveOffset = bytesCurBlock;
- Packet oldCurrentPacket = currentPacket;
// flush checksum buffer, but keep checksum buffer intact
- flushBuffer(true);
+ int numKept = flushBuffer(true, true);
// bytesCurBlock potentially incremented if there was buffered data
if (DFSClient.LOG.isDebugEnabled()) {
DFSClient.LOG.debug(
- "DFSClient flush() : saveOffset " + saveOffset +
+ "DFSClient flush() :" +
" bytesCurBlock " + bytesCurBlock +
" lastFlushOffset " + lastFlushOffset);
}
@@ -1889,14 +1881,6 @@ public class DFSOutputStream extends FSOutputSummer
bytesCurBlock, currentSeqno++, this.checksum.getChecksumSize());
}
} else {
- // We already flushed up to this offset.
- // This means that we haven't written anything since the last flush
- // (or the beginning of the file). Hence, we should not have any
- // packet queued prior to this call, since the last flush set
- // currentPacket = null.
- assert oldCurrentPacket == null :
- "Empty flush should not occur with a currentPacket";
-
if (isSync && bytesCurBlock > 0) {
// Nothing to send right now,
// and the block was partially written,
@@ -1916,7 +1900,7 @@ public class DFSOutputStream extends FSOutputSummer
// Restore state of stream. Record the last flush offset
// of the last full chunk that was flushed.
//
- bytesCurBlock = saveOffset;
+ bytesCurBlock -= numKept;
toWaitFor = lastQueuedSeqno;
} // end synchronized
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
index 35285887acb..b5f9d3cf229 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
@@ -261,7 +261,9 @@ public class TestFileAppend{
start += 29;
}
stm.write(fileContents, start, AppendTestUtil.FILE_SIZE -start);
-
+ // need to make sure we completely write out all full blocks before
+ // the checkFile() call (see FSOutputSummer#flush)
+ stm.flush();
// verify that full blocks are sane
checkFile(fs, file1, 1);
stm.close();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java
index 242934547d0..1fe7ba89851 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java
@@ -394,6 +394,8 @@ public class TestBlockToken {
Path filePath = new Path(fileName);
FSDataOutputStream out = fs.create(filePath, (short) 1);
out.write(new byte[1000]);
+ // ensure that the first block is written out (see FSOutputSummer#flush)
+ out.flush();
LocatedBlocks locatedBlocks = cluster.getNameNodeRpc().getBlockLocations(
fileName, 0, 1000);
while (locatedBlocks.getLastLocatedBlock() == null) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
index 5448e7a885e..872ff9c490f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
@@ -70,6 +70,9 @@ public class TestBlockUnderConstruction {
long blocksBefore = stm.getPos() / BLOCK_SIZE;
TestFileCreation.writeFile(stm, BLOCK_SIZE);
+ // need to make sure the full block is completely flushed to the DataNodes
+ // (see FSOutputSummer#flush)
+ stm.flush();
int blocksAfter = 0;
// wait until the block is allocated by DataStreamer
BlockLocation[] locatedBlocks;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
index b0f0dd4d1ac..6f94b717230 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
@@ -154,6 +154,9 @@ public class TestDecommissioningStatus {
Random rand = new Random(seed);
rand.nextBytes(buffer);
stm.write(buffer);
+ // need to make sure that we actually write out both file blocks
+ // (see FSOutputSummer#flush)
+ stm.flush();
// Do not close stream, return it
// so that it is not garbage collected
return stm;