[ 
https://issues.apache.org/jira/browse/HDFS-17332?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17808033#comment-17808033
 ] 

ASF GitHub Bot commented on HDFS-17332:
---------------------------------------

mccormickt12 commented on PR #6446:
URL: https://github.com/apache/hadoop/pull/6446#issuecomment-1897870432

   > > Ok so it seems we didn't remove any exception logging right? We are just 
additionally keeping track of them and logging them again at the end?
   > 
   > correct. store all exceptions in a map and then print them if we fail the 
request at the end.
   
   Its both right. You keep each stack trace as a warn 
   
   ```        
   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);
   DFSClient.LOG.warn(msg);
   
   ```
   
   AND you add it to the map and print them later right?
   
   ```
   exceptionMap.get(datanode).add(e);
   ```
   
   
   So this part of the PR description "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." 
   doesn't seem like is actually happening since we are still printing the warn 
each time and these will still get printed regardless of if the call fails. 
Perhaps you meant to change this to info?




> 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: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org

Reply via email to