Return-Path: X-Original-To: apmail-hbase-user-archive@www.apache.org Delivered-To: apmail-hbase-user-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id A0A3ADB46 for ; Wed, 7 Nov 2012 17:57:56 +0000 (UTC) Received: (qmail 26511 invoked by uid 500); 7 Nov 2012 17:57:54 -0000 Delivered-To: apmail-hbase-user-archive@hbase.apache.org Received: (qmail 26415 invoked by uid 500); 7 Nov 2012 17:57:52 -0000 Mailing-List: contact user-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@hbase.apache.org Delivered-To: mailing list user@hbase.apache.org Received: (qmail 26386 invoked by uid 99); 7 Nov 2012 17:57:51 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 07 Nov 2012 17:57:51 +0000 X-ASF-Spam-Status: No, hits=1.5 required=5.0 tests=HTML_MESSAGE,NORMAL_HTTP_TO_IP,RCVD_IN_DNSWL_LOW,SPF_PASS,WEIRD_PORT X-Spam-Check-By: apache.org Received-SPF: pass (nike.apache.org: domain of varun@pinterest.com designates 209.85.216.41 as permitted sender) Received: from [209.85.216.41] (HELO mail-qa0-f41.google.com) (209.85.216.41) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 07 Nov 2012 17:57:45 +0000 Received: by mail-qa0-f41.google.com with SMTP id c26so3332044qad.14 for ; Wed, 07 Nov 2012 09:57:24 -0800 (PST) X-Google-DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=google.com; s=20120113; h=mime-version:in-reply-to:references:date:message-id:subject:from:to :content-type:x-gm-message-state; bh=317Cfd1ohTO0XNn2NCTKG4lOZ2t3ZH+8Cj6eYq3HZMA=; b=LkVOQIIIT5HsG6QEEIvPDqj7mEkfuTNXxsAXLCTGXjYpx/I/oynVMNXbPopChXgQ3D XzWUtH5r5aBUgxi/hcWadYo6kxL+ChCBSXDHQUCSShNpsG0I1kKrAn2wl+Fsc7W3XXLD b3nj4C5vTvY5NeNCX65KpjVfBI3fvnzy0qD0MM47Ojiu/Es72cxksV5YaKxGLBNrSoxe SsHbNvmINuKrvG/f9k36lKl9DrvHDCPfwmgtSKneSKqpUXDhfdU05V/YZwBAS4HLL3y7 /bbBG/TOcJrfUJXGNQa3JDZJU8AiBj8ATw+LrOx8yQf9qDBpMe0wxjdTHRFprnLe0xha PP3A== MIME-Version: 1.0 Received: by 10.49.17.38 with SMTP id l6mr9327748qed.42.1352311043938; Wed, 07 Nov 2012 09:57:23 -0800 (PST) Received: by 10.49.48.115 with HTTP; Wed, 7 Nov 2012 09:57:23 -0800 (PST) In-Reply-To: References: Date: Wed, 7 Nov 2012 09:57:23 -0800 Message-ID: Subject: Re: Terribly long HDFS timeouts while appending to HLog From: Varun Sharma To: user@hbase.apache.org, Jeremy Carroll , nkeywal@gmail.com Content-Type: multipart/alternative; boundary=047d7bea35346658f904cdeb73b0 X-Gm-Message-State: ALoCoQmZaWadidkXWAXk5HvW3FUq0c3/mJumEx8wdUSvSa9UKjtMhYJ7rXrzwcizfVxK3X+Krs4N X-Virus-Checked: Checked by ClamAV on apache.org --047d7bea35346658f904cdeb73b0 Content-Type: text/plain; charset=ISO-8859-1 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 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 > 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 > 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 > >> > > > > > --047d7bea35346658f904cdeb73b0--