Browse Source

HDFS-13968. BlockReceiver Array-Based Queue. Contributed by BELUGA BEHR.

Inigo Goiri 6 years ago
parent
commit
3532aa3886

+ 19 - 24
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

@@ -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();
       }
       }
     }
     }