HBASE-25927: Fix the log messages by not stringifying the exceptions in log (#3338)

Signed-off-by: Bharath Vissapragada <bharathv@apache.org>
This commit is contained in:
Sandeep Pal 2021-05-28 21:28:19 -07:00 committed by GitHub
parent c1d299fc1d
commit 9a2027bf71
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 10 additions and 12 deletions

View File

@ -59,7 +59,7 @@ public class RecoveredReplicationSourceShipper extends ReplicationSourceShipper
source.locateRecoveredPaths(walGroupId);
break;
} catch (IOException e) {
LOG.error("Error while locating recovered queue paths, attempt #" + numRetries);
LOG.error("Error while locating recovered queue paths, attempt #" + numRetries, e);
numRetries++;
}
}

View File

@ -645,8 +645,8 @@ public class ReplicationSource implements ReplicationSourceInterface {
if (cause == null) {
LOG.info("{} Closing source {} because: {}", logPeerId(), this.queueId, reason);
} else {
LOG.error("{} Closing source {} because an error occurred: {}",
logPeerId(), this.queueId, reason, cause);
LOG.error(String.format("%s Closing source %s because an error occurred: %s",
logPeerId(), this.queueId, reason), cause);
}
this.sourceRunning = false;
if (initThread != null && Thread.currentThread() != initThread) {

View File

@ -76,7 +76,6 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
@ -925,8 +924,8 @@ public class ReplicationSourceManager implements ReplicationListener {
LOG.error(String.format("ReplicationException: cannot claim dead region (%s)'s " +
"replication queue. Znode : (%s)" +
" Possible solution: check if znode size exceeds jute.maxBuffer value. " +
" If so, increase it for both client and server side." + e), deadRS,
queueStorage.getRsNode(deadRS));
" If so, increase it for both client and server side.",
deadRS, queueStorage.getRsNode(deadRS)), e);
server.abort("Failed to claim queue from dead regionserver.", e);
return;
}
@ -1202,7 +1201,7 @@ public class ReplicationSourceManager implements ReplicationListener {
* This ReplicationSource is NOT created via {@link ReplicationSourceFactory}.
* @see #addSource(String) This is a specialization of the addSource call.
* @see #catalogReplicationSource for a note on this ReplicationSource's lifecycle (and more on
* why the special handling).
* why the special handling).
*/
private ReplicationSourceInterface createCatalogReplicationSource(RegionInfo regionInfo)
throws IOException {

View File

@ -172,7 +172,7 @@ class ReplicationSourceWALReader extends Thread {
}
} catch (IOException e) {
if (sleepMultiplier < maxRetriesMultiplier) {
LOG.debug("Failed to read stream of replication entries: " + e);
LOG.debug("Failed to read stream of replication entries: ", e);
sleepMultiplier++;
} else {
LOG.error("Failed to read stream of replication entries", e);
@ -306,7 +306,7 @@ class ReplicationSourceWALReader extends Thread {
return true;
}
} catch (IOException ioe) {
LOG.warn("Couldn't get file length information about log {}", queue.peek());
LOG.warn("Couldn't get file length information about log " + queue.peek(), ioe);
}
}
return false;
@ -420,7 +420,7 @@ class ReplicationSourceWALReader extends Thread {
}
} catch (IOException e) {
LOG.error("Failed to deserialize bulk load entry from wal edit. "
+ "Then its hfiles count will not be added into metric.");
+ "Then its hfiles count will not be added into metric.", e);
}
}
@ -457,8 +457,7 @@ class ReplicationSourceWALReader extends Thread {
} catch (IOException e) {
LOG.error("Failed to deserialize bulk load entry from wal edit. "
+ "Size of HFiles part of cell will not be considered in replication "
+ "request size calculation.",
e);
+ "request size calculation.", e);
}
}
}