Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -190,12 +190,11 @@ private static String tryGetPid(Process p) {
f.setAccessible(true);
return String.valueOf(f.getInt(p));
} else {
LOG.trace("Unable to determine pid for " + p
+ " since it is not a UNIXProcess");
LOG.trace("Unable to determine pid for {} since it is not a UNIXProcess", p);
return null;
}
} catch (Throwable t) {
LOG.trace("Unable to determine pid for " + p, t);
LOG.trace("Unable to determine pid for {}", p, t);
return null;
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -94,8 +94,7 @@ static Comparer<byte[]> getBestComparer() {
return comparer;
} catch (Throwable t) { // ensure we really catch *everything*
if (LOG.isTraceEnabled()) {
LOG.trace(t.getMessage());
LOG.trace("Lexicographical comparer selected");
LOG.trace("Lexicographical comparer selected", t);
}
return lexicographicalComparerJavaImpl();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -162,7 +162,7 @@ public ReadaheadRequest submitReadahead(
identifier, fd, off, len);
pool.execute(req);
if (LOG.isTraceEnabled()) {
LOG.trace("submit readahead: " + req);
LOG.trace("submit readahead: {}", req);
}
return req;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -238,12 +238,11 @@ public Message invoke(Object proxy, final Method method, Object[] args)
}

if (LOG.isTraceEnabled()) {
LOG.trace(Thread.currentThread().getId() + ": Call -> " +
remoteId + ": " + method.getName() +
" {" + TextFormat.shortDebugString((Message) args[1]) + "}");
LOG.trace("{}: Call -> {}: {} {{}}",
Thread.currentThread().getId(), remoteId,
method.getName(), TextFormat.shortDebugString((Message) args[1]));
}


final Message theRequest = (Message) args[1];
final RpcWritable.Buffer val;
try {
Expand All @@ -253,9 +252,7 @@ public Message invoke(Object proxy, final Method method, Object[] args)

} catch (Throwable e) {
if (LOG.isTraceEnabled()) {
LOG.trace(Thread.currentThread().getId() + ": Exception <- " +
remoteId + ": " + method.getName() +
" {" + e + "}");
LOG.trace("{}: Exception <- {}: {} {{}}", Thread.currentThread().getId(), remoteId, method.getName(), e);
}
if (traceScope != null) {
traceScope.addTimelineAnnotation("Call got exception: " +
Expand Down Expand Up @@ -311,9 +308,11 @@ private Message getReturnMessage(final Method method,
returnMessage = buf.getValue(prototype.getDefaultInstanceForType());

if (LOG.isTraceEnabled()) {
LOG.trace(Thread.currentThread().getId() + ": Response <- " +
remoteId + ": " + method.getName() +
" {" + TextFormat.shortDebugString(returnMessage) + "}");
LOG.trace("{}: Response <- {}: {} {{}}",
Thread.currentThread().getId(),
remoteId,
method.getName(),
TextFormat.shortDebugString(returnMessage));
}

} catch (Throwable e) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -264,9 +264,8 @@ private CacheEntry waitForCompletion(CacheEntry newEntry) {
// If an entry in the cache does not exist, add a new one
if (mapEntry == null) {
if (LOG.isTraceEnabled()) {
LOG.trace("Adding Rpc request clientId "
+ newEntry.clientIdMsb + newEntry.clientIdLsb + " callId "
+ newEntry.callId + " to retryCache");
LOG.trace("Adding Rpc request clientId {}{} callId {} to retryCache",
newEntry.clientIdMsb, newEntry.clientIdLsb, newEntry.callId);
}
set.put(newEntry);
retryCacheMetrics.incrCacheUpdated();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -101,13 +101,12 @@ static public ObjectName register(String serviceName, String nameName,
return name;
} catch (InstanceAlreadyExistsException iaee) {
if (LOG.isTraceEnabled()) {
LOG.trace("Failed to register MBean \"" + name + "\"", iaee);
LOG.trace("Failed to register MBean \"{}\"", name, iaee);
} else {
LOG.warn("Failed to register MBean \"" + name
+ "\": Instance already exists.");
LOG.warn("Failed to register MBean \"{}\": Instance already exists.", name);
}
} catch (Exception e) {
LOG.warn("Failed to register MBean \"" + name + "\"", e);
LOG.warn("Failed to register MBean \"{}\"", name, e);
}
}
return null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -105,24 +105,21 @@ public boolean handle(DomainSocket sock) {
try {
kicked = false;
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": NotificationHandler: doing a read on " +
sock.fd);
LOG.trace("{}: NotificationHandler: doing a read on {}", this, sock.fd);
}
if (sock.getInputStream().read() == -1) {
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": NotificationHandler: got EOF on " + sock.fd);
LOG.trace("{}: NotificationHandler: got EOF on {}", this, sock.fd);
}
throw new EOFException();
}
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": NotificationHandler: read succeeded on " +
sock.fd);
LOG.trace("{}: NotificationHandler: read succeeded on {}", this, sock.fd);
}
return false;
} catch (IOException e) {
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": NotificationHandler: setting closed to " +
"true for " + sock.fd);
LOG.trace("{}: NotificationHandler: setting closed to true for {}", this, sock.fd);
}
closed = true;
return true;
Expand Down Expand Up @@ -386,7 +383,7 @@ private void kick() {
private boolean sendCallback(String caller, TreeMap<Integer, Entry> entries,
FdSet fdSet, int fd) {
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": " + caller + " starting sendCallback for fd " + fd);
LOG.trace("{}: {} starting sendCallback for fd {}", this, caller, fd);
}
Entry entry = entries.get(fd);
Preconditions.checkNotNull(entry,
Expand All @@ -395,13 +392,11 @@ private boolean sendCallback(String caller, TreeMap<Integer, Entry> entries,
DomainSocket sock = entry.getDomainSocket();
if (entry.getHandler().handle(sock)) {
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": " + caller + ": closing fd " + fd +
" at the request of the handler.");
LOG.trace("{}: {}: closing fd {} at the request of the handler.", this, caller, fd);
}
if (toRemove.remove(fd) != null) {
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": " + caller + " : sendCallback processed fd " +
fd + " in toRemove.");
LOG.trace("{}: {}: sendCallback processed fd {} in toRemove.", this, caller, fd);
}
}
try {
Expand All @@ -416,8 +411,7 @@ private boolean sendCallback(String caller, TreeMap<Integer, Entry> entries,
return true;
} else {
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": " + caller + ": sendCallback not " +
"closing fd " + fd);
LOG.trace("{}: {}: sendCallback not closing fd {}", this, caller, fd);
}
return false;
}
Expand Down Expand Up @@ -469,7 +463,7 @@ public void run() {
this + ": tried to watch a file descriptor that we " +
"were already watching: " + sock);
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": adding fd " + sock.fd);
LOG.trace("{}: adding fd {}", this, sock.fd);
}
fdSet.add(sock.fd);
}
Expand Down Expand Up @@ -554,9 +548,8 @@ private void addNotificationSocket(final TreeMap<Integer, Entry> entries,
}
fdSet.add(notificationSockets[1].fd);
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": adding notificationSocket " +
notificationSockets[1].fd + ", connected to " +
notificationSockets[0].fd);
LOG.trace("{}: adding notificationSocket {}, connected to {}",
this, notificationSockets[1].fd, notificationSockets[0].fd);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -575,11 +575,9 @@ InetAddress getByName(String hostname) throws UnknownHostException {
long elapsedMs = lookupTimer.stop().now(TimeUnit.MILLISECONDS);

if (elapsedMs >= slowLookupThresholdMs) {
LOG.warn("Slow name lookup for " + hostname + ". Took " + elapsedMs +
" ms.");
LOG.warn("Slow name lookup for {}. Took {} ms.", hostname, elapsedMs);
} else if (LOG.isTraceEnabled()) {
LOG.trace("Name lookup for " + hostname + " took " + elapsedMs +
" ms.");
LOG.trace("Name lookup for {} took {} ms.", hostname, elapsedMs);
}
return result;
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -364,10 +364,10 @@ public void check(final String[] hosts, final String[] cns,
final boolean strictWithSubDomains)
throws SSLException {
if (LOG.isTraceEnabled()) {
LOG.trace("Hosts:{}, CNs:{} subjectAlts:{}, ie6:{}, " +
"strictWithSubDomains{}", Arrays.toString(hosts),
Arrays.toString(cns), Arrays.toString(subjectAlts), ie6,
strictWithSubDomains);
LOG.trace("Hosts:{}, CNs:{} subjectAlts:{}, ie6:{}, strictWithSubDomains{}",
Arrays.toString(hosts),
Arrays.toString(cns), Arrays.toString(subjectAlts), ie6,
strictWithSubDomains);
}
// Build up lists of allowed hosts For logging/debugging purposes.
StringBuffer buf = new StringBuffer(32);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -152,8 +152,8 @@ public static boolean isACLsFileNewer(long time) {
"' must be an absolute path: " + confDir);
}
File f = new File(confDir, KMS_ACLS_XML);
LOG.trace("Checking file {}, modification time is {}, last reload time is"
+ " {}", f.getPath(), f.lastModified(), time);
LOG.trace("Checking file {}, modification time is {}, last reload time is {}",
f.getPath(), f.lastModified(), time);
// at least 100ms newer than time, we do this to ensure the file
// has been properly closed/flushed
newer = f.lastModified() - time > 100;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -222,8 +222,7 @@ private DFSOutputStream(DFSClient dfsClient, String src,
this.addBlockFlags.add(AddBlockFlag.IGNORE_CLIENT_LOCALITY);
}
if (progress != null) {
DFSClient.LOG.debug("Set non-null progress callback on DFSOutputStream "
+"{}", src);
DFSClient.LOG.debug("Set non-null progress callback on DFSOutputStream {}", src);
}

initWritePacketSize();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,7 @@ private static void checkProcessRet(Process p, boolean expectPass)

/** Exec the given command and assert it executed successfully */
private static void execWaitRet(String cmd) throws IOException {
LOG.debug("EXEC "+cmd);
LOG.debug("EXEC {}", cmd);
Process p = r.exec(cmd);
try {
p.waitFor();
Expand All @@ -98,19 +98,19 @@ private static void execWaitRet(String cmd) throws IOException {

/** Exec the given command and assert it executed successfully */
private static void execIgnoreRet(String cmd) throws IOException {
LOG.debug("EXEC "+cmd);
LOG.debug("EXEC {}", cmd);
r.exec(cmd);
}

/** Exec the given command and assert it executed successfully */
private static void execAssertSucceeds(String cmd) throws IOException {
LOG.debug("EXEC "+cmd);
LOG.debug("EXEC {}", cmd);
checkProcessRet(r.exec(cmd), true);
}

/** Exec the given command, assert it returned an error code */
private static void execAssertFails(String cmd) throws IOException {
LOG.debug("EXEC "+cmd);
LOG.debug("EXEC {}", cmd);
checkProcessRet(r.exec(cmd), false);
}

Expand Down Expand Up @@ -170,7 +170,7 @@ private static Process establishMount(URI uri) throws IOException {
String arch = System.getProperty("os.arch");
String jvm = System.getProperty("java.home") + "/lib/" + arch + "/server";
String lp = System.getProperty("LD_LIBRARY_PATH")+":"+libHdfs+":"+jvm;
LOG.debug("LD_LIBRARY_PATH=" + lp);
LOG.debug("LD_LIBRARY_PATH={}", lp);

String nameNode =
"dfs://" + uri.getHost() + ":" + String.valueOf(uri.getPort());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,7 @@ Entry<FileHandle, OpenFileCtx> getEntryToEvict() {
Iterator<Entry<FileHandle, OpenFileCtx>> it = openFileMap.entrySet()
.iterator();
if (LOG.isTraceEnabled()) {
LOG.trace("openFileMap size:" + size());
LOG.trace("openFileMap size:{}", size());
}

Entry<FileHandle, OpenFileCtx> idlest = null;
Expand All @@ -80,7 +80,7 @@ Entry<FileHandle, OpenFileCtx> getEntryToEvict() {
OpenFileCtx ctx = pairs.getValue();
if (!ctx.getActiveState()) {
if (LOG.isDebugEnabled()) {
LOG.debug("Got one inactive stream: " + ctx);
LOG.debug("Got one inactive stream: {}", ctx);
}
return pairs;
}
Expand Down Expand Up @@ -150,7 +150,7 @@ void scan(long streamTimeout) {
Iterator<Entry<FileHandle, OpenFileCtx>> it = openFileMap.entrySet()
.iterator();
if (LOG.isTraceEnabled()) {
LOG.trace("openFileMap size:" + size());
LOG.trace("openFileMap size:{}", size());
}

while (it.hasNext()) {
Expand Down Expand Up @@ -202,7 +202,7 @@ void cleanAll() {
Iterator<Entry<FileHandle, OpenFileCtx>> it = openFileMap.entrySet()
.iterator();
if (LOG.isTraceEnabled()) {
LOG.trace("openFileMap size:" + size());
LOG.trace("openFileMap size:{}", size());
}

while (it.hasNext()) {
Expand Down Expand Up @@ -255,8 +255,7 @@ public void run() {
long workedTime = Time.monotonicNow() - lastWakeupTime;
if (workedTime < rotation) {
if (LOG.isTraceEnabled()) {
LOG.trace("StreamMonitor can still have a sleep:"
+ ((rotation - workedTime) / 1000));
LOG.trace("StreamMonitor can still have a sleep:{}", (rotation - workedTime) / 1000);
}
Thread.sleep(rotation - workedTime);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -151,8 +151,7 @@ long dumpData(FileOutputStream dumpOut, RandomAccessFile raf)
throws IOException {
if (dataState != DataState.ALLOW_DUMP) {
if (LOG.isTraceEnabled()) {
LOG.trace("No need to dump with status(replied,dataState):" + "("
+ replied + "," + dataState + ")");
LOG.trace("No need to dump with status(replied,dataState):({},{})", replied, dataState);
}
return 0;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1276,8 +1276,7 @@ static String[] getSuffixIDs(final Configuration conf, final String addressKey,
Collection<String> nnIds = DFSUtilClient.getNameNodeIds(conf, nsId);
for (String nnId : DFSUtilClient.emptyAsSingletonNull(nnIds)) {
if (LOG.isTraceEnabled()) {
LOG.trace(String.format("addressKey: %s nsId: %s nnId: %s",
addressKey, nsId, nnId));
LOG.trace("addressKey: {} nsId: {} nnId: {}", addressKey, nsId, nnId);
}
if (knownNNId != null && !knownNNId.equals(nnId)) {
continue;
Expand Down
Loading