You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Ted Yu (JIRA)" <ji...@apache.org> on 2011/06/24 20:07:47 UTC
[jira] [Updated] (HBASE-4030) LoadIncrementalHFiles fails with
FileNotFoundException
[ https://issues.apache.org/jira/browse/HBASE-4030?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Ted Yu updated HBASE-4030:
--------------------------
Fix Version/s: 0.90.4
> LoadIncrementalHFiles fails with FileNotFoundException
> ------------------------------------------------------
>
> Key: HBASE-4030
> URL: https://issues.apache.org/jira/browse/HBASE-4030
> Project: HBase
> Issue Type: Bug
> Affects Versions: 0.90.1
> Reporter: Adam Phelps
> Fix For: 0.90.4
>
>
> -------- Original Message --------
> Subject: Re: LoadIncrementalHFiles bug when regionserver fails to
> access file?
> Date: Thu, 23 Jun 2011 17:00:04 -0700
> From: Ted Yu <yu...@gmail.com>
> Reply-To: cdh-user@cloudera.org
> To: user@hbase.apache.org
> CC: CDH Users <cd...@cloudera.org>
> This is due to the handling of HFile.Reader being wrapped in a
> try-finally block. However, there is no check as to whether the reader
> operation encounters any exception which should determine what to do next.
> Please file a JIRA.
> Thanks Adam.
> On Thu, Jun 23, 2011 at 4:40 PM, Adam Phelps <amp@opendns.com
> <ma...@opendns.com>> wrote:
> (As a note, this is with CDH3u0 which is based on HBase 0.90.1)
> We've been seeing intermittent failures of calls to
> LoadIncrementalHFiles. When this happens the node that made the
> call will see a FileNotFoundException such as this:
> 2011-06-23 15:47:34.379566500 java.net
> <http://java.net>.__SocketTimeoutException: Call to
> s8.XXX/67.215.90.38:60020 <http://67.215.90.38:60020> failed on
> socket timeout exception: java.net.SocketTi
> meoutException: 60000 millis timeout while waiting for channel to be
> ready for read. ch : java.nio.channels.__SocketChannel[connected
> local=/67.215.90.51:51605 <http://67.215.90.51:51605> remo
> te=s8.XXX/67.215.90.38:60020 <http://67.215.90.38:60020>]
> 2011-06-23 15:47:34.379570500 java.io.FileNotFoundException:
> java.io.FileNotFoundException: File does not exist:
> /hfiles/2011/06/23/14/__domainsranked/TopDomainsRan
> k.r3v5PRvK/handling/__3557032074765091256
> 2011-06-23 15:47:34.379573500 at
> org.apache.hadoop.hdfs.__DFSClient$DFSInputStream.__openInfo(DFSClient.java:1602)
> 2011-06-23 15:47:34.379573500 at
> org.apache.hadoop.hdfs.__DFSClient$DFSInputStream.<__init>(DFSClient.java:1593)
> Over on the regionserver that was loading this we see that it
> attempted to load and hit a 60 second timeout:
> 2011-06-23 15:45:54,634 INFO
> org.apache.hadoop.hbase.__regionserver.Store: Validating hfile at
> hdfs://namenode.XXX/hfiles/__2011/06/23/14/domainsranked/__TopDomainsRank.r3v5PRvK/__handling/3557032074765091256
> for inclusion in store handling region
> domainsranked,368449:2011/0/__03/23:category:ffffffff:com.__zynga.static.fishville.__facebook,1305890318961.__d4925aca7852bed32613a509215d42__b
> 8.
> ...
> 2011-06-23 15:46:54,639 INFO org.apache.hadoop.hdfs.__DFSClient:
> Failed to connect to /67.215.90.38:50010
> <http://67.215.90.38:50010>, add to deadNodes and continue
> java.net <http://java.net>.__SocketTimeoutException: 60000 millis
> timeout while waiting for channel to be ready for read. ch :
> java.nio.channels.__SocketChannel[connected
> local=/67.215.90.38:42199 <http://67.215.90.38:42199>
> remote=/67.215.90.38:50010 <http://67.215.90.38:50010>]
> at org.apache.hadoop.net
> <http://org.apache.hadoop.net>.__SocketIOWithTimeout.doIO(__SocketIOWithTimeout.java:164)
> at org.apache.hadoop.net
> <http://org.apache.hadoop.net>.__SocketInputStream.read(__SocketInputStream.java:155)
> at org.apache.hadoop.net
> <http://org.apache.hadoop.net>.__SocketInputStream.read(__SocketInputStream.java:128)
> at java.io.BufferedInputStream.__fill(BufferedInputStream.java:__218)
> at java.io.BufferedInputStream.__read(BufferedInputStream.java:__237)
> at java.io.DataInputStream.__readShort(DataInputStream.__java:295)
> We suspect this particular problem is a resource contention issue on
> our side. However, the loading process proceeds to rename the file
> despite the failure:
> 2011-06-23 15:46:54,657 INFO
> org.apache.hadoop.hbase.__regionserver.Store: Renaming bulk load
> file
> hdfs://namenode.XXX/hfiles/__2011/06/23/14/domainsranked/__TopDomainsRank.r3v5PRvK/__handling/3557032074765091256
> to
> hdfs://namenode.XXX:8020/__hbase/domainsranked/__d4925aca7852bed32613a509215d42__b8/handling/__3615917062821145533
> And then the LoadIncrementalHFiles tries to load the hfile again:
> 2011-06-23 15:46:55,684 INFO
> org.apache.hadoop.hbase.__regionserver.Store: Validating hfile at
> hdfs://namenode.XXX/hfiles/__2011/06/23/14/domainsranked/__TopDomainsRank.r3v5PRvK/__handling/3557032074765091256
> for inclusion in store handling region
> domainsranked,368449:2011/05/__03/23:category:ffffffff:com.__zynga.static.fishville.__facebook,1305890318961.__d4925aca7852bed32613a509215d42__b8.
> 2011-06-23 15:46:55,685 DEBUG org.apache.hadoop.ipc.__HBaseServer:
> IPC Server handler 147 on 60020, call
> bulkLoadHFile(hdfs://namenode.__XXX/hfiles/2011/06/23/14/__domainsranked/TopDomainsRank.__r3v5PRvK/handling/__3557032074765091256,
> [B@4224508b, [B@5e23f799) from 67.215.90.51:51856
> <http://67.215.90.51:51856>: error: java.io.FileNotFoundException:
> File does not exist:
> /hfiles/2011/06/23/14/__domainsranked/TopDomainsRank.__r3v5PRvK/handling/__3557032074765091256
> This eventually leads to the load command failing. It feels to me
> like HBase continuing to rename the hfile despite the failure, and
> then LoadIncrementalHFiles trying again is a bug. I figured I'd ask
> here before opening a ticket for this.
> - Adam
--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira