|
@@ -23,7 +23,6 @@ import java.io.BufferedOutputStream;
|
|
import java.io.Closeable;
|
|
import java.io.Closeable;
|
|
import java.io.DataInputStream;
|
|
import java.io.DataInputStream;
|
|
import java.io.DataOutputStream;
|
|
import java.io.DataOutputStream;
|
|
-import java.io.EOFException;
|
|
|
|
import java.io.FileDescriptor;
|
|
import java.io.FileDescriptor;
|
|
import java.io.FileOutputStream;
|
|
import java.io.FileOutputStream;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
@@ -34,12 +33,14 @@ import java.util.LinkedList;
|
|
import java.util.zip.Checksum;
|
|
import java.util.zip.Checksum;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.Log;
|
|
|
|
+import org.apache.hadoop.fs.ChecksumException;
|
|
import org.apache.hadoop.fs.FSOutputSummer;
|
|
import org.apache.hadoop.fs.FSOutputSummer;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;
|
|
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
|
|
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
|
|
@@ -77,9 +78,10 @@ class BlockReceiver implements Closeable {
|
|
private DataOutputStream checksumOut = null; // to crc file at local disk
|
|
private DataOutputStream checksumOut = null; // to crc file at local disk
|
|
private int bytesPerChecksum;
|
|
private int bytesPerChecksum;
|
|
private int checksumSize;
|
|
private int checksumSize;
|
|
- private ByteBuffer buf; // contains one full packet.
|
|
|
|
- private int bufRead; //amount of valid data in the buf
|
|
|
|
- private int maxPacketReadLen;
|
|
|
|
|
|
+
|
|
|
|
+ private PacketReceiver packetReceiver =
|
|
|
|
+ new PacketReceiver(false);
|
|
|
|
+
|
|
protected final String inAddr;
|
|
protected final String inAddr;
|
|
protected final String myAddr;
|
|
protected final String myAddr;
|
|
private String mirrorAddr;
|
|
private String mirrorAddr;
|
|
@@ -248,6 +250,10 @@ class BlockReceiver implements Closeable {
|
|
*/
|
|
*/
|
|
@Override
|
|
@Override
|
|
public void close() throws IOException {
|
|
public void close() throws IOException {
|
|
|
|
+ if (packetReceiver != null) {
|
|
|
|
+ packetReceiver.close();
|
|
|
|
+ }
|
|
|
|
+
|
|
IOException ioe = null;
|
|
IOException ioe = null;
|
|
if (syncOnClose && (out != null || checksumOut != null)) {
|
|
if (syncOnClose && (out != null || checksumOut != null)) {
|
|
datanode.metrics.incrFsyncCount();
|
|
datanode.metrics.incrFsyncCount();
|
|
@@ -365,33 +371,24 @@ class BlockReceiver implements Closeable {
|
|
/**
|
|
/**
|
|
* Verify multiple CRC chunks.
|
|
* Verify multiple CRC chunks.
|
|
*/
|
|
*/
|
|
- private void verifyChunks( byte[] dataBuf, int dataOff, int len,
|
|
|
|
- byte[] checksumBuf, int checksumOff )
|
|
|
|
- throws IOException {
|
|
|
|
- while (len > 0) {
|
|
|
|
- int chunkLen = Math.min(len, bytesPerChecksum);
|
|
|
|
-
|
|
|
|
- clientChecksum.update(dataBuf, dataOff, chunkLen);
|
|
|
|
-
|
|
|
|
- if (!clientChecksum.compare(checksumBuf, checksumOff)) {
|
|
|
|
- if (srcDataNode != null) {
|
|
|
|
- try {
|
|
|
|
- LOG.info("report corrupt block " + block + " from datanode " +
|
|
|
|
- srcDataNode + " to namenode");
|
|
|
|
- datanode.reportRemoteBadBlock(srcDataNode, block);
|
|
|
|
- } catch (IOException e) {
|
|
|
|
- LOG.warn("Failed to report bad block " + block +
|
|
|
|
- " from datanode " + srcDataNode + " to namenode");
|
|
|
|
- }
|
|
|
|
|
|
+ private void verifyChunks(ByteBuffer dataBuf, ByteBuffer checksumBuf)
|
|
|
|
+ throws IOException {
|
|
|
|
+ try {
|
|
|
|
+ clientChecksum.verifyChunkedSums(dataBuf, checksumBuf, clientname, 0);
|
|
|
|
+ } catch (ChecksumException ce) {
|
|
|
|
+ LOG.warn("Checksum error in block " + block + " from " + inAddr, ce);
|
|
|
|
+ if (srcDataNode != null) {
|
|
|
|
+ try {
|
|
|
|
+ LOG.info("report corrupt block " + block + " from datanode " +
|
|
|
|
+ srcDataNode + " to namenode");
|
|
|
|
+ datanode.reportRemoteBadBlock(srcDataNode, block);
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ LOG.warn("Failed to report bad block " + block +
|
|
|
|
+ " from datanode " + srcDataNode + " to namenode");
|
|
}
|
|
}
|
|
- throw new IOException("Unexpected checksum mismatch " +
|
|
|
|
- "while writing " + block + " from " + inAddr);
|
|
|
|
}
|
|
}
|
|
-
|
|
|
|
- clientChecksum.reset();
|
|
|
|
- dataOff += chunkLen;
|
|
|
|
- checksumOff += checksumSize;
|
|
|
|
- len -= chunkLen;
|
|
|
|
|
|
+ throw new IOException("Unexpected checksum mismatch " +
|
|
|
|
+ "while writing " + block + " from " + inAddr);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -403,163 +400,24 @@ class BlockReceiver implements Closeable {
|
|
* This does not verify the original checksums, under the assumption
|
|
* This does not verify the original checksums, under the assumption
|
|
* that they have already been validated.
|
|
* that they have already been validated.
|
|
*/
|
|
*/
|
|
- private void translateChunks( byte[] dataBuf, int dataOff, int len,
|
|
|
|
- byte[] checksumBuf, int checksumOff ) {
|
|
|
|
- if (len == 0) return;
|
|
|
|
-
|
|
|
|
- int numChunks = (len - 1)/bytesPerChecksum + 1;
|
|
|
|
-
|
|
|
|
- diskChecksum.calculateChunkedSums(
|
|
|
|
- ByteBuffer.wrap(dataBuf, dataOff, len),
|
|
|
|
- ByteBuffer.wrap(checksumBuf, checksumOff, numChunks * checksumSize));
|
|
|
|
|
|
+ private void translateChunks(ByteBuffer dataBuf, ByteBuffer checksumBuf) {
|
|
|
|
+ diskChecksum.calculateChunkedSums(dataBuf, checksumBuf);
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * Makes sure buf.position() is zero without modifying buf.remaining().
|
|
|
|
- * It moves the data if position needs to be changed.
|
|
|
|
- */
|
|
|
|
- private void shiftBufData() {
|
|
|
|
- if (bufRead != buf.limit()) {
|
|
|
|
- throw new IllegalStateException("bufRead should be same as " +
|
|
|
|
- "buf.limit()");
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- //shift the remaining data on buf to the front
|
|
|
|
- if (buf.position() > 0) {
|
|
|
|
- int dataLeft = buf.remaining();
|
|
|
|
- if (dataLeft > 0) {
|
|
|
|
- byte[] b = buf.array();
|
|
|
|
- System.arraycopy(b, buf.position(), b, 0, dataLeft);
|
|
|
|
- }
|
|
|
|
- buf.position(0);
|
|
|
|
- bufRead = dataLeft;
|
|
|
|
- buf.limit(bufRead);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * reads upto toRead byte to buf at buf.limit() and increments the limit.
|
|
|
|
- * throws an IOException if read does not succeed.
|
|
|
|
- */
|
|
|
|
- private int readToBuf(int toRead) throws IOException {
|
|
|
|
- if (toRead < 0) {
|
|
|
|
- toRead = (maxPacketReadLen > 0 ? maxPacketReadLen : buf.capacity())
|
|
|
|
- - buf.limit();
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- int nRead = in.read(buf.array(), buf.limit(), toRead);
|
|
|
|
-
|
|
|
|
- if (nRead < 0) {
|
|
|
|
- throw new EOFException("while trying to read " + toRead + " bytes");
|
|
|
|
- }
|
|
|
|
- bufRead = buf.limit() + nRead;
|
|
|
|
- buf.limit(bufRead);
|
|
|
|
- return nRead;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Reads (at least) one packet and returns the packet length.
|
|
|
|
- * buf.position() points to the start of the packet and
|
|
|
|
- * buf.limit() point to the end of the packet. There could
|
|
|
|
- * be more data from next packet in buf.<br><br>
|
|
|
|
- *
|
|
|
|
- * It tries to read a full packet with single read call.
|
|
|
|
- * Consecutive packets are usually of the same length.
|
|
|
|
- */
|
|
|
|
- private void readNextPacket() throws IOException {
|
|
|
|
- /* This dances around buf a little bit, mainly to read
|
|
|
|
- * full packet with single read and to accept arbitrary size
|
|
|
|
- * for next packet at the same time.
|
|
|
|
- */
|
|
|
|
- if (buf == null) {
|
|
|
|
- /* initialize buffer to the best guess size:
|
|
|
|
- * 'chunksPerPacket' calculation here should match the same
|
|
|
|
- * calculation in DFSClient to make the guess accurate.
|
|
|
|
- */
|
|
|
|
- int chunkSize = bytesPerChecksum + checksumSize;
|
|
|
|
- int chunksPerPacket = (datanode.getDnConf().writePacketSize - PacketHeader.PKT_HEADER_LEN
|
|
|
|
- + chunkSize - 1)/chunkSize;
|
|
|
|
- buf = ByteBuffer.allocate(PacketHeader.PKT_HEADER_LEN +
|
|
|
|
- Math.max(chunksPerPacket, 1) * chunkSize);
|
|
|
|
- buf.limit(0);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- // See if there is data left in the buffer :
|
|
|
|
- if (bufRead > buf.limit()) {
|
|
|
|
- buf.limit(bufRead);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- while (buf.remaining() < HdfsConstants.BYTES_IN_INTEGER) {
|
|
|
|
- if (buf.position() > 0) {
|
|
|
|
- shiftBufData();
|
|
|
|
- }
|
|
|
|
- readToBuf(-1);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /* We mostly have the full packet or at least enough for an int
|
|
|
|
- */
|
|
|
|
- buf.mark();
|
|
|
|
- int payloadLen = buf.getInt();
|
|
|
|
- buf.reset();
|
|
|
|
-
|
|
|
|
- // check corrupt values for pktLen, 100MB upper limit should be ok?
|
|
|
|
- if (payloadLen < 0 || payloadLen > (100*1024*1024)) {
|
|
|
|
- throw new IOException("Incorrect value for packet payload : " +
|
|
|
|
- payloadLen);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- // Subtract BYTES_IN_INTEGER since that accounts for the payloadLen that
|
|
|
|
- // we read above.
|
|
|
|
- int pktSize = payloadLen + PacketHeader.PKT_HEADER_LEN
|
|
|
|
- - HdfsConstants.BYTES_IN_INTEGER;
|
|
|
|
-
|
|
|
|
- if (buf.remaining() < pktSize) {
|
|
|
|
- //we need to read more data
|
|
|
|
- int toRead = pktSize - buf.remaining();
|
|
|
|
-
|
|
|
|
- // first make sure buf has enough space.
|
|
|
|
- int spaceLeft = buf.capacity() - buf.limit();
|
|
|
|
- if (toRead > spaceLeft && buf.position() > 0) {
|
|
|
|
- shiftBufData();
|
|
|
|
- spaceLeft = buf.capacity() - buf.limit();
|
|
|
|
- }
|
|
|
|
- if (toRead > spaceLeft) {
|
|
|
|
- byte oldBuf[] = buf.array();
|
|
|
|
- int toCopy = buf.limit();
|
|
|
|
- buf = ByteBuffer.allocate(toCopy + toRead);
|
|
|
|
- System.arraycopy(oldBuf, 0, buf.array(), 0, toCopy);
|
|
|
|
- buf.limit(toCopy);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- //now read:
|
|
|
|
- while (toRead > 0) {
|
|
|
|
- toRead -= readToBuf(toRead);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- if (buf.remaining() > pktSize) {
|
|
|
|
- buf.limit(buf.position() + pktSize);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- if (pktSize > maxPacketReadLen) {
|
|
|
|
- maxPacketReadLen = pktSize;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Receives and processes a packet. It can contain many chunks.
|
|
* Receives and processes a packet. It can contain many chunks.
|
|
* returns the number of data bytes that the packet has.
|
|
* returns the number of data bytes that the packet has.
|
|
*/
|
|
*/
|
|
private int receivePacket() throws IOException {
|
|
private int receivePacket() throws IOException {
|
|
// read the next packet
|
|
// read the next packet
|
|
- readNextPacket();
|
|
|
|
|
|
+ packetReceiver.receiveNextPacket(in);
|
|
|
|
|
|
- buf.mark();
|
|
|
|
- PacketHeader header = new PacketHeader();
|
|
|
|
- header.readFields(buf);
|
|
|
|
- int endOfHeader = buf.position();
|
|
|
|
- buf.reset();
|
|
|
|
|
|
+ PacketHeader header = packetReceiver.getHeader();
|
|
|
|
+ if (LOG.isDebugEnabled()){
|
|
|
|
+ LOG.debug("Receiving one packet for block " + block +
|
|
|
|
+ ": " + header);
|
|
|
|
+ }
|
|
|
|
|
|
// Sanity check the header
|
|
// Sanity check the header
|
|
if (header.getOffsetInBlock() > replicaInfo.getNumBytes()) {
|
|
if (header.getOffsetInBlock() > replicaInfo.getNumBytes()) {
|
|
@@ -574,38 +432,12 @@ class BlockReceiver implements Closeable {
|
|
header.getDataLen());
|
|
header.getDataLen());
|
|
}
|
|
}
|
|
|
|
|
|
- return receivePacket(
|
|
|
|
- header.getOffsetInBlock(),
|
|
|
|
- header.getSeqno(),
|
|
|
|
- header.isLastPacketInBlock(),
|
|
|
|
- header.getDataLen(),
|
|
|
|
- header.getSyncBlock(),
|
|
|
|
- endOfHeader);
|
|
|
|
- }
|
|
|
|
|
|
+ long offsetInBlock = header.getOffsetInBlock();
|
|
|
|
+ long seqno = header.getSeqno();
|
|
|
|
+ boolean lastPacketInBlock = header.isLastPacketInBlock();
|
|
|
|
+ int len = header.getDataLen();
|
|
|
|
+ boolean syncBlock = header.getSyncBlock();
|
|
|
|
|
|
- /**
|
|
|
|
- * Write the received packet to disk (data only)
|
|
|
|
- */
|
|
|
|
- private void writePacketToDisk(byte[] pktBuf, int startByteToDisk,
|
|
|
|
- int numBytesToDisk) throws IOException {
|
|
|
|
- out.write(pktBuf, startByteToDisk, numBytesToDisk);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Receives and processes a packet. It can contain many chunks.
|
|
|
|
- * returns the number of data bytes that the packet has.
|
|
|
|
- */
|
|
|
|
- private int receivePacket(long offsetInBlock, long seqno,
|
|
|
|
- boolean lastPacketInBlock, int len, boolean syncBlock,
|
|
|
|
- int endOfHeader) throws IOException {
|
|
|
|
- if (LOG.isDebugEnabled()){
|
|
|
|
- LOG.debug("Receiving one packet for block " + block +
|
|
|
|
- " of length " + len +
|
|
|
|
- " seqno " + seqno +
|
|
|
|
- " offsetInBlock " + offsetInBlock +
|
|
|
|
- " syncBlock " + syncBlock +
|
|
|
|
- " lastPacketInBlock " + lastPacketInBlock);
|
|
|
|
- }
|
|
|
|
// make sure the block gets sync'ed upon close
|
|
// make sure the block gets sync'ed upon close
|
|
this.syncOnClose |= syncBlock && lastPacketInBlock;
|
|
this.syncOnClose |= syncBlock && lastPacketInBlock;
|
|
|
|
|
|
@@ -625,14 +457,15 @@ class BlockReceiver implements Closeable {
|
|
//First write the packet to the mirror:
|
|
//First write the packet to the mirror:
|
|
if (mirrorOut != null && !mirrorError) {
|
|
if (mirrorOut != null && !mirrorError) {
|
|
try {
|
|
try {
|
|
- mirrorOut.write(buf.array(), buf.position(), buf.remaining());
|
|
|
|
|
|
+ packetReceiver.mirrorPacketTo(mirrorOut);
|
|
mirrorOut.flush();
|
|
mirrorOut.flush();
|
|
} catch (IOException e) {
|
|
} catch (IOException e) {
|
|
handleMirrorOutError(e);
|
|
handleMirrorOutError(e);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- buf.position(endOfHeader);
|
|
|
|
|
|
+ ByteBuffer dataBuf = packetReceiver.getDataSlice();
|
|
|
|
+ ByteBuffer checksumBuf = packetReceiver.getChecksumSlice();
|
|
|
|
|
|
if (lastPacketInBlock || len == 0) {
|
|
if (lastPacketInBlock || len == 0) {
|
|
if(LOG.isDebugEnabled()) {
|
|
if(LOG.isDebugEnabled()) {
|
|
@@ -646,18 +479,11 @@ class BlockReceiver implements Closeable {
|
|
int checksumLen = ((len + bytesPerChecksum - 1)/bytesPerChecksum)*
|
|
int checksumLen = ((len + bytesPerChecksum - 1)/bytesPerChecksum)*
|
|
checksumSize;
|
|
checksumSize;
|
|
|
|
|
|
- if ( buf.remaining() != (checksumLen + len)) {
|
|
|
|
- throw new IOException("Data remaining in packet does not match" +
|
|
|
|
- "sum of checksumLen and dataLen " +
|
|
|
|
- " size remaining: " + buf.remaining() +
|
|
|
|
- " data len: " + len +
|
|
|
|
- " checksum Len: " + checksumLen);
|
|
|
|
- }
|
|
|
|
- int checksumOff = buf.position();
|
|
|
|
- int dataOff = checksumOff + checksumLen;
|
|
|
|
- byte pktBuf[] = buf.array();
|
|
|
|
-
|
|
|
|
- buf.position(buf.limit()); // move to the end of the data.
|
|
|
|
|
|
+ if ( checksumBuf.capacity() != checksumLen) {
|
|
|
|
+ throw new IOException("Length of checksums in packet " +
|
|
|
|
+ checksumBuf.capacity() + " does not match calculated checksum " +
|
|
|
|
+ "length " + checksumLen);
|
|
|
|
+ }
|
|
|
|
|
|
/* skip verifying checksum iff this is not the last one in the
|
|
/* skip verifying checksum iff this is not the last one in the
|
|
* pipeline and clientName is non-null. i.e. Checksum is verified
|
|
* pipeline and clientName is non-null. i.e. Checksum is verified
|
|
@@ -667,11 +493,11 @@ class BlockReceiver implements Closeable {
|
|
* checksum.
|
|
* checksum.
|
|
*/
|
|
*/
|
|
if (mirrorOut == null || isDatanode || needsChecksumTranslation) {
|
|
if (mirrorOut == null || isDatanode || needsChecksumTranslation) {
|
|
- verifyChunks(pktBuf, dataOff, len, pktBuf, checksumOff);
|
|
|
|
|
|
+ verifyChunks(dataBuf, checksumBuf);
|
|
if (needsChecksumTranslation) {
|
|
if (needsChecksumTranslation) {
|
|
// overwrite the checksums in the packet buffer with the
|
|
// overwrite the checksums in the packet buffer with the
|
|
// appropriate polynomial for the disk storage.
|
|
// appropriate polynomial for the disk storage.
|
|
- translateChunks(pktBuf, dataOff, len, pktBuf, checksumOff);
|
|
|
|
|
|
+ translateChunks(dataBuf, checksumBuf);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -700,9 +526,13 @@ class BlockReceiver implements Closeable {
|
|
computePartialChunkCrc(onDiskLen, offsetInChecksum, bytesPerChecksum);
|
|
computePartialChunkCrc(onDiskLen, offsetInChecksum, bytesPerChecksum);
|
|
}
|
|
}
|
|
|
|
|
|
- int startByteToDisk = dataOff+(int)(onDiskLen-firstByteInBlock);
|
|
|
|
|
|
+ int startByteToDisk = (int)(onDiskLen-firstByteInBlock)
|
|
|
|
+ + dataBuf.arrayOffset() + dataBuf.position();
|
|
|
|
+
|
|
int numBytesToDisk = (int)(offsetInBlock-onDiskLen);
|
|
int numBytesToDisk = (int)(offsetInBlock-onDiskLen);
|
|
- writePacketToDisk(pktBuf, startByteToDisk, numBytesToDisk);
|
|
|
|
|
|
+
|
|
|
|
+ // Write data to disk.
|
|
|
|
+ out.write(dataBuf.array(), startByteToDisk, numBytesToDisk);
|
|
|
|
|
|
// If this is a partial chunk, then verify that this is the only
|
|
// If this is a partial chunk, then verify that this is the only
|
|
// chunk in the packet. Calculate new crc for this chunk.
|
|
// chunk in the packet. Calculate new crc for this chunk.
|
|
@@ -714,7 +544,7 @@ class BlockReceiver implements Closeable {
|
|
" len = " + len +
|
|
" len = " + len +
|
|
" bytesPerChecksum " + bytesPerChecksum);
|
|
" bytesPerChecksum " + bytesPerChecksum);
|
|
}
|
|
}
|
|
- partialCrc.update(pktBuf, startByteToDisk, numBytesToDisk);
|
|
|
|
|
|
+ partialCrc.update(dataBuf.array(), startByteToDisk, numBytesToDisk);
|
|
byte[] buf = FSOutputSummer.convertToByteStream(partialCrc, checksumSize);
|
|
byte[] buf = FSOutputSummer.convertToByteStream(partialCrc, checksumSize);
|
|
lastChunkChecksum = Arrays.copyOfRange(
|
|
lastChunkChecksum = Arrays.copyOfRange(
|
|
buf, buf.length - checksumSize, buf.length
|
|
buf, buf.length - checksumSize, buf.length
|
|
@@ -726,11 +556,12 @@ class BlockReceiver implements Closeable {
|
|
partialCrc = null;
|
|
partialCrc = null;
|
|
} else {
|
|
} else {
|
|
lastChunkChecksum = Arrays.copyOfRange(
|
|
lastChunkChecksum = Arrays.copyOfRange(
|
|
- pktBuf,
|
|
|
|
- checksumOff + checksumLen - checksumSize,
|
|
|
|
- checksumOff + checksumLen
|
|
|
|
- );
|
|
|
|
- checksumOut.write(pktBuf, checksumOff, checksumLen);
|
|
|
|
|
|
+ checksumBuf.array(),
|
|
|
|
+ checksumBuf.arrayOffset() + checksumBuf.position() + checksumLen - checksumSize,
|
|
|
|
+ checksumBuf.arrayOffset() + checksumBuf.position() + checksumLen);
|
|
|
|
+ checksumOut.write(checksumBuf.array(),
|
|
|
|
+ checksumBuf.arrayOffset() + checksumBuf.position(),
|
|
|
|
+ checksumLen);
|
|
}
|
|
}
|
|
/// flush entire packet, sync unless close() will sync
|
|
/// flush entire packet, sync unless close() will sync
|
|
flushOrSync(syncBlock && !lastPacketInBlock);
|
|
flushOrSync(syncBlock && !lastPacketInBlock);
|