|
@@ -21,7 +21,6 @@ import org.apache.hadoop.classification.VisibleForTesting;
|
|
import org.apache.hadoop.util.Preconditions;
|
|
import org.apache.hadoop.util.Preconditions;
|
|
import org.apache.hadoop.thirdparty.protobuf.ByteString;
|
|
import org.apache.hadoop.thirdparty.protobuf.ByteString;
|
|
import javax.crypto.SecretKey;
|
|
import javax.crypto.SecretKey;
|
|
-import org.apache.commons.logging.Log;
|
|
|
|
import org.apache.hadoop.fs.FsTracer;
|
|
import org.apache.hadoop.fs.FsTracer;
|
|
import org.apache.hadoop.fs.StorageType;
|
|
import org.apache.hadoop.fs.StorageType;
|
|
import org.apache.hadoop.hdfs.DFSUtilClient;
|
|
import org.apache.hadoop.hdfs.DFSUtilClient;
|
|
@@ -105,7 +104,7 @@ import static org.apache.hadoop.util.Time.monotonicNow;
|
|
*/
|
|
*/
|
|
class DataXceiver extends Receiver implements Runnable {
|
|
class DataXceiver extends Receiver implements Runnable {
|
|
public static final Logger LOG = DataNode.LOG;
|
|
public static final Logger LOG = DataNode.LOG;
|
|
- static final Log ClientTraceLog = DataNode.ClientTraceLog;
|
|
|
|
|
|
+ static final Logger CLIENT_TRACE_LOG = DataNode.CLIENT_TRACE_LOG;
|
|
|
|
|
|
private Peer peer;
|
|
private Peer peer;
|
|
private final String remoteAddress; // address of remote side
|
|
private final String remoteAddress; // address of remote side
|
|
@@ -426,10 +425,10 @@ class DataXceiver extends Receiver implements Runnable {
|
|
registeredSlotId);
|
|
registeredSlotId);
|
|
datanode.shortCircuitRegistry.unregisterSlot(registeredSlotId);
|
|
datanode.shortCircuitRegistry.unregisterSlot(registeredSlotId);
|
|
}
|
|
}
|
|
- if (ClientTraceLog.isInfoEnabled()) {
|
|
|
|
|
|
+ if (CLIENT_TRACE_LOG.isInfoEnabled()) {
|
|
DatanodeRegistration dnR = datanode.getDNRegistrationForBP(blk
|
|
DatanodeRegistration dnR = datanode.getDNRegistrationForBP(blk
|
|
.getBlockPoolId());
|
|
.getBlockPoolId());
|
|
- BlockSender.ClientTraceLog.info(String.format(
|
|
|
|
|
|
+ BlockSender.CLIENT_TRACE_LOG.info(String.format(
|
|
"src: 127.0.0.1, dest: 127.0.0.1, op: REQUEST_SHORT_CIRCUIT_FDS," +
|
|
"src: 127.0.0.1, dest: 127.0.0.1, op: REQUEST_SHORT_CIRCUIT_FDS," +
|
|
" blockid: %s, srvID: %s, success: %b",
|
|
" blockid: %s, srvID: %s, success: %b",
|
|
blk.getBlockId(), dnR.getDatanodeUuid(), success));
|
|
blk.getBlockId(), dnR.getDatanodeUuid(), success));
|
|
@@ -466,8 +465,8 @@ class DataXceiver extends Receiver implements Runnable {
|
|
bld.build().writeDelimitedTo(socketOut);
|
|
bld.build().writeDelimitedTo(socketOut);
|
|
success = true;
|
|
success = true;
|
|
} finally {
|
|
} finally {
|
|
- if (ClientTraceLog.isInfoEnabled()) {
|
|
|
|
- BlockSender.ClientTraceLog.info(String.format(
|
|
|
|
|
|
+ if (CLIENT_TRACE_LOG.isInfoEnabled()) {
|
|
|
|
+ BlockSender.CLIENT_TRACE_LOG.info(String.format(
|
|
"src: 127.0.0.1, dest: 127.0.0.1, op: RELEASE_SHORT_CIRCUIT_FDS," +
|
|
"src: 127.0.0.1, dest: 127.0.0.1, op: RELEASE_SHORT_CIRCUIT_FDS," +
|
|
" shmId: %016x%016x, slotIdx: %d, srvID: %s, success: %b",
|
|
" shmId: %016x%016x, slotIdx: %d, srvID: %s, success: %b",
|
|
slotId.getShmId().getHi(), slotId.getShmId().getLo(),
|
|
slotId.getShmId().getHi(), slotId.getShmId().getLo(),
|
|
@@ -526,9 +525,9 @@ class DataXceiver extends Receiver implements Runnable {
|
|
sendShmSuccessResponse(sock, shmInfo);
|
|
sendShmSuccessResponse(sock, shmInfo);
|
|
success = true;
|
|
success = true;
|
|
} finally {
|
|
} finally {
|
|
- if (ClientTraceLog.isInfoEnabled()) {
|
|
|
|
|
|
+ if (CLIENT_TRACE_LOG.isInfoEnabled()) {
|
|
if (success) {
|
|
if (success) {
|
|
- BlockSender.ClientTraceLog.info(String.format(
|
|
|
|
|
|
+ BlockSender.CLIENT_TRACE_LOG.info(String.format(
|
|
"cliID: %s, src: 127.0.0.1, dest: 127.0.0.1, " +
|
|
"cliID: %s, src: 127.0.0.1, dest: 127.0.0.1, " +
|
|
"op: REQUEST_SHORT_CIRCUIT_SHM," +
|
|
"op: REQUEST_SHORT_CIRCUIT_SHM," +
|
|
" shmId: %016x%016x, srvID: %s, success: true",
|
|
" shmId: %016x%016x, srvID: %s, success: true",
|
|
@@ -536,7 +535,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|
shmInfo.getShmId().getLo(),
|
|
shmInfo.getShmId().getLo(),
|
|
datanode.getDatanodeUuid()));
|
|
datanode.getDatanodeUuid()));
|
|
} else {
|
|
} else {
|
|
- BlockSender.ClientTraceLog.info(String.format(
|
|
|
|
|
|
+ BlockSender.CLIENT_TRACE_LOG.info(String.format(
|
|
"cliID: %s, src: 127.0.0.1, dest: 127.0.0.1, " +
|
|
"cliID: %s, src: 127.0.0.1, dest: 127.0.0.1, " +
|
|
"op: REQUEST_SHORT_CIRCUIT_SHM, " +
|
|
"op: REQUEST_SHORT_CIRCUIT_SHM, " +
|
|
"shmId: n/a, srvID: %s, success: false",
|
|
"shmId: n/a, srvID: %s, success: false",
|
|
@@ -587,13 +586,10 @@ class DataXceiver extends Receiver implements Runnable {
|
|
BlockSender blockSender = null;
|
|
BlockSender blockSender = null;
|
|
DatanodeRegistration dnR =
|
|
DatanodeRegistration dnR =
|
|
datanode.getDNRegistrationForBP(block.getBlockPoolId());
|
|
datanode.getDNRegistrationForBP(block.getBlockPoolId());
|
|
- final String clientTraceFmt =
|
|
|
|
- clientName.length() > 0 && ClientTraceLog.isInfoEnabled()
|
|
|
|
- ? String.format(DN_CLIENTTRACE_FORMAT, localAddress, remoteAddress,
|
|
|
|
- "", "%d", "HDFS_READ", clientName, "%d",
|
|
|
|
- dnR.getDatanodeUuid(), block, "%d")
|
|
|
|
- : dnR + " Served block " + block + " to " +
|
|
|
|
- remoteAddress;
|
|
|
|
|
|
+ final String clientTraceFmt = clientName.length() > 0 && CLIENT_TRACE_LOG.isInfoEnabled() ?
|
|
|
|
+ String.format(DN_CLIENTTRACE_FORMAT, localAddress, remoteAddress, "", "%d", "HDFS_READ",
|
|
|
|
+ clientName, "%d", dnR.getDatanodeUuid(), block, "%d") :
|
|
|
|
+ dnR + " Served block " + block + " to " + remoteAddress;
|
|
|
|
|
|
try {
|
|
try {
|
|
try {
|
|
try {
|