Skip to content

Commit f4a1904

Browse files
HBASE-28775 Change the output of DatanodeInfo in the log to the hostname of the datanode (#6148) (#6212)
Co-authored-by: wangxin <[email protected]> Signed-off-by: Duo Zhang <[email protected]> Signed-off-by: Nihal Jain <[email protected]> Reviewed-by: Vineet Kumar Maheshwari <[email protected]> Reviewed-by: guluo <[email protected]> (cherry picked from commit 241bbaf) Co-authored-by: WangXin <[email protected]>
1 parent 8ed0e96 commit f4a1904

2 files changed

Lines changed: 21 additions & 3 deletions

File tree

hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java

Lines changed: 17 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -35,13 +35,15 @@
3535
import java.lang.reflect.InvocationTargetException;
3636
import java.lang.reflect.Method;
3737
import java.util.ArrayList;
38+
import java.util.Collection;
3839
import java.util.EnumSet;
3940
import java.util.HashSet;
4041
import java.util.IdentityHashMap;
4142
import java.util.List;
4243
import java.util.Map;
4344
import java.util.Set;
4445
import java.util.concurrent.TimeUnit;
46+
import java.util.stream.Collectors;
4547
import org.apache.hadoop.conf.Configuration;
4648
import org.apache.hadoop.crypto.CryptoProtocolVersion;
4749
import org.apache.hadoop.crypto.Encryptor;
@@ -533,8 +535,10 @@ private static FanOutOneBlockAsyncDFSOutput createOutput(DistributedFileSystem d
533535
Set<DatanodeInfo> toExcludeNodes =
534536
new HashSet<>(excludeDatanodeManager.getExcludeDNs().keySet());
535537
for (int retry = 0;; retry++) {
536-
LOG.debug("When create output stream for {}, exclude list is {}, retry={}", src,
537-
toExcludeNodes, retry);
538+
if (LOG.isDebugEnabled()) {
539+
LOG.debug("When create output stream for {}, exclude list is {}, retry={}", src,
540+
getDataNodeInfo(toExcludeNodes), retry);
541+
}
538542
HdfsFileStatus stat;
539543
try {
540544
stat = FILE_CREATOR.create(namenode, src,
@@ -680,4 +684,15 @@ static void sleepIgnoreInterrupt(int retry) {
680684
} catch (InterruptedException e) {
681685
}
682686
}
687+
688+
public static String getDataNodeInfo(Collection<DatanodeInfo> datanodeInfos) {
689+
if (datanodeInfos.isEmpty()) {
690+
return "[]";
691+
}
692+
return datanodeInfos.stream()
693+
.map(datanodeInfo -> new StringBuilder().append("(").append(datanodeInfo.getHostName())
694+
.append("/").append(datanodeInfo.getInfoAddr()).append(":")
695+
.append(datanodeInfo.getInfoPort()).append(")").toString())
696+
.collect(Collectors.joining(",", "[", "]"));
697+
}
683698
}

hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -56,6 +56,7 @@
5656
import java.util.concurrent.atomic.AtomicInteger;
5757
import java.util.concurrent.atomic.AtomicLong;
5858
import java.util.concurrent.locks.ReentrantLock;
59+
import java.util.stream.Collectors;
5960
import org.apache.commons.lang3.mutable.MutableLong;
6061
import org.apache.hadoop.conf.Configuration;
6162
import org.apache.hadoop.fs.FileStatus;
@@ -69,6 +70,7 @@
6970
import org.apache.hadoop.hbase.PrivateCellUtil;
7071
import org.apache.hadoop.hbase.client.RegionInfo;
7172
import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
73+
import org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper;
7274
import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
7375
import org.apache.hadoop.hbase.ipc.RpcServer;
7476
import org.apache.hadoop.hbase.ipc.ServerCall;
@@ -921,7 +923,8 @@ private Map<byte[], List<byte[]>> rollWriterInternal(boolean force) throws IOExc
921923
tellListenersAboutPostLogRoll(oldPath, newPath);
922924
if (LOG.isDebugEnabled()) {
923925
LOG.debug("Create new " + implClassName + " writer with pipeline: "
924-
+ Arrays.toString(getPipeline()));
926+
+ FanOutOneBlockAsyncDFSOutputHelper
927+
.getDataNodeInfo(Arrays.stream(getPipeline()).collect(Collectors.toList())));
925928
}
926929
// We got a new writer, so reset the slow sync count
927930
lastTimeCheckSlowSync = EnvironmentEdgeManager.currentTime();

0 commit comments

Comments
 (0)