|
Varun Sharma
2012-11-07, 09:43
Nicolas Liochon
2012-11-07, 09:56
Jeremy Carroll
2012-11-07, 15:22
Jeremy Carroll
2012-11-07, 15:25
Varun Sharma
2012-11-07, 17:57
David Charle
2012-11-07, 18:21
Jeremy Carroll
2012-11-07, 19:52
Jeremy Carroll
2012-11-07, 19:53
Varun Sharma
2012-11-07, 21:52
|
-
Terribly long HDFS timeouts while appending to HLogVarun Sharma 2012-11-07, 09:43
Hi,
I am seeing extremely long HDFS timeouts - and this seems to be associated with the loss of a DataNode. Here is the RS log: 12/11/07 02:17:45 WARN hdfs.DFSClient: DFSOutputStream ResponseProcessor exception for block blk_2813460962462751946_78454java.io.IOException: Bad response 1 for block blk_2813460962462751946_78454 from datanode 10.31.190.107:9200 at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:3084) 12/11/07 02:17:45 WARN hdfs.DFSClient: Error Recovery for block blk_2813460962462751946_78454 bad datanode[1] 10.31.190.107:9200 12/11/07 02:17:45 WARN hdfs.DFSClient: Error Recovery for block blk_2813460962462751946_78454 in pipeline 10.31.138.245:9200, 10.31.190.107:9200, 10.159.19.90:9200: bad datanode 10.31.190.107:9200 12/11/07 02:17:45 WARN wal.HLog: IPC Server handler 35 on 60020 took 65955 ms appending an edit to hlog; editcount=476686, len~=76.0 12/11/07 02:17:45 WARN wal.HLog: HDFS pipeline error detected. Found 2 replicas but expecting no less than 3 replicas. Requesting close of hlog. The corresponding DN log goes like this 2012-11-07 02:17:45,142 INFO org.apache.hadoop.hdfs.server.datanode.DataNode (PacketResponder 2 for Block blk_2813460962462751946_78454): PacketResponder blk_2813460962462751946_78454 2 Exception java.net.SocketTimeoutException: 66000 millis timeout while waiting for channel to be ready for read. ch : java.nio.channels.SocketChannel[connected local=/10.31.138.245:33965remote=/ 10.31.190.107:9200] at org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:164) at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:155) at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:128) at java.io.DataInputStream.readFully(DataInputStream.java:178) at java.io.DataInputStream.readLong(DataInputStream.java:399) at org.apache.hadoop.hdfs.protocol.DataTransferProtocol$PipelineAck.readFields(DataTransferProtocol.java:124) at org.apache.hadoop.hdfs.server.datanode.BlockReceiver$PacketResponder.run(BlockReceiver.java:806) at java.lang.Thread.run(Thread.java:662) It seems like the DataNode local to the region server is trying to grab the block from another DN and that is timing out because of this other data node being bad. All in all this causes response times to be terribly poor. Is there a way around this or am I missing something ? Varun
-
Re: Terribly long HDFS timeouts while appending to HLogNicolas Liochon 2012-11-07, 09:56
Hi Varun,
HDFS-3703 and HDFS-3912 are about this. The story is not over yet (and there are other stuff like HDFS-3704, HDFS-3705, HDFS-3706), but it helps by lowering the probability to go to a dead datanode: hdfs waits 10 minutes before deciding a datanode is dead, with the jiras mentionned above, after 30s (configurable), the non responding datanodes are not used for writes, and set to least priorities for reads. Cheers, Nicolas On Wed, Nov 7, 2012 at 10:43 AM, Varun Sharma <[EMAIL PROTECTED]> wrote: > Hi, > > I am seeing extremely long HDFS timeouts - and this seems to be associated > with the loss of a DataNode. Here is the RS log: > > 12/11/07 02:17:45 WARN hdfs.DFSClient: DFSOutputStream ResponseProcessor > exception for block blk_2813460962462751946_78454java.io.IOException: Bad > response 1 for block blk_2813460962462751946_78454 from datanode > 10.31.190.107:9200 > at > > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:3084) > > 12/11/07 02:17:45 WARN hdfs.DFSClient: Error Recovery for block > blk_2813460962462751946_78454 bad datanode[1] 10.31.190.107:9200 > 12/11/07 02:17:45 WARN hdfs.DFSClient: Error Recovery for block > blk_2813460962462751946_78454 in pipeline 10.31.138.245:9200, > 10.31.190.107:9200, 10.159.19.90:9200: bad datanode 10.31.190.107:9200 > 12/11/07 02:17:45 WARN wal.HLog: IPC Server handler 35 on 60020 took 65955 > ms appending an edit to hlog; editcount=476686, len~=76.0 > 12/11/07 02:17:45 WARN wal.HLog: HDFS pipeline error detected. Found 2 > replicas but expecting no less than 3 replicas. Requesting close of hlog. > > The corresponding DN log goes like this > > 2012-11-07 02:17:45,142 INFO > org.apache.hadoop.hdfs.server.datanode.DataNode (PacketResponder 2 for > Block blk_2813460962462751946_78454): PacketResponder > blk_2813460962462751946_78454 2 Exception java.net.SocketTimeoutException: > 66000 millis timeout while waiting for channel to be ready for read. ch : > java.nio.channels.SocketChannel[connected local=/10.31.138.245:33965 > remote=/ > 10.31.190.107:9200] > at > > org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:164) > at > org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:155) > at > org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:128) > at java.io.DataInputStream.readFully(DataInputStream.java:178) > at java.io.DataInputStream.readLong(DataInputStream.java:399) > at > > org.apache.hadoop.hdfs.protocol.DataTransferProtocol$PipelineAck.readFields(DataTransferProtocol.java:124) > at > > org.apache.hadoop.hdfs.server.datanode.BlockReceiver$PacketResponder.run(BlockReceiver.java:806) > at java.lang.Thread.run(Thread.java:662) > > It seems like the DataNode local to the region server is trying to grab the > block from another DN and that is timing out because of this other data > node being bad. All in all this causes response times to be terribly poor. > Is there a way around this or am I missing something ? > > Varun >
-
Re: Terribly long HDFS timeouts while appending to HLogJeremy Carroll 2012-11-07, 15:22
One trick I have used for a while is to
set dfs.datanode.socket.write.timeout in hdfs-site.xml to 0 (disabled). It's not going to solve your underlying IOPS capacity issue with your servers, but it can help for short bursty periods. Basically it's hiding the real issue, but it can help in the short term. On Wed, Nov 7, 2012 at 1:43 AM, Varun Sharma <[EMAIL PROTECTED]> wrote: > Hi, > > I am seeing extremely long HDFS timeouts - and this seems to be associated > with the loss of a DataNode. Here is the RS log: > > 12/11/07 02:17:45 WARN hdfs.DFSClient: DFSOutputStream ResponseProcessor > exception for block blk_2813460962462751946_78454java.io.IOException: Bad > response 1 for block blk_2813460962462751946_78454 from datanode > 10.31.190.107:9200 > at > > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:3084) > > 12/11/07 02:17:45 WARN hdfs.DFSClient: Error Recovery for block > blk_2813460962462751946_78454 bad datanode[1] 10.31.190.107:9200 > 12/11/07 02:17:45 WARN hdfs.DFSClient: Error Recovery for block > blk_2813460962462751946_78454 in pipeline 10.31.138.245:9200, > 10.31.190.107:9200, 10.159.19.90:9200: bad datanode 10.31.190.107:9200 > 12/11/07 02:17:45 WARN wal.HLog: IPC Server handler 35 on 60020 took 65955 > ms appending an edit to hlog; editcount=476686, len~=76.0 > 12/11/07 02:17:45 WARN wal.HLog: HDFS pipeline error detected. Found 2 > replicas but expecting no less than 3 replicas. Requesting close of hlog. > > The corresponding DN log goes like this > > 2012-11-07 02:17:45,142 INFO > org.apache.hadoop.hdfs.server.datanode.DataNode (PacketResponder 2 for > Block blk_2813460962462751946_78454): PacketResponder > blk_2813460962462751946_78454 2 Exception java.net.SocketTimeoutException: > 66000 millis timeout while waiting for channel to be ready for read. ch : > java.nio.channels.SocketChannel[connected local=/10.31.138.245:33965 > remote=/ > 10.31.190.107:9200] > at > > org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:164) > at > org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:155) > at > org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:128) > at java.io.DataInputStream.readFully(DataInputStream.java:178) > at java.io.DataInputStream.readLong(DataInputStream.java:399) > at > > org.apache.hadoop.hdfs.protocol.DataTransferProtocol$PipelineAck.readFields(DataTransferProtocol.java:124) > at > > org.apache.hadoop.hdfs.server.datanode.BlockReceiver$PacketResponder.run(BlockReceiver.java:806) > at java.lang.Thread.run(Thread.java:662) > > It seems like the DataNode local to the region server is trying to grab the > block from another DN and that is timing out because of this other data > node being bad. All in all this causes response times to be terribly poor. > Is there a way around this or am I missing something ? > > Varun >
-
Re: Terribly long HDFS timeouts while appending to HLogJeremy Carroll 2012-11-07, 15:25
Sorry. It's early in the morning here. Did not see the 'read timeout'. +1
to Nicolas's response. On Wed, Nov 7, 2012 at 7:22 AM, Jeremy Carroll <[EMAIL PROTECTED]> wrote: > One trick I have used for a while is to > set dfs.datanode.socket.write.timeout in hdfs-site.xml to 0 (disabled). > It's not going to solve your underlying IOPS capacity issue with your > servers, but it can help for short bursty periods. Basically it's hiding > the real issue, but it can help in the short term. > > > On Wed, Nov 7, 2012 at 1:43 AM, Varun Sharma <[EMAIL PROTECTED]> wrote: > >> Hi, >> >> I am seeing extremely long HDFS timeouts - and this seems to be associated >> with the loss of a DataNode. Here is the RS log: >> >> 12/11/07 02:17:45 WARN hdfs.DFSClient: DFSOutputStream ResponseProcessor >> exception for block blk_2813460962462751946_78454java.io.IOException: Bad >> response 1 for block blk_2813460962462751946_78454 from datanode >> 10.31.190.107:9200 >> at >> >> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:3084) >> >> 12/11/07 02:17:45 WARN hdfs.DFSClient: Error Recovery for block >> blk_2813460962462751946_78454 bad datanode[1] 10.31.190.107:9200 >> 12/11/07 02:17:45 WARN hdfs.DFSClient: Error Recovery for block >> blk_2813460962462751946_78454 in pipeline 10.31.138.245:9200, >> 10.31.190.107:9200, 10.159.19.90:9200: bad datanode 10.31.190.107:9200 >> 12/11/07 02:17:45 WARN wal.HLog: IPC Server handler 35 on 60020 took 65955 >> ms appending an edit to hlog; editcount=476686, len~=76.0 >> 12/11/07 02:17:45 WARN wal.HLog: HDFS pipeline error detected. Found 2 >> replicas but expecting no less than 3 replicas. Requesting close of hlog. >> >> The corresponding DN log goes like this >> >> 2012-11-07 02:17:45,142 INFO >> org.apache.hadoop.hdfs.server.datanode.DataNode (PacketResponder 2 for >> Block blk_2813460962462751946_78454): PacketResponder >> blk_2813460962462751946_78454 2 Exception java.net.SocketTimeoutException: >> 66000 millis timeout while waiting for channel to be ready for read. ch : >> java.nio.channels.SocketChannel[connected local=/10.31.138.245:33965 >> remote=/ >> 10.31.190.107:9200] >> at >> >> org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:164) >> at >> org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:155) >> at >> org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:128) >> at java.io.DataInputStream.readFully(DataInputStream.java:178) >> at java.io.DataInputStream.readLong(DataInputStream.java:399) >> at >> >> org.apache.hadoop.hdfs.protocol.DataTransferProtocol$PipelineAck.readFields(DataTransferProtocol.java:124) >> at >> >> org.apache.hadoop.hdfs.server.datanode.BlockReceiver$PacketResponder.run(BlockReceiver.java:806) >> at java.lang.Thread.run(Thread.java:662) >> >> It seems like the DataNode local to the region server is trying to grab >> the >> block from another DN and that is timing out because of this other data >> node being bad. All in all this causes response times to be terribly poor. >> Is there a way around this or am I missing something ? >> >> Varun >> > >
-
Re: Terribly long HDFS timeouts while appending to HLogVarun Sharma 2012-11-07, 17:57
Thanks for the response. One more point is that I am running hadoop 1.0.4
with hbase 0.92 - not sure if that is known to have these issues. I had one quick question though - these logs are picked from 10.31.138.145 and from my understanding of the logs below, its still going to another bad datanode for retrieving the block even though it should already have the data block - see last line... 12/11/07 02:17:45 WARN hdfs.DFSClient: DFSOutputStream ResponseProcessor exception for block blk_2813460962462751946_78454java.io.IOException: Bad response 1 for block blk_2813460962462751946_78454 from datanode 10.31.190.107:9200 at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:3084) 12/11/07 02:17:45 WARN hdfs.DFSClient: Error Recovery for block blk_2813460962462751946_78454 bad datanode[1] 10.31.190.107:9200 12/11/07 02:17:45 WARN hdfs.DFSClient: Error Recovery for block blk_2813460962462751946_78454 in pipeline *10.31.138.245:9200, 10.31.190.107:9200, 10.159.19.90:9200: bad datanode 10.31.190.107:9200* Looking at the DataNode logs - it seems that the local datanode is trying to connect to the remote bad datanode. Is this for replicating the WALEdit ? 2012-11-07 02:17:45,142 INFO org.apache.hadoop.hdfs.server.datanode.DataNode (PacketResponder 2 for Block blk_2813460962462751946_78454): PacketResponder blk_2813460962462751946_78454 2 Exception java.net.SocketTimeoutException: 66000 millis timeout while waiting for channel to be ready for read. ch : java.nio.channels.SocketChannel[*connected local=/**10.31.138.245:33965 remote=/10.31.190.107:9200]* * * Also, this is preceded by a whole bunch of slow operations with processingtimems close to 20 seconds like these - are these other slow walEdit appends (slowed down due to HDFS) ? 12/11/07 02:16:01 WARN ipc.HBaseServer: (responseTooSlow): {"processingtimems":21957,"call":"multi(org.apache.hadoop.hbase.client.MultiAction@7198c05d), rpc version=1, client version=29, methodsFingerPrint=54742778","client":" 10.31.128.131:55327 ","starttimems":1352254539935,"queuetimems":0,"class":"HRegionServer","responsesize":0,"method":"multi"} Thanks Varun On Wed, Nov 7, 2012 at 7:25 AM, Jeremy Carroll <[EMAIL PROTECTED]> wrote: > Sorry. It's early in the morning here. Did not see the 'read timeout'. +1 > to Nicolas's response. > > On Wed, Nov 7, 2012 at 7:22 AM, Jeremy Carroll <[EMAIL PROTECTED]> > wrote: > > > One trick I have used for a while is to > > set dfs.datanode.socket.write.timeout in hdfs-site.xml to 0 (disabled). > > It's not going to solve your underlying IOPS capacity issue with your > > servers, but it can help for short bursty periods. Basically it's hiding > > the real issue, but it can help in the short term. > > > > > > On Wed, Nov 7, 2012 at 1:43 AM, Varun Sharma <[EMAIL PROTECTED]> > wrote: > > > >> Hi, > >> > >> I am seeing extremely long HDFS timeouts - and this seems to be > associated > >> with the loss of a DataNode. Here is the RS log: > >> > >> 12/11/07 02:17:45 WARN hdfs.DFSClient: DFSOutputStream ResponseProcessor > >> exception for block blk_2813460962462751946_78454java.io.IOException: > Bad > >> response 1 for block blk_2813460962462751946_78454 from datanode > >> 10.31.190.107:9200 > >> at > >> > >> > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:3084) > >> > >> 12/11/07 02:17:45 WARN hdfs.DFSClient: Error Recovery for block > >> blk_2813460962462751946_78454 bad datanode[1] 10.31.190.107:9200 > >> 12/11/07 02:17:45 WARN hdfs.DFSClient: Error Recovery for block > >> blk_2813460962462751946_78454 in pipeline 10.31.138.245:9200, > >> 10.31.190.107:9200, 10.159.19.90:9200: bad datanode 10.31.190.107:9200 > >> 12/11/07 02:17:45 WARN wal.HLog: IPC Server handler 35 on 60020 took > 65955 > >> ms appending an edit to hlog; editcount=476686, len~=76.0 > >> 12/11/07 02:17:45 WARN wal.HLog: HDFS pipeline error detected. Found 2 > >> replicas but expecting no less than 3 replicas. Requesting close of
-
Re: Terribly long HDFS timeouts while appending to HLogDavid Charle 2012-11-07, 18:21
u should upgrade to 0.94 as you also had issues with row locks as newer version had improved miniBatchPut code base.
On Nov 7, 2012, at 9:57 AM, Varun Sharma <[EMAIL PROTECTED]> wrote: > Thanks for the response. One more point is that I am running hadoop 1.0.4 > with hbase 0.92 - not sure if that is known to have these issues. > > I had one quick question though - these logs are picked from 10.31.138.145 > and from my understanding of the logs below, its still going to another bad > datanode for retrieving the block even though it should already have the > data block - see last line... > > 12/11/07 02:17:45 WARN hdfs.DFSClient: DFSOutputStream ResponseProcessor > exception for block blk_2813460962462751946_78454java.io.IOException: Bad > response 1 for block blk_2813460962462751946_78454 from datanode > 10.31.190.107:9200 > at > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:3084) > > 12/11/07 02:17:45 WARN hdfs.DFSClient: Error Recovery for block > blk_2813460962462751946_78454 bad datanode[1] 10.31.190.107:9200 > 12/11/07 02:17:45 WARN hdfs.DFSClient: Error Recovery for block > blk_2813460962462751946_78454 in pipeline *10.31.138.245:9200, > 10.31.190.107:9200, 10.159.19.90:9200: bad datanode 10.31.190.107:9200* > > Looking at the DataNode logs - it seems that the local datanode is trying > to connect to the remote bad datanode. Is this for replicating the WALEdit ? > > 2012-11-07 02:17:45,142 INFO org.apache.hadoop.hdfs.server.datanode.DataNode > (PacketResponder 2 for Block blk_2813460962462751946_78454): > PacketResponder blk_2813460962462751946_78454 2 Exception > java.net.SocketTimeoutException: > 66000 millis timeout while waiting for channel to be ready for read. ch : > java.nio.channels.SocketChannel[*connected local=/**10.31.138.245:33965 > remote=/10.31.190.107:9200]* > * > * > Also, this is preceded by a whole bunch of slow operations with > processingtimems close to 20 seconds like these - are these other slow > walEdit appends (slowed down due to HDFS) ? > > 12/11/07 02:16:01 WARN ipc.HBaseServer: (responseTooSlow): > {"processingtimems":21957,"call":"multi(org.apache.hadoop.hbase.client.MultiAction@7198c05d), > rpc version=1, client version=29, methodsFingerPrint=54742778","client":" > 10.31.128.131:55327 > ","starttimems":1352254539935,"queuetimems":0,"class":"HRegionServer","responsesize":0,"method":"multi"} > > Thanks > Varun > > On Wed, Nov 7, 2012 at 7:25 AM, Jeremy Carroll <[EMAIL PROTECTED]> wrote: > >> Sorry. It's early in the morning here. Did not see the 'read timeout'. +1 >> to Nicolas's response. >> >> On Wed, Nov 7, 2012 at 7:22 AM, Jeremy Carroll <[EMAIL PROTECTED]> >> wrote: >> >>> One trick I have used for a while is to >>> set dfs.datanode.socket.write.timeout in hdfs-site.xml to 0 (disabled). >>> It's not going to solve your underlying IOPS capacity issue with your >>> servers, but it can help for short bursty periods. Basically it's hiding >>> the real issue, but it can help in the short term. >>> >>> >>> On Wed, Nov 7, 2012 at 1:43 AM, Varun Sharma <[EMAIL PROTECTED]> >> wrote: >>> >>>> Hi, >>>> >>>> I am seeing extremely long HDFS timeouts - and this seems to be >> associated >>>> with the loss of a DataNode. Here is the RS log: >>>> >>>> 12/11/07 02:17:45 WARN hdfs.DFSClient: DFSOutputStream ResponseProcessor >>>> exception for block blk_2813460962462751946_78454java.io.IOException: >> Bad >>>> response 1 for block blk_2813460962462751946_78454 from datanode >>>> 10.31.190.107:9200 >>>> at >> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:3084) >>>> >>>> 12/11/07 02:17:45 WARN hdfs.DFSClient: Error Recovery for block >>>> blk_2813460962462751946_78454 bad datanode[1] 10.31.190.107:9200 >>>> 12/11/07 02:17:45 WARN hdfs.DFSClient: Error Recovery for block >>>> blk_2813460962462751946_78454 in pipeline 10.31.138.245:9200, >>>> 10.31.190.107:9200, 10.159.19.90:9200: bad datanode 10.31.190.107:9200
-
Re: Terribly long HDFS timeouts while appending to HLogJeremy Carroll 2012-11-07, 19:52
It's important to realize that HBase is a strongly consistent system. So if
a DataNode is down (But looks alive due to HDFS not marking it as down). The system will choose to be unavailable, rather than consistent. During this timeframe where the underlying HDFS file system was not operating normally (Did not mark nodes as failed), HBase can give up / timeout on a lot of operations. HDFS replicates data by default factor of 3. So during this time the node may have been a replication target, but unable to satisfy that request. On Wed, Nov 7, 2012 at 9:57 AM, Varun Sharma <[EMAIL PROTECTED]> wrote: > Thanks for the response. One more point is that I am running hadoop 1.0.4 > with hbase 0.92 - not sure if that is known to have these issues. > > I had one quick question though - these logs are picked from 10.31.138.145 > and from my understanding of the logs below, its still going to another bad > datanode for retrieving the block even though it should already have the > data block - see last line... > > 12/11/07 02:17:45 WARN hdfs.DFSClient: DFSOutputStream ResponseProcessor > exception for block blk_2813460962462751946_78454java.io.IOException: Bad > response 1 for block blk_2813460962462751946_78454 from datanode > 10.31.190.107:9200 > at > > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:3084) > > 12/11/07 02:17:45 WARN hdfs.DFSClient: Error Recovery for block > blk_2813460962462751946_78454 bad datanode[1] 10.31.190.107:9200 > 12/11/07 02:17:45 WARN hdfs.DFSClient: Error Recovery for block > blk_2813460962462751946_78454 in pipeline *10.31.138.245:9200, > 10.31.190.107:9200, 10.159.19.90:9200: bad datanode 10.31.190.107:9200* > > Looking at the DataNode logs - it seems that the local datanode is trying > to connect to the remote bad datanode. Is this for replicating the WALEdit > ? > > 2012-11-07 02:17:45,142 INFO > org.apache.hadoop.hdfs.server.datanode.DataNode > (PacketResponder 2 for Block blk_2813460962462751946_78454): > PacketResponder blk_2813460962462751946_78454 2 Exception > java.net.SocketTimeoutException: > 66000 millis timeout while waiting for channel to be ready for read. ch : > java.nio.channels.SocketChannel[*connected local=/**10.31.138.245:33965 > remote=/10.31.190.107:9200]* > * > * > Also, this is preceded by a whole bunch of slow operations with > processingtimems close to 20 seconds like these - are these other slow > walEdit appends (slowed down due to HDFS) ? > > 12/11/07 02:16:01 WARN ipc.HBaseServer: (responseTooSlow): > > {"processingtimems":21957,"call":"multi(org.apache.hadoop.hbase.client.MultiAction@7198c05d > ), > rpc version=1, client version=29, methodsFingerPrint=54742778","client":" > 10.31.128.131:55327 > > ","starttimems":1352254539935,"queuetimems":0,"class":"HRegionServer","responsesize":0,"method":"multi"} > > Thanks > Varun > > On Wed, Nov 7, 2012 at 7:25 AM, Jeremy Carroll <[EMAIL PROTECTED]> > wrote: > > > Sorry. It's early in the morning here. Did not see the 'read timeout'. +1 > > to Nicolas's response. > > > > On Wed, Nov 7, 2012 at 7:22 AM, Jeremy Carroll <[EMAIL PROTECTED]> > > wrote: > > > > > One trick I have used for a while is to > > > set dfs.datanode.socket.write.timeout in hdfs-site.xml to 0 (disabled). > > > It's not going to solve your underlying IOPS capacity issue with your > > > servers, but it can help for short bursty periods. Basically it's > hiding > > > the real issue, but it can help in the short term. > > > > > > > > > On Wed, Nov 7, 2012 at 1:43 AM, Varun Sharma <[EMAIL PROTECTED]> > > wrote: > > > > > >> Hi, > > >> > > >> I am seeing extremely long HDFS timeouts - and this seems to be > > associated > > >> with the loss of a DataNode. Here is the RS log: > > >> > > >> 12/11/07 02:17:45 WARN hdfs.DFSClient: DFSOutputStream > ResponseProcessor > > >> exception for block blk_2813460962462751946_78454java.io.IOException: > > Bad > > >> response 1 for block blk_2813460962462751946_78454 from datanode > > >> 10.31.190.107:9200
-
Re: Terribly long HDFS timeouts while appending to HLogJeremy Carroll 2012-11-07, 19:53
Er. Inconsistent. Sorry for typo. Basically when the underlying file system
is unstable, HBase can become unstable as well. On Wed, Nov 7, 2012 at 11:52 AM, Jeremy Carroll <[EMAIL PROTECTED]> wrote: > It's important to realize that HBase is a strongly consistent system. So > if a DataNode is down (But looks alive due to HDFS not marking it as down). > The system will choose to be unavailable, rather than consistent. During > this timeframe where the underlying HDFS file system was not operating > normally (Did not mark nodes as failed), HBase can give up / timeout on a > lot of operations. HDFS replicates data by default factor of 3. So during > this time the node may have been a replication target, but unable to > satisfy that request. > > > On Wed, Nov 7, 2012 at 9:57 AM, Varun Sharma <[EMAIL PROTECTED]> wrote: > >> Thanks for the response. One more point is that I am running hadoop 1.0.4 >> with hbase 0.92 - not sure if that is known to have these issues. >> >> I had one quick question though - these logs are picked from 10.31.138.145 >> and from my understanding of the logs below, its still going to another >> bad >> datanode for retrieving the block even though it should already have the >> data block - see last line... >> >> 12/11/07 02:17:45 WARN hdfs.DFSClient: DFSOutputStream ResponseProcessor >> exception for block blk_2813460962462751946_78454java.io.IOException: Bad >> response 1 for block blk_2813460962462751946_78454 from datanode >> 10.31.190.107:9200 >> at >> >> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:3084) >> >> 12/11/07 02:17:45 WARN hdfs.DFSClient: Error Recovery for block >> blk_2813460962462751946_78454 bad datanode[1] 10.31.190.107:9200 >> 12/11/07 02:17:45 WARN hdfs.DFSClient: Error Recovery for block >> blk_2813460962462751946_78454 in pipeline *10.31.138.245:9200, >> 10.31.190.107:9200, 10.159.19.90:9200: bad datanode 10.31.190.107:9200* >> >> Looking at the DataNode logs - it seems that the local datanode is trying >> to connect to the remote bad datanode. Is this for replicating the >> WALEdit ? >> >> 2012-11-07 02:17:45,142 INFO >> org.apache.hadoop.hdfs.server.datanode.DataNode >> (PacketResponder 2 for Block blk_2813460962462751946_78454): >> PacketResponder blk_2813460962462751946_78454 2 Exception >> java.net.SocketTimeoutException: >> 66000 millis timeout while waiting for channel to be ready for read. ch : >> java.nio.channels.SocketChannel[*connected local=/**10.31.138.245:33965 >> remote=/10.31.190.107:9200]* >> * >> * >> Also, this is preceded by a whole bunch of slow operations with >> processingtimems close to 20 seconds like these - are these other slow >> walEdit appends (slowed down due to HDFS) ? >> >> 12/11/07 02:16:01 WARN ipc.HBaseServer: (responseTooSlow): >> >> {"processingtimems":21957,"call":"multi(org.apache.hadoop.hbase.client.MultiAction@7198c05d >> ), >> rpc version=1, client version=29, methodsFingerPrint=54742778","client":" >> 10.31.128.131:55327 >> >> ","starttimems":1352254539935,"queuetimems":0,"class":"HRegionServer","responsesize":0,"method":"multi"} >> >> Thanks >> Varun >> >> On Wed, Nov 7, 2012 at 7:25 AM, Jeremy Carroll <[EMAIL PROTECTED]> >> wrote: >> >> > Sorry. It's early in the morning here. Did not see the 'read timeout'. >> +1 >> > to Nicolas's response. >> > >> > On Wed, Nov 7, 2012 at 7:22 AM, Jeremy Carroll <[EMAIL PROTECTED]> >> > wrote: >> > >> > > One trick I have used for a while is to >> > > set dfs.datanode.socket.write.timeout in hdfs-site.xml to 0 >> (disabled). >> > > It's not going to solve your underlying IOPS capacity issue with your >> > > servers, but it can help for short bursty periods. Basically it's >> hiding >> > > the real issue, but it can help in the short term. >> > > >> > > >> > > On Wed, Nov 7, 2012 at 1:43 AM, Varun Sharma <[EMAIL PROTECTED]> >> > wrote: >> > > >> > >> Hi, >> > >> >> > >> I am seeing extremely long HDFS timeouts - and this seems to be >> > associated
-
Re: Terribly long HDFS timeouts while appending to HLogVarun Sharma 2012-11-07, 21:52
Great - thanks for all the information.
Reducing the timeout seems like one plausible approach to me. However, I had two follow up questions: 1) For writes - since these need to happen synchronously - if a datanode goes down, we would essentially have some slight inconsistency in terms of data across the 3 replicas - since that node will not have the latest walEdit (say for a walAppend) - in such a case, how does HDFS handle this issue ? What if the DataNode is brought back up before it expired within the NameNode ? 2) Does the NameNode suggest DataNodes that are local to the RegionServer if we have a read operation. Also, I am running the DataNode with Xmx=1024M - could that be a reason behind the data node crashes ? Thanks Varun On Wed, Nov 7, 2012 at 11:53 AM, Jeremy Carroll <[EMAIL PROTECTED]> wrote: > Er. Inconsistent. Sorry for typo. Basically when the underlying file system > is unstable, HBase can become unstable as well. > > On Wed, Nov 7, 2012 at 11:52 AM, Jeremy Carroll <[EMAIL PROTECTED]> > wrote: > > > It's important to realize that HBase is a strongly consistent system. So > > if a DataNode is down (But looks alive due to HDFS not marking it as > down). > > The system will choose to be unavailable, rather than consistent. During > > this timeframe where the underlying HDFS file system was not operating > > normally (Did not mark nodes as failed), HBase can give up / timeout on a > > lot of operations. HDFS replicates data by default factor of 3. So during > > this time the node may have been a replication target, but unable to > > satisfy that request. > > > > > > On Wed, Nov 7, 2012 at 9:57 AM, Varun Sharma <[EMAIL PROTECTED]> > wrote: > > > >> Thanks for the response. One more point is that I am running hadoop > 1.0.4 > >> with hbase 0.92 - not sure if that is known to have these issues. > >> > >> I had one quick question though - these logs are picked from > 10.31.138.145 > >> and from my understanding of the logs below, its still going to another > >> bad > >> datanode for retrieving the block even though it should already have the > >> data block - see last line... > >> > >> 12/11/07 02:17:45 WARN hdfs.DFSClient: DFSOutputStream ResponseProcessor > >> exception for block blk_2813460962462751946_78454java.io.IOException: > Bad > >> response 1 for block blk_2813460962462751946_78454 from datanode > >> 10.31.190.107:9200 > >> at > >> > >> > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:3084) > >> > >> 12/11/07 02:17:45 WARN hdfs.DFSClient: Error Recovery for block > >> blk_2813460962462751946_78454 bad datanode[1] 10.31.190.107:9200 > >> 12/11/07 02:17:45 WARN hdfs.DFSClient: Error Recovery for block > >> blk_2813460962462751946_78454 in pipeline *10.31.138.245:9200, > >> 10.31.190.107:9200, 10.159.19.90:9200: bad datanode 10.31.190.107:9200* > >> > >> Looking at the DataNode logs - it seems that the local datanode is > trying > >> to connect to the remote bad datanode. Is this for replicating the > >> WALEdit ? > >> > >> 2012-11-07 02:17:45,142 INFO > >> org.apache.hadoop.hdfs.server.datanode.DataNode > >> (PacketResponder 2 for Block blk_2813460962462751946_78454): > >> PacketResponder blk_2813460962462751946_78454 2 Exception > >> java.net.SocketTimeoutException: > >> 66000 millis timeout while waiting for channel to be ready for read. ch > : > >> java.nio.channels.SocketChannel[*connected local=/**10.31.138.245:33965 > >> remote=/10.31.190.107:9200]* > >> * > >> * > >> Also, this is preceded by a whole bunch of slow operations with > >> processingtimems close to 20 seconds like these - are these other slow > >> walEdit appends (slowed down due to HDFS) ? > >> > >> 12/11/07 02:16:01 WARN ipc.HBaseServer: (responseTooSlow): > >> > >> > {"processingtimems":21957,"call":"multi(org.apache.hadoop.hbase.client.MultiAction@7198c05d > >> ), > >> rpc version=1, client version=29, > methodsFingerPrint=54742778","client":" > >> 10.31.128.131:55327 > >> > >> > ","starttimems":1352254539935,"queuetimems":0,"class":"HRegionServer","responsesize":0,"method":"multi"} |