[
https://issues.apache.org/jira/browse/HDFS-17332?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17806252#comment-17806252
]
ASF GitHub Bot commented on HDFS-17332:
---------------------------------------
xinglin commented on code in PR #6446:
URL: https://github.com/apache/hadoop/pull/6446#discussion_r1451226414
##########
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java:
##########
@@ -796,11 +797,18 @@ private synchronized int readBuffer(ReaderStrategy
reader, int len,
// we want to remember which block replicas we have tried
corruptedBlocks.addCorruptedBlock(getCurrentBlock(), currentNode);
} catch (IOException e) {
- if (!retryCurrentNode) {
- DFSClient.LOG.warn("Exception while reading from "
- + getCurrentBlock() + " of " + src + " from "
- + currentNode, e);
+ String msg = String.format("Failed to read block %s for file %s from
datanode %s. "
+ + "Exception is %s. Retry with the current or next available
datanode.",
+ getCurrentBlock().getBlockName(), src, currentNode.getXferAddr(),
e);
Review Comment:
> Consider using currentNode.getResolvedAddress?
We chose to use getXferAddr(), to be consistent with what is stored in the
DNAddrPair object returned by getBestNodeDNAddrPair().
> When retry with current v.s. next DN? Is any way to differenciate add to
logs?
readBuffer() simply retries to read from the current (first) DN one more
time, before moving on the next DN. So, the order readBuffer() will follow is
DN1 -> DN1 -> DN2 -> DN3 -> refresh() -> DN1 -> DN2 -> DN3 -> refresh() and
repeat from DN1 to DN3, until we hit refresh limit.
We print datanode tranferAddr in the log. So, we can tell whether it is
retry on the current node or a different node. I guess there is not much value
to print different log lines when we retry on the current node and when we
retry a different node.
> DFSInputStream: avoid logging stacktrace until when we really need to fail a
> read request with a MissingBlockException
> ----------------------------------------------------------------------------------------------------------------------
>
> Key: HDFS-17332
> URL: https://issues.apache.org/jira/browse/HDFS-17332
> Project: Hadoop HDFS
> Issue Type: Improvement
> Components: hdfs
> Reporter: Xing Lin
> Assignee: Xing Lin
> Priority: Minor
> Labels: pull-request-available
>
> In DFSInputStream#actualGetFromOneDataNode(), it would send the exception
> stacktrace to the dfsClient.LOG whenever we fail on a DN. However, in most
> cases, the read request will be served successfully by reading from the next
> available DN. The existence of exception stacktrace in the log has caused
> multiple hadoop users at Linkedin to consider this WARN message as the
> RC/fatal error for their jobs. We would like to improve the log message and
> avoid sending the stacktrace to dfsClient.LOG when a read succeeds. The
> stackTrace when reading reach DN is sent to the log only when we really need
> to fail a read request (when chooseDataNode()/refetchLocations() throws a
> BlockMissingException).
>
> Example stack trace
> {code:java}
> [12]<stderr>:23/11/30 23:01:33 WARN hdfs.DFSClient: Connection failure:
> Failed to connect to 10.150.91.13/10.150.91.13:71 for file
> /XXXX/part-yyyy-95b9909c-zzz-c000.avro for block
> BP-364971551-DatanodeIP-1448516588954:blk_zzzz_129864739321:java.net.SocketTimeoutException:
> 60000 millis timeout while waiting for channel to be ready for read. ch :
> java.nio.channels.SocketChannel[connected local=/ip:40492
> remote=datanodeIP:71] [12]<stderr>:java.net.SocketTimeoutException: 60000
> millis timeout while waiting for channel to be ready for read. ch :
> java.nio.channels.SocketChannel[connected local=/localIp:40492
> remote=datanodeIP:71] [12]<stderr>: at
> org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:164)
> [12]<stderr>: at
> org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:161)
> [12]<stderr>: at
> org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:131)
> [12]<stderr>: at
> org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:118)
> [12]<stderr>: at java.io.FilterInputStream.read(FilterInputStream.java:83)
> [12]<stderr>: at
> org.apache.hadoop.hdfs.protocolPB.PBHelperClient.vintPrefixed(PBHelperClient.java:458)
> [12]<stderr>: at
> org.apache.hadoop.hdfs.client.impl.BlockReaderRemote2.newBlockReader(BlockReaderRemote2.java:412)
> [12]<stderr>: at
> org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.getRemoteBlockReader(BlockReaderFactory.java:864)
> [12]<stderr>: at
> org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.getRemoteBlockReaderFromTcp(BlockReaderFactory.java:753)
> [12]<stderr>: at
> org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.build(BlockReaderFactory.java:387)
> [12]<stderr>: at
> org.apache.hadoop.hdfs.DFSInputStream.getBlockReader(DFSInputStream.java:736)
> [12]<stderr>: at
> org.apache.hadoop.hdfs.DFSInputStream.actualGetFromOneDataNode(DFSInputStream.java:1268)
> [12]<stderr>: at
> org.apache.hadoop.hdfs.DFSInputStream.fetchBlockByteRange(DFSInputStream.java:1216)
> [12]<stderr>: at
> org.apache.hadoop.hdfs.DFSInputStream.pread(DFSInputStream.java:1608)
> [12]<stderr>: at
> org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:1568)
> [12]<stderr>: at
> org.apache.hadoop.fs.FSDataInputStream.read(FSDataInputStream.java:93)
> [12]<stderr>: at
> hdfs_metrics_shade.org.apache.hadoop.fs.InstrumentedFSDataInputStream$InstrumentedFilterInputStream.lambda$read$0(InstrumentedFSDataInputStream.java:108)
> [12]<stderr>: at
> com.linkedin.hadoop.metrics.fs.PerformanceTrackingFSDataInputStream.process(PerformanceTrackingFSDataInputStream.java:39)
> [12]<stderr>: at
> hdfs_metrics_shade.org.apache.hadoop.fs.InstrumentedFSDataInputStream$InstrumentedFilterInputStream.read(InstrumentedFSDataInputStream.java:108)
> [12]<stderr>: at
> org.apache.hadoop.fs.FSDataInputStream.read(FSDataInputStream.java:93)
> [12]<stderr>: at
> org.apache.hadoop.fs.RetryingInputStream.lambda$read$2(RetryingInputStream.java:153)
> [12]<stderr>: at
> org.apache.hadoop.fs.NoOpRetryPolicy.run(NoOpRetryPolicy.java:36)
> [12]<stderr>: at
> org.apache.hadoop.fs.RetryingInputStream.read(RetryingInputStream.java:149)
> [12]<stderr>: at
> org.apache.hadoop.fs.FSDataInputStream.read(FSDataInputStream.java:93){code}
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]