diff --git a/hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java b/hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java index d4a71a77a79d..85abc9ab1214 100644 --- a/hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java +++ b/hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java @@ -36,6 +36,7 @@ import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.util.ArrayList; +import java.util.Collection; import java.util.EnumSet; import java.util.HashSet; import java.util.IdentityHashMap; @@ -43,6 +44,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.crypto.CryptoProtocolVersion; import org.apache.hadoop.crypto.Encryptor; @@ -473,8 +475,10 @@ private static FanOutOneBlockAsyncDFSOutput createOutput(DistributedFileSystem d Set toExcludeNodes = new HashSet<>(excludeDatanodeManager.getExcludeDNs().keySet()); for (int retry = 0;; retry++) { - LOG.debug("When create output stream for {}, exclude list is {}, retry={}", src, - toExcludeNodes, retry); + if (LOG.isDebugEnabled()) { + LOG.debug("When create output stream for {}, exclude list is {}, retry={}", src, + getDataNodeInfo(toExcludeNodes), retry); + } HdfsFileStatus stat; try { stat = FILE_CREATOR.create(namenode, src, @@ -620,4 +624,15 @@ static void sleepIgnoreInterrupt(int retry) { } catch (InterruptedException e) { } } + + public static String getDataNodeInfo(Collection datanodeInfos) { + if (datanodeInfos.isEmpty()) { + return "[]"; + } + return datanodeInfos.stream() + .map(datanodeInfo -> new StringBuilder().append("(").append(datanodeInfo.getHostName()) + .append("/").append(datanodeInfo.getInfoAddr()).append(":") + .append(datanodeInfo.getInfoPort()).append(")").toString()) + .collect(Collectors.joining(",", "[", "]")); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java index 7a057ca7c7b6..1dd7e92a0db3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java @@ -70,6 +70,7 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.function.Supplier; +import java.util.stream.Collectors; import org.apache.commons.lang3.mutable.MutableLong; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -84,6 +85,7 @@ import org.apache.hadoop.hbase.client.ConnectionUtils; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.exceptions.TimeoutIOException; +import org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper; import org.apache.hadoop.hbase.io.util.MemorySizeUtil; import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.ipc.ServerCall; @@ -1103,7 +1105,8 @@ private Map> rollWriterInternal(boolean force) throws IOExc tellListenersAboutPostLogRoll(oldPath, newPath); if (LOG.isDebugEnabled()) { LOG.debug("Create new " + implClassName + " writer with pipeline: " - + Arrays.toString(getPipeline())); + + FanOutOneBlockAsyncDFSOutputHelper + .getDataNodeInfo(Arrays.stream(getPipeline()).collect(Collectors.toList()))); } // We got a new writer, so reset the slow sync count lastTimeCheckSlowSync = EnvironmentEdgeManager.currentTime();