|
@@ -377,7 +377,8 @@ class BlockReceiver implements Closeable {
|
|
|
clientChecksum.verifyChunkedSums(dataBuf, checksumBuf, clientname, 0);
|
|
|
} catch (ChecksumException ce) {
|
|
|
LOG.warn("Checksum error in block " + block + " from " + inAddr, ce);
|
|
|
- if (srcDataNode != null) {
|
|
|
+ // No need to report to namenode when client is writing.
|
|
|
+ if (srcDataNode != null && isDatanode) {
|
|
|
try {
|
|
|
LOG.info("report corrupt " + block + " from datanode " +
|
|
|
srcDataNode + " to namenode");
|
|
@@ -404,6 +405,19 @@ class BlockReceiver implements Closeable {
|
|
|
diskChecksum.calculateChunkedSums(dataBuf, checksumBuf);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Check whether checksum needs to be verified.
|
|
|
+ * Skip verifying checksum iff this is not the last one in the
|
|
|
+ * pipeline and clientName is non-null. i.e. Checksum is verified
|
|
|
+ * on all the datanodes when the data is being written by a
|
|
|
+ * datanode rather than a client. Whe client is writing the data,
|
|
|
+ * protocol includes acks and only the last datanode needs to verify
|
|
|
+ * checksum.
|
|
|
+ * @return true if checksum verification is needed, otherwise false.
|
|
|
+ */
|
|
|
+ private boolean shouldVerifyChecksum() {
|
|
|
+ return (mirrorOut == null || isDatanode || needsChecksumTranslation);
|
|
|
+ }
|
|
|
|
|
|
/**
|
|
|
* Receives and processes a packet. It can contain many chunks.
|
|
@@ -451,9 +465,9 @@ class BlockReceiver implements Closeable {
|
|
|
}
|
|
|
|
|
|
// put in queue for pending acks, unless sync was requested
|
|
|
- if (responder != null && !syncBlock) {
|
|
|
+ if (responder != null && !syncBlock && !shouldVerifyChecksum()) {
|
|
|
((PacketResponder) responder.getRunnable()).enqueue(seqno,
|
|
|
- lastPacketInBlock, offsetInBlock);
|
|
|
+ lastPacketInBlock, offsetInBlock, Status.SUCCESS);
|
|
|
}
|
|
|
|
|
|
//First write the packet to the mirror:
|
|
@@ -485,17 +499,26 @@ class BlockReceiver implements Closeable {
|
|
|
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
|
|
|
- * pipeline and clientName is non-null. i.e. Checksum is verified
|
|
|
- * on all the datanodes when the data is being written by a
|
|
|
- * datanode rather than a client. Whe client is writing the data,
|
|
|
- * protocol includes acks and only the last datanode needs to verify
|
|
|
- * checksum.
|
|
|
- */
|
|
|
- if (mirrorOut == null || isDatanode || needsChecksumTranslation) {
|
|
|
- verifyChunks(dataBuf, checksumBuf);
|
|
|
+ }
|
|
|
+
|
|
|
+ if (shouldVerifyChecksum()) {
|
|
|
+ try {
|
|
|
+ verifyChunks(dataBuf, checksumBuf);
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ // checksum error detected locally. there is no reason to continue.
|
|
|
+ if (responder != null) {
|
|
|
+ try {
|
|
|
+ ((PacketResponder) responder.getRunnable()).enqueue(seqno,
|
|
|
+ lastPacketInBlock, offsetInBlock,
|
|
|
+ Status.ERROR_CHECKSUM);
|
|
|
+ // Wait until the responder sends back the response
|
|
|
+ // and interrupt this thread.
|
|
|
+ Thread.sleep(3000);
|
|
|
+ } catch (InterruptedException e) { }
|
|
|
+ }
|
|
|
+ throw new IOException("Terminating due to a checksum error." + ioe);
|
|
|
+ }
|
|
|
+
|
|
|
if (needsChecksumTranslation) {
|
|
|
// overwrite the checksums in the packet buffer with the
|
|
|
// appropriate polynomial for the disk storage.
|
|
@@ -584,9 +607,9 @@ class BlockReceiver implements Closeable {
|
|
|
|
|
|
// if sync was requested, put in queue for pending acks here
|
|
|
// (after the fsync finished)
|
|
|
- if (responder != null && syncBlock) {
|
|
|
+ if (responder != null && (syncBlock || shouldVerifyChecksum())) {
|
|
|
((PacketResponder) responder.getRunnable()).enqueue(seqno,
|
|
|
- lastPacketInBlock, offsetInBlock);
|
|
|
+ lastPacketInBlock, offsetInBlock, Status.SUCCESS);
|
|
|
}
|
|
|
|
|
|
if (throttler != null) { // throttle I/O
|
|
@@ -783,7 +806,7 @@ class BlockReceiver implements Closeable {
|
|
|
private static Status[] MIRROR_ERROR_STATUS = {Status.SUCCESS, Status.ERROR};
|
|
|
|
|
|
/**
|
|
|
- * Processed responses from downstream datanodes in the pipeline
|
|
|
+ * Processes responses from downstream datanodes in the pipeline
|
|
|
* and sends back replies to the originator.
|
|
|
*/
|
|
|
class PacketResponder implements Runnable, Closeable {
|
|
@@ -836,9 +859,9 @@ class BlockReceiver implements Closeable {
|
|
|
* @param offsetInBlock
|
|
|
*/
|
|
|
void enqueue(final long seqno, final boolean lastPacketInBlock,
|
|
|
- final long offsetInBlock) {
|
|
|
+ final long offsetInBlock, final Status ackStatus) {
|
|
|
final Packet p = new Packet(seqno, lastPacketInBlock, offsetInBlock,
|
|
|
- System.nanoTime());
|
|
|
+ System.nanoTime(), ackStatus);
|
|
|
if(LOG.isDebugEnabled()) {
|
|
|
LOG.debug(myString + ": enqueue " + p);
|
|
|
}
|
|
@@ -976,7 +999,8 @@ class BlockReceiver implements Closeable {
|
|
|
}
|
|
|
|
|
|
sendAckUpstream(ack, expected, totalAckTimeNanos,
|
|
|
- (pkt != null ? pkt.offsetInBlock : 0));
|
|
|
+ (pkt != null ? pkt.offsetInBlock : 0),
|
|
|
+ (pkt != null ? pkt.ackStatus : Status.SUCCESS));
|
|
|
if (pkt != null) {
|
|
|
// remove the packet from the ack queue
|
|
|
removeAckHead();
|
|
@@ -1038,7 +1062,8 @@ class BlockReceiver implements Closeable {
|
|
|
* @param offsetInBlock offset in block for the data in packet
|
|
|
*/
|
|
|
private void sendAckUpstream(PipelineAck ack, long seqno,
|
|
|
- long totalAckTimeNanos, long offsetInBlock) throws IOException {
|
|
|
+ long totalAckTimeNanos, long offsetInBlock,
|
|
|
+ Status myStatus) throws IOException {
|
|
|
Status[] replies = null;
|
|
|
if (mirrorError) { // ack read error
|
|
|
replies = MIRROR_ERROR_STATUS;
|
|
@@ -1046,10 +1071,19 @@ class BlockReceiver implements Closeable {
|
|
|
short ackLen = type == PacketResponderType.LAST_IN_PIPELINE ? 0 : ack
|
|
|
.getNumOfReplies();
|
|
|
replies = new Status[1 + ackLen];
|
|
|
- replies[0] = Status.SUCCESS;
|
|
|
+ replies[0] = myStatus;
|
|
|
for (int i = 0; i < ackLen; i++) {
|
|
|
replies[i + 1] = ack.getReply(i);
|
|
|
}
|
|
|
+ // If the mirror has reported that it received a corrupt packet,
|
|
|
+ // do self-destruct to mark myself bad, instead of making the
|
|
|
+ // mirror node bad. The mirror is guaranteed to be good without
|
|
|
+ // corrupt data on disk.
|
|
|
+ if (ackLen > 0 && replies[1] == Status.ERROR_CHECKSUM) {
|
|
|
+ throw new IOException("Shutting down writer and responder "
|
|
|
+ + "since the down streams reported the data sent by this "
|
|
|
+ + "thread is corrupt");
|
|
|
+ }
|
|
|
}
|
|
|
PipelineAck replyAck = new PipelineAck(seqno, replies,
|
|
|
totalAckTimeNanos);
|
|
@@ -1064,6 +1098,14 @@ class BlockReceiver implements Closeable {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug(myString + ", replyAck=" + replyAck);
|
|
|
}
|
|
|
+
|
|
|
+ // If a corruption was detected in the received data, terminate after
|
|
|
+ // sending ERROR_CHECKSUM back.
|
|
|
+ if (myStatus == Status.ERROR_CHECKSUM) {
|
|
|
+ throw new IOException("Shutting down writer and responder "
|
|
|
+ + "due to a checksum error in received data. The error "
|
|
|
+ + "response has been sent upstream.");
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1085,13 +1127,15 @@ class BlockReceiver implements Closeable {
|
|
|
final boolean lastPacketInBlock;
|
|
|
final long offsetInBlock;
|
|
|
final long ackEnqueueNanoTime;
|
|
|
+ final Status ackStatus;
|
|
|
|
|
|
Packet(long seqno, boolean lastPacketInBlock, long offsetInBlock,
|
|
|
- long ackEnqueueNanoTime) {
|
|
|
+ long ackEnqueueNanoTime, Status ackStatus) {
|
|
|
this.seqno = seqno;
|
|
|
this.lastPacketInBlock = lastPacketInBlock;
|
|
|
this.offsetInBlock = offsetInBlock;
|
|
|
this.ackEnqueueNanoTime = ackEnqueueNanoTime;
|
|
|
+ this.ackStatus = ackStatus;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -1100,6 +1144,7 @@ class BlockReceiver implements Closeable {
|
|
|
+ ", lastPacketInBlock=" + lastPacketInBlock
|
|
|
+ ", offsetInBlock=" + offsetInBlock
|
|
|
+ ", ackEnqueueNanoTime=" + ackEnqueueNanoTime
|
|
|
+ + ", ackStatus=" + ackStatus
|
|
|
+ ")";
|
|
|
}
|
|
|
}
|