|
@@ -28,8 +28,9 @@ import java.io.IOException;
|
|
import java.io.OutputStreamWriter;
|
|
import java.io.OutputStreamWriter;
|
|
import java.io.Writer;
|
|
import java.io.Writer;
|
|
import java.nio.ByteBuffer;
|
|
import java.nio.ByteBuffer;
|
|
|
|
+import java.util.ArrayDeque;
|
|
import java.util.Arrays;
|
|
import java.util.Arrays;
|
|
-import java.util.LinkedList;
|
|
|
|
|
|
+import java.util.Queue;
|
|
import java.util.zip.Checksum;
|
|
import java.util.zip.Checksum;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.Log;
|
|
@@ -1208,9 +1209,9 @@ class BlockReceiver implements Closeable {
|
|
* Processes responses from downstream datanodes in the pipeline
|
|
* Processes responses from downstream datanodes in the pipeline
|
|
* and sends back replies to the originator.
|
|
* and sends back replies to the originator.
|
|
*/
|
|
*/
|
|
- class PacketResponder implements Runnable, Closeable {
|
|
|
|
|
|
+ class PacketResponder implements Runnable, Closeable {
|
|
/** queue for packets waiting for ack - synchronization using monitor lock */
|
|
/** queue for packets waiting for ack - synchronization using monitor lock */
|
|
- private final LinkedList<Packet> ackQueue = new LinkedList<Packet>();
|
|
|
|
|
|
+ private final Queue<Packet> ackQueue = new ArrayDeque<>();
|
|
/** the thread that spawns this responder */
|
|
/** the thread that spawns this responder */
|
|
private final Thread receiverThread = Thread.currentThread();
|
|
private final Thread receiverThread = Thread.currentThread();
|
|
/** is this responder running? - synchronization using monitor lock */
|
|
/** is this responder running? - synchronization using monitor lock */
|
|
@@ -1264,12 +1265,10 @@ class BlockReceiver implements Closeable {
|
|
final long offsetInBlock, final Status ackStatus) {
|
|
final long offsetInBlock, final Status ackStatus) {
|
|
final Packet p = new Packet(seqno, lastPacketInBlock, offsetInBlock,
|
|
final Packet p = new Packet(seqno, lastPacketInBlock, offsetInBlock,
|
|
System.nanoTime(), ackStatus);
|
|
System.nanoTime(), ackStatus);
|
|
- if(LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug(myString + ": enqueue " + p);
|
|
|
|
- }
|
|
|
|
- synchronized(ackQueue) {
|
|
|
|
|
|
+ LOG.debug("{}: enqueue {}", this, p);
|
|
|
|
+ synchronized (ackQueue) {
|
|
if (running) {
|
|
if (running) {
|
|
- ackQueue.addLast(p);
|
|
|
|
|
|
+ ackQueue.add(p);
|
|
ackQueue.notifyAll();
|
|
ackQueue.notifyAll();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -1321,15 +1320,13 @@ class BlockReceiver implements Closeable {
|
|
|
|
|
|
/** Wait for a packet with given {@code seqno} to be enqueued to ackQueue */
|
|
/** Wait for a packet with given {@code seqno} to be enqueued to ackQueue */
|
|
Packet waitForAckHead(long seqno) throws InterruptedException {
|
|
Packet waitForAckHead(long seqno) throws InterruptedException {
|
|
- synchronized(ackQueue) {
|
|
|
|
- while (isRunning() && ackQueue.size() == 0) {
|
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug(myString + ": seqno=" + seqno +
|
|
|
|
- " waiting for local datanode to finish write.");
|
|
|
|
- }
|
|
|
|
|
|
+ synchronized (ackQueue) {
|
|
|
|
+ while (isRunning() && ackQueue.isEmpty()) {
|
|
|
|
+ LOG.debug("{}: seqno={} waiting for local datanode to finish write.",
|
|
|
|
+ myString, seqno);
|
|
ackQueue.wait();
|
|
ackQueue.wait();
|
|
}
|
|
}
|
|
- return isRunning() ? ackQueue.getFirst() : null;
|
|
|
|
|
|
+ return isRunning() ? ackQueue.element() : null;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1338,8 +1335,8 @@ class BlockReceiver implements Closeable {
|
|
*/
|
|
*/
|
|
@Override
|
|
@Override
|
|
public void close() {
|
|
public void close() {
|
|
- synchronized(ackQueue) {
|
|
|
|
- while (isRunning() && ackQueue.size() != 0) {
|
|
|
|
|
|
+ synchronized (ackQueue) {
|
|
|
|
+ while (isRunning() && !ackQueue.isEmpty()) {
|
|
try {
|
|
try {
|
|
ackQueue.wait();
|
|
ackQueue.wait();
|
|
} catch (InterruptedException e) {
|
|
} catch (InterruptedException e) {
|
|
@@ -1347,14 +1344,12 @@ class BlockReceiver implements Closeable {
|
|
Thread.currentThread().interrupt();
|
|
Thread.currentThread().interrupt();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
- if(LOG.isDebugEnabled()) {
|
|
|
|
- LOG.debug(myString + ": closing");
|
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("{}: closing", this);
|
|
running = false;
|
|
running = false;
|
|
ackQueue.notifyAll();
|
|
ackQueue.notifyAll();
|
|
}
|
|
}
|
|
|
|
|
|
- synchronized(this) {
|
|
|
|
|
|
+ synchronized (this) {
|
|
running = false;
|
|
running = false;
|
|
notifyAll();
|
|
notifyAll();
|
|
}
|
|
}
|
|
@@ -1657,12 +1652,12 @@ class BlockReceiver implements Closeable {
|
|
|
|
|
|
/**
|
|
/**
|
|
* Remove a packet from the head of the ack queue
|
|
* Remove a packet from the head of the ack queue
|
|
- *
|
|
|
|
|
|
+ *
|
|
* This should be called only when the ack queue is not empty
|
|
* This should be called only when the ack queue is not empty
|
|
*/
|
|
*/
|
|
private void removeAckHead() {
|
|
private void removeAckHead() {
|
|
- synchronized(ackQueue) {
|
|
|
|
- ackQueue.removeFirst();
|
|
|
|
|
|
+ synchronized (ackQueue) {
|
|
|
|
+ ackQueue.remove();
|
|
ackQueue.notifyAll();
|
|
ackQueue.notifyAll();
|
|
}
|
|
}
|
|
}
|
|
}
|