HDDS-2260. Avoid evaluation of LOG.trace and LOG.debug statement in the read/write path (HDDS). (#1612)

This commit is contained in:
Siddharth 2019-10-08 11:13:54 -07:00 committed by Bharat Viswanadham
parent 91320b4461
commit 15a9beed1b
55 changed files with 420 additions and 222 deletions

View File

@ -158,8 +158,10 @@ public class XceiverClientGrpc extends XceiverClientSpi {
// Add credential context to the client call
String userName = UserGroupInformation.getCurrentUser().getShortUserName();
LOG.debug("Nodes in pipeline : {}", pipeline.getNodes().toString());
LOG.debug("Connecting to server : {}", dn.getIpAddress());
if (LOG.isDebugEnabled()) {
LOG.debug("Nodes in pipeline : {}", pipeline.getNodes().toString());
LOG.debug("Connecting to server : {}", dn.getIpAddress());
}
NettyChannelBuilder channelBuilder =
NettyChannelBuilder.forAddress(dn.getIpAddress(), port).usePlaintext()
.maxInboundMessageSize(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE)
@ -283,7 +285,9 @@ public class XceiverClientGrpc extends XceiverClientSpi {
}
for (DatanodeDetails dn : datanodeList) {
try {
LOG.debug("Executing command " + request + " on datanode " + dn);
if (LOG.isDebugEnabled()) {
LOG.debug("Executing command " + request + " on datanode " + dn);
}
// In case the command gets retried on a 2nd datanode,
// sendCommandAsyncCall will create a new channel and async stub
// in case these don't exist for the specific datanode.
@ -377,9 +381,10 @@ public class XceiverClientGrpc extends XceiverClientSpi {
if (!isConnected(channel)) {
reconnect(dn, token);
}
LOG.debug("Send command {} to datanode {}", request.getCmdType().toString(),
dn.getNetworkFullPath());
if (LOG.isDebugEnabled()) {
LOG.debug("Send command {} to datanode {}",
request.getCmdType().toString(), dn.getNetworkFullPath());
}
final CompletableFuture<ContainerCommandResponseProto> replyFuture =
new CompletableFuture<>();
semaphore.acquire();

View File

@ -170,8 +170,10 @@ public final class XceiverClientRatis extends XceiverClientSpi {
@Override
public void connect() throws Exception {
LOG.debug("Connecting to pipeline:{} datanode:{}", getPipeline().getId(),
RatisHelper.toRaftPeerId(pipeline.getFirstNode()));
if (LOG.isDebugEnabled()) {
LOG.debug("Connecting to pipeline:{} datanode:{}", getPipeline().getId(),
RatisHelper.toRaftPeerId(pipeline.getFirstNode()));
}
// TODO : XceiverClient ratis should pass the config value of
// maxOutstandingRequests so as to set the upper bound on max no of async
// requests to be handled by raft client
@ -223,10 +225,14 @@ public final class XceiverClientRatis extends XceiverClientSpi {
= ContainerCommandRequestMessage.toMessage(
request, TracingUtil.exportCurrentSpan());
if (HddsUtils.isReadOnly(request)) {
LOG.debug("sendCommandAsync ReadOnly {}", message);
if (LOG.isDebugEnabled()) {
LOG.debug("sendCommandAsync ReadOnly {}", message);
}
return getClient().sendReadOnlyAsync(message);
} else {
LOG.debug("sendCommandAsync {}", message);
if (LOG.isDebugEnabled()) {
LOG.debug("sendCommandAsync {}", message);
}
return getClient().sendAsync(message);
}
}
@ -258,7 +264,9 @@ public final class XceiverClientRatis extends XceiverClientSpi {
clientReply.setLogIndex(commitIndex);
return clientReply;
}
LOG.debug("commit index : {} watch timeout : {}", index, timeout);
if (LOG.isDebugEnabled()) {
LOG.debug("commit index : {} watch timeout : {}", index, timeout);
}
RaftClientReply reply;
try {
CompletableFuture<RaftClientReply> replyFuture = getClient()
@ -310,10 +318,12 @@ public final class XceiverClientRatis extends XceiverClientSpi {
metrics.incrPendingContainerOpsMetrics(request.getCmdType());
CompletableFuture<ContainerCommandResponseProto> containerCommandResponse =
raftClientReply.whenComplete((reply, e) -> {
LOG.debug("received reply {} for request: cmdType={} containerID={}"
+ " pipelineID={} traceID={} exception: {}", reply,
request.getCmdType(), request.getContainerID(),
request.getPipelineID(), request.getTraceID(), e);
if (LOG.isDebugEnabled()) {
LOG.debug("received reply {} for request: cmdType={} containerID={}"
+ " pipelineID={} traceID={} exception: {}", reply,
request.getCmdType(), request.getContainerID(),
request.getPipelineID(), request.getTraceID(), e);
}
metrics.decrPendingContainerOpsMetrics(request.getCmdType());
metrics.addContainerOpsLatency(request.getCmdType(),
Time.monotonicNowNanos() - requestTime);

View File

@ -165,8 +165,10 @@ public class ContainerOperationClient implements ScmClient {
// TODO : Should we change the state on the client side ??
// That makes sense, but it is not needed for the client to work.
LOG.debug("Pipeline creation successful. Pipeline: {}",
pipeline.toString());
if (LOG.isDebugEnabled()) {
LOG.debug("Pipeline creation successful. Pipeline: {}",
pipeline.toString());
}
}
@Override
@ -379,7 +381,9 @@ public class ContainerOperationClient implements ScmClient {
throws IOException {
XceiverClientSpi client = null;
try {
LOG.debug("Close container {}", pipeline);
if (LOG.isDebugEnabled()) {
LOG.debug("Close container {}", pipeline);
}
/*
TODO: two orders here, revisit this later:
1. close on SCM first, then on data node

View File

@ -155,8 +155,10 @@ public class BlockInputStream extends InputStream implements Seekable {
boolean success = false;
List<ChunkInfo> chunks;
try {
LOG.debug("Initializing BlockInputStream for get key to access {}",
blockID.getContainerID());
if (LOG.isDebugEnabled()) {
LOG.debug("Initializing BlockInputStream for get key to access {}",
blockID.getContainerID());
}
if (token != null) {
UserGroupInformation.getCurrentUser().addToken(token);

View File

@ -392,22 +392,26 @@ public class BlockOutputStream extends OutputStream {
.equals(responseBlockID.getContainerBlockID()));
// updates the bcsId of the block
blockID = responseBlockID;
LOG.debug(
"Adding index " + asyncReply.getLogIndex() + " commitMap size "
+ commitWatcher.getCommitInfoMapSize() + " flushLength "
+ flushPos + " numBuffers " + byteBufferList.size()
+ " blockID " + blockID + " bufferPool size" + bufferPool
.getSize() + " currentBufferIndex " + bufferPool
.getCurrentBufferIndex());
if (LOG.isDebugEnabled()) {
LOG.debug(
"Adding index " + asyncReply.getLogIndex() + " commitMap size "
+ commitWatcher.getCommitInfoMapSize() + " flushLength "
+ flushPos + " numBuffers " + byteBufferList.size()
+ " blockID " + blockID + " bufferPool size" + bufferPool
.getSize() + " currentBufferIndex " + bufferPool
.getCurrentBufferIndex());
}
// for standalone protocol, logIndex will always be 0.
commitWatcher
.updateCommitInfoMap(asyncReply.getLogIndex(), byteBufferList);
}
return e;
}, responseExecutor).exceptionally(e -> {
LOG.debug(
"putBlock failed for blockID " + blockID + " with exception " + e
.getLocalizedMessage());
if (LOG.isDebugEnabled()) {
LOG.debug(
"putBlock failed for blockID " + blockID + " with exception " + e
.getLocalizedMessage());
}
CompletionException ce = new CompletionException(e);
setIoException(ce);
throw ce;
@ -609,9 +613,11 @@ public class BlockOutputStream extends OutputStream {
}
return e;
}, responseExecutor).exceptionally(e -> {
LOG.debug(
"writing chunk failed " + chunkInfo.getChunkName() + " blockID "
+ blockID + " with exception " + e.getLocalizedMessage());
if (LOG.isDebugEnabled()) {
LOG.debug(
"writing chunk failed " + chunkInfo.getChunkName() + " blockID "
+ blockID + " with exception " + e.getLocalizedMessage());
}
CompletionException ce = new CompletionException(e);
setIoException(ce);
throw ce;
@ -620,9 +626,11 @@ public class BlockOutputStream extends OutputStream {
throw new IOException(
"Unexpected Storage Container Exception: " + e.toString(), e);
}
LOG.debug(
"writing chunk " + chunkInfo.getChunkName() + " blockID " + blockID
+ " length " + effectiveChunkSize);
if (LOG.isDebugEnabled()) {
LOG.debug(
"writing chunk " + chunkInfo.getChunkName() + " blockID " + blockID
+ " length " + effectiveChunkSize);
}
containerBlockData.addChunks(chunkInfo);
}

View File

@ -131,7 +131,9 @@ public class CommitWatcher {
long index =
commitIndex2flushedDataMap.keySet().stream().mapToLong(v -> v).min()
.getAsLong();
LOG.debug("waiting for first index " + index + " to catch up");
if (LOG.isDebugEnabled()) {
LOG.debug("waiting for first index " + index + " to catch up");
}
return watchForCommit(index);
} else {
return null;
@ -153,7 +155,9 @@ public class CommitWatcher {
long index =
commitIndex2flushedDataMap.keySet().stream().mapToLong(v -> v).max()
.getAsLong();
LOG.debug("waiting for last flush Index " + index + " to catch up");
if (LOG.isDebugEnabled()) {
LOG.debug("waiting for last flush Index " + index + " to catch up");
}
return watchForCommit(index);
} else {
return null;

View File

@ -423,8 +423,10 @@ public final class HddsUtils {
InvocationTargetException e) {
// Fallback
LOG.trace("Registering MBean {} without additional properties {}",
mBeanName, jmxProperties);
if (LOG.isTraceEnabled()) {
LOG.trace("Registering MBean {} without additional properties {}",
mBeanName, jmxProperties);
}
return MBeans.register(serviceName, mBeanName, mBean);
}
}

View File

@ -171,7 +171,9 @@ public class HddsConfServlet extends HttpServlet {
Properties properties = config.getAllPropertiesByTag(tag);
propMap.put(tag, properties);
} else {
LOG.debug("Not a valid tag" + tag);
if (LOG.isDebugEnabled()) {
LOG.debug("Not a valid tag" + tag);
}
}
}
out.write(gson.toJsonTree(propMap).toString());

View File

@ -183,7 +183,10 @@ public interface RatisHelper {
static RaftClient newRaftClient(RpcType rpcType, RaftPeerId leader,
RaftGroup group, RetryPolicy retryPolicy, int maxOutStandingRequest,
GrpcTlsConfig tlsConfig, TimeDuration clientRequestTimeout) {
LOG.trace("newRaftClient: {}, leader={}, group={}", rpcType, leader, group);
if (LOG.isTraceEnabled()) {
LOG.trace("newRaftClient: {}, leader={}, group={}",
rpcType, leader, group);
}
final RaftProperties properties = new RaftProperties();
RaftConfigKeys.Rpc.setType(properties, rpcType);
RaftClientConfigKeys.Rpc

View File

@ -109,7 +109,9 @@ public class NetworkTopologyImpl implements NetworkTopology{
if (add) {
LOG.info("Added a new node: " + node.getNetworkFullPath());
LOG.debug("NetworkTopology became:\n{}", this);
if (LOG.isDebugEnabled()) {
LOG.debug("NetworkTopology became:\n{}", this);
}
}
}
@ -131,7 +133,9 @@ public class NetworkTopologyImpl implements NetworkTopology{
netlock.writeLock().unlock();
}
LOG.info("Removed a node: " + node.getNetworkFullPath());
LOG.debug("NetworkTopology became:\n{}", this);
if (LOG.isDebugEnabled()) {
LOG.debug("NetworkTopology became:\n{}", this);
}
}
/**
@ -558,11 +562,14 @@ public class NetworkTopologyImpl implements NetworkTopology{
ret = ((InnerNode)scopeNode).getLeaf(nodeIndex, mutableExcludedScopes,
mutableExNodes, ancestorGen);
}
LOG.debug("Choosing node[index={},random={}] from \"{}\" available nodes" +
" scope=\"{}\", excludedScope=\"{}\", excludeNodes=\"{}\".",
nodeIndex, (leafIndex == -1 ? "true" : "false"), availableNodes,
scopeNode.getNetworkFullPath(), excludedScopes, excludedNodes);
LOG.debug("Chosen node = {}", (ret == null ? "not found" : ret.toString()));
if (LOG.isDebugEnabled()) {
LOG.debug("Choosing node[index={},random={}] from \"{}\" available " +
"nodes, scope=\"{}\", excludedScope=\"{}\", excludeNodes=\"{}\".",
nodeIndex, (leafIndex == -1 ? "true" : "false"), availableNodes,
scopeNode.getNetworkFullPath(), excludedScopes, excludedNodes);
LOG.debug("Chosen node = {}", (ret == null ? "not found" :
ret.toString()));
}
return ret;
}

View File

@ -192,8 +192,10 @@ public final class Pipeline {
}
}
}
LOG.debug("Serialize pipeline {} with nodesInOrder{ }", id.toString(),
nodes);
if (LOG.isDebugEnabled()) {
LOG.debug("Serialize pipeline {} with nodesInOrder{ }", id.toString(),
nodes);
}
}
return builder.build();
}
@ -339,8 +341,10 @@ public final class Pipeline {
nodeIndex--;
}
}
LOG.debug("Deserialize nodesInOrder {} in pipeline {}", nodesWithOrder,
id.toString());
if (LOG.isDebugEnabled()) {
LOG.debug("Deserialize nodesInOrder {} in pipeline {}",
nodesWithOrder, id.toString());
}
pipeline.setNodesInOrder(nodesWithOrder);
} else if (nodesInOrder != null){
// This branch is for pipeline clone

View File

@ -68,7 +68,9 @@ public class BlockTokenVerifier implements TokenVerifier {
OzoneBlockTokenIdentifier tokenId = new OzoneBlockTokenIdentifier();
try {
token.decodeFromUrlString(tokenStr);
LOGGER.debug("Verifying token:{} for user:{} ", token, user);
if (LOGGER.isDebugEnabled()) {
LOGGER.debug("Verifying token:{} for user:{} ", token, user);
}
ByteArrayInputStream buf = new ByteArrayInputStream(
token.getIdentifier());
DataInputStream in = new DataInputStream(buf);

View File

@ -47,7 +47,9 @@ public class OzoneBlockTokenSelector implements
for (Token<? extends TokenIdentifier> token : tokens) {
if (OzoneBlockTokenIdentifier.KIND_NAME.equals(token.getKind())
&& token.getService().equals(service)) {
LOG.trace("Getting token for service:{}", service);
if (LOG.isTraceEnabled()) {
LOG.trace("Getting token for service:{}", service);
}
return (Token<OzoneBlockTokenIdentifier>) token;
}
}
@ -66,7 +68,9 @@ public class OzoneBlockTokenSelector implements
for (Token<? extends TokenIdentifier> token : tokens) {
if (OzoneBlockTokenIdentifier.KIND_NAME.equals(token.getKind())
&& token.getService().equals(service)) {
LOG.trace("Getting token for service:{}", service);
if (LOG.isTraceEnabled()) {
LOG.trace("Getting token for service:{}", service);
}
return (Token<OzoneBlockTokenIdentifier>) token;
}
}

View File

@ -236,7 +236,9 @@ public class DefaultProfile implements PKIProfile {
try {
final InetAddress byAddress = InetAddress.getByAddress(
Hex.decodeHex(value.substring(1)));
LOG.debug("Host Name/IP Address : {}", byAddress.toString());
if (LOG.isDebugEnabled()) {
LOG.debug("Host Name/IP Address : {}", byAddress.toString());
}
return true;
} catch (UnknownHostException | DecoderException e) {
return false;

View File

@ -108,8 +108,10 @@ public class HDDSKeyGenerator {
*/
public KeyPair generateKey(int size, String algorithm, String provider)
throws NoSuchProviderException, NoSuchAlgorithmException {
LOG.debug("Generating key pair using size:{}, Algorithm:{}, Provider:{}",
size, algorithm, provider);
if (LOG.isDebugEnabled()) {
LOG.debug("Generating key pair using size:{}, Algorithm:{}, Provider:{}",
size, algorithm, provider);
}
KeyPairGenerator generator = KeyPairGenerator
.getInstance(algorithm, provider);
generator.initialize(size);

View File

@ -45,7 +45,9 @@ public class StringCodec implements Codec<StringBuilder> {
if (value != null && !value.equals("")) {
String[] parts = value.split(":");
if (parts.length != 4) {
LOG.debug("MalformedTracerStateString: {}", value);
if (LOG.isDebugEnabled()) {
LOG.debug("MalformedTracerStateString: {}", value);
}
throw new MalformedTracerStateStringException(value);
} else {
String traceId = parts[0];

View File

@ -102,15 +102,18 @@ public abstract class BackgroundService {
public class PeriodicalTask implements Runnable {
@Override
public synchronized void run() {
LOG.debug("Running background service : {}", serviceName);
if (LOG.isDebugEnabled()) {
LOG.debug("Running background service : {}", serviceName);
}
BackgroundTaskQueue tasks = getTasks();
if (tasks.isEmpty()) {
// No task found, or some problems to init tasks
// return and retry in next interval.
return;
}
LOG.debug("Number of background tasks to execute : {}", tasks.size());
if (LOG.isDebugEnabled()) {
LOG.debug("Number of background tasks to execute : {}", tasks.size());
}
CompletionService<BackgroundTaskResult> taskCompletionService =
new ExecutorCompletionService<>(exec);

View File

@ -50,7 +50,9 @@ public final class HddsVersionInfo {
"Compiled with protoc " + HDDS_VERSION_INFO.getProtocVersion());
System.out.println(
"From source with checksum " + HDDS_VERSION_INFO.getSrcChecksum());
LOG.debug("This command was run using " +
ClassUtil.findContainingJar(HddsVersionInfo.class));
if (LOG.isDebugEnabled()) {
LOG.debug("This command was run using " +
ClassUtil.findContainingJar(HddsVersionInfo.class));
}
}
}

View File

@ -77,7 +77,9 @@ public class LevelDBStore implements MetadataStore {
private void openDB(File dbPath, Options options) throws IOException {
if (dbPath.getParentFile().mkdirs()) {
LOG.debug("Db path {} created.", dbPath.getParentFile());
if (LOG.isDebugEnabled()) {
LOG.debug("Db path {} created.", dbPath.getParentFile());
}
}
db = JniDBFactory.factory.open(dbPath, options);
if (LOG.isDebugEnabled()) {
@ -370,17 +372,21 @@ public class LevelDBStore implements MetadataStore {
int scanned = filter.getKeysScannedNum();
int hinted = filter.getKeysHintedNum();
if (scanned > 0 || hinted > 0) {
LOG.debug(
"getRangeKVs ({}) numOfKeysScanned={}, numOfKeysHinted={}",
filter.getClass().getSimpleName(), filter.getKeysScannedNum(),
filter.getKeysHintedNum());
if (LOG.isDebugEnabled()) {
LOG.debug(
"getRangeKVs ({}) numOfKeysScanned={}, numOfKeysHinted={}",
filter.getClass().getSimpleName(),
filter.getKeysScannedNum(), filter.getKeysHintedNum());
}
}
}
}
long end = System.currentTimeMillis();
long timeConsumed = end - start;
LOG.debug("Time consumed for getRangeKVs() is {}ms,"
+ " result length is {}.", timeConsumed, result.size());
if (LOG.isDebugEnabled()) {
LOG.debug("Time consumed for getRangeKVs() is {}ms,"
+ " result length is {}.", timeConsumed, result.size());
}
}
}
return result;

View File

@ -79,9 +79,11 @@ public final class ContainerCommandRequestPBHelper {
auditParams.put("blockData",
BlockData.getFromProtoBuf(msg.getPutBlock().getBlockData())
.toString());
}catch (IOException ex){
LOG.trace("Encountered error parsing BlockData from protobuf:"
+ ex.getMessage());
} catch (IOException ex){
if (LOG.isTraceEnabled()) {
LOG.trace("Encountered error parsing BlockData from protobuf: "
+ ex.getMessage());
}
return null;
}
return auditParams;
@ -134,9 +136,11 @@ public final class ContainerCommandRequestPBHelper {
auditParams.put("blockData",
BlockData.getFromProtoBuf(msg.getPutSmallFile()
.getBlock().getBlockData()).toString());
}catch (IOException ex){
LOG.trace("Encountered error parsing BlockData from protobuf:"
+ ex.getMessage());
} catch (IOException ex){
if (LOG.isTraceEnabled()) {
LOG.trace("Encountered error parsing BlockData from protobuf: "
+ ex.getMessage());
}
}
return auditParams;

View File

@ -49,7 +49,7 @@ public class LeaseCallbackExecutor<T> implements Runnable {
@Override
public void run() {
if(LOG.isDebugEnabled()) {
if (LOG.isDebugEnabled()) {
LOG.debug("Executing callbacks for lease on {}", resource);
}
for(Callable<Void> callback : callbacks) {

View File

@ -110,7 +110,7 @@ public class LeaseManager<T> {
public synchronized Lease<T> acquire(T resource, long timeout)
throws LeaseAlreadyExistException {
checkStatus();
if(LOG.isDebugEnabled()) {
if (LOG.isDebugEnabled()) {
LOG.debug("Acquiring lease on {} for {} milliseconds", resource, timeout);
}
if(activeLeases.containsKey(resource)) {
@ -150,7 +150,7 @@ public class LeaseManager<T> {
public synchronized void release(T resource)
throws LeaseNotFoundException {
checkStatus();
if(LOG.isDebugEnabled()) {
if (LOG.isDebugEnabled()) {
LOG.debug("Releasing lease on {}", resource);
}
Lease<T> lease = activeLeases.remove(resource);
@ -206,7 +206,7 @@ public class LeaseManager<T> {
@Override
public void run() {
while(monitor) {
while (monitor) {
LOG.debug("{}-LeaseMonitor: checking for lease expiry", name);
long sleepTime = Long.MAX_VALUE;

View File

@ -65,9 +65,11 @@ public class ContainerSet {
Preconditions.checkNotNull(container, "container cannot be null");
long containerId = container.getContainerData().getContainerID();
if(containerMap.putIfAbsent(containerId, container) == null) {
LOG.debug("Container with container Id {} is added to containerMap",
containerId);
if (containerMap.putIfAbsent(containerId, container) == null) {
if (LOG.isDebugEnabled()) {
LOG.debug("Container with container Id {} is added to containerMap",
containerId);
}
// wish we could have done this from ContainerData.setState
container.getContainerData().commitSpace();
return true;
@ -100,7 +102,7 @@ public class ContainerSet {
Preconditions.checkState(containerId >= 0,
"Container Id cannot be negative.");
Container<?> removed = containerMap.remove(containerId);
if(removed == null) {
if (removed == null) {
LOG.debug("Container with containerId {} is not present in " +
"containerMap", containerId);
return false;

View File

@ -155,8 +155,10 @@ public class HddsDispatcher implements ContainerDispatcher, Auditor {
private ContainerCommandResponseProto dispatchRequest(
ContainerCommandRequestProto msg, DispatcherContext dispatcherContext) {
Preconditions.checkNotNull(msg);
LOG.trace("Command {}, trace ID: {} ", msg.getCmdType().toString(),
msg.getTraceID());
if (LOG.isTraceEnabled()) {
LOG.trace("Command {}, trace ID: {} ", msg.getCmdType().toString(),
msg.getTraceID());
}
AuditAction action = ContainerCommandRequestPBHelper.getAuditAction(
msg.getCmdType());
@ -553,7 +555,10 @@ public class HddsDispatcher implements ContainerDispatcher, Auditor {
}
break;
default: LOG.debug("Invalid audit event status - " + result);
default:
if (LOG.isDebugEnabled()) {
LOG.debug("Invalid audit event status - " + result);
}
}
}

View File

@ -55,11 +55,12 @@ public class RandomContainerDeletionChoosingPolicy
if (currentCount < count) {
result.add(entry);
currentCount++;
LOG.debug("Select container {} for block deletion, "
+ "pending deletion blocks num: {}.",
entry.getContainerID(),
((KeyValueContainerData)entry).getNumPendingDeletionBlocks());
if (LOG.isDebugEnabled()) {
LOG.debug("Select container {} for block deletion, "
+ "pending deletion blocks num: {}.",
entry.getContainerID(),
((KeyValueContainerData) entry).getNumPendingDeletionBlocks());
}
} else {
break;
}

View File

@ -69,12 +69,13 @@ public class TopNOrderedContainerDeletionChoosingPolicy
if (entry.getNumPendingDeletionBlocks() > 0) {
result.add(entry);
currentCount++;
LOG.debug(
"Select container {} for block deletion, "
+ "pending deletion blocks num: {}.",
entry.getContainerID(),
entry.getNumPendingDeletionBlocks());
if (LOG.isDebugEnabled()) {
LOG.debug(
"Select container {} for block deletion, "
+ "pending deletion blocks num: {}.",
entry.getContainerID(),
entry.getNumPendingDeletionBlocks());
}
} else {
LOG.debug("Stop looking for next container, there is no"
+ " pending deletion block contained in remaining containers.");

View File

@ -207,7 +207,9 @@ public class EndpointStateMachine
TimeUnit.MILLISECONDS.toSeconds(
this.getMissedCount() * getScmHeartbeatInterval(this.conf)), ex);
}
LOG.trace("Incrementing the Missed count. Ex : {}", ex);
if (LOG.isTraceEnabled()) {
LOG.trace("Incrementing the Missed count. Ex : {}", ex);
}
this.incMissed();
}

View File

@ -114,8 +114,10 @@ public class CloseContainerCommandHandler implements CommandHandler {
break;
case UNHEALTHY:
case INVALID:
LOG.debug("Cannot close the container #{}, the container is"
+ " in {} state.", containerId, container.getContainerState());
if (LOG.isDebugEnabled()) {
LOG.debug("Cannot close the container #{}, the container is"
+ " in {} state.", containerId, container.getContainerState());
}
default:
break;
}

View File

@ -196,9 +196,11 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
}
if (delTX.getTxID() < containerData.getDeleteTransactionId()) {
LOG.debug(String.format("Ignoring delete blocks for containerId: %d."
+ " Outdated delete transactionId %d < %d", containerId,
delTX.getTxID(), containerData.getDeleteTransactionId()));
if (LOG.isDebugEnabled()) {
LOG.debug(String.format("Ignoring delete blocks for containerId: %d."
+ " Outdated delete transactionId %d < %d", containerId,
delTX.getTxID(), containerData.getDeleteTransactionId()));
}
return;
}
@ -216,9 +218,11 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
DFSUtil.string2Bytes(OzoneConsts.DELETED_KEY_PREFIX + blk);
if (containerDB.getStore().get(deletingKeyBytes) != null
|| containerDB.getStore().get(deletedKeyBytes) != null) {
LOG.debug(String.format(
"Ignoring delete for block %d in container %d."
+ " Entry already added.", blk, containerId));
if (LOG.isDebugEnabled()) {
LOG.debug(String.format(
"Ignoring delete for block %d in container %d."
+ " Entry already added.", blk, containerId));
}
continue;
}
// Found the block in container db,
@ -228,8 +232,10 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
try {
containerDB.getStore().writeBatch(batch);
newDeletionBlocks++;
LOG.debug("Transited Block {} to DELETING state in container {}",
blk, containerId);
if (LOG.isDebugEnabled()) {
LOG.debug("Transited Block {} to DELETING state in container {}",
blk, containerId);
}
} catch (IOException e) {
// if some blocks failed to delete, we fail this TX,
// without sending this ACK to SCM, SCM will resend the TX
@ -238,8 +244,10 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
"Failed to delete blocks for TXID = " + delTX.getTxID(), e);
}
} else {
LOG.debug("Block {} not found or already under deletion in"
+ " container {}, skip deleting it.", blk, containerId);
if (LOG.isDebugEnabled()) {
LOG.debug("Block {} not found or already under deletion in"
+ " container {}, skip deleting it.", blk, containerId);
}
}
}

View File

@ -391,9 +391,11 @@ public class ContainerStateMachine extends BaseStateMachine {
private ContainerCommandResponseProto dispatchCommand(
ContainerCommandRequestProto requestProto, DispatcherContext context) {
LOG.trace("{}: dispatch {} containerID={} pipelineID={} traceID={}", gid,
requestProto.getCmdType(), requestProto.getContainerID(),
requestProto.getPipelineID(), requestProto.getTraceID());
if (LOG.isTraceEnabled()) {
LOG.trace("{}: dispatch {} containerID={} pipelineID={} traceID={}", gid,
requestProto.getCmdType(), requestProto.getContainerID(),
requestProto.getPipelineID(), requestProto.getTraceID());
}
if (isBlockTokenEnabled) {
try {
// ServerInterceptors intercepts incoming request and creates ugi.
@ -409,7 +411,9 @@ public class ContainerStateMachine extends BaseStateMachine {
}
ContainerCommandResponseProto response =
dispatcher.dispatch(requestProto, context);
LOG.trace("{}: response {}", gid, response);
if (LOG.isTraceEnabled()) {
LOG.trace("{}: response {}", gid, response);
}
return response;
}
@ -462,9 +466,11 @@ public class ContainerStateMachine extends BaseStateMachine {
}, chunkExecutor);
writeChunkFutureMap.put(entryIndex, writeChunkFuture);
LOG.debug(gid + ": writeChunk writeStateMachineData : blockId " +
write.getBlockID() + " logIndex " + entryIndex + " chunkName "
+ write.getChunkData().getChunkName());
if (LOG.isDebugEnabled()) {
LOG.debug(gid + ": writeChunk writeStateMachineData : blockId " +
write.getBlockID() + " logIndex " + entryIndex + " chunkName "
+ write.getChunkData().getChunkName());
}
// Remove the future once it finishes execution from the
// writeChunkFutureMap.
writeChunkFuture.thenApply(r -> {
@ -480,10 +486,12 @@ public class ContainerStateMachine extends BaseStateMachine {
} else {
metrics.incNumBytesWrittenCount(
requestProto.getWriteChunk().getChunkData().getLen());
LOG.debug(gid +
": writeChunk writeStateMachineData completed: blockId" +
write.getBlockID() + " logIndex " + entryIndex + " chunkName " +
write.getChunkData().getChunkName());
if (LOG.isDebugEnabled()) {
LOG.debug(gid +
": writeChunk writeStateMachineData completed: blockId" +
write.getBlockID() + " logIndex " + entryIndex + " chunkName " +
write.getChunkData().getChunkName());
}
raftFuture.complete(r::toByteString);
metrics.recordWriteStateMachineCompletion(
Time.monotonicNowNanos() - startTime);
@ -761,10 +769,12 @@ public class ContainerStateMachine extends BaseStateMachine {
stateMachineHealthy.compareAndSet(true, false);
ratisServer.handleApplyTransactionFailure(gid, trx.getServerRole());
} else {
LOG.debug(
"gid {} : ApplyTransaction completed. cmd {} logIndex {} msg : "
+ "{} Container Result: {}", gid, r.getCmdType(), index,
r.getMessage(), r.getResult());
if (LOG.isDebugEnabled()) {
LOG.debug(
"gid {} : ApplyTransaction completed. cmd {} logIndex {} msg : "
+ "{} Container Result: {}", gid, r.getCmdType(), index,
r.getMessage(), r.getResult());
}
applyTransactionFuture.complete(r::toByteString);
if (cmdType == Type.WriteChunk || cmdType == Type.PutSmallFile) {
metrics.incNumBytesCommittedCount(

View File

@ -181,10 +181,12 @@ public class HddsVolumeChecker {
final long gap = timer.monotonicNow() - lastAllVolumesCheck;
if (gap < minDiskCheckGapMs) {
numSkippedChecks.incrementAndGet();
LOG.trace(
"Skipped checking all volumes, time since last check {} is less " +
"than the minimum gap between checks ({} ms).",
gap, minDiskCheckGapMs);
if (LOG.isTraceEnabled()) {
LOG.trace(
"Skipped checking all volumes, time since last check {} is less " +
"than the minimum gap between checks ({} ms).",
gap, minDiskCheckGapMs);
}
return Collections.emptySet();
}
@ -314,7 +316,9 @@ public class HddsVolumeChecker {
switch (result) {
case HEALTHY:
case DEGRADED:
LOG.debug("Volume {} is {}.", volume, result);
if (LOG.isDebugEnabled()) {
LOG.debug("Volume {} is {}.", volume, result);
}
markHealthy();
break;
case FAILED:

View File

@ -129,9 +129,11 @@ public class ThrottledAsyncChecker<K, V> implements AsyncChecker<K, V> {
completedChecks.get(target);
final long msSinceLastCheck = timer.monotonicNow() - result.completedAt;
if (msSinceLastCheck < minMsBetweenChecks) {
LOG.debug("Skipped checking {}. Time since last check {}ms " +
"is less than the min gap {}ms.",
target, msSinceLastCheck, minMsBetweenChecks);
if (LOG.isDebugEnabled()) {
LOG.debug("Skipped checking {}. Time since last check {}ms " +
"is less than the min gap {}ms.",
target, msSinceLastCheck, minMsBetweenChecks);
}
return Optional.empty();
}
}

View File

@ -128,8 +128,10 @@ public class KeyValueBlockIterator implements BlockIterator<BlockData>,
KeyValue block = blockIterator.next();
if (blockFilter.filterKey(null, block.getKey(), null)) {
nextBlock = BlockUtils.getBlockData(block.getValue());
LOG.trace("Block matching with filter found: blockID is : {} for " +
"containerID {}", nextBlock.getLocalID(), containerId);
if (LOG.isTraceEnabled()) {
LOG.trace("Block matching with filter found: blockID is : {} for " +
"containerID {}", nextBlock.getLocalID(), containerId);
}
return true;
}
hasNext();

View File

@ -206,8 +206,10 @@ public class KeyValueHandler extends Handler {
ContainerCommandResponseProto handleCreateContainer(
ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
if (!request.hasCreateContainer()) {
LOG.debug("Malformed Create Container request. trace ID: {}",
request.getTraceID());
if (LOG.isDebugEnabled()) {
LOG.debug("Malformed Create Container request. trace ID: {}",
request.getTraceID());
}
return ContainerUtils.malformedRequest(request);
}
// Create Container request should be passed a null container as the
@ -269,8 +271,10 @@ public class KeyValueHandler extends Handler {
ContainerCommandResponseProto handleReadContainer(
ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
if (!request.hasReadContainer()) {
LOG.debug("Malformed Read Container request. trace ID: {}",
request.getTraceID());
if (LOG.isDebugEnabled()) {
LOG.debug("Malformed Read Container request. trace ID: {}",
request.getTraceID());
}
return ContainerUtils.malformedRequest(request);
}
@ -296,8 +300,10 @@ public class KeyValueHandler extends Handler {
ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
if (!request.hasUpdateContainer()) {
LOG.debug("Malformed Update Container request. trace ID: {}",
request.getTraceID());
if (LOG.isDebugEnabled()) {
LOG.debug("Malformed Update Container request. trace ID: {}",
request.getTraceID());
}
return ContainerUtils.malformedRequest(request);
}
@ -330,8 +336,10 @@ public class KeyValueHandler extends Handler {
ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
if (!request.hasDeleteContainer()) {
LOG.debug("Malformed Delete container request. trace ID: {}",
request.getTraceID());
if (LOG.isDebugEnabled()) {
LOG.debug("Malformed Delete container request. trace ID: {}",
request.getTraceID());
}
return ContainerUtils.malformedRequest(request);
}
@ -352,8 +360,10 @@ public class KeyValueHandler extends Handler {
ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
if (!request.hasCloseContainer()) {
LOG.debug("Malformed Update Container request. trace ID: {}",
request.getTraceID());
if (LOG.isDebugEnabled()) {
LOG.debug("Malformed Update Container request. trace ID: {}",
request.getTraceID());
}
return ContainerUtils.malformedRequest(request);
}
try {
@ -379,8 +389,10 @@ public class KeyValueHandler extends Handler {
long blockLength;
if (!request.hasPutBlock()) {
LOG.debug("Malformed Put Key request. trace ID: {}",
request.getTraceID());
if (LOG.isDebugEnabled()) {
LOG.debug("Malformed Put Key request. trace ID: {}",
request.getTraceID());
}
return ContainerUtils.malformedRequest(request);
}
@ -415,8 +427,10 @@ public class KeyValueHandler extends Handler {
ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
if (!request.hasGetBlock()) {
LOG.debug("Malformed Get Key request. trace ID: {}",
request.getTraceID());
if (LOG.isDebugEnabled()) {
LOG.debug("Malformed Get Key request. trace ID: {}",
request.getTraceID());
}
return ContainerUtils.malformedRequest(request);
}
@ -454,8 +468,10 @@ public class KeyValueHandler extends Handler {
ContainerCommandResponseProto handleGetCommittedBlockLength(
ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
if (!request.hasGetCommittedBlockLength()) {
LOG.debug("Malformed Get Key request. trace ID: {}",
request.getTraceID());
if (LOG.isDebugEnabled()) {
LOG.debug("Malformed Get Key request. trace ID: {}",
request.getTraceID());
}
return ContainerUtils.malformedRequest(request);
}
@ -490,8 +506,10 @@ public class KeyValueHandler extends Handler {
ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
if (!request.hasDeleteBlock()) {
LOG.debug("Malformed Delete Key request. trace ID: {}",
request.getTraceID());
if (LOG.isDebugEnabled()) {
LOG.debug("Malformed Delete Key request. trace ID: {}",
request.getTraceID());
}
return ContainerUtils.malformedRequest(request);
}
@ -521,8 +539,10 @@ public class KeyValueHandler extends Handler {
DispatcherContext dispatcherContext) {
if (!request.hasReadChunk()) {
LOG.debug("Malformed Read Chunk request. trace ID: {}",
request.getTraceID());
if (LOG.isDebugEnabled()) {
LOG.debug("Malformed Read Chunk request. trace ID: {}",
request.getTraceID());
}
return ContainerUtils.malformedRequest(request);
}
@ -589,8 +609,10 @@ public class KeyValueHandler extends Handler {
ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
if (!request.hasDeleteChunk()) {
LOG.debug("Malformed Delete Chunk request. trace ID: {}",
request.getTraceID());
if (LOG.isDebugEnabled()) {
LOG.debug("Malformed Delete Chunk request. trace ID: {}",
request.getTraceID());
}
return ContainerUtils.malformedRequest(request);
}
@ -632,8 +654,10 @@ public class KeyValueHandler extends Handler {
DispatcherContext dispatcherContext) {
if (!request.hasWriteChunk()) {
LOG.debug("Malformed Write Chunk request. trace ID: {}",
request.getTraceID());
if (LOG.isDebugEnabled()) {
LOG.debug("Malformed Write Chunk request. trace ID: {}",
request.getTraceID());
}
return ContainerUtils.malformedRequest(request);
}
@ -687,8 +711,10 @@ public class KeyValueHandler extends Handler {
DispatcherContext dispatcherContext) {
if (!request.hasPutSmallFile()) {
LOG.debug("Malformed Put Small File request. trace ID: {}",
request.getTraceID());
if (LOG.isDebugEnabled()) {
LOG.debug("Malformed Put Small File request. trace ID: {}",
request.getTraceID());
}
return ContainerUtils.malformedRequest(request);
}
PutSmallFileRequestProto putSmallFileReq =
@ -745,8 +771,10 @@ public class KeyValueHandler extends Handler {
ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
if (!request.hasGetSmallFile()) {
LOG.debug("Malformed Get Small File request. trace ID: {}",
request.getTraceID());
if (LOG.isDebugEnabled()) {
LOG.debug("Malformed Get Small File request. trace ID: {}",
request.getTraceID());
}
return ContainerUtils.malformedRequest(request);
}

View File

@ -127,8 +127,10 @@ public final class ChunkUtils {
return null;
});
log.debug("Write Chunk completed for chunkFile: {}, size {}", chunkFile,
bufferSize);
if (log.isDebugEnabled()) {
log.debug("Write Chunk completed for chunkFile: {}, size {}", chunkFile,
bufferSize);
}
}
/**

View File

@ -120,9 +120,11 @@ public class BlockManagerImpl implements BlockManager {
container.updateBlockCommitSequenceId(bcsId);
// Increment keycount here
container.getContainerData().incrKeyCount();
LOG.debug(
"Block " + data.getBlockID() + " successfully committed with bcsId "
+ bcsId + " chunk size " + data.getChunks().size());
if (LOG.isDebugEnabled()) {
LOG.debug(
"Block " + data.getBlockID() + " successfully committed with bcsId "
+ bcsId + " chunk size " + data.getChunks().size());
}
return data.getSize();
}
}

View File

@ -87,10 +87,11 @@ public class ChunkManagerImpl implements ChunkManager {
boolean isOverwrite = ChunkUtils.validateChunkForOverwrite(
chunkFile, info);
File tmpChunkFile = getTmpChunkFile(chunkFile, dispatcherContext);
LOG.debug(
"writing chunk:{} chunk stage:{} chunk file:{} tmp chunk file:{}",
info.getChunkName(), stage, chunkFile, tmpChunkFile);
if (LOG.isDebugEnabled()) {
LOG.debug(
"writing chunk:{} chunk stage:{} chunk file:{} tmp chunk file:{}",
info.getChunkName(), stage, chunkFile, tmpChunkFile);
}
switch (stage) {
case WRITE_DATA:

View File

@ -285,8 +285,10 @@ public class BlockDeletingService extends BackgroundService {
File chunkFile = dataDir.toPath()
.resolve(chunkInfo.getChunkName()).toFile();
if (FileUtils.deleteQuietly(chunkFile)) {
LOG.debug("block {} chunk {} deleted", blockName,
chunkFile.getAbsolutePath());
if (LOG.isDebugEnabled()) {
LOG.debug("block {} chunk {} deleted", blockName,
chunkFile.getAbsolutePath());
}
}
}
succeedBlocks.add(blockName);

View File

@ -69,7 +69,9 @@ public class ContainerDataScanner extends Thread {
@Override
public void run() {
LOG.trace("{}: thread starting.", this);
if (LOG.isTraceEnabled()) {
LOG.trace("{}: thread starting.", this);
}
try {
while (!stopping) {
runIteration();

View File

@ -161,7 +161,7 @@ public class EventQueue implements EventPublisher, AutoCloseable {
for (EventHandler handler : executorAndHandlers.getValue()) {
queuedCount.incrementAndGet();
if (LOG.isTraceEnabled()) {
if (LOG.isDebugEnabled()) {
LOG.debug(
"Delivering event {} to executor/handler {}: <json>{}</json>",
event.getName(),

View File

@ -153,7 +153,9 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
public AllocatedBlock allocateBlock(final long size, ReplicationType type,
ReplicationFactor factor, String owner, ExcludeList excludeList)
throws IOException {
LOG.trace("Size;{} , type : {}, factor : {} ", size, type, factor);
if (LOG.isTraceEnabled()) {
LOG.trace("Size;{} , type : {}, factor : {} ", size, type, factor);
}
ScmUtils.preCheck(ScmOps.allocateBlock, safeModePrecheck);
if (size < 0 || size > containerSize) {
LOG.warn("Invalid block size requested : {}", size);
@ -241,8 +243,10 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
AllocatedBlock.Builder abb = new AllocatedBlock.Builder()
.setContainerBlockID(new ContainerBlockID(containerID, localID))
.setPipeline(pipeline);
LOG.trace("New block allocated : {} Container ID: {}", localID,
containerID);
if (LOG.isTraceEnabled()) {
LOG.trace("New block allocated : {} Container ID: {}", localID,
containerID);
}
pipelineManager.incNumBlocksAllocatedMetric(pipeline.getId());
return abb.build();
} catch (PipelineNotFoundException ex) {

View File

@ -168,11 +168,13 @@ public class SCMBlockDeletingService extends BackgroundService {
// offline for sometime, the cached commands be flooded.
eventPublisher.fireEvent(SCMEvents.RETRIABLE_DATANODE_COMMAND,
new CommandForDatanode<>(dnId, new DeleteBlocksCommand(dnTXs)));
LOG.debug(
"Added delete block command for datanode {} in the queue,"
+ " number of delete block transactions: {}, TxID list: {}",
dnId, dnTXs.size(), String.join(",",
transactions.getTransactionIDList(dnId)));
if (LOG.isDebugEnabled()) {
LOG.debug(
"Added delete block command for datanode {} in the queue," +
" number of delete block transactions: {}, TxID list: {}",
dnId, dnTXs.size(), String.join(",",
transactions.getTransactionIDList(dnId)));
}
}
}
containerManager.updateDeleteTransactionId(transactionMap);

View File

@ -48,13 +48,17 @@ public class CommandStatusReportHandler implements
Preconditions.checkNotNull(report);
List<CommandStatus> cmdStatusList = report.getReport().getCmdStatusList();
Preconditions.checkNotNull(cmdStatusList);
LOGGER.trace("Processing command status report for dn: {}", report
.getDatanodeDetails());
if (LOGGER.isTraceEnabled()) {
LOGGER.trace("Processing command status report for dn: {}", report
.getDatanodeDetails());
}
// Route command status to its watchers.
cmdStatusList.forEach(cmdStatus -> {
LOGGER.trace("Emitting command status for id:{} type: {}", cmdStatus
.getCmdId(), cmdStatus.getType());
if (LOGGER.isTraceEnabled()) {
LOGGER.trace("Emitting command status for id:{} type: {}", cmdStatus
.getCmdId(), cmdStatus.getType());
}
if (cmdStatus.getType() == SCMCommandProto.Type.deleteBlocksCommand) {
if (cmdStatus.getStatus() == CommandStatus.Status.EXECUTED) {
publisher.fireEvent(SCMEvents.DELETE_BLOCK_STATUS,

View File

@ -76,8 +76,10 @@ public class AbstractContainerReportHandler {
.setSequenceId(replicaProto.getBlockCommitSequenceId())
.build();
logger.debug("Processing replica of container {} from datanode {}",
containerId, datanodeDetails);
if (logger.isDebugEnabled()) {
logger.debug("Processing replica of container {} from datanode {}",
containerId, datanodeDetails);
}
// Synchronized block should be replaced by container lock,
// once we have introduced lock inside ContainerInfo.
synchronized (containerManager.getContainer(containerId)) {

View File

@ -48,8 +48,10 @@ public class ContainerActionsHandler implements
ContainerID containerId = ContainerID.valueof(action.getContainerID());
switch (action.getAction()) {
case CLOSE:
LOG.debug("Closing container {} in datanode {} because the" +
" container is {}.", containerId, dd, action.getReason());
if (LOG.isDebugEnabled()) {
LOG.debug("Closing container {} in datanode {} because the" +
" container is {}.", containerId, dd, action.getReason());
}
publisher.fireEvent(SCMEvents.CLOSE_CONTAINER, containerId);
break;
default:

View File

@ -305,7 +305,9 @@ public class ContainerStateManager {
pipelineManager.addContainerToPipeline(pipeline.getId(),
ContainerID.valueof(containerID));
containerStateCount.incrementAndGet(containerInfo.getState());
LOG.trace("New container allocated: {}", containerInfo);
if (LOG.isTraceEnabled()) {
LOG.trace("New container allocated: {}", containerInfo);
}
return containerInfo;
}

View File

@ -54,8 +54,10 @@ public class IncrementalContainerReportHandler extends
@Override
public void onMessage(final IncrementalContainerReportFromDatanode report,
final EventPublisher publisher) {
LOG.debug("Processing incremental container report from data node {}",
report.getDatanodeDetails().getUuid());
if (LOG.isDebugEnabled()) {
LOG.debug("Processing incremental container report from data node {}",
report.getDatanodeDetails().getUuid());
}
boolean success = true;
for (ContainerReplicaProto replicaProto :

View File

@ -272,8 +272,10 @@ public final class SCMContainerPlacementRackAware extends SCMCommonPolicy {
" excludedNodes and affinityNode constrains.", null);
}
if (hasEnoughSpace((DatanodeDetails)node, sizeRequired)) {
LOG.debug("Datanode {} is chosen for container. Required size is {}",
node.toString(), sizeRequired);
if (LOG.isDebugEnabled()) {
LOG.debug("Datanode {} is chosen for container. Required size is {}",
node.toString(), sizeRequired);
}
metrics.incrDatanodeChooseSuccessCount();
if (isFallbacked) {
metrics.incrDatanodeChooseFallbackCount();

View File

@ -167,7 +167,9 @@ public class ContainerAttribute<T> {
if (attributeMap.containsKey(key)) {
attributeMap.get(key).clear();
} else {
LOG.debug("key: {} does not exist in the attributeMap", key);
if (LOG.isDebugEnabled()) {
LOG.debug("key: {} does not exist in the attributeMap", key);
}
}
}
@ -183,13 +185,17 @@ public class ContainerAttribute<T> {
if (attributeMap.containsKey(key)) {
if (!attributeMap.get(key).remove(value)) {
LOG.debug("ContainerID: {} does not exist in the set pointed by " +
"key:{}", value, key);
if (LOG.isDebugEnabled()) {
LOG.debug("ContainerID: {} does not exist in the set pointed by " +
"key:{}", value, key);
}
return false;
}
return true;
} else {
LOG.debug("key: {} does not exist in the attributeMap", key);
if (LOG.isDebugEnabled()) {
LOG.debug("key: {} does not exist in the attributeMap", key);
}
return false;
}
}
@ -206,7 +212,9 @@ public class ContainerAttribute<T> {
if (this.attributeMap.containsKey(key)) {
return Collections.unmodifiableNavigableSet(this.attributeMap.get(key));
}
LOG.debug("No such Key. Key {}", key);
if (LOG.isDebugEnabled()) {
LOG.debug("No such Key. Key {}", key);
}
return EMPTY_SET;
}
@ -237,7 +245,9 @@ public class ContainerAttribute<T> {
LOG.error("error in update.", ex);
if (removed) {
insert(currentKey, value);
LOG.trace("reinserted the removed key. {}", currentKey);
if (LOG.isTraceEnabled()) {
LOG.trace("reinserted the removed key. {}", currentKey);
}
}
throw ex;
}

View File

@ -308,8 +308,10 @@ public class ContainerStateMap {
// be in an inconsistent state,
lifeCycleStateMap.update(currentState, newState, containerID);
LOG.trace("Updated the container {} to new state. Old = {}, new = " +
"{}", containerID, currentState, newState);
if (LOG.isTraceEnabled()) {
LOG.trace("Updated the container {} to new state. Old = {}, new = " +
"{}", containerID, currentState, newState);
}
// Just flush both old and new data sets from the result cache.
flushCache(currentInfo);

View File

@ -267,8 +267,10 @@ public class SCMNodeManager implements NodeManager {
processNodeReport(datanodeDetails, nodeReport);
LOG.info("Registered Data node : {}", datanodeDetails);
} catch (NodeAlreadyExistsException e) {
LOG.trace("Datanode is already registered. Datanode: {}",
datanodeDetails.toString());
if (LOG.isTraceEnabled()) {
LOG.trace("Datanode is already registered. Datanode: {}",
datanodeDetails.toString());
}
}
return RegisteredCommand.newBuilder().setErrorCode(ErrorCode.success)
@ -642,7 +644,9 @@ public class SCMNodeManager implements NodeManager {
List<String> resolvedHosts = dnsToSwitchMapping.resolve(hosts);
if (resolvedHosts != null && !resolvedHosts.isEmpty()) {
String location = resolvedHosts.get(0);
LOG.debug("Resolve datanode {} return location {}", hostname, location);
if (LOG.isDebugEnabled()) {
LOG.debug("Resolve datanode {} return location {}", hostname, location);
}
return location;
} else {
LOG.error("Node {} Resolution failed. Please make sure that DNS table " +

View File

@ -74,7 +74,9 @@ public class PipelineReportHandler implements
pipelineReportFromDatanode.getReport();
Preconditions.checkNotNull(dn, "Pipeline Report is "
+ "missing DatanodeDetails.");
LOGGER.trace("Processing pipeline report for dn: {}", dn);
if (LOGGER.isTraceEnabled()) {
LOGGER.trace("Processing pipeline report for dn: {}", dn);
}
for (PipelineReport report : pipelineReport.getPipelineReportList()) {
try {
processPipelineReport(report, dn);

View File

@ -190,7 +190,9 @@ public class RatisPipelineProvider implements PipelineProvider {
protected void initializePipeline(Pipeline pipeline) throws IOException {
final RaftGroup group = RatisHelper.newRaftGroup(pipeline);
LOG.debug("creating pipeline:{} with {}", pipeline.getId(), group);
if (LOG.isDebugEnabled()) {
LOG.debug("creating pipeline:{} with {}", pipeline.getId(), group);
}
callRatisRpc(pipeline.getNodes(),
(raftClient, peer) -> {
RaftClientReply reply = raftClient.groupAdd(group, peer.getId());

View File

@ -59,7 +59,9 @@ public final class RatisPipelineUtils {
static void destroyPipeline(Pipeline pipeline, Configuration ozoneConf,
GrpcTlsConfig grpcTlsConfig) {
final RaftGroup group = RatisHelper.newRaftGroup(pipeline);
LOG.debug("destroying pipeline:{} with {}", pipeline.getId(), group);
if (LOG.isDebugEnabled()) {
LOG.debug("destroying pipeline:{} with {}", pipeline.getId(), group);
}
for (DatanodeDetails dn : pipeline.getNodes()) {
try {
destroyPipeline(dn, pipeline.getId(), ozoneConf, grpcTlsConfig);

View File

@ -494,10 +494,12 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
*/
private void loginAsSCMUser(Configuration conf)
throws IOException, AuthenticationException {
LOG.debug("Ozone security is enabled. Attempting login for SCM user. "
+ "Principal: {}, keytab: {}",
conf.get(HDDS_SCM_KERBEROS_PRINCIPAL_KEY),
conf.get(HDDS_SCM_KERBEROS_KEYTAB_FILE_KEY));
if (LOG.isDebugEnabled()) {
LOG.debug("Ozone security is enabled. Attempting login for SCM user. "
+ "Principal: {}, keytab: {}",
conf.get(HDDS_SCM_KERBEROS_PRINCIPAL_KEY),
conf.get(HDDS_SCM_KERBEROS_KEYTAB_FILE_KEY));
}
if (SecurityUtil.getAuthenticationMethod(conf).equals(
AuthenticationMethod.KERBEROS)) {
@ -697,10 +699,10 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
ContainerStat stat = removalNotification.getValue();
// remove invalid container report
metrics.decrContainerStat(stat);
LOG.debug(
"Remove expired container stat entry for datanode: " +
"{}.",
removalNotification.getKey());
if (LOG.isDebugEnabled()) {
LOG.debug("Remove expired container stat entry for " +
"datanode: {}.", removalNotification.getKey());
}
}
}
})