[jira] [Commented] (HADOOP-17052) NetUtils.connect() throws an exception the prevents any retries when hostname resolution fails
[ https://issues.apache.org/jira/browse/HADOOP-17052?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17119913#comment-17119913 ] Dhiraj Hegde commented on HADOOP-17052: --- Sorry for the confusion on the retry issue, I did not mean DNS needs to be retried, what I meant was all the node retry logic in hdfs client is failing in this situation. Let me clarify the problem here: NetUtils.connect() throws this unchecked exception java.nio.channels.UnresolvedAddressException. This causes the error to bubble up all the way out of the hdfs client. This means that while normally hdfs client code would catch IOException and respond to it by trying a different node, but in this case that process is aborted. By making this unchecked exception to be java.net.UnknownHostException we make sure that the hdfs code will catch the exception and work as designed. When request to one node fails it will catch the exception and try another. > NetUtils.connect() throws an exception the prevents any retries when hostname > resolution fails > -- > > Key: HADOOP-17052 > URL: https://issues.apache.org/jira/browse/HADOOP-17052 > Project: Hadoop Common > Issue Type: Bug > Components: hdfs-client >Affects Versions: 2.10.0, 2.9.2, 3.2.1, 3.1.3 >Reporter: Dhiraj Hegde >Assignee: Dhiraj Hegde >Priority: Major > Attachments: stack_trace2 > > > Hadoop components are increasingly being deployed on VMs and containers. One > aspect of this environment is that DNS is dynamic. Hostname records get > modified (or deleted/recreated) as a container in Kubernetes (or even VM) is > being created/recreated. In such dynamic environments, the initial DNS > resolution request might return resolution failure briefly as DNS client > doesn't always get the latest records. This has been observed in Kubernetes > in particular. In such cases NetUtils.connect() appears to throw > java.nio.channels.UnresolvedAddressException. In much of Hadoop code (like > DFSInputStream and DFSOutputStream), the code is designed to retry > IOException. However, since UnresolvedAddressException is not child of > IOException, no retry happens and the code aborts immediately. It is much > better if NetUtils.connect() throws java.net.UnknownHostException as that is > derived from IOException and the code will treat this as a retry-able error. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-17052) NetUtils.connect() throws an exception the prevents any retries when hostname resolution fails
[ https://issues.apache.org/jira/browse/HADOOP-17052?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17119393#comment-17119393 ] Mingliang Liu commented on HADOOP-17052: [~dhegde] Thanks for reporting and providing a patch. # The two stack trace you posted are both for read. If you have a write pipeline stack (e.g. DataStreamer as mentioned by [~hemanthboyina]), that will be useful here for reference. # The idea you two have figured looks good to me. I have approved the PR. If there is no more objections, I'll commit it in 3 days. I think it is fine and actually better to retry on one peer in case of generic host unresolvable problem, be it unchecked UnresolvedAddressException or checked UnknownHostException (IOE). # [~dhegde] It is highly recommended to make the JIRA title and PR subject the same. You can edit either places since they both look good to me. > NetUtils.connect() throws an exception the prevents any retries when hostname > resolution fails > -- > > Key: HADOOP-17052 > URL: https://issues.apache.org/jira/browse/HADOOP-17052 > Project: Hadoop Common > Issue Type: Bug > Components: hdfs-client >Affects Versions: 2.10.0, 2.9.2, 3.2.1, 3.1.3 >Reporter: Dhiraj Hegde >Assignee: Dhiraj Hegde >Priority: Major > Attachments: stack_trace2 > > > Hadoop components are increasingly being deployed on VMs and containers. One > aspect of this environment is that DNS is dynamic. Hostname records get > modified (or deleted/recreated) as a container in Kubernetes (or even VM) is > being created/recreated. In such dynamic environments, the initial DNS > resolution request might return resolution failure briefly as DNS client > doesn't always get the latest records. This has been observed in Kubernetes > in particular. In such cases NetUtils.connect() appears to throw > java.nio.channels.UnresolvedAddressException. In much of Hadoop code (like > DFSInputStream and DFSOutputStream), the code is designed to retry > IOException. However, since UnresolvedAddressException is not child of > IOException, no retry happens and the code aborts immediately. It is much > better if NetUtils.connect() throws java.net.UnknownHostException as that is > derived from IOException and the code will treat this as a retry-able error. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-17052) NetUtils.connect() throws an exception the prevents any retries when hostname resolution fails
[ https://issues.apache.org/jira/browse/HADOOP-17052?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17119305#comment-17119305 ] hemanthboyina commented on HADOOP-17052: {quote}In such dynamic environments, the initial DNS resolution request might return resolution failure briefly as DNS client doesn't always get the latest records. {quote} the fix might solve the problem but in a normal scenario when we expect or get UnresolvedAddressException , it is not required to retry , because even with retry we doesn't solve the unresolved address problem > NetUtils.connect() throws an exception the prevents any retries when hostname > resolution fails > -- > > Key: HADOOP-17052 > URL: https://issues.apache.org/jira/browse/HADOOP-17052 > Project: Hadoop Common > Issue Type: Bug > Components: hdfs-client >Affects Versions: 2.10.0, 2.9.2, 3.2.1, 3.1.3 >Reporter: Dhiraj Hegde >Assignee: Dhiraj Hegde >Priority: Major > Attachments: stack_trace2 > > > Hadoop components are increasingly being deployed on VMs and containers. One > aspect of this environment is that DNS is dynamic. Hostname records get > modified (or deleted/recreated) as a container in Kubernetes (or even VM) is > being created/recreated. In such dynamic environments, the initial DNS > resolution request might return resolution failure briefly as DNS client > doesn't always get the latest records. This has been observed in Kubernetes > in particular. In such cases NetUtils.connect() appears to throw > java.nio.channels.UnresolvedAddressException. In much of Hadoop code (like > DFSInputStream and DFSOutputStream), the code is designed to retry > IOException. However, since UnresolvedAddressException is not child of > IOException, no retry happens and the code aborts immediately. It is much > better if NetUtils.connect() throws java.net.UnknownHostException as that is > derived from IOException and the code will treat this as a retry-able error. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-17052) NetUtils.connect() throws an exception the prevents any retries when hostname resolution fails
[ https://issues.apache.org/jira/browse/HADOOP-17052?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17117602#comment-17117602 ] hemanthboyina commented on HADOOP-17052: thanks for providing more details [~dhegde] {quote}The code change could be made a level above in places like newConnectedPeer() {quote} i don't think this will cover the scenario of Write call IMO it will be better to handle in Netutils.connect() by catching the exception and checking if that is an instanceof , and throw the required exception to retry [~aajisaka] [~liuml07] thoughts ? > NetUtils.connect() throws an exception the prevents any retries when hostname > resolution fails > -- > > Key: HADOOP-17052 > URL: https://issues.apache.org/jira/browse/HADOOP-17052 > Project: Hadoop Common > Issue Type: Bug > Components: hdfs-client >Affects Versions: 2.10.0, 2.9.2, 3.2.1, 3.1.3 >Reporter: Dhiraj Hegde >Assignee: Dhiraj Hegde >Priority: Major > Attachments: stack_trace2 > > > Hadoop components are increasingly being deployed on VMs and containers. One > aspect of this environment is that DNS is dynamic. Hostname records get > modified (or deleted/recreated) as a container in Kubernetes (or even VM) is > being created/recreated. In such dynamic environments, the initial DNS > resolution request might return resolution failure briefly as DNS client > doesn't always get the latest records. This has been observed in Kubernetes > in particular. In such cases NetUtils.connect() appears to throw > java.nio.channels.UnresolvedAddressException. In much of Hadoop code (like > DFSInputStream and DFSOutputStream), the code is designed to retry > IOException. However, since UnresolvedAddressException is not child of > IOException, no retry happens and the code aborts immediately. It is much > better if NetUtils.connect() throws java.net.UnknownHostException as that is > derived from IOException and the code will treat this as a retry-able error. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-17052) NetUtils.connect() throws an exception the prevents any retries when hostname resolution fails
[ https://issues.apache.org/jira/browse/HADOOP-17052?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17117246#comment-17117246 ] Dhiraj Hegde commented on HADOOP-17052: --- Some additional stack trace which came from a different callstack: [^stack_trace2] > NetUtils.connect() throws an exception the prevents any retries when hostname > resolution fails > -- > > Key: HADOOP-17052 > URL: https://issues.apache.org/jira/browse/HADOOP-17052 > Project: Hadoop Common > Issue Type: Bug > Components: hdfs-client >Affects Versions: 2.10.0, 2.9.2, 3.2.1, 3.1.3 >Reporter: Dhiraj Hegde >Assignee: Dhiraj Hegde >Priority: Major > Attachments: stack_trace2 > > > Hadoop components are increasingly being deployed on VMs and containers. One > aspect of this environment is that DNS is dynamic. Hostname records get > modified (or deleted/recreated) as a container in Kubernetes (or even VM) is > being created/recreated. In such dynamic environments, the initial DNS > resolution request might return resolution failure briefly as DNS client > doesn't always get the latest records. This has been observed in Kubernetes > in particular. In such cases NetUtils.connect() appears to throw > java.nio.channels.UnresolvedAddressException. In much of Hadoop code (like > DFSInputStream and DFSOutputStream), the code is designed to retry > IOException. However, since UnresolvedAddressException is not child of > IOException, no retry happens and the code aborts immediately. It is much > better if NetUtils.connect() throws java.net.UnknownHostException as that is > derived from IOException and the code will treat this as a retry-able error. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-17052) NetUtils.connect() throws an exception the prevents any retries when hostname resolution fails
[ https://issues.apache.org/jira/browse/HADOOP-17052?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17117006#comment-17117006 ] Dhiraj Hegde commented on HADOOP-17052: --- The code change could be made a level above in places like newConnectedPeer(). My original thought was that java.net.UnknownHostException might be a better one for NetUtils.connect() to throw and one being thrown right now wasn't really an intentional decision in connect(). Let me know what you think [~hemanthboyina]. > NetUtils.connect() throws an exception the prevents any retries when hostname > resolution fails > -- > > Key: HADOOP-17052 > URL: https://issues.apache.org/jira/browse/HADOOP-17052 > Project: Hadoop Common > Issue Type: Bug > Components: hdfs-client >Affects Versions: 2.10.0, 2.9.2, 3.2.1, 3.1.3 >Reporter: Dhiraj Hegde >Assignee: Dhiraj Hegde >Priority: Major > > Hadoop components are increasingly being deployed on VMs and containers. One > aspect of this environment is that DNS is dynamic. Hostname records get > modified (or deleted/recreated) as a container in Kubernetes (or even VM) is > being created/recreated. In such dynamic environments, the initial DNS > resolution request might return resolution failure briefly as DNS client > doesn't always get the latest records. This has been observed in Kubernetes > in particular. In such cases NetUtils.connect() appears to throw > java.nio.channels.UnresolvedAddressException. In much of Hadoop code (like > DFSInputStream and DFSOutputStream), the code is designed to retry > IOException. However, since UnresolvedAddressException is not child of > IOException, no retry happens and the code aborts immediately. It is much > better if NetUtils.connect() throws java.net.UnknownHostException as that is > derived from IOException and the code will treat this as a retry-able error. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-17052) NetUtils.connect() throws an exception the prevents any retries when hostname resolution fails
[ https://issues.apache.org/jira/browse/HADOOP-17052?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17116998#comment-17116998 ] Dhiraj Hegde commented on HADOOP-17052: --- Here is one for the write pipeline: {code:java} java.nio.channels.UnresolvedAddressException at sun.nio.ch.Net.checkAddress(Net.java:101) at sun.nio.ch.SocketChannelImpl.connect(SocketChannelImpl.java:622) at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:192) at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:531) at org.apache.hadoop.hdfs.DFSClient.newConnectedPeer(DFSClient.java:3483) at org.apache.hadoop.hdfs.BlockReaderFactory.nextTcpPeer(BlockReaderFactory.java:796) at org.apache.hadoop.hdfs.BlockReaderFactory.getRemoteBlockReaderFromTcp(BlockReaderFactory.java:694) at org.apache.hadoop.hdfs.BlockReaderFactory.build(BlockReaderFactory.java:355) at org.apache.hadoop.hdfs.DFSInputStream.actualGetFromOneDataNode(DFSInputStream.java:1162) at org.apache.hadoop.hdfs.DFSInputStream.fetchBlockByteRange(DFSInputStream.java:1087) at org.apache.hadoop.hdfs.DFSInputStream.pread(DFSInputStream.java:1444) at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:1407) at org.apache.hadoop.fs.FSDataInputStream.read(FSDataInputStream.java:89) at org.apache.hadoop.hbase.io.hfile.HFileBlock.positionalReadWithExtra(HFileBlock.java:834) at org.apache.hadoop.hbase.io.hfile.HFileBlock$AbstractFSReader.readAtOffset(HFileBlock.java:1530) at org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.readBlockDataInternal(HFileBlock.java:1797) at org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.readBlockData(HFileBlock.java:1624) at org.apache.hadoop.hbase.io.hfile.HFileReaderV2.readBlock(HFileReaderV2.java:455) at org.apache.hadoop.hbase.io.hfile.HFileReaderV2$AbstractScannerV2.readNextDataBlock(HFileReaderV2.java:733) at org.apache.hadoop.hbase.io.hfile.HFileReaderV2$ScannerV2.isNextBlock(HFileReaderV2.java:858) at org.apache.hadoop.hbase.io.hfile.HFileReaderV2$ScannerV2.positionForNextBlock(HFileReaderV2.java:853) at org.apache.hadoop.hbase.io.hfile.HFileReaderV2$ScannerV2._next(HFileReaderV2.java:870) at org.apache.hadoop.hbase.io.hfile.HFileReaderV2$ScannerV2.next(HFileReaderV2.java:890) at org.apache.hadoop.hbase.regionserver.StoreFileScanner.next(StoreFileScanner.java:169) at org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:113) at org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:667) at org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:152) at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:6175) at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:6335) at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:6113) at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.next(HRegion.java:6090) at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.next(HRegion.java:6076) at org.apache.hadoop.hbase.regionserver.HRegion.get(HRegion.java:7258) at org.apache.hadoop.hbase.regionserver.HRegion.get(HRegion.java:7216) at org.apache.hadoop.hbase.regionserver.HRegion.get(HRegion.java:7209) at org.apache.hadoop.hbase.regionserver.RSRpcServices.get(RSRpcServices.java:2273) at org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:35068) at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2417) at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:124) at org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:186) at org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:166) {code} > NetUtils.connect() throws an exception the prevents any retries when hostname > resolution fails > -- > > Key: HADOOP-17052 > URL: https://issues.apache.org/jira/browse/HADOOP-17052 > Project: Hadoop Common > Issue Type: Bug > Components: hdfs-client >Affects Versions: 2.10.0, 2.9.2, 3.2.1, 3.1.3 >Reporter: Dhiraj Hegde >Assignee: Dhiraj Hegde >Priority: Major > > Hadoop components are increasingly being deployed on VMs and containers. One > aspect of this environment is that DNS is dynamic. Hostname records get > modified (or deleted/recreated) as a container in Kubernetes (or even VM) is > being created/recreated. In
[jira] [Commented] (HADOOP-17052) NetUtils.connect() throws an exception the prevents any retries when hostname resolution fails
[ https://issues.apache.org/jira/browse/HADOOP-17052?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17116992#comment-17116992 ] Dhiraj Hegde commented on HADOOP-17052: --- [~hemanthboyina] Here is a stack trace of read failing due to the exception thrown: Caused by: java.nio.channels.UnresolvedAddressException at sun.nio.ch.Net.checkAddress(Net.java:101) at sun.nio.ch.SocketChannelImpl.connect(SocketChannelImpl.java:622) at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:192) at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:531) at org.apache.hadoop.hdfs.DFSClient.newConnectedPeer(DFSClient.java:3363) at org.apache.hadoop.hdfs.BlockReaderFactory.nextTcpPeer(BlockReaderFactory.java:817) at org.apache.hadoop.hdfs.BlockReaderFactory.getRemoteBlockReaderFromTcp(BlockReaderFactory.java:710) at org.apache.hadoop.hdfs.BlockReaderFactory.build(BlockReaderFactory.java:356) at org.apache.hadoop.hdfs.DFSInputStream.blockSeekTo(DFSInputStream.java:666) at org.apache.hadoop.hdfs.DFSInputStream.readWithStrategy(DFSInputStream.java:875) at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:927) ... FATAL [MemStoreFlusher.0] regionserver.HRegionServer - ABORTING region server > NetUtils.connect() throws an exception the prevents any retries when hostname > resolution fails > -- > > Key: HADOOP-17052 > URL: https://issues.apache.org/jira/browse/HADOOP-17052 > Project: Hadoop Common > Issue Type: Bug > Components: hdfs-client >Affects Versions: 2.10.0, 2.9.2, 3.2.1, 3.1.3 >Reporter: Dhiraj Hegde >Assignee: Dhiraj Hegde >Priority: Major > > Hadoop components are increasingly being deployed on VMs and containers. One > aspect of this environment is that DNS is dynamic. Hostname records get > modified (or deleted/recreated) as a container in Kubernetes (or even VM) is > being created/recreated. In such dynamic environments, the initial DNS > resolution request might return resolution failure briefly as DNS client > doesn't always get the latest records. This has been observed in Kubernetes > in particular. In such cases NetUtils.connect() appears to throw > java.nio.channels.UnresolvedAddressException. In much of Hadoop code (like > DFSInputStream and DFSOutputStream), the code is designed to retry > IOException. However, since UnresolvedAddressException is not child of > IOException, no retry happens and the code aborts immediately. It is much > better if NetUtils.connect() throws java.net.UnknownHostException as that is > derived from IOException and the code will treat this as a retry-able error. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-17052) NetUtils.connect() throws an exception the prevents any retries when hostname resolution fails
[ https://issues.apache.org/jira/browse/HADOOP-17052?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17116957#comment-17116957 ] hemanthboyina commented on HADOOP-17052: Thanks for reporting [~dhegde] do you have any stack trace of the exception ? An Example : In DataStreamer#createSocketForPipeline we call NetUtils.connect() {code:java} final InetSocketAddress isa = NetUtils.createSocketAddr(dnAddr); final Socket sock = client.socketFactory.createSocket(); final int timeout = client.getDatanodeReadTimeout(length); NetUtils.connect(sock, isa, client.getRandomLocalInterfaceAddr(), conf.getSocketTimeout()); {code} we resolve the DN address and calls the NetUtils.connect() , is there any chance of getting reported scenario in this part of code ? > NetUtils.connect() throws an exception the prevents any retries when hostname > resolution fails > -- > > Key: HADOOP-17052 > URL: https://issues.apache.org/jira/browse/HADOOP-17052 > Project: Hadoop Common > Issue Type: Bug > Components: hdfs-client >Affects Versions: 2.10.0, 2.9.2, 3.2.1, 3.1.3 >Reporter: Dhiraj Hegde >Assignee: Dhiraj Hegde >Priority: Major > > Hadoop components are increasingly being deployed on VMs and containers. One > aspect of this environment is that DNS is dynamic. Hostname records get > modified (or deleted/recreated) as a container in Kubernetes (or even VM) is > being created/recreated. In such dynamic environments, the initial DNS > resolution request might return resolution failure briefly as DNS client > doesn't always get the latest records. This has been observed in Kubernetes > in particular. In such cases NetUtils.connect() appears to throw > java.nio.channels.UnresolvedAddressException. In much of Hadoop code (like > DFSInputStream and DFSOutputStream), the code is designed to retry > IOException. However, since UnresolvedAddressException is not child of > IOException, no retry happens and the code aborts immediately. It is much > better if NetUtils.connect() throws java.net.UnknownHostException as that is > derived from IOException and the code will treat this as a retry-able error. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org