Skip to content

Commit

Permalink
HDDS-2266. Avoid evaluation of LOG.trace and LOG.debug statement in t…
Browse files Browse the repository at this point in the history
…he read/write path. (apache#1633)
  • Loading branch information
swagle authored and RogPodge committed Mar 25, 2020
1 parent b052483 commit d2a3e0b
Show file tree
Hide file tree
Showing 31 changed files with 182 additions and 101 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -41,8 +41,7 @@
*/
public final class Pipeline {

private static final Logger LOG = LoggerFactory
.getLogger(Pipeline.class);
private static final Logger LOG = LoggerFactory.getLogger(Pipeline.class);
private final PipelineID id;
private final ReplicationType type;
private final ReplicationFactor factor;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -193,10 +193,12 @@ public List<OmKeyLocationInfo> getLocationInfoList() {
.setPipeline(streamEntry.getPipeline()).build();
locationInfoList.add(info);
}
LOG.debug(
"block written " + streamEntry.getBlockID() + ", length " + length
+ " bcsID " + streamEntry.getBlockID()
.getBlockCommitSequenceId());
if (LOG.isDebugEnabled()) {
LOG.debug(
"block written " + streamEntry.getBlockID() + ", length " + length
+ " bcsID " + streamEntry.getBlockID()
.getBlockCommitSequenceId());
}
}
return locationInfoList;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -97,8 +97,10 @@ private synchronized void initialize(String keyName,
long keyLength = 0;
for (int i = 0; i < blockInfos.size(); i++) {
OmKeyLocationInfo omKeyLocationInfo = blockInfos.get(i);
LOG.debug("Adding stream for accessing {}. The stream will be " +
"initialized later.", omKeyLocationInfo);
if (LOG.isDebugEnabled()) {
LOG.debug("Adding stream for accessing {}. The stream will be " +
"initialized later.", omKeyLocationInfo);
}

addStream(omKeyLocationInfo, xceiverClientManager,
verifyChecksum);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -439,10 +439,14 @@ public Token<OzoneTokenIdentifier> getDelegationToken(Text renewer)
ozoneManagerClient.getDelegationToken(renewer);
if (token != null) {
token.setService(dtService);
LOG.debug("Created token {} for dtService {}", token, dtService);
if (LOG.isDebugEnabled()) {
LOG.debug("Created token {} for dtService {}", token, dtService);
}
} else {
LOG.debug("Cannot get ozone delegation token for renewer {} to access " +
"service {}", renewer, dtService);
if (LOG.isDebugEnabled()) {
LOG.debug("Cannot get ozone delegation token for renewer {} to " +
"access service {}", renewer, dtService);
}
}
return token;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,9 @@ public S3SecretValue getS3Secret(String kerberosID) throws IOException {
} finally {
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;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -214,8 +214,10 @@ private Text computeDelegationTokenService() {
@Override
public void performFailover(OzoneManagerProtocolPB currentProxy) {
int newProxyIndex = incrementProxyIndex();
LOG.debug("Failing over OM proxy to index: {}, nodeId: {}",
newProxyIndex, omNodeIDList.get(newProxyIndex));
if (LOG.isDebugEnabled()) {
LOG.debug("Failing over OM proxy to index: {}, nodeId: {}",
newProxyIndex, omNodeIDList.get(newProxyIndex));
}
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,9 @@ private OMRatisHelper() {
*/
public static RaftClient newRaftClient(RpcType rpcType, String omId, RaftGroup
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();
RaftConfigKeys.Rpc.setType(properties, rpcType);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -168,8 +168,10 @@ private boolean lock(Resource resource, String resourceName,
throw new RuntimeException(errorMessage);
} else {
lockFn.accept(resourceName);
LOG.debug("Acquired {} {} lock on resource {}", lockType, resource.name,
resourceName);
if (LOG.isDebugEnabled()) {
LOG.debug("Acquired {} {} lock on resource {}", lockType, resource.name,
resourceName);
}
lockSet.set(resource.setLock(lockSet.get()));
return true;
}
Expand Down Expand Up @@ -264,8 +266,10 @@ public boolean acquireMultiUserLock(String firstUser, String secondUser) {
throw ex;
}
}
LOG.debug("Acquired Write {} lock on resource {} and {}", resource.name,
firstUser, secondUser);
if (LOG.isDebugEnabled()) {
LOG.debug("Acquired Write {} lock on resource {} and {}", resource.name,
firstUser, secondUser);
}
lockSet.set(resource.setLock(lockSet.get()));
return true;
}
Expand Down Expand Up @@ -300,8 +304,10 @@ public void releaseMultiUserLock(String firstUser, String secondUser) {
manager.writeUnlock(firstUser);
manager.writeUnlock(secondUser);
}
LOG.debug("Release Write {} lock on resource {} and {}", resource.name,
firstUser, secondUser);
if (LOG.isDebugEnabled()) {
LOG.debug("Release Write {} lock on resource {} and {}", resource.name,
firstUser, secondUser);
}
lockSet.set(resource.clearLock(lockSet.get()));
}

Expand Down Expand Up @@ -352,8 +358,10 @@ private void unlock(Resource resource, String resourceName,
// locks, as some locks support acquiring lock again.
lockFn.accept(resourceName);
// clear lock
LOG.debug("Release {} {}, lock on resource {}", lockType, resource.name,
resourceName);
if (LOG.isDebugEnabled()) {
LOG.debug("Release {} {}, lock on resource {}", lockType, resource.name,
resourceName);
}
lockSet.set(resource.clearLock(lockSet.get()));
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -89,7 +89,7 @@ public Token<OzoneBlockTokenIdentifier> generateToken(String user,
if (LOG.isTraceEnabled()) {
long expiryTime = tokenIdentifier.getExpiryDate();
String tokenId = tokenIdentifier.toString();
LOG.trace("Issued delegation token -> expiryTime:{},tokenId:{}",
LOG.trace("Issued delegation token -> expiryTime:{}, tokenId:{}",
expiryTime, tokenId);
}
// Pass blockId as service.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -289,8 +289,10 @@ public OzoneTokenIdentifier cancelToken(Token<OzoneTokenIdentifier> token,
String canceller) throws IOException {
OzoneTokenIdentifier id = OzoneTokenIdentifier.readProtoBuf(
token.getIdentifier());
LOG.debug("Token cancellation requested for identifier: {}",
formatTokenId(id));
if (LOG.isDebugEnabled()) {
LOG.debug("Token cancellation requested for identifier: {}",
formatTokenId(id));
}

if (id.getUser() == null) {
throw new InvalidToken("Token with no owner " + formatTokenId(id));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,9 +43,13 @@ public OzoneDelegationTokenSelector() {
@Override
public Token<OzoneTokenIdentifier> selectToken(Text service,
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);
LOG.debug("Got tokens: {} for service {}", token, service);
if (LOG.isDebugEnabled()) {
LOG.debug("Got tokens: {} for service {}", token, service);
}
return token;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -110,8 +110,10 @@ public byte[] createPassword(byte[] identifier, PrivateKey privateKey)

@Override
public byte[] createPassword(T identifier) {
logger.debug("Creating password for identifier: {}, currentKey: {}",
formatTokenId(identifier), currentKey.getKeyId());
if (logger.isDebugEnabled()) {
logger.debug("Creating password for identifier: {}, currentKey: {}",
formatTokenId(identifier), currentKey.getKeyId());
}
byte[] password = null;
try {
password = createPassword(identifier.getBytes(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -570,8 +570,10 @@ public boolean checkAccess(OzoneObj ozObject, RequestContext context)
}
boolean hasAccess = OzoneAclUtil.checkAclRights(bucketInfo.getAcls(),
context);
LOG.debug("user:{} has access rights for bucket:{} :{} ",
context.getClientUgi(), ozObject.getBucketName(), hasAccess);
if (LOG.isDebugEnabled()) {
LOG.debug("user:{} has access rights for bucket:{} :{} ",
context.getClientUgi(), ozObject.getBucketName(), hasAccess);
}
return hasAccess;
} catch (IOException ex) {
if(ex instanceof OMException) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1661,8 +1661,10 @@ public boolean checkAccess(OzoneObj ozObject, RequestContext context)
if (keyInfo == null) {
// the key does not exist, but it is a parent "dir" of some key
// let access be determined based on volume/bucket/prefix ACL
LOG.debug("key:{} is non-existent parent, permit access to user:{}",
keyName, context.getClientUgi());
if (LOG.isDebugEnabled()) {
LOG.debug("key:{} is non-existent parent, permit access to user:{}",
keyName, context.getClientUgi());
}
return true;
}
} catch (OMException e) {
Expand All @@ -1678,8 +1680,10 @@ public boolean checkAccess(OzoneObj ozObject, RequestContext context)

boolean hasAccess = OzoneAclUtil.checkAclRight(
keyInfo.getAcls(), context);
LOG.debug("user:{} has access rights for key:{} :{} ",
context.getClientUgi(), ozObject.getKeyName(), hasAccess);
if (LOG.isDebugEnabled()) {
LOG.debug("user:{} has access rights for key:{} :{} ",
context.getClientUgi(), ozObject.getKeyName(), hasAccess);
}
return hasAccess;
} catch (IOException ex) {
if(ex instanceof OMException) {
Expand Down Expand Up @@ -1766,10 +1770,11 @@ public OzoneFileStatus getFileStatus(OmKeyArgs args) throws IOException {
if (keys.iterator().hasNext()) {
return new OzoneFileStatus(keyName);
}

LOG.debug("Unable to get file status for the key: volume:" + volumeName +
" bucket:" + bucketName + " key:" + keyName + " with error no " +
"such file exists:");
if (LOG.isDebugEnabled()) {
LOG.debug("Unable to get file status for the key: volume: {}, bucket:" +
" {}, key: {}, with error: No such file exists.", volumeName,
bucketName, keyName);
}
throw new OMException("Unable to get file status: volume: " +
volumeName + " bucket: " + bucketName + " key: " + keyName,
FILE_NOT_FOUND);
Expand Down Expand Up @@ -2132,8 +2137,10 @@ private void sortDatanodeInPipeline(OmKeyInfo keyInfo, String clientMachine) {
List<DatanodeDetails> sortedNodes = scmClient.getBlockClient()
.sortDatanodes(nodeList, clientMachine);
k.getPipeline().setNodesInOrder(sortedNodes);
LOG.debug("Sort datanodes {} for client {}, return {}", nodes,
clientMachine, sortedNodes);
if (LOG.isDebugEnabled()) {
LOG.debug("Sort datanodes {} for client {}, return {}", nodes,
clientMachine, sortedNodes);
}
} catch (IOException e) {
LOG.warn("Unable to sort datanodes based on distance to " +
"client, volume=" + keyInfo.getVolumeName() +
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -88,7 +88,9 @@ public BackgroundTaskResult call() throws Exception {
if (result.isSuccess()) {
try {
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;
} catch (IOException e) {
LOG.warn("Failed to delete hanging-open key {}",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -734,10 +734,12 @@ private static void loginOMUser(OzoneConfiguration conf)

if (SecurityUtil.getAuthenticationMethod(conf).equals(
AuthenticationMethod.KERBEROS)) {
LOG.debug("Ozone security is enabled. Attempting login for OM user. "
+ "Principal: {},keytab: {}", conf.get(
OZONE_OM_KERBEROS_PRINCIPAL_KEY),
conf.get(OZONE_OM_KERBEROS_KEYTAB_FILE_KEY));
if (LOG.isDebugEnabled()) {
LOG.debug("Ozone security is enabled. Attempting login for OM user. "
+ "Principal: {}, keytab: {}", conf.get(
OZONE_OM_KERBEROS_PRINCIPAL_KEY),
conf.get(OZONE_OM_KERBEROS_KEYTAB_FILE_KEY));
}

UserGroupInformation.setConfiguration(conf);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -139,7 +139,10 @@ public boolean removeAcl(OzoneObj obj, OzoneAcl acl) throws IOException {
OMPrefixAclOpResult omPrefixAclOpResult = removeAcl(obj, acl, prefixInfo);

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;
}

Expand Down Expand Up @@ -236,8 +239,10 @@ public boolean checkAccess(OzoneObj ozObject, RequestContext context)
if (lastNode != null && lastNode.getValue() != null) {
boolean hasAccess = OzoneAclUtil.checkAclRights(lastNode.getValue().
getAcls(), context);
LOG.debug("user:{} has access rights for ozObj:{} ::{} ",
context.getClientUgi(), ozObject, hasAccess);
if (LOG.isDebugEnabled()) {
LOG.debug("user:{} has access rights for ozObj:{} ::{} ",
context.getClientUgi(), ozObject, hasAccess);
}
return hasAccess;
} else {
return true;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -108,7 +108,7 @@ private UserVolumeInfo delVolumeFromOwnerList(String volume, String owner)
if (volumeList != null) {
prevVolList.addAll(volumeList.getVolumeNamesList());
} else {
LOG.debug("volume:{} not found for user:{}");
LOG.debug("volume:{} not found for user:{}", volume, owner);
throw new OMException(ResultCodes.USER_NOT_FOUND);
}

Expand Down Expand Up @@ -503,7 +503,9 @@ public boolean addAcl(OzoneObj obj, OzoneAcl acl) throws IOException {
try {
volumeArgs.addAcl(acl);
} catch (OMException ex) {
LOG.debug("Add acl failed.", ex);
if (LOG.isDebugEnabled()) {
LOG.debug("Add acl failed.", ex);
}
return false;
}
metadataManager.getVolumeTable().put(dbVolumeKey, volumeArgs);
Expand Down Expand Up @@ -553,7 +555,9 @@ public boolean removeAcl(OzoneObj obj, OzoneAcl acl) throws IOException {
try {
volumeArgs.removeAcl(acl);
} catch (OMException ex) {
LOG.debug("Remove acl failed.", ex);
if (LOG.isDebugEnabled()) {
LOG.debug("Remove acl failed.", ex);
}
return false;
}
metadataManager.getVolumeTable().put(dbVolumeKey, volumeArgs);
Expand Down Expand Up @@ -685,8 +689,10 @@ public boolean checkAccess(OzoneObj ozObject, RequestContext context)
Preconditions.checkState(volume.equals(volumeArgs.getVolume()));
boolean hasAccess = volumeArgs.getAclMap().hasAccess(
context.getAclRights(), context.getClientUgi());
LOG.debug("user:{} has access rights for volume:{} :{} ",
context.getClientUgi(), ozObject.getVolumeName(), hasAccess);
if (LOG.isDebugEnabled()) {
LOG.debug("user:{} has access rights for volume:{} :{} ",
context.getClientUgi(), ozObject.getVolumeName(), hasAccess);
}
return hasAccess;
} catch (IOException ex) {
LOG.error("Check access operation failed for volume:{}", volume, ex);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -148,9 +148,11 @@ private void flushTransactions() {
flushedTransactionCount.addAndGet(flushedTransactionsSize);
flushIterations.incrementAndGet();

LOG.debug("Sync Iteration {} flushed transactions in this " +
"iteration{}", flushIterations.get(),
flushedTransactionsSize);
if (LOG.isDebugEnabled()) {
LOG.debug("Sync Iteration {} flushed transactions in this " +
"iteration{}", flushIterations.get(),
flushedTransactionsSize);
}

long lastRatisTransactionIndex =
readyBuffer.stream().map(DoubleBufferEntry::getTrxLogIndex)
Expand Down
Loading

0 comments on commit d2a3e0b

Please sign in to comment.