|
@@ -630,14 +630,19 @@ class BlockReceiver implements Closeable {
|
|
|
// bytes should be skipped when writing the data and checksum
|
|
|
// buffers out to disk.
|
|
|
long partialChunkSizeOnDisk = onDiskLen % bytesPerChecksum;
|
|
|
+ long lastChunkBoundary = onDiskLen - partialChunkSizeOnDisk;
|
|
|
boolean alignedOnDisk = partialChunkSizeOnDisk == 0;
|
|
|
boolean alignedInPacket = firstByteInBlock % bytesPerChecksum == 0;
|
|
|
|
|
|
- // Since data is always appended, not overwritten, partial CRC
|
|
|
- // recalculation is necessary if the on-disk data is not chunk-
|
|
|
- // aligned, regardless of whether the beginning of the data in
|
|
|
- // the packet is chunk-aligned.
|
|
|
- boolean doPartialCrc = !alignedOnDisk && !shouldNotWriteChecksum;
|
|
|
+ // If the end of the on-disk data is not chunk-aligned, the last
|
|
|
+ // checksum needs to be overwritten.
|
|
|
+ boolean overwriteLastCrc = !alignedOnDisk && !shouldNotWriteChecksum;
|
|
|
+ // If the starting offset of the packat data is at the last chunk
|
|
|
+ // boundary of the data on disk, the partial checksum recalculation
|
|
|
+ // can be skipped and the checksum supplied by the client can be used
|
|
|
+ // instead. This reduces disk reads and cpu load.
|
|
|
+ boolean doCrcRecalc = overwriteLastCrc &&
|
|
|
+ (lastChunkBoundary != firstByteInBlock);
|
|
|
|
|
|
// If this is a partial chunk, then verify that this is the only
|
|
|
// chunk in the packet. If the starting offset is not chunk
|
|
@@ -653,9 +658,10 @@ class BlockReceiver implements Closeable {
|
|
|
// If the last portion of the block file is not a full chunk,
|
|
|
// then read in pre-existing partial data chunk and recalculate
|
|
|
// the checksum so that the checksum calculation can continue
|
|
|
- // from the right state.
|
|
|
+ // from the right state. If the client provided the checksum for
|
|
|
+ // the whole chunk, this is not necessary.
|
|
|
Checksum partialCrc = null;
|
|
|
- if (doPartialCrc) {
|
|
|
+ if (doCrcRecalc) {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("receivePacket for " + block
|
|
|
+ ": previous write did not end at the chunk boundary."
|
|
@@ -691,11 +697,18 @@ class BlockReceiver implements Closeable {
|
|
|
int skip = 0;
|
|
|
byte[] crcBytes = null;
|
|
|
|
|
|
- // First, overwrite the partial crc at the end, if necessary.
|
|
|
- if (doPartialCrc) { // not chunk-aligned on disk
|
|
|
- // Calculate new crc for this chunk.
|
|
|
- int bytesToReadForRecalc =
|
|
|
- (int)(bytesPerChecksum - partialChunkSizeOnDisk);
|
|
|
+ // First, prepare to overwrite the partial crc at the end.
|
|
|
+ if (overwriteLastCrc) { // not chunk-aligned on disk
|
|
|
+ // prepare to overwrite last checksum
|
|
|
+ adjustCrcFilePosition();
|
|
|
+ }
|
|
|
+
|
|
|
+ // The CRC was recalculated for the last partial chunk. Update the
|
|
|
+ // CRC by reading the rest of the chunk, then write it out.
|
|
|
+ if (doCrcRecalc) {
|
|
|
+ // Calculate new crc for this chunk.
|
|
|
+ int bytesToReadForRecalc =
|
|
|
+ (int)(bytesPerChecksum - partialChunkSizeOnDisk);
|
|
|
if (numBytesToDisk < bytesToReadForRecalc) {
|
|
|
bytesToReadForRecalc = numBytesToDisk;
|
|
|
}
|
|
@@ -705,8 +718,6 @@ class BlockReceiver implements Closeable {
|
|
|
byte[] buf = FSOutputSummer.convertToByteStream(partialCrc,
|
|
|
checksumSize);
|
|
|
crcBytes = copyLastChunkChecksum(buf, checksumSize, buf.length);
|
|
|
- // prepare to overwrite last checksum
|
|
|
- adjustCrcFilePosition();
|
|
|
checksumOut.write(buf);
|
|
|
if(LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Writing out partial crc for data len " + len +
|
|
@@ -719,7 +730,6 @@ class BlockReceiver implements Closeable {
|
|
|
// boundary. The checksum after the boundary was already counted
|
|
|
// above. Only count the number of checksums skipped up to the
|
|
|
// boundary here.
|
|
|
- long lastChunkBoundary = onDiskLen - (onDiskLen%bytesPerChecksum);
|
|
|
long skippedDataBytes = lastChunkBoundary - firstByteInBlock;
|
|
|
|
|
|
if (skippedDataBytes > 0) {
|