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

This commit is contained in:
Siddharth 2019-10-10 03:00:11 -07:00 committed by bshashikant
parent eeb58a07e2
commit a031388a2e
31 changed files with 181 additions and 100 deletions

View File

@ -41,8 +41,7 @@
*/ */
public final class Pipeline { public final class Pipeline {
private static final Logger LOG = LoggerFactory private static final Logger LOG = LoggerFactory.getLogger(Pipeline.class);
.getLogger(Pipeline.class);
private final PipelineID id; private final PipelineID id;
private final ReplicationType type; private final ReplicationType type;
private final ReplicationFactor factor; private final ReplicationFactor factor;

View File

@ -193,10 +193,12 @@ public List<OmKeyLocationInfo> getLocationInfoList() {
.setPipeline(streamEntry.getPipeline()).build(); .setPipeline(streamEntry.getPipeline()).build();
locationInfoList.add(info); locationInfoList.add(info);
} }
LOG.debug( if (LOG.isDebugEnabled()) {
"block written " + streamEntry.getBlockID() + ", length " + length LOG.debug(
+ " bcsID " + streamEntry.getBlockID() "block written " + streamEntry.getBlockID() + ", length " + length
.getBlockCommitSequenceId()); + " bcsID " + streamEntry.getBlockID()
.getBlockCommitSequenceId());
}
} }
return locationInfoList; return locationInfoList;
} }

View File

@ -97,8 +97,10 @@ private synchronized void initialize(String keyName,
long keyLength = 0; long keyLength = 0;
for (int i = 0; i < blockInfos.size(); i++) { for (int i = 0; i < blockInfos.size(); i++) {
OmKeyLocationInfo omKeyLocationInfo = blockInfos.get(i); OmKeyLocationInfo omKeyLocationInfo = blockInfos.get(i);
LOG.debug("Adding stream for accessing {}. The stream will be " + if (LOG.isDebugEnabled()) {
"initialized later.", omKeyLocationInfo); LOG.debug("Adding stream for accessing {}. The stream will be " +
"initialized later.", omKeyLocationInfo);
}
addStream(omKeyLocationInfo, xceiverClientManager, addStream(omKeyLocationInfo, xceiverClientManager,
verifyChecksum); verifyChecksum);

View File

@ -439,10 +439,14 @@ public Token<OzoneTokenIdentifier> getDelegationToken(Text renewer)
ozoneManagerClient.getDelegationToken(renewer); ozoneManagerClient.getDelegationToken(renewer);
if (token != null) { if (token != null) {
token.setService(dtService); token.setService(dtService);
LOG.debug("Created token {} for dtService {}", token, dtService); if (LOG.isDebugEnabled()) {
LOG.debug("Created token {} for dtService {}", token, dtService);
}
} else { } else {
LOG.debug("Cannot get ozone delegation token for renewer {} to access " + if (LOG.isDebugEnabled()) {
"service {}", renewer, dtService); LOG.debug("Cannot get ozone delegation token for renewer {} to " +
"access service {}", renewer, dtService);
}
} }
return token; return token;
} }

View File

@ -75,7 +75,9 @@ public S3SecretValue getS3Secret(String kerberosID) throws IOException {
} finally { } finally {
omMetadataManager.getLock().releaseLock(S3_SECRET_LOCK, kerberosID); omMetadataManager.getLock().releaseLock(S3_SECRET_LOCK, kerberosID);
} }
LOG.trace("Secret for accessKey:{}, proto:{}", kerberosID, result); if (LOG.isTraceEnabled()) {
LOG.trace("Secret for accessKey:{}, proto:{}", kerberosID, result);
}
return result; return result;
} }

View File

@ -214,8 +214,10 @@ private Text computeDelegationTokenService() {
@Override @Override
public void performFailover(OzoneManagerProtocolPB currentProxy) { public void performFailover(OzoneManagerProtocolPB currentProxy) {
int newProxyIndex = incrementProxyIndex(); int newProxyIndex = incrementProxyIndex();
LOG.debug("Failing over OM proxy to index: {}, nodeId: {}", if (LOG.isDebugEnabled()) {
newProxyIndex, omNodeIDList.get(newProxyIndex)); LOG.debug("Failing over OM proxy to index: {}, nodeId: {}",
newProxyIndex, omNodeIDList.get(newProxyIndex));
}
} }
/** /**

View File

@ -61,7 +61,9 @@ private OMRatisHelper() {
*/ */
public static RaftClient newRaftClient(RpcType rpcType, String omId, RaftGroup public static RaftClient newRaftClient(RpcType rpcType, String omId, RaftGroup
group, RetryPolicy retryPolicy, Configuration conf) { group, RetryPolicy retryPolicy, Configuration conf) {
LOG.trace("newRaftClient: {}, leader={}, group={}", rpcType, omId, group); if (LOG.isTraceEnabled()) {
LOG.trace("newRaftClient: {}, leader={}, group={}", rpcType, omId, group);
}
final RaftProperties properties = new RaftProperties(); final RaftProperties properties = new RaftProperties();
RaftConfigKeys.Rpc.setType(properties, rpcType); RaftConfigKeys.Rpc.setType(properties, rpcType);

View File

@ -168,8 +168,10 @@ private boolean lock(Resource resource, String resourceName,
throw new RuntimeException(errorMessage); throw new RuntimeException(errorMessage);
} else { } else {
lockFn.accept(resourceName); lockFn.accept(resourceName);
LOG.debug("Acquired {} {} lock on resource {}", lockType, resource.name, if (LOG.isDebugEnabled()) {
resourceName); LOG.debug("Acquired {} {} lock on resource {}", lockType, resource.name,
resourceName);
}
lockSet.set(resource.setLock(lockSet.get())); lockSet.set(resource.setLock(lockSet.get()));
return true; return true;
} }
@ -264,8 +266,10 @@ public boolean acquireMultiUserLock(String firstUser, String secondUser) {
throw ex; throw ex;
} }
} }
LOG.debug("Acquired Write {} lock on resource {} and {}", resource.name, if (LOG.isDebugEnabled()) {
firstUser, secondUser); LOG.debug("Acquired Write {} lock on resource {} and {}", resource.name,
firstUser, secondUser);
}
lockSet.set(resource.setLock(lockSet.get())); lockSet.set(resource.setLock(lockSet.get()));
return true; return true;
} }
@ -300,8 +304,10 @@ public void releaseMultiUserLock(String firstUser, String secondUser) {
manager.writeUnlock(firstUser); manager.writeUnlock(firstUser);
manager.writeUnlock(secondUser); manager.writeUnlock(secondUser);
} }
LOG.debug("Release Write {} lock on resource {} and {}", resource.name, if (LOG.isDebugEnabled()) {
firstUser, secondUser); LOG.debug("Release Write {} lock on resource {} and {}", resource.name,
firstUser, secondUser);
}
lockSet.set(resource.clearLock(lockSet.get())); lockSet.set(resource.clearLock(lockSet.get()));
} }
@ -352,8 +358,10 @@ private void unlock(Resource resource, String resourceName,
// locks, as some locks support acquiring lock again. // locks, as some locks support acquiring lock again.
lockFn.accept(resourceName); lockFn.accept(resourceName);
// clear lock // clear lock
LOG.debug("Release {} {}, lock on resource {}", lockType, resource.name, if (LOG.isDebugEnabled()) {
resourceName); LOG.debug("Release {} {}, lock on resource {}", lockType, resource.name,
resourceName);
}
lockSet.set(resource.clearLock(lockSet.get())); lockSet.set(resource.clearLock(lockSet.get()));
} }

View File

@ -89,7 +89,7 @@ public Token<OzoneBlockTokenIdentifier> generateToken(String user,
if (LOG.isTraceEnabled()) { if (LOG.isTraceEnabled()) {
long expiryTime = tokenIdentifier.getExpiryDate(); long expiryTime = tokenIdentifier.getExpiryDate();
String tokenId = tokenIdentifier.toString(); String tokenId = tokenIdentifier.toString();
LOG.trace("Issued delegation token -> expiryTime:{},tokenId:{}", LOG.trace("Issued delegation token -> expiryTime:{}, tokenId:{}",
expiryTime, tokenId); expiryTime, tokenId);
} }
// Pass blockId as service. // Pass blockId as service.

View File

@ -289,8 +289,10 @@ public OzoneTokenIdentifier cancelToken(Token<OzoneTokenIdentifier> token,
String canceller) throws IOException { String canceller) throws IOException {
OzoneTokenIdentifier id = OzoneTokenIdentifier.readProtoBuf( OzoneTokenIdentifier id = OzoneTokenIdentifier.readProtoBuf(
token.getIdentifier()); token.getIdentifier());
LOG.debug("Token cancellation requested for identifier: {}", if (LOG.isDebugEnabled()) {
formatTokenId(id)); LOG.debug("Token cancellation requested for identifier: {}",
formatTokenId(id));
}
if (id.getUser() == null) { if (id.getUser() == null) {
throw new InvalidToken("Token with no owner " + formatTokenId(id)); throw new InvalidToken("Token with no owner " + formatTokenId(id));

View File

@ -43,9 +43,13 @@ public OzoneDelegationTokenSelector() {
@Override @Override
public Token<OzoneTokenIdentifier> selectToken(Text service, public Token<OzoneTokenIdentifier> selectToken(Text service,
Collection<Token<? extends TokenIdentifier>> tokens) { Collection<Token<? extends TokenIdentifier>> tokens) {
LOG.trace("Getting token for service {}", service); if (LOG.isTraceEnabled()) {
LOG.trace("Getting token for service {}", service);
}
Token token = getSelectedTokens(service, tokens); Token token = getSelectedTokens(service, tokens);
LOG.debug("Got tokens: {} for service {}", token, service); if (LOG.isDebugEnabled()) {
LOG.debug("Got tokens: {} for service {}", token, service);
}
return token; return token;
} }

View File

@ -110,8 +110,10 @@ public byte[] createPassword(byte[] identifier, PrivateKey privateKey)
@Override @Override
public byte[] createPassword(T identifier) { public byte[] createPassword(T identifier) {
logger.debug("Creating password for identifier: {}, currentKey: {}", if (logger.isDebugEnabled()) {
formatTokenId(identifier), currentKey.getKeyId()); logger.debug("Creating password for identifier: {}, currentKey: {}",
formatTokenId(identifier), currentKey.getKeyId());
}
byte[] password = null; byte[] password = null;
try { try {
password = createPassword(identifier.getBytes(), password = createPassword(identifier.getBytes(),

View File

@ -570,8 +570,10 @@ public boolean checkAccess(OzoneObj ozObject, RequestContext context)
} }
boolean hasAccess = OzoneAclUtil.checkAclRights(bucketInfo.getAcls(), boolean hasAccess = OzoneAclUtil.checkAclRights(bucketInfo.getAcls(),
context); context);
LOG.debug("user:{} has access rights for bucket:{} :{} ", if (LOG.isDebugEnabled()) {
context.getClientUgi(), ozObject.getBucketName(), hasAccess); LOG.debug("user:{} has access rights for bucket:{} :{} ",
context.getClientUgi(), ozObject.getBucketName(), hasAccess);
}
return hasAccess; return hasAccess;
} catch (IOException ex) { } catch (IOException ex) {
if(ex instanceof OMException) { if(ex instanceof OMException) {

View File

@ -1661,8 +1661,10 @@ public boolean checkAccess(OzoneObj ozObject, RequestContext context)
if (keyInfo == null) { if (keyInfo == null) {
// the key does not exist, but it is a parent "dir" of some key // the key does not exist, but it is a parent "dir" of some key
// let access be determined based on volume/bucket/prefix ACL // let access be determined based on volume/bucket/prefix ACL
LOG.debug("key:{} is non-existent parent, permit access to user:{}", if (LOG.isDebugEnabled()) {
keyName, context.getClientUgi()); LOG.debug("key:{} is non-existent parent, permit access to user:{}",
keyName, context.getClientUgi());
}
return true; return true;
} }
} catch (OMException e) { } catch (OMException e) {
@ -1678,8 +1680,10 @@ public boolean checkAccess(OzoneObj ozObject, RequestContext context)
boolean hasAccess = OzoneAclUtil.checkAclRight( boolean hasAccess = OzoneAclUtil.checkAclRight(
keyInfo.getAcls(), context); keyInfo.getAcls(), context);
LOG.debug("user:{} has access rights for key:{} :{} ", if (LOG.isDebugEnabled()) {
context.getClientUgi(), ozObject.getKeyName(), hasAccess); LOG.debug("user:{} has access rights for key:{} :{} ",
context.getClientUgi(), ozObject.getKeyName(), hasAccess);
}
return hasAccess; return hasAccess;
} catch (IOException ex) { } catch (IOException ex) {
if(ex instanceof OMException) { if(ex instanceof OMException) {
@ -1766,10 +1770,11 @@ public OzoneFileStatus getFileStatus(OmKeyArgs args) throws IOException {
if (keys.iterator().hasNext()) { if (keys.iterator().hasNext()) {
return new OzoneFileStatus(keyName); return new OzoneFileStatus(keyName);
} }
if (LOG.isDebugEnabled()) {
LOG.debug("Unable to get file status for the key: volume:" + volumeName + LOG.debug("Unable to get file status for the key: volume: {}, bucket:" +
" bucket:" + bucketName + " key:" + keyName + " with error no " + " {}, key: {}, with error: No such file exists.", volumeName,
"such file exists:"); bucketName, keyName);
}
throw new OMException("Unable to get file status: volume: " + throw new OMException("Unable to get file status: volume: " +
volumeName + " bucket: " + bucketName + " key: " + keyName, volumeName + " bucket: " + bucketName + " key: " + keyName,
FILE_NOT_FOUND); FILE_NOT_FOUND);
@ -2132,8 +2137,10 @@ private void sortDatanodeInPipeline(OmKeyInfo keyInfo, String clientMachine) {
List<DatanodeDetails> sortedNodes = scmClient.getBlockClient() List<DatanodeDetails> sortedNodes = scmClient.getBlockClient()
.sortDatanodes(nodeList, clientMachine); .sortDatanodes(nodeList, clientMachine);
k.getPipeline().setNodesInOrder(sortedNodes); k.getPipeline().setNodesInOrder(sortedNodes);
LOG.debug("Sort datanodes {} for client {}, return {}", nodes, if (LOG.isDebugEnabled()) {
clientMachine, sortedNodes); LOG.debug("Sort datanodes {} for client {}, return {}", nodes,
clientMachine, sortedNodes);
}
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Unable to sort datanodes based on distance to " + LOG.warn("Unable to sort datanodes based on distance to " +
"client, volume=" + keyInfo.getVolumeName() + "client, volume=" + keyInfo.getVolumeName() +

View File

@ -88,7 +88,9 @@ public BackgroundTaskResult call() throws Exception {
if (result.isSuccess()) { if (result.isSuccess()) {
try { try {
keyManager.deleteExpiredOpenKey(result.getObjectKey()); keyManager.deleteExpiredOpenKey(result.getObjectKey());
LOG.debug("Key {} deleted from OM DB", result.getObjectKey()); if (LOG.isDebugEnabled()) {
LOG.debug("Key {} deleted from OM DB", result.getObjectKey());
}
deletedSize += 1; deletedSize += 1;
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Failed to delete hanging-open key {}", LOG.warn("Failed to delete hanging-open key {}",

View File

@ -734,10 +734,12 @@ private static void loginOMUser(OzoneConfiguration conf)
if (SecurityUtil.getAuthenticationMethod(conf).equals( if (SecurityUtil.getAuthenticationMethod(conf).equals(
AuthenticationMethod.KERBEROS)) { AuthenticationMethod.KERBEROS)) {
LOG.debug("Ozone security is enabled. Attempting login for OM user. " if (LOG.isDebugEnabled()) {
+ "Principal: {},keytab: {}", conf.get( LOG.debug("Ozone security is enabled. Attempting login for OM user. "
OZONE_OM_KERBEROS_PRINCIPAL_KEY), + "Principal: {}, keytab: {}", conf.get(
conf.get(OZONE_OM_KERBEROS_KEYTAB_FILE_KEY)); OZONE_OM_KERBEROS_PRINCIPAL_KEY),
conf.get(OZONE_OM_KERBEROS_KEYTAB_FILE_KEY));
}
UserGroupInformation.setConfiguration(conf); UserGroupInformation.setConfiguration(conf);

View File

@ -139,7 +139,10 @@ public boolean removeAcl(OzoneObj obj, OzoneAcl acl) throws IOException {
OMPrefixAclOpResult omPrefixAclOpResult = removeAcl(obj, acl, prefixInfo); OMPrefixAclOpResult omPrefixAclOpResult = removeAcl(obj, acl, prefixInfo);
if (!omPrefixAclOpResult.isOperationsResult()) { if (!omPrefixAclOpResult.isOperationsResult()) {
LOG.debug("acl {} does not exist for prefix path {} ", acl, prefixPath); if (LOG.isDebugEnabled()) {
LOG.debug("acl {} does not exist for prefix path {} ",
acl, prefixPath);
}
return false; return false;
} }
@ -236,8 +239,10 @@ public boolean checkAccess(OzoneObj ozObject, RequestContext context)
if (lastNode != null && lastNode.getValue() != null) { if (lastNode != null && lastNode.getValue() != null) {
boolean hasAccess = OzoneAclUtil.checkAclRights(lastNode.getValue(). boolean hasAccess = OzoneAclUtil.checkAclRights(lastNode.getValue().
getAcls(), context); getAcls(), context);
LOG.debug("user:{} has access rights for ozObj:{} ::{} ", if (LOG.isDebugEnabled()) {
context.getClientUgi(), ozObject, hasAccess); LOG.debug("user:{} has access rights for ozObj:{} ::{} ",
context.getClientUgi(), ozObject, hasAccess);
}
return hasAccess; return hasAccess;
} else { } else {
return true; return true;

View File

@ -108,7 +108,7 @@ private UserVolumeInfo delVolumeFromOwnerList(String volume, String owner)
if (volumeList != null) { if (volumeList != null) {
prevVolList.addAll(volumeList.getVolumeNamesList()); prevVolList.addAll(volumeList.getVolumeNamesList());
} else { } else {
LOG.debug("volume:{} not found for user:{}"); LOG.debug("volume:{} not found for user:{}", volume, owner);
throw new OMException(ResultCodes.USER_NOT_FOUND); throw new OMException(ResultCodes.USER_NOT_FOUND);
} }
@ -503,7 +503,9 @@ public boolean addAcl(OzoneObj obj, OzoneAcl acl) throws IOException {
try { try {
volumeArgs.addAcl(acl); volumeArgs.addAcl(acl);
} catch (OMException ex) { } catch (OMException ex) {
LOG.debug("Add acl failed.", ex); if (LOG.isDebugEnabled()) {
LOG.debug("Add acl failed.", ex);
}
return false; return false;
} }
metadataManager.getVolumeTable().put(dbVolumeKey, volumeArgs); metadataManager.getVolumeTable().put(dbVolumeKey, volumeArgs);
@ -553,7 +555,9 @@ public boolean removeAcl(OzoneObj obj, OzoneAcl acl) throws IOException {
try { try {
volumeArgs.removeAcl(acl); volumeArgs.removeAcl(acl);
} catch (OMException ex) { } catch (OMException ex) {
LOG.debug("Remove acl failed.", ex); if (LOG.isDebugEnabled()) {
LOG.debug("Remove acl failed.", ex);
}
return false; return false;
} }
metadataManager.getVolumeTable().put(dbVolumeKey, volumeArgs); metadataManager.getVolumeTable().put(dbVolumeKey, volumeArgs);
@ -685,8 +689,10 @@ public boolean checkAccess(OzoneObj ozObject, RequestContext context)
Preconditions.checkState(volume.equals(volumeArgs.getVolume())); Preconditions.checkState(volume.equals(volumeArgs.getVolume()));
boolean hasAccess = volumeArgs.getAclMap().hasAccess( boolean hasAccess = volumeArgs.getAclMap().hasAccess(
context.getAclRights(), context.getClientUgi()); context.getAclRights(), context.getClientUgi());
LOG.debug("user:{} has access rights for volume:{} :{} ", if (LOG.isDebugEnabled()) {
context.getClientUgi(), ozObject.getVolumeName(), hasAccess); LOG.debug("user:{} has access rights for volume:{} :{} ",
context.getClientUgi(), ozObject.getVolumeName(), hasAccess);
}
return hasAccess; return hasAccess;
} catch (IOException ex) { } catch (IOException ex) {
LOG.error("Check access operation failed for volume:{}", volume, ex); LOG.error("Check access operation failed for volume:{}", volume, ex);

View File

@ -148,9 +148,11 @@ private void flushTransactions() {
flushedTransactionCount.addAndGet(flushedTransactionsSize); flushedTransactionCount.addAndGet(flushedTransactionsSize);
flushIterations.incrementAndGet(); flushIterations.incrementAndGet();
LOG.debug("Sync Iteration {} flushed transactions in this " + if (LOG.isDebugEnabled()) {
"iteration{}", flushIterations.get(), LOG.debug("Sync Iteration {} flushed transactions in this " +
flushedTransactionsSize); "iteration{}", flushIterations.get(),
flushedTransactionsSize);
}
long lastRatisTransactionIndex = long lastRatisTransactionIndex =
readyBuffer.stream().map(DoubleBufferEntry::getTrxLogIndex) readyBuffer.stream().map(DoubleBufferEntry::getTrxLogIndex)

View File

@ -99,8 +99,10 @@ public static OzoneManagerRatisClient newOzoneManagerRatisClient(
} }
public void connect() { public void connect() {
LOG.debug("Connecting to OM Ratis Server GroupId:{} OM:{}", if (LOG.isDebugEnabled()) {
raftGroup.getGroupId().getUuid().toString(), omNodeID); LOG.debug("Connecting to OM Ratis Server GroupId:{} OM:{}",
raftGroup.getGroupId().getUuid().toString(), omNodeID);
}
// TODO : XceiverClient ratis should pass the config value of // TODO : XceiverClient ratis should pass the config value of
// maxOutstandingRequests so as to set the upper bound on max no of async // maxOutstandingRequests so as to set the upper bound on max no of async
@ -147,8 +149,7 @@ private OzoneManagerProtocolProtos.Status parseErrorStatus(String message) {
if (message.contains(STATUS_CODE)) { if (message.contains(STATUS_CODE)) {
String errorCode = message.substring(message.indexOf(STATUS_CODE) + String errorCode = message.substring(message.indexOf(STATUS_CODE) +
STATUS_CODE.length()); STATUS_CODE.length());
LOG.debug("Parsing error message for error code " + LOG.debug("Parsing error message for error code {}", errorCode);
errorCode);
return OzoneManagerProtocolProtos.Status.valueOf(errorCode.trim()); return OzoneManagerProtocolProtos.Status.valueOf(errorCode.trim());
} else { } else {
return OzoneManagerProtocolProtos.Status.INTERNAL_ERROR; return OzoneManagerProtocolProtos.Status.INTERNAL_ERROR;
@ -166,25 +167,27 @@ private CompletableFuture<OMResponse> sendCommandAsync(OMRequest request) {
CompletableFuture<RaftClientReply> raftClientReply = CompletableFuture<RaftClientReply> raftClientReply =
sendRequestAsync(request); sendRequestAsync(request);
return raftClientReply.whenComplete((reply, e) -> LOG.debug( return raftClientReply.whenComplete((reply, e) -> {
"received reply {} for request: cmdType={} traceID={} " + if (LOG.isDebugEnabled()) {
"exception: {}", reply, request.getCmdType(), LOG.debug("received reply {} for request: cmdType={} traceID={} " +
request.getTraceID(), e)) "exception: {}", reply, request.getCmdType(),
.thenApply(reply -> { request.getTraceID(), e);
try { }
Preconditions.checkNotNull(reply); }).thenApply(reply -> {
if (!reply.isSuccess()) { try {
RaftException exception = reply.getException(); Preconditions.checkNotNull(reply);
Preconditions.checkNotNull(exception, "Raft reply failure " + if (!reply.isSuccess()) {
"but no exception propagated."); RaftException exception = reply.getException();
throw new CompletionException(exception); Preconditions.checkNotNull(exception, "Raft reply failure " +
} "but no exception propagated.");
return OMRatisHelper.getOMResponseFromRaftClientReply(reply); throw new CompletionException(exception);
}
return OMRatisHelper.getOMResponseFromRaftClientReply(reply);
} catch (InvalidProtocolBufferException e) { } catch (InvalidProtocolBufferException e) {
throw new CompletionException(e); throw new CompletionException(e);
} }
}); });
} }
/** /**
@ -198,7 +201,9 @@ private CompletableFuture<RaftClientReply> sendRequestAsync(
OMRequest request) { OMRequest request) {
boolean isReadOnlyRequest = OmUtils.isReadOnly(request); boolean isReadOnlyRequest = OmUtils.isReadOnly(request);
ByteString byteString = OMRatisHelper.convertRequestToByteString(request); ByteString byteString = OMRatisHelper.convertRequestToByteString(request);
LOG.debug("sendOMRequestAsync {} {}", isReadOnlyRequest, request); if (LOG.isDebugEnabled()) {
LOG.debug("sendOMRequestAsync {} {}", isReadOnlyRequest, request);
}
return isReadOnlyRequest ? raftClient.sendReadOnlyAsync(() -> byteString) : return isReadOnlyRequest ? raftClient.sendReadOnlyAsync(() -> byteString) :
raftClient.sendAsync(() -> byteString); raftClient.sendAsync(() -> byteString);
} }

View File

@ -169,8 +169,10 @@ private OMResponse processReply(OMRequest omRequest, RaftClientReply reply)
omResponse.setMessage(stateMachineException.getCause().getMessage()); omResponse.setMessage(stateMachineException.getCause().getMessage());
omResponse.setStatus(parseErrorStatus( omResponse.setStatus(parseErrorStatus(
stateMachineException.getCause().getMessage())); stateMachineException.getCause().getMessage()));
LOG.debug("Error while executing ratis request. " + if (LOG.isDebugEnabled()) {
"stateMachineException: ", stateMachineException); LOG.debug("Error while executing ratis request. " +
"stateMachineException: ", stateMachineException);
}
return omResponse.build(); return omResponse.build();
} }

View File

@ -103,7 +103,9 @@ OMClientResponse onFailure(OMResponse.Builder omResponse,
void onComplete(boolean operationResult, IOException exception, void onComplete(boolean operationResult, IOException exception,
OMMetrics omMetrics) { OMMetrics omMetrics) {
if (operationResult) { if (operationResult) {
LOG.debug("Set acl: {} for path: {} success!", getAcls(), getPath()); if (LOG.isDebugEnabled()) {
LOG.debug("Set acl: {} for path: {} success!", getAcls(), getPath());
}
} else { } else {
omMetrics.incNumBucketUpdateFails(); omMetrics.incNumBucketUpdateFails();
if (exception == null) { if (exception == null) {

View File

@ -96,8 +96,10 @@ OMClientResponse onFailure(OMResponse.Builder omResponse,
@Override @Override
void onComplete(IOException ex) { void onComplete(IOException ex) {
if (ex == null) { if (ex == null) {
LOG.debug("Set acls: {} to volume: {} success!", if (LOG.isDebugEnabled()) {
getAcls(), getVolumeName()); LOG.debug("Set acls: {} to volume: {} success!",
getAcls(), getVolumeName());
}
} else { } else {
LOG.error("Set acls {} to volume {} failed!", LOG.error("Set acls {} to volume {} failed!",
getAcls(), getVolumeName(), ex); getAcls(), getVolumeName(), ex);

View File

@ -48,7 +48,9 @@ public OzoneManagerHARequestHandlerImpl(OzoneManager om,
@Override @Override
public OMResponse handleApplyTransaction(OMRequest omRequest, public OMResponse handleApplyTransaction(OMRequest omRequest,
long transactionLogIndex) { long transactionLogIndex) {
LOG.debug("Received OMRequest: {}, ", omRequest); if (LOG.isDebugEnabled()) {
LOG.debug("Received OMRequest: {}, ", omRequest);
}
Type cmdType = omRequest.getCmdType(); Type cmdType = omRequest.getCmdType();
switch (cmdType) { switch (cmdType) {
case CreateVolume: case CreateVolume:

View File

@ -225,7 +225,9 @@ private OMResponse submitRequestDirectlyToOM(OMRequest request) {
} }
try { try {
omClientResponse.getFlushFuture().get(); omClientResponse.getFlushFuture().get();
LOG.trace("Future for {} is completed", request); if (LOG.isTraceEnabled()) {
LOG.trace("Future for {} is completed", request);
}
} catch (ExecutionException | InterruptedException ex) { } catch (ExecutionException | InterruptedException ex) {
// terminate OM. As if we are in this stage means, while getting // terminate OM. As if we are in this stage means, while getting
// response from flush future, we got an exception. // response from flush future, we got an exception.

View File

@ -149,7 +149,9 @@ public OzoneManagerRequestHandler(OzoneManager om) {
@SuppressWarnings("methodlength") @SuppressWarnings("methodlength")
@Override @Override
public OMResponse handle(OMRequest request) { public OMResponse handle(OMRequest request) {
LOG.debug("Received OMRequest: {}, ", request); if (LOG.isDebugEnabled()) {
LOG.debug("Received OMRequest: {}, ", request);
}
Type cmdType = request.getCmdType(); Type cmdType = request.getCmdType();
OMResponse.Builder responseBuilder = OMResponse.newBuilder() OMResponse.Builder responseBuilder = OMResponse.newBuilder()
.setCmdType(cmdType) .setCmdType(cmdType)

View File

@ -79,20 +79,20 @@ public boolean checkAccess(IOzoneObj ozObject, RequestContext context)
switch (objInfo.getResourceType()) { switch (objInfo.getResourceType()) {
case VOLUME: case VOLUME:
LOG.trace("Checking access for volume:" + objInfo); LOG.trace("Checking access for volume: {}", objInfo);
return volumeManager.checkAccess(objInfo, context); return volumeManager.checkAccess(objInfo, context);
case BUCKET: case BUCKET:
LOG.trace("Checking access for bucket:" + objInfo); LOG.trace("Checking access for bucket: {}", objInfo);
return (bucketManager.checkAccess(objInfo, context) return (bucketManager.checkAccess(objInfo, context)
&& volumeManager.checkAccess(objInfo, context)); && volumeManager.checkAccess(objInfo, context));
case KEY: case KEY:
LOG.trace("Checking access for Key:" + objInfo); LOG.trace("Checking access for Key: {}", objInfo);
return (keyManager.checkAccess(objInfo, context) return (keyManager.checkAccess(objInfo, context)
&& prefixManager.checkAccess(objInfo, context) && prefixManager.checkAccess(objInfo, context)
&& bucketManager.checkAccess(objInfo, context) && bucketManager.checkAccess(objInfo, context)
&& volumeManager.checkAccess(objInfo, context)); && volumeManager.checkAccess(objInfo, context));
case PREFIX: case PREFIX:
LOG.trace("Checking access for Prefix:" + objInfo); LOG.trace("Checking access for Prefix: {]", objInfo);
return (prefixManager.checkAccess(objInfo, context) return (prefixManager.checkAccess(objInfo, context)
&& bucketManager.checkAccess(objInfo, context) && bucketManager.checkAccess(objInfo, context)
&& volumeManager.checkAccess(objInfo, context)); && volumeManager.checkAccess(objInfo, context));

View File

@ -425,7 +425,9 @@ private boolean innerDelete(Path f, boolean recursive) throws IOException {
DeleteIterator iterator = new DeleteIterator(f, recursive); DeleteIterator iterator = new DeleteIterator(f, recursive);
return iterator.iterate(); return iterator.iterate();
} catch (FileNotFoundException e) { } catch (FileNotFoundException e) {
LOG.debug("Couldn't delete {} - does not exist", f); if (LOG.isDebugEnabled()) {
LOG.debug("Couldn't delete {} - does not exist", f);
}
return false; return false;
} }
} }

View File

@ -110,10 +110,14 @@ public void parse() throws Exception {
canonicalRequest = buildCanonicalRequest(); canonicalRequest = buildCanonicalRequest();
strToSign.append(hash(canonicalRequest)); strToSign.append(hash(canonicalRequest));
LOG.debug("canonicalRequest:[{}]", canonicalRequest); if (LOG.isDebugEnabled()) {
LOG.debug("canonicalRequest:[{}]", canonicalRequest);
}
headerMap.keySet().forEach(k -> LOG.trace("Header:{},value:{}", k, if (LOG.isTraceEnabled()) {
headerMap.get(k))); headerMap.keySet().forEach(k -> LOG.trace("Header:{},value:{}", k,
headerMap.get(k)));
}
LOG.debug("StringToSign:[{}]", strToSign); LOG.debug("StringToSign:[{}]", strToSign);
stringToSign = strToSign.toString(); stringToSign = strToSign.toString();

View File

@ -86,8 +86,9 @@ private OzoneClient getClient(OzoneConfiguration config) throws IOException {
identifier.setSignature(v4RequestParser.getSignature()); identifier.setSignature(v4RequestParser.getSignature());
identifier.setAwsAccessId(v4RequestParser.getAwsAccessId()); identifier.setAwsAccessId(v4RequestParser.getAwsAccessId());
identifier.setOwner(new Text(v4RequestParser.getAwsAccessId())); identifier.setOwner(new Text(v4RequestParser.getAwsAccessId()));
if (LOG.isTraceEnabled()) {
LOG.trace("Adding token for service:{}", omService); LOG.trace("Adding token for service:{}", omService);
}
Token<OzoneTokenIdentifier> token = new Token(identifier.getBytes(), Token<OzoneTokenIdentifier> token = new Token(identifier.getBytes(),
identifier.getSignature().getBytes(UTF_8), identifier.getSignature().getBytes(UTF_8),
identifier.getKind(), identifier.getKind(),

View File

@ -42,7 +42,9 @@ public class OS3ExceptionMapper implements ExceptionMapper<OS3Exception> {
@Override @Override
public Response toResponse(OS3Exception exception) { public Response toResponse(OS3Exception exception) {
LOG.debug("Returning exception. ex: {}", exception.toString()); if (LOG.isDebugEnabled()) {
LOG.debug("Returning exception. ex: {}", exception.toString());
}
exception.setRequestId(requestIdentifier.getRequestId()); exception.setRequestId(requestIdentifier.getRequestId());
return Response.status(exception.getHttpCode()) return Response.status(exception.getHttpCode())
.entity(exception.toXml()).build(); .entity(exception.toXml()).build();