This is an automated email from the ASF dual-hosted git repository. zhangduo pushed a commit to branch branch-3 in repository https://gitbox.apache.org/repos/asf/hbase.git
commit 8ffd93d9096485826274e4b5f7524c09ecf46098 Author: WangXin <1458451...@qq.com> AuthorDate: Fri Sep 6 16:22:27 2024 +0800 HBASE-28775 Change the output of DatanodeInfo in the log to the hostname of the datanode (#6148) Co-authored-by: wangxin <wangxin9...@gmail.com> Signed-off-by: Duo Zhang <zhang...@apache.org> Signed-off-by: Nihal Jain <nihalj...@apache.org> Reviewed-by: Vineet Kumar Maheshwari <vineet.4...@gmail.com> Reviewed-by: guluo <lupeng_n...@qq.com> (cherry picked from commit 241bbaf0718d6215775cd4996e9ca08ba12eb086) --- .../asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java | 19 +++++++++++++++++-- .../hadoop/hbase/regionserver/wal/AbstractFSWAL.java | 5 ++++- 2 files changed, 21 insertions(+), 3 deletions(-) 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 7a4f624e6e0..879dfda77ec 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 @@ -35,6 +35,7 @@ import java.io.InterruptedIOException; 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; @@ -42,6 +43,7 @@ import java.util.List; 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 @@ public final class FanOutOneBlockAsyncDFSOutputHelper { Set<DatanodeInfo> 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 @@ public final class FanOutOneBlockAsyncDFSOutputHelper { } catch (InterruptedException e) { } } + + public static String getDataNodeInfo(Collection<DatanodeInfo> 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 bba9bd534e9..77c296b096c 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 @@ -71,6 +71,7 @@ import java.util.concurrent.locks.Condition; 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; @@ -85,6 +86,7 @@ import org.apache.hadoop.hbase.PrivateCellUtil; 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; @@ -1105,7 +1107,8 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL { 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();