HADOOP-11741. Add LOG.isDebugEnabled() guard for some LOG.debug(). Contributed by Walter Su.
(cherry picked from commit 5582b0f1d4
)
This commit is contained in:
parent
456cec127b
commit
77024aada9
|
@ -29,6 +29,9 @@ Release 2.8.0 - UNRELEASED
|
|||
HADOOP-11737. mockito's version in hadoop-nfs’ pom.xml shouldn't be
|
||||
specified. (Kengo Seki via ozawa)
|
||||
|
||||
HADOOP-11741. Add LOG.isDebugEnabled() guard for some LOG.debug().
|
||||
(Walter Su via ozawa)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
BUG FIXES
|
||||
|
|
|
@ -2460,8 +2460,10 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
|
|||
private Document parse(DocumentBuilder builder, URL url)
|
||||
throws IOException, SAXException {
|
||||
if (!quietmode) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("parsing URL " + url);
|
||||
}
|
||||
}
|
||||
if (url == null) {
|
||||
return null;
|
||||
}
|
||||
|
|
|
@ -214,9 +214,11 @@ public class JavaKeyStoreProvider extends KeyProvider {
|
|||
renameOrFail(path, new Path(path.toString() + "_CORRUPTED_"
|
||||
+ System.currentTimeMillis()));
|
||||
renameOrFail(backupPath, path);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(String.format(
|
||||
"KeyStore loaded successfully from '%s' since '%s'"
|
||||
+ "was corrupted !!", backupPath, path));
|
||||
}
|
||||
} else {
|
||||
throw ioe;
|
||||
}
|
||||
|
@ -265,8 +267,10 @@ public class JavaKeyStoreProvider extends KeyProvider {
|
|||
try {
|
||||
perm = loadFromPath(pathToLoad, password);
|
||||
renameOrFail(pathToLoad, path);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(String.format("KeyStore loaded successfully from '%s'!!",
|
||||
pathToLoad));
|
||||
}
|
||||
if (fs.exists(pathToDelete)) {
|
||||
fs.delete(pathToDelete, true);
|
||||
}
|
||||
|
|
|
@ -256,7 +256,9 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
|
|||
appData = new byte[data.length];
|
||||
System.arraycopy(data, 0, appData, 0, data.length);
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Attempting active election for " + this);
|
||||
}
|
||||
joinElectionInternal();
|
||||
}
|
||||
|
||||
|
@ -406,9 +408,11 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
|
|||
public synchronized void processResult(int rc, String path, Object ctx,
|
||||
String name) {
|
||||
if (isStaleClient(ctx)) return;
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("CreateNode result: " + rc + " for path: " + path
|
||||
+ " connectionState: " + zkConnectionState +
|
||||
" for " + this);
|
||||
}
|
||||
|
||||
Code code = Code.get(rc);
|
||||
if (isSuccess(code)) {
|
||||
|
@ -468,9 +472,10 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
|
|||
assert wantToBeInElection :
|
||||
"Got a StatNode result after quitting election";
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("StatNode result: " + rc + " for path: " + path
|
||||
+ " connectionState: " + zkConnectionState + " for " + this);
|
||||
|
||||
}
|
||||
|
||||
Code code = Code.get(rc);
|
||||
if (isSuccess(code)) {
|
||||
|
@ -535,10 +540,12 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
|
|||
synchronized void processWatchEvent(ZooKeeper zk, WatchedEvent event) {
|
||||
Event.EventType eventType = event.getType();
|
||||
if (isStaleClient(zk)) return;
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Watcher event type: " + eventType + " with state:"
|
||||
+ event.getState() + " for path:" + event.getPath()
|
||||
+ " connectionState: " + zkConnectionState
|
||||
+ " for " + this);
|
||||
}
|
||||
|
||||
if (eventType == Event.EventType.None) {
|
||||
// the connection state has changed
|
||||
|
@ -597,7 +604,9 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
|
|||
monitorActiveStatus();
|
||||
break;
|
||||
default:
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Unexpected node event: " + eventType + " for path: " + path);
|
||||
}
|
||||
monitorActiveStatus();
|
||||
}
|
||||
|
||||
|
@ -646,7 +655,9 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
|
|||
|
||||
private void monitorActiveStatus() {
|
||||
assert wantToBeInElection;
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Monitoring active leader for " + this);
|
||||
}
|
||||
statRetryCount = 0;
|
||||
monitorLockNodeAsync();
|
||||
}
|
||||
|
@ -737,7 +748,9 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
|
|||
int connectionRetryCount = 0;
|
||||
boolean success = false;
|
||||
while(!success && connectionRetryCount < maxRetryNum) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Establishing zookeeper connection for " + this);
|
||||
}
|
||||
try {
|
||||
createConnection();
|
||||
success = true;
|
||||
|
@ -765,15 +778,19 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
|
|||
watcher = null;
|
||||
}
|
||||
zkClient = getNewZooKeeper();
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Created new connection for " + this);
|
||||
}
|
||||
}
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public synchronized void terminateConnection() {
|
||||
if (zkClient == null) {
|
||||
return;
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Terminating ZK connection for " + this);
|
||||
}
|
||||
ZooKeeper tempZk = zkClient;
|
||||
zkClient = null;
|
||||
watcher = null;
|
||||
|
@ -801,7 +818,9 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
|
|||
Stat oldBreadcrumbStat = fenceOldActive();
|
||||
writeBreadCrumbNode(oldBreadcrumbStat);
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Becoming active for " + this);
|
||||
}
|
||||
appClient.becomeActive();
|
||||
state = State.ACTIVE;
|
||||
return true;
|
||||
|
@ -906,7 +925,9 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
|
|||
|
||||
private void becomeStandby() {
|
||||
if (state != State.STANDBY) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Becoming standby for " + this);
|
||||
}
|
||||
state = State.STANDBY;
|
||||
appClient.becomeStandby();
|
||||
}
|
||||
|
@ -914,7 +935,9 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
|
|||
|
||||
private void enterNeutralMode() {
|
||||
if (state != State.NEUTRAL) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Entering neutral mode for " + this);
|
||||
}
|
||||
state = State.NEUTRAL;
|
||||
appClient.enterNeutralMode();
|
||||
}
|
||||
|
|
|
@ -872,10 +872,13 @@ public class RPC {
|
|||
|
||||
getProtocolImplMap(rpcKind).put(new ProtoNameVer(protocolName, version),
|
||||
new ProtoClassProtoImpl(protocolClass, protocolImpl));
|
||||
LOG.debug("RpcKind = " + rpcKind + " Protocol Name = " + protocolName + " version=" + version +
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("RpcKind = " + rpcKind + " Protocol Name = " + protocolName +
|
||||
" version=" + version +
|
||||
" ProtocolImpl=" + protocolImpl.getClass().getName() +
|
||||
" protocolClass=" + protocolClass.getName());
|
||||
}
|
||||
}
|
||||
|
||||
static class VerProtocolImpl {
|
||||
final long version;
|
||||
|
|
|
@ -231,10 +231,12 @@ public abstract class Server {
|
|||
throw new IllegalArgumentException("ReRegistration of rpcKind: " +
|
||||
rpcKind);
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("rpcKind=" + rpcKind +
|
||||
", rpcRequestWrapperClass=" + rpcRequestWrapperClass +
|
||||
", rpcInvoker=" + rpcInvoker);
|
||||
}
|
||||
}
|
||||
|
||||
public Class<? extends Writable> getRpcRequestWrapper(
|
||||
RpcKindProto rpcKind) {
|
||||
|
|
|
@ -80,8 +80,10 @@ public class GangliaContext31 extends GangliaContext {
|
|||
return;
|
||||
}
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Emitting metric " + name + ", type " + type + ", value " +
|
||||
value + " from hostname" + hostName);
|
||||
}
|
||||
|
||||
String units = getUnits(name);
|
||||
int slope = getSlope(name);
|
||||
|
|
|
@ -164,7 +164,9 @@ public class FileBasedKeyStoresFactory implements KeyStoresFactory {
|
|||
// configuration property for key password.
|
||||
keystoreKeyPassword = getPassword(
|
||||
conf, keyPasswordProperty, keystorePassword);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(mode.toString() + " KeyStore: " + keystoreLocation);
|
||||
}
|
||||
|
||||
InputStream is = new FileInputStream(keystoreLocation);
|
||||
try {
|
||||
|
@ -172,7 +174,9 @@ public class FileBasedKeyStoresFactory implements KeyStoresFactory {
|
|||
} finally {
|
||||
is.close();
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(mode.toString() + " Loaded KeyStore: " + keystoreLocation);
|
||||
}
|
||||
} else {
|
||||
keystore.load(null, null);
|
||||
}
|
||||
|
@ -204,18 +208,24 @@ public class FileBasedKeyStoresFactory implements KeyStoresFactory {
|
|||
resolvePropertyName(mode, SSL_TRUSTSTORE_RELOAD_INTERVAL_TPL_KEY),
|
||||
DEFAULT_SSL_TRUSTSTORE_RELOAD_INTERVAL);
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(mode.toString() + " TrustStore: " + truststoreLocation);
|
||||
}
|
||||
|
||||
trustManager = new ReloadingX509TrustManager(truststoreType,
|
||||
truststoreLocation,
|
||||
truststorePassword,
|
||||
truststoreReloadInterval);
|
||||
trustManager.init();
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(mode.toString() + " Loaded TrustStore: " + truststoreLocation);
|
||||
}
|
||||
trustManagers = new TrustManager[]{trustManager};
|
||||
} else {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("The property '" + locationProperty + "' has not been set, " +
|
||||
"no TrustStore will be loaded");
|
||||
}
|
||||
trustManagers = null;
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue