[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17192673#comment-17192673 ] Solvannan R M commented on HDFS-15413: -- We are also facing the same issue in our hbase clusters where compaction fails with connection reset exception. The scenario is, during compaction the erasure coded file is opened and kept idle for a long time exceeding the timeout on datanode (dfs.datanode.socket.write.timeout) leads to connection reset by peer exception in hbase regionservers. We are using hadoop 3.1.3 with RS-10-4-1024k ec enabled. Currently, we have increased the dfs.datanode.socket.write.timeout value in all datanodes, to mitigate the issue. Can anyone please check this issue and assist us ? > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >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 = 100 (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: 1 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: > {nof
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >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 = 100 (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: 1 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: 1 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
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17232754#comment-17232754 ] Hui Fei commented on HDFS-15413: [~lalapala] Thanks for involving me. I will take a look later. > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >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 = 100 (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: 1 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: 1 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:0
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17436908#comment-17436908 ] Jeff Kubina commented on HDFS-15413: Is this issue being worked to was it resolved already? > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >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 = 100 (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: 1 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: 1 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:0
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17437284#comment-17437284 ] Hemanth Boyina commented on HDFS-15413: --- Hi [~jmkubin] AFAIK the issue is not resolved yet, you can work on it if you want to > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >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 = 100 (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: 1 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: 1 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} > datan
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17741981#comment-17741981 ] ASF GitHub Bot commented on HDFS-15413: --- Neilxzn opened a new pull request, #5829: URL: https://github.com/apache/hadoop/pull/5829 ### Description of PR https://issues.apache.org/jira/browse/HDFS-15413 Offer a available patch to fix HDFS-15413. This patch add dfs.client.read.striped.datanode.max.attempts config to allow users to adjust the number of dn retries to solve the problem of Datanode timeout when reading EC files. ### How was this patch tested? no add test. just test in our cluster ### For code changes: add dfs.client.read.striped.datanode.max.attempts config > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >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 = 100 (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: 1 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} >
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17741985#comment-17741985 ] Max Xie commented on HDFS-15413: - I add a available patch to solve it https://github.com/apache/hadoop/pull/5829. After set `dfs.client.read.striped.datanode.max.attempts = 2`, the above test passes. > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: 1 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: 1 millis timeout > while waiting for channel to be ready for write.
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17742166#comment-17742166 ] ASF GitHub Bot commented on HDFS-15413: --- Neilxzn commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-1631452001 这是自动回复邮件。来件已接收,谢谢。 > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: 1 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: 1 millis timeout > while waiting for channel to be ready for write. ch : > java.nio.channels.SocketChan
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17742165#comment-17742165 ] ASF GitHub Bot commented on HDFS-15413: --- hadoop-yetus commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-1631451493 :broken_heart: **-1 overall** | Vote | Subsystem | Runtime | Logfile | Comment | |::|--:|:|::|:---:| | +0 :ok: | reexec | 0m 49s | | Docker mode activated. | _ Prechecks _ | | +1 :green_heart: | dupname | 0m 0s | | No case conflicting files found. | | +0 :ok: | codespell | 0m 1s | | codespell was not available. | | +0 :ok: | detsecrets | 0m 1s | | detect-secrets was not available. | | +0 :ok: | xmllint | 0m 1s | | xmllint was not available. | | +1 :green_heart: | @author | 0m 0s | | The patch does not contain any @author tags. | | -1 :x: | test4tests | 0m 0s | | The patch doesn't appear to include any new or modified tests. Please justify why no new tests are needed for this patch. Also please list what manual steps were performed to verify this patch. | _ trunk Compile Tests _ | | +0 :ok: | mvndep | 15m 55s | | Maven dependency ordering for branch | | +1 :green_heart: | mvninstall | 37m 31s | | trunk passed | | +1 :green_heart: | compile | 6m 12s | | trunk passed with JDK Ubuntu-11.0.19+7-post-Ubuntu-0ubuntu120.04.1 | | +1 :green_heart: | compile | 5m 55s | | trunk passed with JDK Private Build-1.8.0_362-8u372-ga~us1-0ubuntu1~20.04-b09 | | +1 :green_heart: | checkstyle | 1m 30s | | trunk passed | | +1 :green_heart: | mvnsite | 2m 24s | | trunk passed | | +1 :green_heart: | javadoc | 1m 53s | | trunk passed with JDK Ubuntu-11.0.19+7-post-Ubuntu-0ubuntu120.04.1 | | +1 :green_heart: | javadoc | 2m 17s | | trunk passed with JDK Private Build-1.8.0_362-8u372-ga~us1-0ubuntu1~20.04-b09 | | +1 :green_heart: | spotbugs | 6m 2s | | trunk passed | | +1 :green_heart: | shadedclient | 44m 7s | | branch has no errors when building and testing our client artifacts. | _ Patch Compile Tests _ | | +0 :ok: | mvndep | 0m 31s | | Maven dependency ordering for patch | | +1 :green_heart: | mvninstall | 2m 7s | | the patch passed | | +1 :green_heart: | compile | 6m 25s | | the patch passed with JDK Ubuntu-11.0.19+7-post-Ubuntu-0ubuntu120.04.1 | | +1 :green_heart: | javac | 6m 25s | | the patch passed | | +1 :green_heart: | compile | 6m 6s | | the patch passed with JDK Private Build-1.8.0_362-8u372-ga~us1-0ubuntu1~20.04-b09 | | +1 :green_heart: | javac | 6m 6s | | the patch passed | | -1 :x: | blanks | 0m 0s | [/blanks-eol.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/1/artifact/out/blanks-eol.txt) | The patch has 1 line(s) that end in blanks. Use git apply --whitespace=fix <>. Refer https://git-scm.com/docs/git-apply | | -0 :warning: | checkstyle | 1m 27s | [/results-checkstyle-hadoop-hdfs-project.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/1/artifact/out/results-checkstyle-hadoop-hdfs-project.txt) | hadoop-hdfs-project: The patch generated 8 new + 45 unchanged - 0 fixed = 53 total (was 45) | | +1 :green_heart: | mvnsite | 2m 6s | | the patch passed | | +1 :green_heart: | javadoc | 1m 33s | | the patch passed with JDK Ubuntu-11.0.19+7-post-Ubuntu-0ubuntu120.04.1 | | +1 :green_heart: | javadoc | 2m 5s | | the patch passed with JDK Private Build-1.8.0_362-8u372-ga~us1-0ubuntu1~20.04-b09 | | +1 :green_heart: | spotbugs | 6m 8s | | the patch passed | | +1 :green_heart: | shadedclient | 40m 45s | | patch has no errors when building and testing our client artifacts. | _ Other Tests _ | | +1 :green_heart: | unit | 2m 22s | | hadoop-hdfs-client in the patch passed. | | -1 :x: | unit | 253m 30s | [/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/1/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt) | hadoop-hdfs in the patch passed. | | +1 :green_heart: | asflicense | 0m 52s | | The patch does not generate ASF License warnings. | | | | 451m 11s | | | | Reason | Tests | |---:|:--| | Failed junit tests | hadoop.hdfs.server.namenode.ha.TestObserverNode | | Subsystem | Report/Notes | |--:|:-| | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/1/artifact/out/Dockerfile | | GITHUB PR | https://github.com/apache/hadoop/pull/5829 | | Optional Tests | dupname asflicense compile java
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17782303#comment-17782303 ] Bryan Beaudreault commented on HDFS-15413: -- I was also having this issue when trying out Erasure Coding for HBase storefiles. Similarly, compations were failing but other requests were fine. I applied [~max2049] 's patch to my cluster, and it resolved the issue (after setting max attempts to 2). It would be great to get this reviewed and merged. > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: 1 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.1
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17782391#comment-17782391 ] ASF GitHub Bot commented on HDFS-15413: --- Hexiaoqiao commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-1791847743 cc @zhangshuyan0 Would you mind to take a review? > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: 1 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: 1 millis timeout > while waiting for channel to be ready for write. ch
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17782523#comment-17782523 ] ASF GitHub Bot commented on HDFS-15413: --- ayushtkn commented on code in PR #5829: URL: https://github.com/apache/hadoop/pull/5829#discussion_r1381508823 ## hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripeReader.java: ## @@ -233,41 +236,60 @@ private ByteBufferStrategy[] getReadStrategies(StripingChunk chunk) { private int readToBuffer(BlockReader blockReader, DatanodeInfo currentNode, ByteBufferStrategy strategy, - ExtendedBlock currentBlock) throws IOException { + LocatedBlock currentBlock, int chunkIndex) throws IOException { final int targetLength = strategy.getTargetLength(); -int length = 0; -try { - while (length < targetLength) { -int ret = strategy.readFromBlock(blockReader); -if (ret < 0) { - throw new IOException("Unexpected EOS from the reader"); +int curAttempts = 0; +while (curAttempts < readDNMaxAttempts) { + int length = 0; + try { +while (length < targetLength) { + int ret = strategy.readFromBlock(blockReader); + if (ret < 0) { +throw new IOException("Unexpected EOS from the reader"); + } + length += ret; } -length += ret; +return length; + } catch (ChecksumException ce) { +DFSClient.LOG.warn("Found Checksum error for " ++ currentBlock + " from " + currentNode ++ " at " + ce.getPos()); +//Clear buffer to make next decode success +strategy.getReadBuffer().clear(); +// we want to remember which block replicas we have tried +corruptedBlocks.addCorruptedBlock(currentBlock.getBlock(), currentNode); +throw ce; + } catch (IOException e) { +//Clear buffer to make next decode success +strategy.getReadBuffer().clear(); +if (curAttempts < readDNMaxAttempts - 1) { + curAttempts++; + if (readerInfos[chunkIndex].reader != null) { +readerInfos[chunkIndex].reader.close(); + } + if (dfsStripedInputStream.createBlockReader(currentBlock, + alignedStripe.getOffsetInBlock(), targetBlocks, + readerInfos, chunkIndex, readTo)) { +blockReader = readerInfos[chunkIndex].reader; +String msg = "Reconnect to " + currentNode.getInfoAddr() ++ " for block " + currentBlock.getBlock(); +DFSClient.LOG.warn(msg); +continue; + } +DFSClient.LOG.warn("Exception while reading from " ++ currentBlock + " of " + dfsStripedInputStream.getSrc() + " from " ++ currentNode, e); +throw e; } - return length; -} catch (ChecksumException ce) { - DFSClient.LOG.warn("Found Checksum error for " - + currentBlock + " from " + currentNode - + " at " + ce.getPos()); - //Clear buffer to make next decode success - strategy.getReadBuffer().clear(); - // we want to remember which block replicas we have tried - corruptedBlocks.addCorruptedBlock(currentBlock, currentNode); - throw ce; -} catch (IOException e) { - DFSClient.LOG.warn("Exception while reading from " - + currentBlock + " of " + dfsStripedInputStream.getSrc() + " from " - + currentNode, e); - //Clear buffer to make next decode success - strategy.getReadBuffer().clear(); - throw e; } } +return -1; Review Comment: I don't think we should return -1, there is logic which uses the return value ``` for (ByteBufferStrategy strategy : strategies) { int bytesReead = readToBuffer(reader, datanode, strategy, currentBlock); ret += bytesReead; } ``` We should throw exception or a valid value ## hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripeReader.java: ## @@ -233,41 +236,60 @@ private ByteBufferStrategy[] getReadStrategies(StripingChunk chunk) { private int readToBuffer(BlockReader blockReader, DatanodeInfo currentNode, ByteBufferStrategy strategy, - ExtendedBlock currentBlock) throws IOException { + LocatedBlock currentBlock, int chunkIndex) throws IOException { final int targetLength = strategy.getTargetLength(); -int length = 0; -try { - while (length < targetLength) { -int ret = strategy.readFromBlock(blockReader); -if (ret < 0) { - throw new IOException("Unexpected EOS from the reader"); +int curAttempts = 0; +while (curAttempts < readDNMaxAttempts) { + int length = 0; + try { +while (length < targetLength) { + int ret = strategy.readFromBlock(blockReader); +
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17782974#comment-17782974 ] ASF GitHub Bot commented on HDFS-15413: --- zhangshuyan0 commented on code in PR #5829: URL: https://github.com/apache/hadoop/pull/5829#discussion_r1382549418 ## hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripeReader.java: ## @@ -233,41 +236,60 @@ private ByteBufferStrategy[] getReadStrategies(StripingChunk chunk) { private int readToBuffer(BlockReader blockReader, DatanodeInfo currentNode, ByteBufferStrategy strategy, - ExtendedBlock currentBlock) throws IOException { + LocatedBlock currentBlock, int chunkIndex) throws IOException { final int targetLength = strategy.getTargetLength(); -int length = 0; -try { - while (length < targetLength) { -int ret = strategy.readFromBlock(blockReader); -if (ret < 0) { - throw new IOException("Unexpected EOS from the reader"); +int curAttempts = 0; +while (curAttempts < readDNMaxAttempts) { + int length = 0; + try { +while (length < targetLength) { + int ret = strategy.readFromBlock(blockReader); + if (ret < 0) { +throw new IOException("Unexpected EOS from the reader"); + } + length += ret; } -length += ret; +return length; + } catch (ChecksumException ce) { +DFSClient.LOG.warn("Found Checksum error for " ++ currentBlock + " from " + currentNode ++ " at " + ce.getPos()); +//Clear buffer to make next decode success +strategy.getReadBuffer().clear(); +// we want to remember which block replicas we have tried +corruptedBlocks.addCorruptedBlock(currentBlock.getBlock(), currentNode); +throw ce; + } catch (IOException e) { +//Clear buffer to make next decode success +strategy.getReadBuffer().clear(); +if (curAttempts < readDNMaxAttempts - 1) { + curAttempts++; + if (readerInfos[chunkIndex].reader != null) { +readerInfos[chunkIndex].reader.close(); + } + if (dfsStripedInputStream.createBlockReader(currentBlock, + alignedStripe.getOffsetInBlock(), targetBlocks, + readerInfos, chunkIndex, readTo)) { +blockReader = readerInfos[chunkIndex].reader; +String msg = "Reconnect to " + currentNode.getInfoAddr() ++ " for block " + currentBlock.getBlock(); +DFSClient.LOG.warn(msg); +continue; + } +DFSClient.LOG.warn("Exception while reading from " ++ currentBlock + " of " + dfsStripedInputStream.getSrc() + " from " ++ currentNode, e); +throw e; } - return length; -} catch (ChecksumException ce) { - DFSClient.LOG.warn("Found Checksum error for " - + currentBlock + " from " + currentNode - + " at " + ce.getPos()); - //Clear buffer to make next decode success - strategy.getReadBuffer().clear(); - // we want to remember which block replicas we have tried - corruptedBlocks.addCorruptedBlock(currentBlock, currentNode); - throw ce; -} catch (IOException e) { - DFSClient.LOG.warn("Exception while reading from " - + currentBlock + " of " + dfsStripedInputStream.getSrc() + " from " - + currentNode, e); - //Clear buffer to make next decode success - strategy.getReadBuffer().clear(); - throw e; } } +return -1; Review Comment: Agree with @ayushtkn. Line279-282 should be here. > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17783085#comment-17783085 ] ASF GitHub Bot commented on HDFS-15413: --- Hexiaoqiao commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-1794081852 Please also check the checkstyle and blannks reported by Yetus. Thanks. @Neilxzn > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: 1 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: 1 millis timeout > while waiting for
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17783223#comment-17783223 ] ASF GitHub Bot commented on HDFS-15413: --- Neilxzn commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-1794890337 > Please also check the checkstyle and blannks reported by Yetus. Thanks. @Neilxzn Fix these checkstyle and add unit test. Please review it again. Thanks > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: 1 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)
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17783347#comment-17783347 ] ASF GitHub Bot commented on HDFS-15413: --- hadoop-yetus commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-1796080821 :broken_heart: **-1 overall** | Vote | Subsystem | Runtime | Logfile | Comment | |::|--:|:|::|:---:| | +0 :ok: | reexec | 9m 0s | | Docker mode activated. | _ Prechecks _ | | +1 :green_heart: | dupname | 0m 0s | | No case conflicting files found. | | +0 :ok: | codespell | 0m 0s | | codespell was not available. | | +0 :ok: | detsecrets | 0m 0s | | detect-secrets was not available. | | +0 :ok: | xmllint | 0m 0s | | xmllint was not available. | | +1 :green_heart: | @author | 0m 0s | | The patch does not contain any @author tags. | | +1 :green_heart: | test4tests | 0m 0s | | The patch appears to include 1 new or modified test files. | _ trunk Compile Tests _ | | +0 :ok: | mvndep | 14m 3s | | Maven dependency ordering for branch | | +1 :green_heart: | mvninstall | 22m 34s | | trunk passed | | +1 :green_heart: | compile | 3m 7s | | trunk passed with JDK Ubuntu-11.0.20.1+1-post-Ubuntu-0ubuntu120.04 | | +1 :green_heart: | compile | 3m 4s | | trunk passed with JDK Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05 | | +1 :green_heart: | checkstyle | 0m 52s | | trunk passed | | +1 :green_heart: | mvnsite | 1m 31s | | trunk passed | | +1 :green_heart: | javadoc | 1m 22s | | trunk passed with JDK Ubuntu-11.0.20.1+1-post-Ubuntu-0ubuntu120.04 | | +1 :green_heart: | javadoc | 1m 40s | | trunk passed with JDK Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05 | | +1 :green_heart: | spotbugs | 3m 23s | | trunk passed | | +1 :green_heart: | shadedclient | 21m 16s | | branch has no errors when building and testing our client artifacts. | _ Patch Compile Tests _ | | +0 :ok: | mvndep | 0m 26s | | Maven dependency ordering for patch | | +1 :green_heart: | mvninstall | 1m 13s | | the patch passed | | +1 :green_heart: | compile | 2m 59s | | the patch passed with JDK Ubuntu-11.0.20.1+1-post-Ubuntu-0ubuntu120.04 | | +1 :green_heart: | javac | 2m 59s | | the patch passed | | +1 :green_heart: | compile | 2m 51s | | the patch passed with JDK Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05 | | +1 :green_heart: | javac | 2m 51s | | the patch passed | | -1 :x: | blanks | 0m 0s | [/blanks-eol.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/2/artifact/out/blanks-eol.txt) | The patch has 1 line(s) that end in blanks. Use git apply --whitespace=fix <>. Refer https://git-scm.com/docs/git-apply | | -0 :warning: | checkstyle | 0m 41s | [/results-checkstyle-hadoop-hdfs-project.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/2/artifact/out/results-checkstyle-hadoop-hdfs-project.txt) | hadoop-hdfs-project: The patch generated 1 new + 45 unchanged - 0 fixed = 46 total (was 45) | | +1 :green_heart: | mvnsite | 1m 17s | | the patch passed | | +1 :green_heart: | javadoc | 1m 5s | | the patch passed with JDK Ubuntu-11.0.20.1+1-post-Ubuntu-0ubuntu120.04 | | +1 :green_heart: | javadoc | 1m 30s | | the patch passed with JDK Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05 | | +1 :green_heart: | spotbugs | 3m 25s | | the patch passed | | +1 :green_heart: | shadedclient | 21m 20s | | patch has no errors when building and testing our client artifacts. | _ Other Tests _ | | +1 :green_heart: | unit | 1m 56s | | hadoop-hdfs-client in the patch passed. | | -1 :x: | unit | 192m 14s | [/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/2/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt) | hadoop-hdfs in the patch passed. | | +1 :green_heart: | asflicense | 0m 36s | | The patch does not generate ASF License warnings. | | | | 315m 36s | | | | Reason | Tests | |---:|:--| | Failed junit tests | hadoop.hdfs.TestDFSUtil | | | hadoop.hdfs.server.datanode.TestDirectoryScanner | | Subsystem | Report/Notes | |--:|:-| | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/2/artifact/out/Dockerfile | | GITHUB PR | https://github.com/apache/hadoop/pull/5829 | | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets xmllint | | uname | Linux a7218a7ce
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17783445#comment-17783445 ] qinyuren commented on HDFS-15413: - We also encountered datanode socket timeout problems in the process of using EC, That's because our dfsclient has a lot concurrent read requests, and the default value of *stripedReadThreadpoolSize* is 18, so we adjusted the value of config *dfs.client.read.striped.threadpool.size* to solve the problem. I hope it was helpful. > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: 1 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
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17783449#comment-17783449 ] Bryan Beaudreault commented on HDFS-15413: -- [~qinyuren] thank you for the suggestion! In our usage of EC with HBase, this happens even with very little load. Because HBase will read a chunk of data for compaction, then write it out again over time. It may get rate limited due to throughput limits, which may cause it to pause before reading more data. This exceeds the timeout, and just 1 retry gets it to re-connect. > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: 1 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.hadoo
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17784235#comment-17784235 ] ASF GitHub Bot commented on HDFS-15413: --- bbeaudreault commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-1802929219 @ayushtkn @zhangshuyan0 looks like the remaining failing checks are unrelated, and the feedback was addressed. Any chance for another look? > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: 1 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.Socket
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17784688#comment-17784688 ] ASF GitHub Bot commented on HDFS-15413: --- ayushtkn commented on code in PR #5829: URL: https://github.com/apache/hadoop/pull/5829#discussion_r1388949742 ## hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStreamWithTimeout.java: ## @@ -0,0 +1,168 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs; + +import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.protocol.LocatedBlocks; +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; +import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; +import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; +import org.apache.hadoop.io.erasurecode.CodecUtil; +import org.apache.hadoop.io.erasurecode.ErasureCodeNative; +import org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawErasureCoderFactory; +import org.apache.hadoop.test.GenericTestUtils; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; + +import java.io.IOException; +import java.util.Arrays; + +public class TestDFSStripedInputStreamWithTimeout { + + public static final Logger LOG = + LoggerFactory.getLogger(TestDFSStripedInputStreamWithTimeout.class); + + private MiniDFSCluster cluster; + private Configuration conf = new Configuration(); + private DistributedFileSystem fs; + private final Path dirPath = new Path("/striped"); + private Path filePath = new Path(dirPath, "file"); + private ErasureCodingPolicy ecPolicy; + private short dataBlocks; + private short parityBlocks; + private int cellSize; + private final int stripesPerBlock = 2; + private int blockSize; + private int blockGroupSize; + + @Rule + public Timeout globalTimeout = new Timeout(30); + + public ErasureCodingPolicy getEcPolicy() { +return StripedFileTestUtil.getDefaultECPolicy(); + } + + @Before + public void setup() throws IOException { +/* + * Initialize erasure coding policy. + */ +ecPolicy = getEcPolicy(); +dataBlocks = (short) ecPolicy.getNumDataUnits(); +parityBlocks = (short) ecPolicy.getNumParityUnits(); +cellSize = ecPolicy.getCellSize(); +blockSize = stripesPerBlock * cellSize; +blockGroupSize = dataBlocks * blockSize; +System.out.println("EC policy = " + ecPolicy); + +conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); Review Comment: This is deprecated config I believe, We should use ``HdfsClientConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY`` ## hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStreamWithTimeout.java: ## @@ -0,0 +1,168 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs; + +import org.apache.hadoop.hdfs.client.HdfsCl
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17787014#comment-17787014 ] ASF GitHub Bot commented on HDFS-15413: --- Hexiaoqiao commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-1815704414 Hi @Neilxzn Any progress here? Thanks. > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: 1 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: 1 millis timeout > while waiting for channel to be ready for write. ch : > java.
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17792547#comment-17792547 ] ASF GitHub Bot commented on HDFS-15413: --- bbeaudreault commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-1837536440 Hi @Neilxzn , any chance you have time to finish this up? > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: 1 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: 1 millis timeout > while waiting for channel to be ready for
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17792777#comment-17792777 ] ASF GitHub Bot commented on HDFS-15413: --- Neilxzn commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-1838407963 > Hi @Neilxzn , any chance you have time to finish this up? Sorry for the late reply. I have been busy with other things recently. I will try to submit a new unit test tomorrow. > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: 1 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
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17793169#comment-17793169 ] ASF GitHub Bot commented on HDFS-15413: --- Neilxzn commented on code in PR #5829: URL: https://github.com/apache/hadoop/pull/5829#discussion_r1415026140 ## hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStreamWithTimeout.java: ## @@ -0,0 +1,168 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs; + +import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.protocol.LocatedBlocks; +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; +import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; +import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; +import org.apache.hadoop.io.erasurecode.CodecUtil; +import org.apache.hadoop.io.erasurecode.ErasureCodeNative; +import org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawErasureCoderFactory; +import org.apache.hadoop.test.GenericTestUtils; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; + +import java.io.IOException; +import java.util.Arrays; + +public class TestDFSStripedInputStreamWithTimeout { + + public static final Logger LOG = + LoggerFactory.getLogger(TestDFSStripedInputStreamWithTimeout.class); + + private MiniDFSCluster cluster; + private Configuration conf = new Configuration(); + private DistributedFileSystem fs; + private final Path dirPath = new Path("/striped"); + private Path filePath = new Path(dirPath, "file"); + private ErasureCodingPolicy ecPolicy; + private short dataBlocks; + private short parityBlocks; + private int cellSize; + private final int stripesPerBlock = 2; + private int blockSize; + private int blockGroupSize; + + @Rule + public Timeout globalTimeout = new Timeout(30); + + public ErasureCodingPolicy getEcPolicy() { +return StripedFileTestUtil.getDefaultECPolicy(); + } + + @Before + public void setup() throws IOException { +/* + * Initialize erasure coding policy. + */ +ecPolicy = getEcPolicy(); +dataBlocks = (short) ecPolicy.getNumDataUnits(); +parityBlocks = (short) ecPolicy.getNumParityUnits(); +cellSize = ecPolicy.getCellSize(); +blockSize = stripesPerBlock * cellSize; +blockGroupSize = dataBlocks * blockSize; +System.out.println("EC policy = " + ecPolicy); + +conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); Review Comment: done > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17793170#comment-17793170 ] ASF GitHub Bot commented on HDFS-15413: --- Neilxzn commented on code in PR #5829: URL: https://github.com/apache/hadoop/pull/5829#discussion_r1415029913 ## hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStreamWithTimeout.java: ## @@ -0,0 +1,168 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs; + +import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.protocol.LocatedBlocks; +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; +import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; +import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; +import org.apache.hadoop.io.erasurecode.CodecUtil; +import org.apache.hadoop.io.erasurecode.ErasureCodeNative; +import org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawErasureCoderFactory; +import org.apache.hadoop.test.GenericTestUtils; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; + +import java.io.IOException; +import java.util.Arrays; + +public class TestDFSStripedInputStreamWithTimeout { + + public static final Logger LOG = + LoggerFactory.getLogger(TestDFSStripedInputStreamWithTimeout.class); + + private MiniDFSCluster cluster; + private Configuration conf = new Configuration(); + private DistributedFileSystem fs; + private final Path dirPath = new Path("/striped"); + private Path filePath = new Path(dirPath, "file"); + private ErasureCodingPolicy ecPolicy; + private short dataBlocks; + private short parityBlocks; + private int cellSize; + private final int stripesPerBlock = 2; + private int blockSize; + private int blockGroupSize; + + @Rule + public Timeout globalTimeout = new Timeout(30); + + public ErasureCodingPolicy getEcPolicy() { +return StripedFileTestUtil.getDefaultECPolicy(); + } + + @Before + public void setup() throws IOException { +/* + * Initialize erasure coding policy. + */ +ecPolicy = getEcPolicy(); +dataBlocks = (short) ecPolicy.getNumDataUnits(); +parityBlocks = (short) ecPolicy.getNumParityUnits(); +cellSize = ecPolicy.getCellSize(); +blockSize = stripesPerBlock * cellSize; +blockGroupSize = dataBlocks * blockSize; +System.out.println("EC policy = " + ecPolicy); + +conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); +conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0); + +conf.setInt(DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY, 1000); +// SET CONFIG FOR HDFS CLIENT +conf.setInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 1000); +conf.setInt(HdfsClientConfigKeys.StripedRead.DATANODE_MAX_ATTEMPTS, 3); + +if (ErasureCodeNative.isNativeCodeLoaded()) { + conf.set( + CodecUtil.IO_ERASURECODE_CODEC_RS_RAWCODERS_KEY, + NativeRSRawErasureCoderFactory.CODER_NAME); +} +conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, +GenericTestUtils.getRandomizedTempPath()); +SimulatedFSDataset.setFactory(conf); +startUp(); + } + + private void startUp() throws IOException { +cluster = new MiniDFSCluster.Builder(conf).numDataNodes( +dataBlocks + parityBlocks).build(); +cluster.waitActive(); +for (DataNode dn : cluster.getDataNodes()) { + DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, true); +} +fs = cluster.getFileSystem(); +fs.enableErasureCodingPolicy(getEcPolicy().getName()); +fs.mkdirs(dirPath); +fs.getClient() +.setErasureCodingPolicy(dirPath.to
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17793318#comment-17793318 ] ASF GitHub Bot commented on HDFS-15413: --- hadoop-yetus commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-1840975664 :broken_heart: **-1 overall** | Vote | Subsystem | Runtime | Logfile | Comment | |::|--:|:|::|:---:| | +0 :ok: | reexec | 0m 49s | | Docker mode activated. | _ Prechecks _ | | +1 :green_heart: | dupname | 0m 0s | | No case conflicting files found. | | +0 :ok: | codespell | 0m 1s | | codespell was not available. | | +0 :ok: | detsecrets | 0m 1s | | detect-secrets was not available. | | +0 :ok: | xmllint | 0m 1s | | xmllint was not available. | | +1 :green_heart: | @author | 0m 0s | | The patch does not contain any @author tags. | | +1 :green_heart: | test4tests | 0m 0s | | The patch appears to include 1 new or modified test files. | _ trunk Compile Tests _ | | +0 :ok: | mvndep | 14m 2s | | Maven dependency ordering for branch | | +1 :green_heart: | mvninstall | 35m 6s | | trunk passed | | +1 :green_heart: | compile | 6m 4s | | trunk passed with JDK Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04 | | +1 :green_heart: | compile | 5m 51s | | trunk passed with JDK Private Build-1.8.0_392-8u392-ga-1~20.04-b08 | | +1 :green_heart: | checkstyle | 1m 27s | | trunk passed | | +1 :green_heart: | mvnsite | 2m 22s | | trunk passed | | +1 :green_heart: | javadoc | 1m 52s | | trunk passed with JDK Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04 | | +1 :green_heart: | javadoc | 2m 21s | | trunk passed with JDK Private Build-1.8.0_392-8u392-ga-1~20.04-b08 | | +1 :green_heart: | spotbugs | 5m 56s | | trunk passed | | +1 :green_heart: | shadedclient | 40m 54s | | branch has no errors when building and testing our client artifacts. | _ Patch Compile Tests _ | | +0 :ok: | mvndep | 0m 31s | | Maven dependency ordering for patch | | +1 :green_heart: | mvninstall | 1m 59s | | the patch passed | | +1 :green_heart: | compile | 5m 54s | | the patch passed with JDK Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04 | | +1 :green_heart: | javac | 5m 54s | | the patch passed | | +1 :green_heart: | compile | 5m 42s | | the patch passed with JDK Private Build-1.8.0_392-8u392-ga-1~20.04-b08 | | +1 :green_heart: | javac | 5m 42s | | the patch passed | | -1 :x: | blanks | 0m 0s | [/blanks-eol.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/3/artifact/out/blanks-eol.txt) | The patch has 1 line(s) that end in blanks. Use git apply --whitespace=fix <>. Refer https://git-scm.com/docs/git-apply | | -0 :warning: | checkstyle | 1m 17s | [/results-checkstyle-hadoop-hdfs-project.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/3/artifact/out/results-checkstyle-hadoop-hdfs-project.txt) | hadoop-hdfs-project: The patch generated 2 new + 45 unchanged - 0 fixed = 47 total (was 45) | | +1 :green_heart: | mvnsite | 2m 6s | | the patch passed | | +1 :green_heart: | javadoc | 1m 32s | | the patch passed with JDK Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04 | | +1 :green_heart: | javadoc | 2m 6s | | the patch passed with JDK Private Build-1.8.0_392-8u392-ga-1~20.04-b08 | | +1 :green_heart: | spotbugs | 5m 59s | | the patch passed | | +1 :green_heart: | shadedclient | 40m 5s | | patch has no errors when building and testing our client artifacts. | _ Other Tests _ | | +1 :green_heart: | unit | 2m 22s | | hadoop-hdfs-client in the patch passed. | | -1 :x: | unit | 251m 52s | [/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/3/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt) | hadoop-hdfs in the patch passed. | | +1 :green_heart: | asflicense | 0m 43s | | The patch does not generate ASF License warnings. | | | | 439m 20s | | | | Reason | Tests | |---:|:--| | Failed junit tests | hadoop.hdfs.TestDFSStripedInputStreamWithTimeout | | | hadoop.hdfs.server.datanode.TestDirectoryScanner | | Subsystem | Report/Notes | |--:|:-| | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/3/artifact/out/Dockerfile | | GITHUB PR | https://github.com/apache/hadoop/pull/5829 | | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets xmllint | | uname | Linux
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17793662#comment-17793662 ] ASF GitHub Bot commented on HDFS-15413: --- hadoop-yetus commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-1842713669 :broken_heart: **-1 overall** | Vote | Subsystem | Runtime | Logfile | Comment | |::|--:|:|::|:---:| | +0 :ok: | reexec | 0m 19s | | Docker mode activated. | _ Prechecks _ | | +1 :green_heart: | dupname | 0m 0s | | No case conflicting files found. | | +0 :ok: | codespell | 0m 0s | | codespell was not available. | | +0 :ok: | detsecrets | 0m 0s | | detect-secrets was not available. | | +0 :ok: | xmllint | 0m 0s | | xmllint was not available. | | +1 :green_heart: | @author | 0m 0s | | The patch does not contain any @author tags. | | +1 :green_heart: | test4tests | 0m 0s | | The patch appears to include 1 new or modified test files. | _ trunk Compile Tests _ | | +0 :ok: | mvndep | 13m 24s | | Maven dependency ordering for branch | | +1 :green_heart: | mvninstall | 19m 14s | | trunk passed | | +1 :green_heart: | compile | 2m 51s | | trunk passed with JDK Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04 | | +1 :green_heart: | compile | 2m 47s | | trunk passed with JDK Private Build-1.8.0_392-8u392-ga-1~20.04-b08 | | +1 :green_heart: | checkstyle | 0m 44s | | trunk passed | | +1 :green_heart: | mvnsite | 1m 16s | | trunk passed | | +1 :green_heart: | javadoc | 1m 3s | | trunk passed with JDK Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04 | | +1 :green_heart: | javadoc | 1m 29s | | trunk passed with JDK Private Build-1.8.0_392-8u392-ga-1~20.04-b08 | | +1 :green_heart: | spotbugs | 3m 3s | | trunk passed | | +1 :green_heart: | shadedclient | 20m 9s | | branch has no errors when building and testing our client artifacts. | _ Patch Compile Tests _ | | +0 :ok: | mvndep | 0m 21s | | Maven dependency ordering for patch | | +1 :green_heart: | mvninstall | 1m 2s | | the patch passed | | +1 :green_heart: | compile | 2m 47s | | the patch passed with JDK Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04 | | +1 :green_heart: | javac | 2m 47s | | the patch passed | | +1 :green_heart: | compile | 2m 42s | | the patch passed with JDK Private Build-1.8.0_392-8u392-ga-1~20.04-b08 | | +1 :green_heart: | javac | 2m 42s | | the patch passed | | -1 :x: | blanks | 0m 0s | [/blanks-eol.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/4/artifact/out/blanks-eol.txt) | The patch has 1 line(s) that end in blanks. Use git apply --whitespace=fix <>. Refer https://git-scm.com/docs/git-apply | | -0 :warning: | checkstyle | 0m 35s | [/results-checkstyle-hadoop-hdfs-project.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/4/artifact/out/results-checkstyle-hadoop-hdfs-project.txt) | hadoop-hdfs-project: The patch generated 1 new + 45 unchanged - 0 fixed = 46 total (was 45) | | +1 :green_heart: | mvnsite | 1m 6s | | the patch passed | | +1 :green_heart: | javadoc | 0m 50s | | the patch passed with JDK Ubuntu-11.0.21+9-post-Ubuntu-0ubuntu120.04 | | +1 :green_heart: | javadoc | 1m 20s | | the patch passed with JDK Private Build-1.8.0_392-8u392-ga-1~20.04-b08 | | +1 :green_heart: | spotbugs | 3m 5s | | the patch passed | | +1 :green_heart: | shadedclient | 20m 7s | | patch has no errors when building and testing our client artifacts. | _ Other Tests _ | | +1 :green_heart: | unit | 1m 49s | | hadoop-hdfs-client in the patch passed. | | -1 :x: | unit | 189m 58s | [/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/4/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt) | hadoop-hdfs in the patch passed. | | +1 :green_heart: | asflicense | 0m 27s | | The patch does not generate ASF License warnings. | | | | 293m 19s | | | | Reason | Tests | |---:|:--| | Failed junit tests | hadoop.hdfs.TestDFSStripedInputStreamWithTimeout | | Subsystem | Report/Notes | |--:|:-| | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/4/artifact/out/Dockerfile | | GITHUB PR | https://github.com/apache/hadoop/pull/5829 | | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets xmllint | | uname | Linux 807603bf2dcf 5.15.0-88-generic #98-Ubuntu SMP Mon Oct 2 15:
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17794010#comment-17794010 ] ASF GitHub Bot commented on HDFS-15413: --- Neilxzn commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-1844334814 I can pass the unit test hadoop.hdfs.TestDFSStripedInputStreamWithTimeout in my local development environment, but it fails on GitHub Jenkins. ![image](https://github.com/apache/hadoop/assets/10757009/a511b4e1-8413-44bb-9136-5e7cc1f3ff17) Check if the test log of the development environment is consistent with the assumption. When the client reads the file for the first time and stops for 10 seconds, the connection between the client and the datanode server will be automatically disconnected, resulting in a failed subsequent read by the client. @ayushtkn Any other suggestions? > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: 1 millis timeout > while waiti
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17794125#comment-17794125 ] ASF GitHub Bot commented on HDFS-15413: --- ayushtkn commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-1845117963 @Neilxzn I tried & it fails locally ``` [INFO] --- [INFO] T E S T S [INFO] --- [INFO] Running org.apache.hadoop.hdfs.TestDFSStripedInputStreamWithTimeout [ERROR] Tests run: 1, Failures: 1, Errors: 0, Skipped: 0, Time elapsed: 17.64 s <<< FAILURE! - in org.apache.hadoop.hdfs.TestDFSStripedInputStreamWithTimeout [ERROR] testPreadTimeout(org.apache.hadoop.hdfs.TestDFSStripedInputStreamWithTimeout) Time elapsed: 17.509 s <<< FAILURE! java.lang.AssertionError: It Should fail to read striped time out with 1 attempt . at org.junit.Assert.fail(Assert.java:89) at org.apache.hadoop.hdfs.TestDFSStripedInputStreamWithTimeout.testPreadTimeout(TestDFSStripedInputStreamWithTimeout.java:145) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:299) at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:293) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.lang.Thread.run(Thread.java:750) [INFO] [INFO] Results: [INFO] [ERROR] Failures: [ERROR] TestDFSStripedInputStreamWithTimeout.testPreadTimeout:145 It Should fail to read striped time out with 1 attempt . [INFO] [ERROR] Tests run: 1, Failures: 1, Errors: 0, Skipped: 0 [INFO] [ERROR] There are test failures. ``` To reproduce: in the hadoop root directory there is file named ``start-build-env.sh``, run that ``bash start-build-env.sh``, it will give you a docker env, run the test inside that & it will fail > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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) { >
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17795268#comment-17795268 ] ASF GitHub Bot commented on HDFS-15413: --- Neilxzn commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-1849677715 > @Neilxzn I tried & it fails locally > > ``` > [INFO] --- > [INFO] T E S T S > [INFO] --- > [INFO] Running org.apache.hadoop.hdfs.TestDFSStripedInputStreamWithTimeout > [ERROR] Tests run: 1, Failures: 1, Errors: 0, Skipped: 0, Time elapsed: 17.64 s <<< FAILURE! - in org.apache.hadoop.hdfs.TestDFSStripedInputStreamWithTimeout > [ERROR] testPreadTimeout(org.apache.hadoop.hdfs.TestDFSStripedInputStreamWithTimeout) Time elapsed: 17.509 s <<< FAILURE! > java.lang.AssertionError: It Should fail to read striped time out with 1 attempt . > at org.junit.Assert.fail(Assert.java:89) > at org.apache.hadoop.hdfs.TestDFSStripedInputStreamWithTimeout.testPreadTimeout(TestDFSStripedInputStreamWithTimeout.java:145) > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) > at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) > at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) > at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) > at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) > at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) > at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) > at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:299) > at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:293) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at java.lang.Thread.run(Thread.java:750) > > [INFO] > [INFO] Results: > [INFO] > [ERROR] Failures: > [ERROR] TestDFSStripedInputStreamWithTimeout.testPreadTimeout:145 It Should fail to read striped time out with 1 attempt . > [INFO] > [ERROR] Tests run: 1, Failures: 1, Errors: 0, Skipped: 0 > [INFO] > [ERROR] There are test failures. > ``` > > To reproduce: in the hadoop root directory there is file named `start-build-env.sh`, run that `bash start-build-env.sh`, it will give you a docker env, run the test inside that & it will fail Thank you. I will check it again soon. > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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); >
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17824734#comment-17824734 ] ASF GitHub Bot commented on HDFS-15413: --- haiyang1987 commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-1985490125 Hi @Neilxzn Any progress here? Thanks. this PR is still necessary, there are some similar problems in our environment~ > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: 1 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.SocketTimeoutExceptio
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17824735#comment-17824735 ] ASF GitHub Bot commented on HDFS-15413: --- Neilxzn commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-1985490862 这是自动回复邮件。来件已接收,谢谢。 > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: 1 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: 1 millis timeout > while waiting for channel to be ready for write. ch : > java.nio.channels.SocketChan
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17824902#comment-17824902 ] ASF GitHub Bot commented on HDFS-15413: --- haiyang1987 commented on code in PR #5829: URL: https://github.com/apache/hadoop/pull/5829#discussion_r1518480694 ## hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripeReader.java: ## @@ -233,41 +235,62 @@ private ByteBufferStrategy[] getReadStrategies(StripingChunk chunk) { private int readToBuffer(BlockReader blockReader, DatanodeInfo currentNode, ByteBufferStrategy strategy, - ExtendedBlock currentBlock) throws IOException { + LocatedBlock currentBlock, int chunkIndex) throws IOException { final int targetLength = strategy.getTargetLength(); -int length = 0; -try { - while (length < targetLength) { -int ret = strategy.readFromBlock(blockReader); -if (ret < 0) { - throw new IOException("Unexpected EOS from the reader"); +int curAttempts = 0; +while (curAttempts < readDNMaxAttempts) { + curAttempts++; + int length = 0; + try { +while (length < targetLength) { + int ret = strategy.readFromBlock(blockReader); + if (ret < 0) { +throw new IOException("Unexpected EOS from the reader"); + } + length += ret; +} +return length; + } catch (ChecksumException ce) { +DFSClient.LOG.warn("Found Checksum error for " ++ currentBlock + " from " + currentNode ++ " at " + ce.getPos()); +//Clear buffer to make next decode success +strategy.getReadBuffer().clear(); +// we want to remember which block replicas we have tried +corruptedBlocks.addCorruptedBlock(currentBlock.getBlock(), currentNode); +throw ce; + } catch (IOException e) { +//Clear buffer to make next decode success +strategy.getReadBuffer().clear(); +if (curAttempts < readDNMaxAttempts) { + if (readerInfos[chunkIndex].reader != null) { +readerInfos[chunkIndex].reader.close(); + } + if (dfsStripedInputStream.createBlockReader(currentBlock, + alignedStripe.getOffsetInBlock(), targetBlocks, + readerInfos, chunkIndex, readTo)) { +blockReader = readerInfos[chunkIndex].reader; +String msg = "Reconnect to " + currentNode.getInfoAddr() ++ " for block " + currentBlock.getBlock(); +DFSClient.LOG.warn(msg); +continue; + } + DFSClient.LOG.warn("Exception while reading from " Review Comment: Line[278-281] will move` if (curAttempts < readDNMaxAttempts) {` outside > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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"
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17825125#comment-17825125 ] ASF GitHub Bot commented on HDFS-15413: --- Neilxzn closed pull request #5829: HDFS-15413. add dfs.client.read.striped.datanode.max.attempts to fix read ecfile timeout URL: https://github.com/apache/hadoop/pull/5829 > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: 1 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: 1 millis timeout > while waiting for channel to be ready
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17825126#comment-17825126 ] ASF GitHub Bot commented on HDFS-15413: --- hadoop-yetus commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-1987499017 :broken_heart: **-1 overall** | Vote | Subsystem | Runtime | Logfile | Comment | |::|--:|:|::|:---:| | +0 :ok: | reexec | 0m 0s | | Docker mode activated. | | -1 :x: | patch | 0m 19s | | https://github.com/apache/hadoop/pull/5829 does not apply to trunk. Rebase required? Wrong Branch? See https://cwiki.apache.org/confluence/display/HADOOP/How+To+Contribute for help. | | Subsystem | Report/Notes | |--:|:-| | GITHUB PR | https://github.com/apache/hadoop/pull/5829 | | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/5/console | | versions | git=2.34.1 | | Powered by | Apache Yetus 0.14.0 https://yetus.apache.org | This message was automatically generated. > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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 cli
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17825133#comment-17825133 ] ASF GitHub Bot commented on HDFS-15413: --- Neilxzn opened a new pull request, #5829: URL: https://github.com/apache/hadoop/pull/5829 ### Description of PR https://issues.apache.org/jira/browse/HDFS-15413 Offer a available patch to fix HDFS-15413. This patch add dfs.client.read.striped.datanode.max.attempts config to allow users to adjust the number of dn retries to solve the problem of Datanode timeout when reading EC files. ### How was this patch tested? no add test. just test in our cluster ### For code changes: add dfs.client.read.striped.datanode.max.attempts config > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: 1 millis timeout > while waiting for channel to be ready for write. ch : > java.nio.channels.SocketChannel[connected local=/10.128.14.46:9866
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17825135#comment-17825135 ] ASF GitHub Bot commented on HDFS-15413: --- Neilxzn commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-1987532047 > Hi @Neilxzn Any progress here? Thanks. > > this PR is still necessary, there are some similar problems in our environment~ @haiyang1987 Our online environment (70 PB EC Data cluster, spark + hive olap) has already applied this patch. So far, everything is running normally. > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: 1 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-
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17825137#comment-17825137 ] ASF GitHub Bot commented on HDFS-15413: --- Neilxzn commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-1987533616 Rebase it to branch trunk. And The new test `org.apache.hadoop.hdfs.TestDFSStripedInputStreamWithTimeout` passed on my lolcal env. https://github.com/apache/hadoop/assets/10757009/6bedc732-cbe7-403b-8c6a-b5e78a33527f";> > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: 1 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_B
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17825139#comment-17825139 ] ASF GitHub Bot commented on HDFS-15413: --- haiyang1987 commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-1987535597 > > Hi @Neilxzn Any progress here? Thanks. > > this PR is still necessary, there are some similar problems in our environment~ > > @haiyang1987 Our online environment (70 PB EC Data cluster, spark + hive olap) has already applied this patch. So far, everything is running normally. Noted, thanks for you work ~ > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: 1 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 stoppe
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17825194#comment-17825194 ] ASF GitHub Bot commented on HDFS-15413: --- hadoop-yetus commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-1987825036 :broken_heart: **-1 overall** | Vote | Subsystem | Runtime | Logfile | Comment | |::|--:|:|::|:---:| | +0 :ok: | reexec | 0m 20s | | Docker mode activated. | _ Prechecks _ | | +1 :green_heart: | dupname | 0m 0s | | No case conflicting files found. | | +0 :ok: | codespell | 0m 0s | | codespell was not available. | | +0 :ok: | detsecrets | 0m 0s | | detect-secrets was not available. | | +0 :ok: | xmllint | 0m 0s | | xmllint was not available. | | +1 :green_heart: | @author | 0m 0s | | The patch does not contain any @author tags. | | +1 :green_heart: | test4tests | 0m 0s | | The patch appears to include 1 new or modified test files. | _ trunk Compile Tests _ | | +0 :ok: | mvndep | 14m 55s | | Maven dependency ordering for branch | | +1 :green_heart: | mvninstall | 20m 51s | | trunk passed | | +1 :green_heart: | compile | 3m 1s | | trunk passed with JDK Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1 | | +1 :green_heart: | compile | 2m 53s | | trunk passed with JDK Private Build-1.8.0_392-8u392-ga-1~20.04-b08 | | +1 :green_heart: | checkstyle | 0m 46s | | trunk passed | | +1 :green_heart: | mvnsite | 1m 21s | | trunk passed | | +1 :green_heart: | javadoc | 1m 10s | | trunk passed with JDK Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1 | | +1 :green_heart: | javadoc | 1m 33s | | trunk passed with JDK Private Build-1.8.0_392-8u392-ga-1~20.04-b08 | | -1 :x: | spotbugs | 1m 30s | [/branch-spotbugs-hadoop-hdfs-project_hadoop-hdfs-client-warnings.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/6/artifact/out/branch-spotbugs-hadoop-hdfs-project_hadoop-hdfs-client-warnings.html) | hadoop-hdfs-project/hadoop-hdfs-client in trunk has 1 extant spotbugs warnings. | | +1 :green_heart: | shadedclient | 22m 5s | | branch has no errors when building and testing our client artifacts. | | -0 :warning: | patch | 22m 18s | | Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary. | _ Patch Compile Tests _ | | +0 :ok: | mvndep | 0m 24s | | Maven dependency ordering for patch | | +1 :green_heart: | mvninstall | 1m 10s | | the patch passed | | +1 :green_heart: | compile | 3m 5s | | the patch passed with JDK Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1 | | +1 :green_heart: | javac | 3m 5s | | the patch passed | | +1 :green_heart: | compile | 3m 1s | | the patch passed with JDK Private Build-1.8.0_392-8u392-ga-1~20.04-b08 | | +1 :green_heart: | javac | 3m 1s | | the patch passed | | -1 :x: | blanks | 0m 0s | [/blanks-eol.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/6/artifact/out/blanks-eol.txt) | The patch has 1 line(s) that end in blanks. Use git apply --whitespace=fix <>. Refer https://git-scm.com/docs/git-apply | | -0 :warning: | checkstyle | 0m 40s | [/results-checkstyle-hadoop-hdfs-project.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/6/artifact/out/results-checkstyle-hadoop-hdfs-project.txt) | hadoop-hdfs-project: The patch generated 1 new + 45 unchanged - 0 fixed = 46 total (was 45) | | +1 :green_heart: | mvnsite | 1m 11s | | the patch passed | | +1 :green_heart: | javadoc | 0m 58s | | the patch passed with JDK Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1 | | +1 :green_heart: | javadoc | 1m 26s | | the patch passed with JDK Private Build-1.8.0_392-8u392-ga-1~20.04-b08 | | +1 :green_heart: | spotbugs | 3m 4s | | the patch passed | | +1 :green_heart: | shadedclient | 22m 21s | | patch has no errors when building and testing our client artifacts. | _ Other Tests _ | | +1 :green_heart: | unit | 1m 49s | | hadoop-hdfs-client in the patch passed. | | -1 :x: | unit | 217m 9s | [/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/6/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt) | hadoop-hdfs in the patch passed. | | +1 :green_heart: | asflicense | 0m 32s | | The patch does not generate ASF License warnings. | | | | 330m 42s | | | | Reason | Tests | |---:|:--| | Failed junit tests | hadoop.hdfs.TestReconstructStripedFileWithRandomECPolicy | | | hadoop.hdfs.serve
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17825214#comment-17825214 ] ASF GitHub Bot commented on HDFS-15413: --- hadoop-yetus commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-1987902095 :broken_heart: **-1 overall** | Vote | Subsystem | Runtime | Logfile | Comment | |::|--:|:|::|:---:| | +0 :ok: | reexec | 0m 19s | | Docker mode activated. | _ Prechecks _ | | +1 :green_heart: | dupname | 0m 0s | | No case conflicting files found. | | +0 :ok: | codespell | 0m 0s | | codespell was not available. | | +0 :ok: | detsecrets | 0m 1s | | detect-secrets was not available. | | +0 :ok: | xmllint | 0m 1s | | xmllint was not available. | | +1 :green_heart: | @author | 0m 0s | | The patch does not contain any @author tags. | | +1 :green_heart: | test4tests | 0m 0s | | The patch appears to include 1 new or modified test files. | _ trunk Compile Tests _ | | +0 :ok: | mvndep | 14m 40s | | Maven dependency ordering for branch | | +1 :green_heart: | mvninstall | 21m 13s | | trunk passed | | +1 :green_heart: | compile | 3m 2s | | trunk passed with JDK Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1 | | +1 :green_heart: | compile | 3m 5s | | trunk passed with JDK Private Build-1.8.0_392-8u392-ga-1~20.04-b08 | | +1 :green_heart: | checkstyle | 0m 48s | | trunk passed | | +1 :green_heart: | mvnsite | 1m 19s | | trunk passed | | +1 :green_heart: | javadoc | 1m 8s | | trunk passed with JDK Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1 | | +1 :green_heart: | javadoc | 1m 48s | | trunk passed with JDK Private Build-1.8.0_392-8u392-ga-1~20.04-b08 | | -1 :x: | spotbugs | 1m 39s | [/branch-spotbugs-hadoop-hdfs-project_hadoop-hdfs-client-warnings.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/7/artifact/out/branch-spotbugs-hadoop-hdfs-project_hadoop-hdfs-client-warnings.html) | hadoop-hdfs-project/hadoop-hdfs-client in trunk has 1 extant spotbugs warnings. | | +1 :green_heart: | shadedclient | 22m 50s | | branch has no errors when building and testing our client artifacts. | | -0 :warning: | patch | 23m 3s | | Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary. | _ Patch Compile Tests _ | | +0 :ok: | mvndep | 0m 24s | | Maven dependency ordering for patch | | +1 :green_heart: | mvninstall | 1m 1s | | the patch passed | | +1 :green_heart: | compile | 2m 55s | | the patch passed with JDK Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1 | | +1 :green_heart: | javac | 2m 55s | | the patch passed | | +1 :green_heart: | compile | 2m 56s | | the patch passed with JDK Private Build-1.8.0_392-8u392-ga-1~20.04-b08 | | +1 :green_heart: | javac | 2m 56s | | the patch passed | | -1 :x: | blanks | 0m 0s | [/blanks-eol.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/7/artifact/out/blanks-eol.txt) | The patch has 1 line(s) that end in blanks. Use git apply --whitespace=fix <>. Refer https://git-scm.com/docs/git-apply | | -0 :warning: | checkstyle | 0m 44s | [/results-checkstyle-hadoop-hdfs-project.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/7/artifact/out/results-checkstyle-hadoop-hdfs-project.txt) | hadoop-hdfs-project: The patch generated 1 new + 45 unchanged - 0 fixed = 46 total (was 45) | | +1 :green_heart: | mvnsite | 1m 13s | | the patch passed | | +1 :green_heart: | javadoc | 0m 53s | | the patch passed with JDK Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1 | | +1 :green_heart: | javadoc | 1m 34s | | the patch passed with JDK Private Build-1.8.0_392-8u392-ga-1~20.04-b08 | | +1 :green_heart: | spotbugs | 3m 23s | | the patch passed | | +1 :green_heart: | shadedclient | 23m 7s | | patch has no errors when building and testing our client artifacts. | _ Other Tests _ | | +1 :green_heart: | unit | 1m 54s | | hadoop-hdfs-client in the patch passed. | | -1 :x: | unit | 235m 57s | [/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/7/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt) | hadoop-hdfs in the patch passed. | | +1 :green_heart: | asflicense | 0m 32s | | The patch does not generate ASF License warnings. | | | | 351m 39s | | | | Reason | Tests | |---:|:--| | Failed junit tests | hadoop.hdfs.TestEncryptionZonesWithKMS | | | hadoop.hdfs.TestDFSStripedInputStrea
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17828904#comment-17828904 ] ASF GitHub Bot commented on HDFS-15413: --- haiyang1987 commented on code in PR #5829: URL: https://github.com/apache/hadoop/pull/5829#discussion_r1531798611 ## hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripeReader.java: ## @@ -233,41 +235,62 @@ private ByteBufferStrategy[] getReadStrategies(StripingChunk chunk) { private int readToBuffer(BlockReader blockReader, DatanodeInfo currentNode, ByteBufferStrategy strategy, - ExtendedBlock currentBlock) throws IOException { + LocatedBlock currentBlock, int chunkIndex) throws IOException { Review Comment: ``` private int readToBuffer(BlockReader blockReader, DatanodeInfo currentNode, ByteBufferStrategy strategy, LocatedBlock currentBlock, int chunkIndex, long offsetInBlock) ``` > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: > /
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17828902#comment-17828902 ] ASF GitHub Bot commented on HDFS-15413: --- haiyang1987 commented on code in PR #5829: URL: https://github.com/apache/hadoop/pull/5829#discussion_r1531794650 ## hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripeReader.java: ## @@ -284,7 +307,8 @@ private Callable readCells(final BlockReader reader, int ret = 0; for (ByteBufferStrategy strategy : strategies) { -int bytesReead = readToBuffer(reader, datanode, strategy, currentBlock); +int bytesReead = readToBuffer(reader, datanode, strategy, currentBlock, +chunkIndex); Review Comment: For `readToBuffer` maybe need to consider the current actual offsetInBlock. `readToBuffer(reader, datanode, strategy, currentBlock, chunkIndex, ret);` > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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.SocketTi
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17828906#comment-17828906 ] ASF GitHub Bot commented on HDFS-15413: --- haiyang1987 commented on code in PR #5829: URL: https://github.com/apache/hadoop/pull/5829#discussion_r1531803838 ## hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripeReader.java: ## @@ -233,41 +235,62 @@ private ByteBufferStrategy[] getReadStrategies(StripingChunk chunk) { private int readToBuffer(BlockReader blockReader, DatanodeInfo currentNode, ByteBufferStrategy strategy, - ExtendedBlock currentBlock) throws IOException { + LocatedBlock currentBlock, int chunkIndex) throws IOException { final int targetLength = strategy.getTargetLength(); -int length = 0; -try { - while (length < targetLength) { -int ret = strategy.readFromBlock(blockReader); -if (ret < 0) { - throw new IOException("Unexpected EOS from the reader"); +int curAttempts = 0; +while (curAttempts < readDNMaxAttempts) { + curAttempts++; + int length = 0; + try { +while (length < targetLength) { + int ret = strategy.readFromBlock(blockReader); + if (ret < 0) { +throw new IOException("Unexpected EOS from the reader"); + } + length += ret; +} +return length; + } catch (ChecksumException ce) { +DFSClient.LOG.warn("Found Checksum error for " ++ currentBlock + " from " + currentNode ++ " at " + ce.getPos()); +//Clear buffer to make next decode success +strategy.getReadBuffer().clear(); +// we want to remember which block replicas we have tried +corruptedBlocks.addCorruptedBlock(currentBlock.getBlock(), currentNode); +throw ce; + } catch (IOException e) { +//Clear buffer to make next decode success +strategy.getReadBuffer().clear(); +if (curAttempts < readDNMaxAttempts) { + if (readerInfos[chunkIndex].reader != null) { +readerInfos[chunkIndex].reader.close(); + } + if (dfsStripedInputStream.createBlockReader(currentBlock, + alignedStripe.getOffsetInBlock(), targetBlocks, Review Comment: If use pread to read data, if the currently set buffer size is a block size, For a block in a dn, the data of multiple cell units may be read, so the size of the ByteBufferStrategy array in the StripingChunk corresponding to the AlignedStripe is calculated to be multiple (there are multiple List slices in ChunkByteBuffer), https://github.com/apache/hadoop/assets/3760130/40f7a944-ea57-4891-9719-86a1b009244d";> So when processing retry createBlockReader in readToBuffer, we may need to consider the current actual offsetInBlock to avoid reading duplicate data from datanode. > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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) { >
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17828905#comment-17828905 ] ASF GitHub Bot commented on HDFS-15413: --- haiyang1987 commented on code in PR #5829: URL: https://github.com/apache/hadoop/pull/5829#discussion_r1531799282 ## hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripeReader.java: ## @@ -233,41 +235,62 @@ private ByteBufferStrategy[] getReadStrategies(StripingChunk chunk) { private int readToBuffer(BlockReader blockReader, DatanodeInfo currentNode, ByteBufferStrategy strategy, - ExtendedBlock currentBlock) throws IOException { + LocatedBlock currentBlock, int chunkIndex) throws IOException { final int targetLength = strategy.getTargetLength(); -int length = 0; -try { - while (length < targetLength) { -int ret = strategy.readFromBlock(blockReader); -if (ret < 0) { - throw new IOException("Unexpected EOS from the reader"); +int curAttempts = 0; +while (curAttempts < readDNMaxAttempts) { + curAttempts++; + int length = 0; + try { +while (length < targetLength) { + int ret = strategy.readFromBlock(blockReader); + if (ret < 0) { +throw new IOException("Unexpected EOS from the reader"); + } + length += ret; +} +return length; + } catch (ChecksumException ce) { +DFSClient.LOG.warn("Found Checksum error for " ++ currentBlock + " from " + currentNode ++ " at " + ce.getPos()); +//Clear buffer to make next decode success +strategy.getReadBuffer().clear(); +// we want to remember which block replicas we have tried +corruptedBlocks.addCorruptedBlock(currentBlock.getBlock(), currentNode); +throw ce; + } catch (IOException e) { +//Clear buffer to make next decode success +strategy.getReadBuffer().clear(); +if (curAttempts < readDNMaxAttempts) { + if (readerInfos[chunkIndex].reader != null) { +readerInfos[chunkIndex].reader.close(); + } + if (dfsStripedInputStream.createBlockReader(currentBlock, + alignedStripe.getOffsetInBlock(), targetBlocks, Review Comment: ``` if (dfsStripedInputStream.createBlockReader(currentBlock, offsetInBlock, targetBlocks, ``` > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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"); > T
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17828907#comment-17828907 ] ASF GitHub Bot commented on HDFS-15413: --- haiyang1987 commented on code in PR #5829: URL: https://github.com/apache/hadoop/pull/5829#discussion_r1531806620 ## hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripeReader.java: ## @@ -233,41 +235,62 @@ private ByteBufferStrategy[] getReadStrategies(StripingChunk chunk) { private int readToBuffer(BlockReader blockReader, DatanodeInfo currentNode, ByteBufferStrategy strategy, - ExtendedBlock currentBlock) throws IOException { + LocatedBlock currentBlock, int chunkIndex) throws IOException { final int targetLength = strategy.getTargetLength(); -int length = 0; -try { - while (length < targetLength) { -int ret = strategy.readFromBlock(blockReader); -if (ret < 0) { - throw new IOException("Unexpected EOS from the reader"); +int curAttempts = 0; +while (curAttempts < readDNMaxAttempts) { + curAttempts++; + int length = 0; + try { +while (length < targetLength) { + int ret = strategy.readFromBlock(blockReader); + if (ret < 0) { +throw new IOException("Unexpected EOS from the reader"); + } + length += ret; +} +return length; + } catch (ChecksumException ce) { +DFSClient.LOG.warn("Found Checksum error for " ++ currentBlock + " from " + currentNode ++ " at " + ce.getPos()); +//Clear buffer to make next decode success +strategy.getReadBuffer().clear(); +// we want to remember which block replicas we have tried +corruptedBlocks.addCorruptedBlock(currentBlock.getBlock(), currentNode); +throw ce; + } catch (IOException e) { +//Clear buffer to make next decode success +strategy.getReadBuffer().clear(); +if (curAttempts < readDNMaxAttempts) { + if (readerInfos[chunkIndex].reader != null) { +readerInfos[chunkIndex].reader.close(); + } + if (dfsStripedInputStream.createBlockReader(currentBlock, + alignedStripe.getOffsetInBlock(), targetBlocks, Review Comment: Hi @Neilxzn @Hexiaoqiao @ayushtkn @zhangshuyan0 @ZanderXu what dou you think? Please also help to look into this issue when you have free time , thanks~ > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 + "
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17834915#comment-17834915 ] ASF GitHub Bot commented on HDFS-15413: --- zhangshuyan0 commented on code in PR #5829: URL: https://github.com/apache/hadoop/pull/5829#discussion_r1555819771 ## hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripeReader.java: ## @@ -233,41 +235,62 @@ private ByteBufferStrategy[] getReadStrategies(StripingChunk chunk) { private int readToBuffer(BlockReader blockReader, DatanodeInfo currentNode, ByteBufferStrategy strategy, - ExtendedBlock currentBlock) throws IOException { + LocatedBlock currentBlock, int chunkIndex) throws IOException { final int targetLength = strategy.getTargetLength(); -int length = 0; -try { - while (length < targetLength) { -int ret = strategy.readFromBlock(blockReader); -if (ret < 0) { - throw new IOException("Unexpected EOS from the reader"); +int curAttempts = 0; +while (curAttempts < readDNMaxAttempts) { + curAttempts++; + int length = 0; + try { +while (length < targetLength) { + int ret = strategy.readFromBlock(blockReader); + if (ret < 0) { +throw new IOException("Unexpected EOS from the reader"); + } + length += ret; +} +return length; + } catch (ChecksumException ce) { +DFSClient.LOG.warn("Found Checksum error for " ++ currentBlock + " from " + currentNode ++ " at " + ce.getPos()); +//Clear buffer to make next decode success +strategy.getReadBuffer().clear(); +// we want to remember which block replicas we have tried +corruptedBlocks.addCorruptedBlock(currentBlock.getBlock(), currentNode); +throw ce; + } catch (IOException e) { +//Clear buffer to make next decode success +strategy.getReadBuffer().clear(); +if (curAttempts < readDNMaxAttempts) { + if (readerInfos[chunkIndex].reader != null) { +readerInfos[chunkIndex].reader.close(); + } + if (dfsStripedInputStream.createBlockReader(currentBlock, + alignedStripe.getOffsetInBlock(), targetBlocks, Review Comment: Great catch! Agree with @haiyang1987 's idea. But the solution seems flawed. Should we use `alignedStripe.getOffsetInBlock()+ret` instead of `ret` here? Looking forward to your reply! @haiyang1987 > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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.
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17834916#comment-17834916 ] ASF GitHub Bot commented on HDFS-15413: --- zhangshuyan0 commented on code in PR #5829: URL: https://github.com/apache/hadoop/pull/5829#discussion_r1555821906 ## hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripeReader.java: ## @@ -284,7 +307,8 @@ private Callable readCells(final BlockReader reader, int ret = 0; for (ByteBufferStrategy strategy : strategies) { -int bytesReead = readToBuffer(reader, datanode, strategy, currentBlock); +int bytesReead = readToBuffer(reader, datanode, strategy, currentBlock, Review Comment: Could you please correct this variable name by the way? `bytesReead` -> `bytesRead` > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: 1 millis timeout > while waiting for channel to be ready for write. ch : > j
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17835123#comment-17835123 ] ASF GitHub Bot commented on HDFS-15413: --- haiyang1987 commented on code in PR #5829: URL: https://github.com/apache/hadoop/pull/5829#discussion_r1556791082 ## hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripeReader.java: ## @@ -233,41 +235,62 @@ private ByteBufferStrategy[] getReadStrategies(StripingChunk chunk) { private int readToBuffer(BlockReader blockReader, DatanodeInfo currentNode, ByteBufferStrategy strategy, - ExtendedBlock currentBlock) throws IOException { + LocatedBlock currentBlock, int chunkIndex) throws IOException { final int targetLength = strategy.getTargetLength(); -int length = 0; -try { - while (length < targetLength) { -int ret = strategy.readFromBlock(blockReader); -if (ret < 0) { - throw new IOException("Unexpected EOS from the reader"); +int curAttempts = 0; +while (curAttempts < readDNMaxAttempts) { + curAttempts++; + int length = 0; + try { +while (length < targetLength) { + int ret = strategy.readFromBlock(blockReader); + if (ret < 0) { +throw new IOException("Unexpected EOS from the reader"); + } + length += ret; +} +return length; + } catch (ChecksumException ce) { +DFSClient.LOG.warn("Found Checksum error for " ++ currentBlock + " from " + currentNode ++ " at " + ce.getPos()); +//Clear buffer to make next decode success +strategy.getReadBuffer().clear(); +// we want to remember which block replicas we have tried +corruptedBlocks.addCorruptedBlock(currentBlock.getBlock(), currentNode); +throw ce; + } catch (IOException e) { +//Clear buffer to make next decode success +strategy.getReadBuffer().clear(); +if (curAttempts < readDNMaxAttempts) { + if (readerInfos[chunkIndex].reader != null) { +readerInfos[chunkIndex].reader.close(); + } + if (dfsStripedInputStream.createBlockReader(currentBlock, + alignedStripe.getOffsetInBlock(), targetBlocks, Review Comment: yeah, Agree with @zhangshuyan0 comment. `readToBuffer(reader, datanode, strategy, currentBlock, chunkIndex, ret) ` here we shoud use `alignedStripe.getOffsetInBlock()+ret` instead of `ret` > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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.print
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17837170#comment-17837170 ] ASF GitHub Bot commented on HDFS-15413: --- zhangshuyan0 commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-2056526550 @Neilxzn Hi, this patch is very useful, would you mind further fixing this PR? > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: 1 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: 1 millis timeout > while waiting for c
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17838091#comment-17838091 ] ASF GitHub Bot commented on HDFS-15413: --- Neilxzn commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-2060941447 > @Neilxzn Hi, this patch is very useful, would you mind further fixing this PR? Sorry for my late reply. I have updated the patch based on the suggestions above. Please review it again. @haiyang1987 @zhangshuyan0 > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: 1 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
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17838143#comment-17838143 ] ASF GitHub Bot commented on HDFS-15413: --- haiyang1987 commented on code in PR #5829: URL: https://github.com/apache/hadoop/pull/5829#discussion_r1568745996 ## hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripeReader.java: ## @@ -233,41 +235,63 @@ private ByteBufferStrategy[] getReadStrategies(StripingChunk chunk) { private int readToBuffer(BlockReader blockReader, DatanodeInfo currentNode, ByteBufferStrategy strategy, - ExtendedBlock currentBlock) throws IOException { + LocatedBlock currentBlock, int chunkIndex, long offsetInBlock) + throws IOException { final int targetLength = strategy.getTargetLength(); -int length = 0; -try { - while (length < targetLength) { -int ret = strategy.readFromBlock(blockReader); -if (ret < 0) { - throw new IOException("Unexpected EOS from the reader"); +int curAttempts = 0; +while (curAttempts < readDNMaxAttempts) { + curAttempts++; + int length = 0; + try { +while (length < targetLength) { + int ret = strategy.readFromBlock(blockReader); + if (ret < 0) { +throw new IOException("Unexpected EOS from the reader"); + } + length += ret; +} +return length; + } catch (ChecksumException ce) { +DFSClient.LOG.warn("Found Checksum error for " ++ currentBlock + " from " + currentNode ++ " at " + ce.getPos()); +//Clear buffer to make next decode success +strategy.getReadBuffer().clear(); +// we want to remember which block replicas we have tried +corruptedBlocks.addCorruptedBlock(currentBlock.getBlock(), currentNode); +throw ce; + } catch (IOException e) { +//Clear buffer to make next decode success +strategy.getReadBuffer().clear(); +if (curAttempts < readDNMaxAttempts) { + if (readerInfos[chunkIndex].reader != null) { +readerInfos[chunkIndex].reader.close(); + } + if (dfsStripedInputStream.createBlockReader(currentBlock, + offsetInBlock, targetBlocks, + readerInfos, chunkIndex, readTo)) { +blockReader = readerInfos[chunkIndex].reader; +String msg = "Reconnect to " + currentNode.getInfoAddr() ++ " for block " + currentBlock.getBlock(); +DFSClient.LOG.warn(msg); Review Comment: Can use the ``` DFSClient.LOG.warn("Reconnect to {} for block {}", currentNode.getInfoAddr(), currentBlock.getBlock()); ``` > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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"); >
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17838145#comment-17838145 ] ASF GitHub Bot commented on HDFS-15413: --- haiyang1987 commented on code in PR #5829: URL: https://github.com/apache/hadoop/pull/5829#discussion_r1568748150 ## hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripeReader.java: ## @@ -233,41 +235,63 @@ private ByteBufferStrategy[] getReadStrategies(StripingChunk chunk) { private int readToBuffer(BlockReader blockReader, DatanodeInfo currentNode, ByteBufferStrategy strategy, - ExtendedBlock currentBlock) throws IOException { + LocatedBlock currentBlock, int chunkIndex, long offsetInBlock) + throws IOException { final int targetLength = strategy.getTargetLength(); -int length = 0; -try { - while (length < targetLength) { -int ret = strategy.readFromBlock(blockReader); -if (ret < 0) { - throw new IOException("Unexpected EOS from the reader"); +int curAttempts = 0; +while (curAttempts < readDNMaxAttempts) { + curAttempts++; + int length = 0; + try { +while (length < targetLength) { + int ret = strategy.readFromBlock(blockReader); + if (ret < 0) { +throw new IOException("Unexpected EOS from the reader"); + } + length += ret; +} +return length; + } catch (ChecksumException ce) { +DFSClient.LOG.warn("Found Checksum error for " ++ currentBlock + " from " + currentNode ++ " at " + ce.getPos()); +//Clear buffer to make next decode success +strategy.getReadBuffer().clear(); +// we want to remember which block replicas we have tried +corruptedBlocks.addCorruptedBlock(currentBlock.getBlock(), currentNode); +throw ce; + } catch (IOException e) { +//Clear buffer to make next decode success +strategy.getReadBuffer().clear(); +if (curAttempts < readDNMaxAttempts) { + if (readerInfos[chunkIndex].reader != null) { +readerInfos[chunkIndex].reader.close(); + } + if (dfsStripedInputStream.createBlockReader(currentBlock, + offsetInBlock, targetBlocks, + readerInfos, chunkIndex, readTo)) { +blockReader = readerInfos[chunkIndex].reader; +String msg = "Reconnect to " + currentNode.getInfoAddr() ++ " for block " + currentBlock.getBlock(); +DFSClient.LOG.warn(msg); +continue; + } } -length += ret; +DFSClient.LOG.warn("Exception while reading from " Review Comment: Here also can use to `warn("{}", arg)` format? > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 l
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17838153#comment-17838153 ] ASF GitHub Bot commented on HDFS-15413: --- haiyang1987 commented on code in PR #5829: URL: https://github.com/apache/hadoop/pull/5829#discussion_r1568756254 ## hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripeReader.java: ## @@ -233,41 +235,63 @@ private ByteBufferStrategy[] getReadStrategies(StripingChunk chunk) { private int readToBuffer(BlockReader blockReader, DatanodeInfo currentNode, ByteBufferStrategy strategy, - ExtendedBlock currentBlock) throws IOException { + LocatedBlock currentBlock, int chunkIndex, long offsetInBlock) + throws IOException { final int targetLength = strategy.getTargetLength(); -int length = 0; -try { - while (length < targetLength) { -int ret = strategy.readFromBlock(blockReader); -if (ret < 0) { - throw new IOException("Unexpected EOS from the reader"); +int curAttempts = 0; +while (curAttempts < readDNMaxAttempts) { Review Comment: here update `while (true)` and can remove line[286~288], how about it ? > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: > datano
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17838146#comment-17838146 ] ASF GitHub Bot commented on HDFS-15413: --- haiyang1987 commented on code in PR #5829: URL: https://github.com/apache/hadoop/pull/5829#discussion_r1568748476 ## hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripeReader.java: ## @@ -233,41 +235,63 @@ private ByteBufferStrategy[] getReadStrategies(StripingChunk chunk) { private int readToBuffer(BlockReader blockReader, DatanodeInfo currentNode, ByteBufferStrategy strategy, - ExtendedBlock currentBlock) throws IOException { + LocatedBlock currentBlock, int chunkIndex, long offsetInBlock) + throws IOException { final int targetLength = strategy.getTargetLength(); -int length = 0; -try { - while (length < targetLength) { -int ret = strategy.readFromBlock(blockReader); -if (ret < 0) { - throw new IOException("Unexpected EOS from the reader"); +int curAttempts = 0; +while (curAttempts < readDNMaxAttempts) { + curAttempts++; + int length = 0; + try { +while (length < targetLength) { + int ret = strategy.readFromBlock(blockReader); + if (ret < 0) { +throw new IOException("Unexpected EOS from the reader"); + } + length += ret; +} +return length; + } catch (ChecksumException ce) { +DFSClient.LOG.warn("Found Checksum error for " Review Comment: here also > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (just below 1MB which is t
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17838261#comment-17838261 ] ASF GitHub Bot commented on HDFS-15413: --- hadoop-yetus commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-2061596554 :broken_heart: **-1 overall** | Vote | Subsystem | Runtime | Logfile | Comment | |::|--:|:|::|:---:| | +0 :ok: | reexec | 0m 21s | | Docker mode activated. | _ Prechecks _ | | +1 :green_heart: | dupname | 0m 0s | | No case conflicting files found. | | +0 :ok: | codespell | 0m 0s | | codespell was not available. | | +0 :ok: | detsecrets | 0m 0s | | detect-secrets was not available. | | +0 :ok: | xmllint | 0m 0s | | xmllint was not available. | | +1 :green_heart: | @author | 0m 0s | | The patch does not contain any @author tags. | | +1 :green_heart: | test4tests | 0m 0s | | The patch appears to include 1 new or modified test files. | _ trunk Compile Tests _ | | +0 :ok: | mvndep | 13m 56s | | Maven dependency ordering for branch | | +1 :green_heart: | mvninstall | 20m 11s | | trunk passed | | +1 :green_heart: | compile | 2m 56s | | trunk passed with JDK Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1 | | +1 :green_heart: | compile | 2m 53s | | trunk passed with JDK Private Build-1.8.0_402-8u402-ga-2ubuntu1~20.04-b06 | | +1 :green_heart: | checkstyle | 0m 47s | | trunk passed | | +1 :green_heart: | mvnsite | 1m 24s | | trunk passed | | +1 :green_heart: | javadoc | 1m 9s | | trunk passed with JDK Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1 | | +1 :green_heart: | javadoc | 1m 31s | | trunk passed with JDK Private Build-1.8.0_402-8u402-ga-2ubuntu1~20.04-b06 | | +1 :green_heart: | spotbugs | 3m 17s | | trunk passed | | +1 :green_heart: | shadedclient | 21m 19s | | branch has no errors when building and testing our client artifacts. | | -0 :warning: | patch | 21m 32s | | Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary. | _ Patch Compile Tests _ | | +0 :ok: | mvndep | 0m 21s | | Maven dependency ordering for patch | | +1 :green_heart: | mvninstall | 1m 3s | | the patch passed | | +1 :green_heart: | compile | 2m 51s | | the patch passed with JDK Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1 | | +1 :green_heart: | javac | 2m 51s | | the patch passed | | +1 :green_heart: | compile | 2m 44s | | the patch passed with JDK Private Build-1.8.0_402-8u402-ga-2ubuntu1~20.04-b06 | | +1 :green_heart: | javac | 2m 44s | | the patch passed | | -1 :x: | blanks | 0m 0s | [/blanks-eol.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/8/artifact/out/blanks-eol.txt) | The patch has 1 line(s) that end in blanks. Use git apply --whitespace=fix <>. Refer https://git-scm.com/docs/git-apply | | -0 :warning: | checkstyle | 0m 34s | [/results-checkstyle-hadoop-hdfs-project.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/8/artifact/out/results-checkstyle-hadoop-hdfs-project.txt) | hadoop-hdfs-project: The patch generated 1 new + 45 unchanged - 0 fixed = 46 total (was 45) | | +1 :green_heart: | mvnsite | 1m 8s | | the patch passed | | +1 :green_heart: | javadoc | 0m 54s | | the patch passed with JDK Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1 | | +1 :green_heart: | javadoc | 1m 28s | | the patch passed with JDK Private Build-1.8.0_402-8u402-ga-2ubuntu1~20.04-b06 | | +1 :green_heart: | spotbugs | 3m 10s | | the patch passed | | +1 :green_heart: | shadedclient | 21m 13s | | patch has no errors when building and testing our client artifacts. | _ Other Tests _ | | +1 :green_heart: | unit | 1m 54s | | hadoop-hdfs-client in the patch passed. | | -1 :x: | unit | 209m 14s | [/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/8/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt) | hadoop-hdfs in the patch passed. | | +1 :green_heart: | asflicense | 0m 30s | | The patch does not generate ASF License warnings. | | | | 318m 25s | | | | Reason | Tests | |---:|:--| | Failed junit tests | hadoop.hdfs.TestDFSStripedInputStreamWithTimeout | | | hadoop.hdfs.server.datanode.TestDirectoryScanner | | Subsystem | Report/Notes | |--:|:-| | Docker | ClientAPI=1.45 ServerAPI=1.45 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/8/artifact/out/Dockerfile | | GITHUB
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17838419#comment-17838419 ] ASF GitHub Bot commented on HDFS-15413: --- haiyang1987 commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-2062844297 The UT `hadoop.hdfs.TestDFSStripedInputStreamWithTimeout ` run failed. > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: 1 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: 1 millis timeout > while waiting for channel
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17838993#comment-17838993 ] ASF GitHub Bot commented on HDFS-15413: --- hadoop-yetus commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-2066527804 :broken_heart: **-1 overall** | Vote | Subsystem | Runtime | Logfile | Comment | |::|--:|:|::|:---:| | +0 :ok: | reexec | 0m 21s | | Docker mode activated. | _ Prechecks _ | | +1 :green_heart: | dupname | 0m 0s | | No case conflicting files found. | | +0 :ok: | codespell | 0m 0s | | codespell was not available. | | +0 :ok: | detsecrets | 0m 0s | | detect-secrets was not available. | | +0 :ok: | xmllint | 0m 0s | | xmllint was not available. | | +1 :green_heart: | @author | 0m 0s | | The patch does not contain any @author tags. | | +1 :green_heart: | test4tests | 0m 0s | | The patch appears to include 1 new or modified test files. | _ trunk Compile Tests _ | | +0 :ok: | mvndep | 14m 0s | | Maven dependency ordering for branch | | +1 :green_heart: | mvninstall | 19m 50s | | trunk passed | | +1 :green_heart: | compile | 2m 58s | | trunk passed with JDK Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1 | | +1 :green_heart: | compile | 2m 51s | | trunk passed with JDK Private Build-1.8.0_402-8u402-ga-2ubuntu1~20.04-b06 | | +1 :green_heart: | checkstyle | 0m 48s | | trunk passed | | +1 :green_heart: | mvnsite | 1m 21s | | trunk passed | | +1 :green_heart: | javadoc | 1m 12s | | trunk passed with JDK Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1 | | +1 :green_heart: | javadoc | 1m 34s | | trunk passed with JDK Private Build-1.8.0_402-8u402-ga-2ubuntu1~20.04-b06 | | +1 :green_heart: | spotbugs | 3m 10s | | trunk passed | | +1 :green_heart: | shadedclient | 21m 16s | | branch has no errors when building and testing our client artifacts. | | -0 :warning: | patch | 21m 30s | | Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary. | _ Patch Compile Tests _ | | +0 :ok: | mvndep | 0m 21s | | Maven dependency ordering for patch | | +1 :green_heart: | mvninstall | 1m 7s | | the patch passed | | +1 :green_heart: | compile | 2m 52s | | the patch passed with JDK Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1 | | +1 :green_heart: | javac | 2m 52s | | the patch passed | | +1 :green_heart: | compile | 2m 49s | | the patch passed with JDK Private Build-1.8.0_402-8u402-ga-2ubuntu1~20.04-b06 | | +1 :green_heart: | javac | 2m 49s | | the patch passed | | -1 :x: | blanks | 0m 0s | [/blanks-eol.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/9/artifact/out/blanks-eol.txt) | The patch has 1 line(s) that end in blanks. Use git apply --whitespace=fix <>. Refer https://git-scm.com/docs/git-apply | | -0 :warning: | checkstyle | 0m 37s | [/results-checkstyle-hadoop-hdfs-project.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/9/artifact/out/results-checkstyle-hadoop-hdfs-project.txt) | hadoop-hdfs-project: The patch generated 1 new + 45 unchanged - 0 fixed = 46 total (was 45) | | +1 :green_heart: | mvnsite | 1m 6s | | the patch passed | | +1 :green_heart: | javadoc | 0m 54s | | the patch passed with JDK Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1 | | +1 :green_heart: | javadoc | 1m 26s | | the patch passed with JDK Private Build-1.8.0_402-8u402-ga-2ubuntu1~20.04-b06 | | +1 :green_heart: | spotbugs | 3m 12s | | the patch passed | | +1 :green_heart: | shadedclient | 21m 22s | | patch has no errors when building and testing our client artifacts. | _ Other Tests _ | | +1 :green_heart: | unit | 1m 51s | | hadoop-hdfs-client in the patch passed. | | -1 :x: | unit | 211m 45s | [/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/9/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt) | hadoop-hdfs in the patch passed. | | +1 :green_heart: | asflicense | 0m 31s | | The patch does not generate ASF License warnings. | | | | 321m 9s | | | | Reason | Tests | |---:|:--| | Failed junit tests | hadoop.hdfs.TestDFSStripedInputStreamWithTimeout | | Subsystem | Report/Notes | |--:|:-| | Docker | ClientAPI=1.45 ServerAPI=1.45 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/9/artifact/out/Dockerfile | | GITHUB PR | https://github.com/apache/hadoop/pull/5829 | | Opti
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17838995#comment-17838995 ] ASF GitHub Bot commented on HDFS-15413: --- Neilxzn commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-2066542689 > The UT `hadoop.hdfs.TestDFSStripedInputStreamWithTimeout ` run failed. add Datanode$closeDataXceiverServer method to close connnect for testing. > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: 1 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.ne
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17839081#comment-17839081 ] ASF GitHub Bot commented on HDFS-15413: --- hadoop-yetus commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-2067076767 :broken_heart: **-1 overall** | Vote | Subsystem | Runtime | Logfile | Comment | |::|--:|:|::|:---:| | +0 :ok: | reexec | 0m 22s | | Docker mode activated. | _ Prechecks _ | | +1 :green_heart: | dupname | 0m 0s | | No case conflicting files found. | | +0 :ok: | codespell | 0m 1s | | codespell was not available. | | +0 :ok: | detsecrets | 0m 1s | | detect-secrets was not available. | | +0 :ok: | xmllint | 0m 1s | | xmllint was not available. | | +1 :green_heart: | @author | 0m 0s | | The patch does not contain any @author tags. | | +1 :green_heart: | test4tests | 0m 0s | | The patch appears to include 1 new or modified test files. | _ trunk Compile Tests _ | | +0 :ok: | mvndep | 14m 14s | | Maven dependency ordering for branch | | +1 :green_heart: | mvninstall | 20m 8s | | trunk passed | | +1 :green_heart: | compile | 2m 58s | | trunk passed with JDK Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1 | | +1 :green_heart: | compile | 2m 49s | | trunk passed with JDK Private Build-1.8.0_402-8u402-ga-2ubuntu1~20.04-b06 | | +1 :green_heart: | checkstyle | 0m 46s | | trunk passed | | +1 :green_heart: | mvnsite | 1m 22s | | trunk passed | | +1 :green_heart: | javadoc | 1m 8s | | trunk passed with JDK Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1 | | +1 :green_heart: | javadoc | 1m 35s | | trunk passed with JDK Private Build-1.8.0_402-8u402-ga-2ubuntu1~20.04-b06 | | +1 :green_heart: | spotbugs | 3m 10s | | trunk passed | | +1 :green_heart: | shadedclient | 21m 22s | | branch has no errors when building and testing our client artifacts. | | -0 :warning: | patch | 21m 36s | | Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary. | _ Patch Compile Tests _ | | +0 :ok: | mvndep | 0m 22s | | Maven dependency ordering for patch | | +1 :green_heart: | mvninstall | 1m 7s | | the patch passed | | +1 :green_heart: | compile | 2m 52s | | the patch passed with JDK Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1 | | +1 :green_heart: | javac | 2m 52s | | the patch passed | | +1 :green_heart: | compile | 2m 43s | | the patch passed with JDK Private Build-1.8.0_402-8u402-ga-2ubuntu1~20.04-b06 | | +1 :green_heart: | javac | 2m 43s | | the patch passed | | -1 :x: | blanks | 0m 0s | [/blanks-eol.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/10/artifact/out/blanks-eol.txt) | The patch has 1 line(s) that end in blanks. Use git apply --whitespace=fix <>. Refer https://git-scm.com/docs/git-apply | | -0 :warning: | checkstyle | 0m 38s | [/results-checkstyle-hadoop-hdfs-project.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/10/artifact/out/results-checkstyle-hadoop-hdfs-project.txt) | hadoop-hdfs-project: The patch generated 2 new + 113 unchanged - 0 fixed = 115 total (was 113) | | +1 :green_heart: | mvnsite | 1m 9s | | the patch passed | | +1 :green_heart: | javadoc | 0m 55s | | the patch passed with JDK Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1 | | +1 :green_heart: | javadoc | 1m 27s | | the patch passed with JDK Private Build-1.8.0_402-8u402-ga-2ubuntu1~20.04-b06 | | +1 :green_heart: | spotbugs | 3m 8s | | the patch passed | | +1 :green_heart: | shadedclient | 23m 50s | | patch has no errors when building and testing our client artifacts. | _ Other Tests _ | | +1 :green_heart: | unit | 1m 51s | | hadoop-hdfs-client in the patch passed. | | -1 :x: | unit | 212m 5s | [/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/10/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt) | hadoop-hdfs in the patch passed. | | +1 :green_heart: | asflicense | 0m 29s | | The patch does not generate ASF License warnings. | | | | 325m 11s | | | | Reason | Tests | |---:|:--| | Failed junit tests | hadoop.hdfs.server.datanode.TestDirectoryScanner | | Subsystem | Report/Notes | |--:|:-| | Docker | ClientAPI=1.45 ServerAPI=1.45 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/10/artifact/out/Dockerfile | | GITHUB PR | https://github.com/apache/hadoop/pull/5829 |
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17839189#comment-17839189 ] ASF GitHub Bot commented on HDFS-15413: --- haiyang1987 commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-2067551100 please fix checkstyle, thanks~ > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: 1 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: 1 millis timeout > while waiting for channel to be ready for write. ch : > java.nio.cha
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17839442#comment-17839442 ] ASF GitHub Bot commented on HDFS-15413: --- Neilxzn commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-2068331088 ``` ./hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java:171: public DfsClientConf(Configuration conf) {:3: Method length is 156 lines (max allowed is 150). [MethodLength] ``` Should we suppress this checkstyle warning? Or are there any better suggestions? > 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 = 1 > - dfs.datanode.socket.write.timeout = 1 >Reporter: Andrey Elenskiy >Priority: Critical > Labels: pull-request-available > 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 = 100 (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: 1 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
[jira] [Commented] (HDFS-15413) DFSStripedInputStream throws exception when datanodes close idle connections
[ https://issues.apache.org/jira/browse/HDFS-15413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17839517#comment-17839517 ] ASF GitHub Bot commented on HDFS-15413: --- hadoop-yetus commented on PR #5829: URL: https://github.com/apache/hadoop/pull/5829#issuecomment-2068640885 :confetti_ball: **+1 overall** | Vote | Subsystem | Runtime | Logfile | Comment | |::|--:|:|::|:---:| | +0 :ok: | reexec | 6m 49s | | Docker mode activated. | _ Prechecks _ | | +1 :green_heart: | dupname | 0m 1s | | No case conflicting files found. | | +0 :ok: | codespell | 0m 0s | | codespell was not available. | | +0 :ok: | detsecrets | 0m 0s | | detect-secrets was not available. | | +0 :ok: | xmllint | 0m 0s | | xmllint was not available. | | +1 :green_heart: | @author | 0m 0s | | The patch does not contain any @author tags. | | +1 :green_heart: | test4tests | 0m 0s | | The patch appears to include 1 new or modified test files. | _ trunk Compile Tests _ | | +0 :ok: | mvndep | 14m 4s | | Maven dependency ordering for branch | | +1 :green_heart: | mvninstall | 19m 54s | | trunk passed | | +1 :green_heart: | compile | 3m 0s | | trunk passed with JDK Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1 | | +1 :green_heart: | compile | 2m 48s | | trunk passed with JDK Private Build-1.8.0_402-8u402-ga-2ubuntu1~20.04-b06 | | +1 :green_heart: | checkstyle | 0m 47s | | trunk passed | | +1 :green_heart: | mvnsite | 1m 18s | | trunk passed | | +1 :green_heart: | javadoc | 1m 8s | | trunk passed with JDK Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1 | | +1 :green_heart: | javadoc | 1m 37s | | trunk passed with JDK Private Build-1.8.0_402-8u402-ga-2ubuntu1~20.04-b06 | | +1 :green_heart: | spotbugs | 3m 2s | | trunk passed | | +1 :green_heart: | shadedclient | 20m 53s | | branch has no errors when building and testing our client artifacts. | | -0 :warning: | patch | 21m 7s | | Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary. | _ Patch Compile Tests _ | | +0 :ok: | mvndep | 0m 22s | | Maven dependency ordering for patch | | +1 :green_heart: | mvninstall | 1m 4s | | the patch passed | | +1 :green_heart: | compile | 2m 50s | | the patch passed with JDK Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1 | | +1 :green_heart: | javac | 2m 50s | | the patch passed | | +1 :green_heart: | compile | 2m 51s | | the patch passed with JDK Private Build-1.8.0_402-8u402-ga-2ubuntu1~20.04-b06 | | +1 :green_heart: | javac | 2m 51s | | the patch passed | | +1 :green_heart: | blanks | 0m 0s | | The patch has no blanks issues. | | -0 :warning: | checkstyle | 0m 38s | [/results-checkstyle-hadoop-hdfs-project.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/11/artifact/out/results-checkstyle-hadoop-hdfs-project.txt) | hadoop-hdfs-project: The patch generated 1 new + 113 unchanged - 0 fixed = 114 total (was 113) | | +1 :green_heart: | mvnsite | 1m 8s | | the patch passed | | +1 :green_heart: | javadoc | 0m 55s | | the patch passed with JDK Ubuntu-11.0.22+7-post-Ubuntu-0ubuntu220.04.1 | | +1 :green_heart: | javadoc | 1m 27s | | the patch passed with JDK Private Build-1.8.0_402-8u402-ga-2ubuntu1~20.04-b06 | | +1 :green_heart: | spotbugs | 3m 8s | | the patch passed | | +1 :green_heart: | shadedclient | 21m 5s | | patch has no errors when building and testing our client artifacts. | _ Other Tests _ | | +1 :green_heart: | unit | 1m 49s | | hadoop-hdfs-client in the patch passed. | | +1 :green_heart: | unit | 214m 41s | | hadoop-hdfs in the patch passed. | | +1 :green_heart: | asflicense | 0m 26s | | The patch does not generate ASF License warnings. | | | | 329m 18s | | | | Subsystem | Report/Notes | |--:|:-| | Docker | ClientAPI=1.45 ServerAPI=1.45 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5829/11/artifact/out/Dockerfile | | GITHUB PR | https://github.com/apache/hadoop/pull/5829 | | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets xmllint | | uname | Linux f0536dd0a688 5.15.0-94-generic #104-Ubuntu SMP Tue Jan 9 15:25:40 UTC 2024 x86_64 x86_64 x86_64 GNU/Linux | | Build tool | maven | | Personality | dev-support/bin/hadoop.sh | | git revision | trunk / d698a9445b4a06fd8978ee4c5005964270d236d9 | | Default Java | Private Build-1.8.0_402-8u402-ga-2ubuntu1~20.04