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

gaozhan ding commented on HDFS-15413:
-------------------------------------

[~ferhui] Do you have any suggestions on this issue? We are facing the same 
problem.

> DFSStripedInputStream throws exception when datanodes close idle connections
> ----------------------------------------------------------------------------
>
>                 Key: HDFS-15413
>                 URL: https://issues.apache.org/jira/browse/HDFS-15413
>             Project: Hadoop HDFS
>          Issue Type: Bug
>          Components: ec, erasure-coding, hdfs-client
>    Affects Versions: 3.1.3
>         Environment: - Hadoop 3.1.3
> - erasure coding with ISA-L and RS-3-2-1024k scheme
> - running in kubernetes
> - dfs.client.socket-timeout = 10000
> - dfs.datanode.socket.write.timeout = 10000
>            Reporter: Andrey Elenskiy
>            Priority: Critical
>         Attachments: out.log
>
>
> We've run into an issue with compactions failing in HBase when erasure coding 
> is enabled on a table directory. After digging further I was able to narrow 
> it down to a seek + read logic and able to reproduce the issue with hdfs 
> client only:
> {code:java}
> import org.apache.hadoop.conf.Configuration;
> import org.apache.hadoop.fs.Path;
> import org.apache.hadoop.fs.FileSystem;
> import org.apache.hadoop.fs.FSDataInputStream;
> public class ReaderRaw {
>     public static void main(final String[] args) throws Exception {
>         Path p = new Path(args[0]);
>         int bufLen = Integer.parseInt(args[1]);
>         int sleepDuration = Integer.parseInt(args[2]);
>         int countBeforeSleep = Integer.parseInt(args[3]);
>         int countAfterSleep = Integer.parseInt(args[4]);
>         Configuration conf = new Configuration();
>         FSDataInputStream istream = FileSystem.get(conf).open(p);
>         byte[] buf = new byte[bufLen];
>         int readTotal = 0;
>         int count = 0;
>         try {
>           while (true) {
>             istream.seek(readTotal);
>             int bytesRemaining = bufLen;
>             int bufOffset = 0;
>             while (bytesRemaining > 0) {
>               int nread = istream.read(buf, 0, bufLen);
>               if (nread < 0) {
>                   throw new Exception("nread is less than zero");
>               }
>               readTotal += nread;
>               bufOffset += nread;
>               bytesRemaining -= nread;
>             }
>             count++;
>             if (count == countBeforeSleep) {
>                 System.out.println("sleeping for " + sleepDuration + " 
> milliseconds");
>                 Thread.sleep(sleepDuration);
>                 System.out.println("resuming");
>             }
>             if (count == countBeforeSleep + countAfterSleep) {
>                 System.out.println("done");
>                 break;
>             }
>           }
>         } catch (Exception e) {
>             System.out.println("exception on read " + count + " read total " 
> + readTotal);
>             throw e;
>         }
>     }
> }
> {code}
> The issue appears to be due to the fact that datanodes close the connection 
> of EC client if it doesn't fetch next packet for longer than 
> dfs.client.socket-timeout. The EC client doesn't retry and instead assumes 
> that those datanodes went away resulting in "missing blocks" exception.
> I was able to consistently reproduce with the following arguments:
> {noformat}
> bufLen = 1000000 (just below 1MB which is the size of the stripe) 
> sleepDuration = (dfs.client.socket-timeout + 1) * 1000 (in our case 11000)
> countBeforeSleep = 1
> countAfterSleep = 7
> {noformat}
> I've attached the entire log output of running the snippet above against 
> erasure coded file with RS-3-2-1024k policy. And here are the logs from 
> datanodes of disconnecting the client:
> datanode 1:
> {noformat}
> 2020-06-15 19:06:20,697 INFO datanode.DataNode: Likely the client has stopped 
> reading, disconnecting it (datanode-v11-0-hadoop.hadoop:9866:DataXceiver 
> error processing READ_BLOCK operation  src: /10.128.23.40:53748 dst: 
> /10.128.14.46:9866); java.net.SocketTimeoutException: 10000 millis timeout 
> while waiting for channel to be ready for write. ch : 
> java.nio.channels.SocketChannel[connected local=/10.128.14.46:9866 
> remote=/10.128.23.40:53748]
> {noformat}
> datanode 2:
> {noformat}
> 2020-06-15 19:06:20,341 INFO datanode.DataNode: Likely the client has stopped 
> reading, disconnecting it (datanode-v11-1-hadoop.hadoop:9866:DataXceiver 
> error processing READ_BLOCK operation  src: /10.128.23.40:48772 dst: 
> /10.128.9.42:9866); java.net.SocketTimeoutException: 10000 millis timeout 
> while waiting for channel to be ready for write. ch : 
> java.nio.channels.SocketChannel[connected local=/10.128.9.42:9866 
> remote=/10.128.23.40:48772]
> {noformat}
> datanode 3:
> {noformat}
> 2020-06-15 19:06:20,467 INFO datanode.DataNode: Likely the client has stopped 
> reading, disconnecting it (datanode-v11-3-hadoop.hadoop:9866:DataXceiver 
> error processing READ_BLOCK operation  src: /10.128.23.40:57184 dst: 
> /10.128.16.13:9866); java.net.SocketTimeoutException: 10000 millis timeout 
> while waiting for channel to be ready for write. ch : 
> java.nio.channels.SocketChannel[connected local=/10.128.16.13:9866 
> remote=/10.128.23.40:57184]
> {noformat}
> I've tried running the same code again non-ec files with replication of 3 and 
> was not able to reproduce the issue with any parameters. Looking through the 
> code, it's pretty clear that non-ec DFSInputStream retries reads after 
> exception: 
> https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java#L844
> Let me know if you need any more information that can help you out with 
> addressing this issue.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
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