|
@@ -27,11 +27,11 @@ import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.net.Peer;
|
|
|
import org.apache.hadoop.hdfs.net.PeerServer;
|
|
|
-import org.apache.hadoop.hdfs.server.balancer.Balancer;
|
|
|
import org.apache.hadoop.hdfs.util.DataTransferThrottler;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.util.Daemon;
|
|
|
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
|
|
|
/**
|
|
|
* Server used for receiving/sending a block of data.
|
|
@@ -45,6 +45,7 @@ class DataXceiverServer implements Runnable {
|
|
|
private final PeerServer peerServer;
|
|
|
private final DataNode datanode;
|
|
|
private final HashMap<Peer, Thread> peers = new HashMap<Peer, Thread>();
|
|
|
+ private final HashMap<Peer, DataXceiver> peersXceiver = new HashMap<Peer, DataXceiver>();
|
|
|
private boolean closed = false;
|
|
|
|
|
|
/**
|
|
@@ -217,18 +218,38 @@ class DataXceiverServer implements Runnable {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- synchronized void addPeer(Peer peer, Thread t) throws IOException {
|
|
|
+ synchronized void addPeer(Peer peer, Thread t, DataXceiver xceiver)
|
|
|
+ throws IOException {
|
|
|
if (closed) {
|
|
|
throw new IOException("Server closed.");
|
|
|
}
|
|
|
peers.put(peer, t);
|
|
|
+ peersXceiver.put(peer, xceiver);
|
|
|
}
|
|
|
|
|
|
synchronized void closePeer(Peer peer) {
|
|
|
peers.remove(peer);
|
|
|
+ peersXceiver.remove(peer);
|
|
|
IOUtils.cleanup(null, peer);
|
|
|
}
|
|
|
|
|
|
+ // Sending OOB to all peers
|
|
|
+ public synchronized void sendOOBToPeers() {
|
|
|
+ if (!datanode.shutdownForUpgrade) {
|
|
|
+ return;
|
|
|
+ }
|
|
|
+
|
|
|
+ for (Peer p : peers.keySet()) {
|
|
|
+ try {
|
|
|
+ peersXceiver.get(p).sendOOB();
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.warn("Got error when sending OOB message.", e);
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ LOG.warn("Interrupted when sending OOB message.");
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
// Notify all peers of the shutdown and restart.
|
|
|
// datanode.shouldRun should still be true and datanode.restarting should
|
|
|
// be set true before calling this method.
|
|
@@ -247,6 +268,7 @@ class DataXceiverServer implements Runnable {
|
|
|
IOUtils.cleanup(LOG, p);
|
|
|
}
|
|
|
peers.clear();
|
|
|
+ peersXceiver.clear();
|
|
|
}
|
|
|
|
|
|
// Return the number of peers.
|
|
@@ -254,7 +276,14 @@ class DataXceiverServer implements Runnable {
|
|
|
return peers.size();
|
|
|
}
|
|
|
|
|
|
+ // Return the number of peers and DataXceivers.
|
|
|
+ @VisibleForTesting
|
|
|
+ synchronized int getNumPeersXceiver() {
|
|
|
+ return peersXceiver.size();
|
|
|
+ }
|
|
|
+
|
|
|
synchronized void releasePeer(Peer peer) {
|
|
|
peers.remove(peer);
|
|
|
+ peersXceiver.remove(peer);
|
|
|
}
|
|
|
}
|