|
@@ -26,6 +26,8 @@ import org.apache.hadoop.ipc.*;
|
|
import org.apache.hadoop.conf.*;
|
|
import org.apache.hadoop.conf.*;
|
|
import org.apache.hadoop.net.DNS;
|
|
import org.apache.hadoop.net.DNS;
|
|
import org.apache.hadoop.net.NetUtils;
|
|
import org.apache.hadoop.net.NetUtils;
|
|
|
|
+import org.apache.hadoop.net.SocketInputStream;
|
|
|
|
+import org.apache.hadoop.net.SocketOutputStream;
|
|
import org.apache.hadoop.util.*;
|
|
import org.apache.hadoop.util.*;
|
|
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
|
|
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
|
|
import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
|
|
import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
|
|
@@ -38,6 +40,8 @@ import org.apache.hadoop.dfs.datanode.metrics.DataNodeMetrics;
|
|
import java.io.*;
|
|
import java.io.*;
|
|
import java.net.*;
|
|
import java.net.*;
|
|
import java.nio.ByteBuffer;
|
|
import java.nio.ByteBuffer;
|
|
|
|
+import java.nio.channels.ServerSocketChannel;
|
|
|
|
+import java.nio.channels.SocketChannel;
|
|
import java.util.*;
|
|
import java.util.*;
|
|
import java.util.concurrent.Semaphore;
|
|
import java.util.concurrent.Semaphore;
|
|
import java.security.NoSuchAlgorithmException;
|
|
import java.security.NoSuchAlgorithmException;
|
|
@@ -249,7 +253,8 @@ public class DataNode implements FSConstants, Runnable {
|
|
|
|
|
|
|
|
|
|
// find free port
|
|
// find free port
|
|
- ServerSocket ss = new ServerSocket(tmpPort, 0, socAddr.getAddress());
|
|
|
|
|
|
+ ServerSocket ss = ServerSocketChannel.open().socket();
|
|
|
|
+ Server.bind(ss, socAddr, 0);
|
|
ss.setReceiveBufferSize(DEFAULT_DATA_SOCKET_SIZE);
|
|
ss.setReceiveBufferSize(DEFAULT_DATA_SOCKET_SIZE);
|
|
// adjust machine name with the actual port
|
|
// adjust machine name with the actual port
|
|
tmpPort = ss.getLocalPort();
|
|
tmpPort = ss.getLocalPort();
|
|
@@ -812,7 +817,7 @@ public class DataNode implements FSConstants, Runnable {
|
|
private static void receiveResponse(Socket s, int numTargets) throws IOException {
|
|
private static void receiveResponse(Socket s, int numTargets) throws IOException {
|
|
// check the response
|
|
// check the response
|
|
DataInputStream reply = new DataInputStream(new BufferedInputStream(
|
|
DataInputStream reply = new DataInputStream(new BufferedInputStream(
|
|
- s.getInputStream(), BUFFER_SIZE));
|
|
|
|
|
|
+ new SocketInputStream(s), BUFFER_SIZE));
|
|
try {
|
|
try {
|
|
for (int i = 0; i < numTargets; i++) {
|
|
for (int i = 0; i < numTargets; i++) {
|
|
short opStatus = reply.readShort();
|
|
short opStatus = reply.readShort();
|
|
@@ -828,7 +833,8 @@ public class DataNode implements FSConstants, Runnable {
|
|
|
|
|
|
/* utility function for sending a respose */
|
|
/* utility function for sending a respose */
|
|
private static void sendResponse(Socket s, short opStatus) throws IOException {
|
|
private static void sendResponse(Socket s, short opStatus) throws IOException {
|
|
- DataOutputStream reply = new DataOutputStream(s.getOutputStream());
|
|
|
|
|
|
+ DataOutputStream reply =
|
|
|
|
+ new DataOutputStream(new SocketOutputStream(s, WRITE_TIMEOUT));
|
|
try {
|
|
try {
|
|
reply.writeShort(opStatus);
|
|
reply.writeShort(opStatus);
|
|
reply.flush();
|
|
reply.flush();
|
|
@@ -926,7 +932,7 @@ public class DataNode implements FSConstants, Runnable {
|
|
DataInputStream in=null;
|
|
DataInputStream in=null;
|
|
try {
|
|
try {
|
|
in = new DataInputStream(
|
|
in = new DataInputStream(
|
|
- new BufferedInputStream(s.getInputStream(), BUFFER_SIZE));
|
|
|
|
|
|
+ new BufferedInputStream(new SocketInputStream(s), BUFFER_SIZE));
|
|
short version = in.readShort();
|
|
short version = in.readShort();
|
|
if ( version != DATA_TRANSFER_VERSION ) {
|
|
if ( version != DATA_TRANSFER_VERSION ) {
|
|
throw new IOException( "Version Mismatch" );
|
|
throw new IOException( "Version Mismatch" );
|
|
@@ -994,7 +1000,8 @@ public class DataNode implements FSConstants, Runnable {
|
|
|
|
|
|
// send the block
|
|
// send the block
|
|
DataOutputStream out = new DataOutputStream(
|
|
DataOutputStream out = new DataOutputStream(
|
|
- new BufferedOutputStream(s.getOutputStream(), SMALL_BUFFER_SIZE));
|
|
|
|
|
|
+ new BufferedOutputStream(new SocketOutputStream(s, WRITE_TIMEOUT),
|
|
|
|
+ SMALL_BUFFER_SIZE));
|
|
BlockSender blockSender = null;
|
|
BlockSender blockSender = null;
|
|
try {
|
|
try {
|
|
try {
|
|
try {
|
|
@@ -1084,7 +1091,8 @@ public class DataNode implements FSConstants, Runnable {
|
|
s.getInetAddress().toString(), isRecovery, client);
|
|
s.getInetAddress().toString(), isRecovery, client);
|
|
|
|
|
|
// get a connection back to the previous target
|
|
// get a connection back to the previous target
|
|
- replyOut = new DataOutputStream(s.getOutputStream());
|
|
|
|
|
|
+ replyOut = new DataOutputStream(
|
|
|
|
+ new SocketOutputStream(s, WRITE_TIMEOUT));
|
|
|
|
|
|
//
|
|
//
|
|
// Open network conn to backup machine, if
|
|
// Open network conn to backup machine, if
|
|
@@ -1095,16 +1103,19 @@ public class DataNode implements FSConstants, Runnable {
|
|
// Connect to backup machine
|
|
// Connect to backup machine
|
|
mirrorNode = targets[0].getName();
|
|
mirrorNode = targets[0].getName();
|
|
mirrorTarget = NetUtils.createSocketAddr(mirrorNode);
|
|
mirrorTarget = NetUtils.createSocketAddr(mirrorNode);
|
|
- mirrorSock = new Socket();
|
|
|
|
|
|
+ mirrorSock = SocketChannel.open().socket();
|
|
try {
|
|
try {
|
|
int timeoutValue = numTargets * socketTimeout;
|
|
int timeoutValue = numTargets * socketTimeout;
|
|
|
|
+ int writeTimeout = WRITE_TIMEOUT +
|
|
|
|
+ (WRITE_TIMEOUT_EXTENSION * numTargets);
|
|
mirrorSock.connect(mirrorTarget, timeoutValue);
|
|
mirrorSock.connect(mirrorTarget, timeoutValue);
|
|
mirrorSock.setSoTimeout(timeoutValue);
|
|
mirrorSock.setSoTimeout(timeoutValue);
|
|
mirrorSock.setSendBufferSize(DEFAULT_DATA_SOCKET_SIZE);
|
|
mirrorSock.setSendBufferSize(DEFAULT_DATA_SOCKET_SIZE);
|
|
mirrorOut = new DataOutputStream(
|
|
mirrorOut = new DataOutputStream(
|
|
- new BufferedOutputStream(mirrorSock.getOutputStream(),
|
|
|
|
- BUFFER_SIZE));
|
|
|
|
- mirrorIn = new DataInputStream(mirrorSock.getInputStream());
|
|
|
|
|
|
+ new BufferedOutputStream(
|
|
|
|
+ new SocketOutputStream(mirrorSock, writeTimeout),
|
|
|
|
+ BUFFER_SIZE));
|
|
|
|
+ mirrorIn = new DataInputStream(new SocketInputStream(mirrorSock));
|
|
|
|
|
|
// Write header: Copied from DFSClient.java!
|
|
// Write header: Copied from DFSClient.java!
|
|
mirrorOut.writeShort( DATA_TRANSFER_VERSION );
|
|
mirrorOut.writeShort( DATA_TRANSFER_VERSION );
|
|
@@ -1214,7 +1225,7 @@ public class DataNode implements FSConstants, Runnable {
|
|
byte [] buf = new byte[(int)fileSize];
|
|
byte [] buf = new byte[(int)fileSize];
|
|
IOUtils.readFully(checksumIn, buf, 0, buf.length);
|
|
IOUtils.readFully(checksumIn, buf, 0, buf.length);
|
|
|
|
|
|
- out = new DataOutputStream(s.getOutputStream());
|
|
|
|
|
|
+ out = new DataOutputStream(new SocketOutputStream(s, WRITE_TIMEOUT));
|
|
|
|
|
|
out.writeByte(OP_STATUS_SUCCESS);
|
|
out.writeByte(OP_STATUS_SUCCESS);
|
|
out.writeInt(buf.length);
|
|
out.writeInt(buf.length);
|
|
@@ -1224,6 +1235,7 @@ public class DataNode implements FSConstants, Runnable {
|
|
out.writeInt(0);
|
|
out.writeInt(0);
|
|
} finally {
|
|
} finally {
|
|
xceiverCount.decr();
|
|
xceiverCount.decr();
|
|
|
|
+ IOUtils.closeStream(out);
|
|
IOUtils.closeStream(checksumIn);
|
|
IOUtils.closeStream(checksumIn);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -1255,12 +1267,13 @@ public class DataNode implements FSConstants, Runnable {
|
|
|
|
|
|
// get the output stream to the target
|
|
// get the output stream to the target
|
|
InetSocketAddress targetAddr = NetUtils.createSocketAddr(target.getName());
|
|
InetSocketAddress targetAddr = NetUtils.createSocketAddr(target.getName());
|
|
- targetSock = new Socket();
|
|
|
|
|
|
+ targetSock = SocketChannel.open().socket();
|
|
targetSock.connect(targetAddr, socketTimeout);
|
|
targetSock.connect(targetAddr, socketTimeout);
|
|
targetSock.setSoTimeout(socketTimeout);
|
|
targetSock.setSoTimeout(socketTimeout);
|
|
|
|
|
|
targetOut = new DataOutputStream(new BufferedOutputStream(
|
|
targetOut = new DataOutputStream(new BufferedOutputStream(
|
|
- targetSock.getOutputStream(), SMALL_BUFFER_SIZE));
|
|
|
|
|
|
+ new SocketOutputStream(targetSock, WRITE_TIMEOUT),
|
|
|
|
+ SMALL_BUFFER_SIZE));
|
|
|
|
|
|
/* send request to the target */
|
|
/* send request to the target */
|
|
// fist write header info
|
|
// fist write header info
|
|
@@ -2541,12 +2554,14 @@ public class DataNode implements FSConstants, Runnable {
|
|
try {
|
|
try {
|
|
InetSocketAddress curTarget =
|
|
InetSocketAddress curTarget =
|
|
NetUtils.createSocketAddr(targets[0].getName());
|
|
NetUtils.createSocketAddr(targets[0].getName());
|
|
- sock = new Socket();
|
|
|
|
|
|
+ sock = SocketChannel.open().socket();
|
|
sock.connect(curTarget, socketTimeout);
|
|
sock.connect(curTarget, socketTimeout);
|
|
sock.setSoTimeout(targets.length * socketTimeout);
|
|
sock.setSoTimeout(targets.length * socketTimeout);
|
|
|
|
|
|
|
|
+ long writeTimeout = WRITE_TIMEOUT +
|
|
|
|
+ WRITE_TIMEOUT_EXTENSION * (targets.length-1);
|
|
out = new DataOutputStream(new BufferedOutputStream(
|
|
out = new DataOutputStream(new BufferedOutputStream(
|
|
- sock.getOutputStream(), SMALL_BUFFER_SIZE));
|
|
|
|
|
|
+ new SocketOutputStream(sock, writeTimeout), SMALL_BUFFER_SIZE));
|
|
|
|
|
|
blockSender = new BlockSender(b, 0, -1, false, false, false);
|
|
blockSender = new BlockSender(b, 0, -1, false, false, false);
|
|
|
|
|