|
@@ -29,6 +29,7 @@ import java.io.DataInputStream;
|
|
import java.io.DataOutputStream;
|
|
import java.io.DataOutputStream;
|
|
import java.io.EOFException;
|
|
import java.io.EOFException;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
|
|
+import java.io.InputStream;
|
|
import java.io.InterruptedIOException;
|
|
import java.io.InterruptedIOException;
|
|
import java.io.OutputStream;
|
|
import java.io.OutputStream;
|
|
import java.net.InetSocketAddress;
|
|
import java.net.InetSocketAddress;
|
|
@@ -43,7 +44,10 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor.InvalidMagicNumberException;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.Op;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.Op;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.Receiver;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.Receiver;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
|
|
@@ -84,7 +88,8 @@ class DataXceiver extends Receiver implements Runnable {
|
|
private final DataXceiverServer dataXceiverServer;
|
|
private final DataXceiverServer dataXceiverServer;
|
|
|
|
|
|
private long opStartTime; //the start time of receiving an Op
|
|
private long opStartTime; //the start time of receiving an Op
|
|
- private final SocketInputWrapper socketInputWrapper;
|
|
|
|
|
|
+ private final SocketInputWrapper socketIn;
|
|
|
|
+ private OutputStream socketOut;
|
|
|
|
|
|
/**
|
|
/**
|
|
* Client Name used in previous operation. Not available on first request
|
|
* Client Name used in previous operation. Not available on first request
|
|
@@ -94,23 +99,19 @@ class DataXceiver extends Receiver implements Runnable {
|
|
|
|
|
|
public static DataXceiver create(Socket s, DataNode dn,
|
|
public static DataXceiver create(Socket s, DataNode dn,
|
|
DataXceiverServer dataXceiverServer) throws IOException {
|
|
DataXceiverServer dataXceiverServer) throws IOException {
|
|
-
|
|
|
|
- SocketInputWrapper iw = NetUtils.getInputStream(s);
|
|
|
|
- return new DataXceiver(s, iw, dn, dataXceiverServer);
|
|
|
|
|
|
+ return new DataXceiver(s, dn, dataXceiverServer);
|
|
}
|
|
}
|
|
|
|
|
|
private DataXceiver(Socket s,
|
|
private DataXceiver(Socket s,
|
|
- SocketInputWrapper socketInput,
|
|
|
|
DataNode datanode,
|
|
DataNode datanode,
|
|
DataXceiverServer dataXceiverServer) throws IOException {
|
|
DataXceiverServer dataXceiverServer) throws IOException {
|
|
- super(new DataInputStream(new BufferedInputStream(
|
|
|
|
- socketInput, HdfsConstants.SMALL_BUFFER_SIZE)));
|
|
|
|
|
|
|
|
this.s = s;
|
|
this.s = s;
|
|
- this.socketInputWrapper = socketInput;
|
|
|
|
|
|
+ this.dnConf = datanode.getDnConf();
|
|
|
|
+ this.socketIn = NetUtils.getInputStream(s);
|
|
|
|
+ this.socketOut = NetUtils.getOutputStream(s, dnConf.socketWriteTimeout);
|
|
this.isLocal = s.getInetAddress().equals(s.getLocalAddress());
|
|
this.isLocal = s.getInetAddress().equals(s.getLocalAddress());
|
|
this.datanode = datanode;
|
|
this.datanode = datanode;
|
|
- this.dnConf = datanode.getDnConf();
|
|
|
|
this.dataXceiverServer = dataXceiverServer;
|
|
this.dataXceiverServer = dataXceiverServer;
|
|
remoteAddress = s.getRemoteSocketAddress().toString();
|
|
remoteAddress = s.getRemoteSocketAddress().toString();
|
|
localAddress = s.getLocalSocketAddress().toString();
|
|
localAddress = s.getLocalSocketAddress().toString();
|
|
@@ -141,6 +142,10 @@ class DataXceiver extends Receiver implements Runnable {
|
|
|
|
|
|
/** Return the datanode object. */
|
|
/** Return the datanode object. */
|
|
DataNode getDataNode() {return datanode;}
|
|
DataNode getDataNode() {return datanode;}
|
|
|
|
+
|
|
|
|
+ private OutputStream getOutputStream() throws IOException {
|
|
|
|
+ return socketOut;
|
|
|
|
+ }
|
|
|
|
|
|
/**
|
|
/**
|
|
* Read/write data from/to the DataXceiverServer.
|
|
* Read/write data from/to the DataXceiverServer.
|
|
@@ -149,8 +154,31 @@ class DataXceiver extends Receiver implements Runnable {
|
|
public void run() {
|
|
public void run() {
|
|
int opsProcessed = 0;
|
|
int opsProcessed = 0;
|
|
Op op = null;
|
|
Op op = null;
|
|
|
|
+
|
|
dataXceiverServer.childSockets.add(s);
|
|
dataXceiverServer.childSockets.add(s);
|
|
|
|
+
|
|
try {
|
|
try {
|
|
|
|
+
|
|
|
|
+ InputStream input = socketIn;
|
|
|
|
+ if (dnConf.encryptDataTransfer) {
|
|
|
|
+ IOStreamPair encryptedStreams = null;
|
|
|
|
+ try {
|
|
|
|
+ encryptedStreams = DataTransferEncryptor.getEncryptedStreams(socketOut,
|
|
|
|
+ socketIn, datanode.blockPoolTokenSecretManager,
|
|
|
|
+ dnConf.encryptionAlgorithm);
|
|
|
|
+ } catch (InvalidMagicNumberException imne) {
|
|
|
|
+ LOG.info("Failed to read expected encryption handshake from client " +
|
|
|
|
+ "at " + s.getInetAddress() + ". Perhaps the client is running an " +
|
|
|
|
+ "older version of Hadoop which does not support encryption.");
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+ input = encryptedStreams.in;
|
|
|
|
+ socketOut = encryptedStreams.out;
|
|
|
|
+ }
|
|
|
|
+ input = new BufferedInputStream(input, HdfsConstants.SMALL_BUFFER_SIZE);
|
|
|
|
+
|
|
|
|
+ super.initialize(new DataInputStream(input));
|
|
|
|
+
|
|
// We process requests in a loop, and stay around for a short timeout.
|
|
// We process requests in a loop, and stay around for a short timeout.
|
|
// This optimistic behaviour allows the other end to reuse connections.
|
|
// This optimistic behaviour allows the other end to reuse connections.
|
|
// Setting keepalive timeout to 0 disable this behavior.
|
|
// Setting keepalive timeout to 0 disable this behavior.
|
|
@@ -160,9 +188,9 @@ class DataXceiver extends Receiver implements Runnable {
|
|
try {
|
|
try {
|
|
if (opsProcessed != 0) {
|
|
if (opsProcessed != 0) {
|
|
assert dnConf.socketKeepaliveTimeout > 0;
|
|
assert dnConf.socketKeepaliveTimeout > 0;
|
|
- socketInputWrapper.setTimeout(dnConf.socketKeepaliveTimeout);
|
|
|
|
|
|
+ socketIn.setTimeout(dnConf.socketKeepaliveTimeout);
|
|
} else {
|
|
} else {
|
|
- socketInputWrapper.setTimeout(dnConf.socketTimeout);
|
|
|
|
|
|
+ socketIn.setTimeout(dnConf.socketTimeout);
|
|
}
|
|
}
|
|
op = readOp();
|
|
op = readOp();
|
|
} catch (InterruptedIOException ignored) {
|
|
} catch (InterruptedIOException ignored) {
|
|
@@ -215,8 +243,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|
final long length) throws IOException {
|
|
final long length) throws IOException {
|
|
previousOpClientName = clientName;
|
|
previousOpClientName = clientName;
|
|
|
|
|
|
- OutputStream baseStream = NetUtils.getOutputStream(s,
|
|
|
|
- dnConf.socketWriteTimeout);
|
|
|
|
|
|
+ OutputStream baseStream = getOutputStream();
|
|
DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
|
|
DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
|
|
baseStream, HdfsConstants.SMALL_BUFFER_SIZE));
|
|
baseStream, HdfsConstants.SMALL_BUFFER_SIZE));
|
|
checkAccess(out, true, block, blockToken,
|
|
checkAccess(out, true, block, blockToken,
|
|
@@ -242,13 +269,12 @@ class DataXceiver extends Receiver implements Runnable {
|
|
} catch(IOException e) {
|
|
} catch(IOException e) {
|
|
String msg = "opReadBlock " + block + " received exception " + e;
|
|
String msg = "opReadBlock " + block + " received exception " + e;
|
|
LOG.info(msg);
|
|
LOG.info(msg);
|
|
- sendResponse(s, ERROR, msg, dnConf.socketWriteTimeout);
|
|
|
|
|
|
+ sendResponse(ERROR, msg);
|
|
throw e;
|
|
throw e;
|
|
}
|
|
}
|
|
|
|
|
|
// send op status
|
|
// send op status
|
|
- writeSuccessWithChecksumInfo(blockSender,
|
|
|
|
- getStreamWithTimeout(s, dnConf.socketWriteTimeout));
|
|
|
|
|
|
+ writeSuccessWithChecksumInfo(blockSender, new DataOutputStream(getOutputStream()));
|
|
|
|
|
|
long read = blockSender.sendBlock(out, baseStream, null); // send data
|
|
long read = blockSender.sendBlock(out, baseStream, null); // send data
|
|
|
|
|
|
@@ -347,7 +373,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|
// reply to upstream datanode or client
|
|
// reply to upstream datanode or client
|
|
final DataOutputStream replyOut = new DataOutputStream(
|
|
final DataOutputStream replyOut = new DataOutputStream(
|
|
new BufferedOutputStream(
|
|
new BufferedOutputStream(
|
|
- NetUtils.getOutputStream(s, dnConf.socketWriteTimeout),
|
|
|
|
|
|
+ getOutputStream(),
|
|
HdfsConstants.SMALL_BUFFER_SIZE));
|
|
HdfsConstants.SMALL_BUFFER_SIZE));
|
|
checkAccess(replyOut, isClient, block, blockToken,
|
|
checkAccess(replyOut, isClient, block, blockToken,
|
|
Op.WRITE_BLOCK, BlockTokenSecretManager.AccessMode.WRITE);
|
|
Op.WRITE_BLOCK, BlockTokenSecretManager.AccessMode.WRITE);
|
|
@@ -389,11 +415,23 @@ class DataXceiver extends Receiver implements Runnable {
|
|
NetUtils.connect(mirrorSock, mirrorTarget, timeoutValue);
|
|
NetUtils.connect(mirrorSock, mirrorTarget, timeoutValue);
|
|
mirrorSock.setSoTimeout(timeoutValue);
|
|
mirrorSock.setSoTimeout(timeoutValue);
|
|
mirrorSock.setSendBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
|
|
mirrorSock.setSendBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
|
|
- mirrorOut = new DataOutputStream(
|
|
|
|
- new BufferedOutputStream(
|
|
|
|
- NetUtils.getOutputStream(mirrorSock, writeTimeout),
|
|
|
|
- HdfsConstants.SMALL_BUFFER_SIZE));
|
|
|
|
- mirrorIn = new DataInputStream(NetUtils.getInputStream(mirrorSock));
|
|
|
|
|
|
+
|
|
|
|
+ OutputStream unbufMirrorOut = NetUtils.getOutputStream(mirrorSock,
|
|
|
|
+ writeTimeout);
|
|
|
|
+ InputStream unbufMirrorIn = NetUtils.getInputStream(mirrorSock);
|
|
|
|
+ if (dnConf.encryptDataTransfer) {
|
|
|
|
+ IOStreamPair encryptedStreams =
|
|
|
|
+ DataTransferEncryptor.getEncryptedStreams(
|
|
|
|
+ unbufMirrorOut, unbufMirrorIn,
|
|
|
|
+ datanode.blockPoolTokenSecretManager
|
|
|
|
+ .generateDataEncryptionKey(block.getBlockPoolId()));
|
|
|
|
+
|
|
|
|
+ unbufMirrorOut = encryptedStreams.out;
|
|
|
|
+ unbufMirrorIn = encryptedStreams.in;
|
|
|
|
+ }
|
|
|
|
+ mirrorOut = new DataOutputStream(new BufferedOutputStream(unbufMirrorOut,
|
|
|
|
+ HdfsConstants.SMALL_BUFFER_SIZE));
|
|
|
|
+ mirrorIn = new DataInputStream(unbufMirrorIn);
|
|
|
|
|
|
new Sender(mirrorOut).writeBlock(originalBlock, blockToken,
|
|
new Sender(mirrorOut).writeBlock(originalBlock, blockToken,
|
|
clientname, targets, srcDataNode, stage, pipelineSize,
|
|
clientname, targets, srcDataNode, stage, pipelineSize,
|
|
@@ -520,7 +558,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|
updateCurrentThreadName(Op.TRANSFER_BLOCK + " " + blk);
|
|
updateCurrentThreadName(Op.TRANSFER_BLOCK + " " + blk);
|
|
|
|
|
|
final DataOutputStream out = new DataOutputStream(
|
|
final DataOutputStream out = new DataOutputStream(
|
|
- NetUtils.getOutputStream(s, dnConf.socketWriteTimeout));
|
|
|
|
|
|
+ getOutputStream());
|
|
try {
|
|
try {
|
|
datanode.transferReplicaForPipelineRecovery(blk, targets, clientName);
|
|
datanode.transferReplicaForPipelineRecovery(blk, targets, clientName);
|
|
writeResponse(Status.SUCCESS, null, out);
|
|
writeResponse(Status.SUCCESS, null, out);
|
|
@@ -533,7 +571,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|
public void blockChecksum(final ExtendedBlock block,
|
|
public void blockChecksum(final ExtendedBlock block,
|
|
final Token<BlockTokenIdentifier> blockToken) throws IOException {
|
|
final Token<BlockTokenIdentifier> blockToken) throws IOException {
|
|
final DataOutputStream out = new DataOutputStream(
|
|
final DataOutputStream out = new DataOutputStream(
|
|
- NetUtils.getOutputStream(s, dnConf.socketWriteTimeout));
|
|
|
|
|
|
+ getOutputStream());
|
|
checkAccess(out, true, block, blockToken,
|
|
checkAccess(out, true, block, blockToken,
|
|
Op.BLOCK_CHECKSUM, BlockTokenSecretManager.AccessMode.READ);
|
|
Op.BLOCK_CHECKSUM, BlockTokenSecretManager.AccessMode.READ);
|
|
updateCurrentThreadName("Reading metadata for block " + block);
|
|
updateCurrentThreadName("Reading metadata for block " + block);
|
|
@@ -593,7 +631,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|
LOG.warn("Invalid access token in request from " + remoteAddress
|
|
LOG.warn("Invalid access token in request from " + remoteAddress
|
|
+ " for OP_COPY_BLOCK for block " + block + " : "
|
|
+ " for OP_COPY_BLOCK for block " + block + " : "
|
|
+ e.getLocalizedMessage());
|
|
+ e.getLocalizedMessage());
|
|
- sendResponse(s, ERROR_ACCESS_TOKEN, "Invalid access token", dnConf.socketWriteTimeout);
|
|
|
|
|
|
+ sendResponse(ERROR_ACCESS_TOKEN, "Invalid access token");
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -603,7 +641,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|
String msg = "Not able to copy block " + block.getBlockId() + " to "
|
|
String msg = "Not able to copy block " + block.getBlockId() + " to "
|
|
+ s.getRemoteSocketAddress() + " because threads quota is exceeded.";
|
|
+ s.getRemoteSocketAddress() + " because threads quota is exceeded.";
|
|
LOG.info(msg);
|
|
LOG.info(msg);
|
|
- sendResponse(s, ERROR, msg, dnConf.socketWriteTimeout);
|
|
|
|
|
|
+ sendResponse(ERROR, msg);
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -617,8 +655,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|
null);
|
|
null);
|
|
|
|
|
|
// set up response stream
|
|
// set up response stream
|
|
- OutputStream baseStream = NetUtils.getOutputStream(
|
|
|
|
- s, dnConf.socketWriteTimeout);
|
|
|
|
|
|
+ OutputStream baseStream = getOutputStream();
|
|
reply = new DataOutputStream(new BufferedOutputStream(
|
|
reply = new DataOutputStream(new BufferedOutputStream(
|
|
baseStream, HdfsConstants.SMALL_BUFFER_SIZE));
|
|
baseStream, HdfsConstants.SMALL_BUFFER_SIZE));
|
|
|
|
|
|
@@ -670,8 +707,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|
LOG.warn("Invalid access token in request from " + remoteAddress
|
|
LOG.warn("Invalid access token in request from " + remoteAddress
|
|
+ " for OP_REPLACE_BLOCK for block " + block + " : "
|
|
+ " for OP_REPLACE_BLOCK for block " + block + " : "
|
|
+ e.getLocalizedMessage());
|
|
+ e.getLocalizedMessage());
|
|
- sendResponse(s, ERROR_ACCESS_TOKEN, "Invalid access token",
|
|
|
|
- dnConf.socketWriteTimeout);
|
|
|
|
|
|
+ sendResponse(ERROR_ACCESS_TOKEN, "Invalid access token");
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -680,7 +716,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|
String msg = "Not able to receive block " + block.getBlockId() + " from "
|
|
String msg = "Not able to receive block " + block.getBlockId() + " from "
|
|
+ s.getRemoteSocketAddress() + " because threads quota is exceeded.";
|
|
+ s.getRemoteSocketAddress() + " because threads quota is exceeded.";
|
|
LOG.warn(msg);
|
|
LOG.warn(msg);
|
|
- sendResponse(s, ERROR, msg, dnConf.socketWriteTimeout);
|
|
|
|
|
|
+ sendResponse(ERROR, msg);
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -699,17 +735,29 @@ class DataXceiver extends Receiver implements Runnable {
|
|
NetUtils.connect(proxySock, proxyAddr, dnConf.socketTimeout);
|
|
NetUtils.connect(proxySock, proxyAddr, dnConf.socketTimeout);
|
|
proxySock.setSoTimeout(dnConf.socketTimeout);
|
|
proxySock.setSoTimeout(dnConf.socketTimeout);
|
|
|
|
|
|
- OutputStream baseStream = NetUtils.getOutputStream(proxySock,
|
|
|
|
|
|
+ OutputStream unbufProxyOut = NetUtils.getOutputStream(proxySock,
|
|
dnConf.socketWriteTimeout);
|
|
dnConf.socketWriteTimeout);
|
|
- proxyOut = new DataOutputStream(new BufferedOutputStream(baseStream,
|
|
|
|
|
|
+ InputStream unbufProxyIn = NetUtils.getInputStream(proxySock);
|
|
|
|
+ if (dnConf.encryptDataTransfer) {
|
|
|
|
+ IOStreamPair encryptedStreams =
|
|
|
|
+ DataTransferEncryptor.getEncryptedStreams(
|
|
|
|
+ unbufProxyOut, unbufProxyIn,
|
|
|
|
+ datanode.blockPoolTokenSecretManager
|
|
|
|
+ .generateDataEncryptionKey(block.getBlockPoolId()));
|
|
|
|
+ unbufProxyOut = encryptedStreams.out;
|
|
|
|
+ unbufProxyIn = encryptedStreams.in;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ proxyOut = new DataOutputStream(new BufferedOutputStream(unbufProxyOut,
|
|
HdfsConstants.SMALL_BUFFER_SIZE));
|
|
HdfsConstants.SMALL_BUFFER_SIZE));
|
|
|
|
+ proxyReply = new DataInputStream(new BufferedInputStream(unbufProxyIn,
|
|
|
|
+ HdfsConstants.IO_FILE_BUFFER_SIZE));
|
|
|
|
|
|
/* send request to the proxy */
|
|
/* send request to the proxy */
|
|
new Sender(proxyOut).copyBlock(block, blockToken);
|
|
new Sender(proxyOut).copyBlock(block, blockToken);
|
|
|
|
|
|
// receive the response from the proxy
|
|
// receive the response from the proxy
|
|
- proxyReply = new DataInputStream(new BufferedInputStream(
|
|
|
|
- NetUtils.getInputStream(proxySock), HdfsConstants.IO_FILE_BUFFER_SIZE));
|
|
|
|
|
|
+
|
|
BlockOpResponseProto copyResponse = BlockOpResponseProto.parseFrom(
|
|
BlockOpResponseProto copyResponse = BlockOpResponseProto.parseFrom(
|
|
HdfsProtoUtil.vintPrefixed(proxyReply));
|
|
HdfsProtoUtil.vintPrefixed(proxyReply));
|
|
|
|
|
|
@@ -762,7 +810,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|
|
|
|
|
// send response back
|
|
// send response back
|
|
try {
|
|
try {
|
|
- sendResponse(s, opStatus, errMsg, dnConf.socketWriteTimeout);
|
|
|
|
|
|
+ sendResponse(opStatus, errMsg);
|
|
} catch (IOException ioe) {
|
|
} catch (IOException ioe) {
|
|
LOG.warn("Error writing reply back to " + s.getRemoteSocketAddress());
|
|
LOG.warn("Error writing reply back to " + s.getRemoteSocketAddress());
|
|
}
|
|
}
|
|
@@ -781,20 +829,13 @@ class DataXceiver extends Receiver implements Runnable {
|
|
|
|
|
|
/**
|
|
/**
|
|
* Utility function for sending a response.
|
|
* Utility function for sending a response.
|
|
- * @param s socket to write to
|
|
|
|
|
|
+ *
|
|
* @param opStatus status message to write
|
|
* @param opStatus status message to write
|
|
- * @param timeout send timeout
|
|
|
|
- **/
|
|
|
|
- private static void sendResponse(Socket s, Status status, String message,
|
|
|
|
- long timeout) throws IOException {
|
|
|
|
- DataOutputStream reply = getStreamWithTimeout(s, timeout);
|
|
|
|
-
|
|
|
|
- writeResponse(status, message, reply);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private static DataOutputStream getStreamWithTimeout(Socket s, long timeout)
|
|
|
|
- throws IOException {
|
|
|
|
- return new DataOutputStream(NetUtils.getOutputStream(s, timeout));
|
|
|
|
|
|
+ * @param message message to send to the client or other DN
|
|
|
|
+ */
|
|
|
|
+ private void sendResponse(Status status,
|
|
|
|
+ String message) throws IOException {
|
|
|
|
+ writeResponse(status, message, getOutputStream());
|
|
}
|
|
}
|
|
|
|
|
|
private static void writeResponse(Status status, String message, OutputStream out)
|
|
private static void writeResponse(Status status, String message, OutputStream out)
|