hbase-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Andrew Purtell <apurt...@yahoo.com>
Subject Re: CompactSplitThread Exception and other questions
Date Mon, 01 Dec 2008 14:39:58 GMT
Aside from the issue of the bloomfilter related exception 
(which seems quite strange to me considering you have not 
enabled them!), a 48 second timeout at the DFS layer and the
UnknownScannerExceptions both indicate to me that you are overloading. In other words, there
are insufficient nodes in
your cluster to spread the load you are generating. At the
mapred level, this manifests by tasks that block waiting for
resources longer than the default regionserver lease time
(120 seconds). At the DFS level, blocks cannot be served to
the regionserver fast enough, and in fact are not served
even after 48 seconds in one particular instance.

    - Andy

--- On Sun, 11/30/08, Thibaut_ <tbritz@blue.lu> wrote:

> From: Thibaut_ <tbritz@blue.lu>
> Subject: Re: CompactSplitThread Exception and other questions
> To: hbase-user@hadoop.apache.org
> Date: Sunday, November 30, 2008, 3:04 PM
> Hi Andy & St.Ack,
> 
> I created all the tables from scratch and didn't change
> them at run time. The schema for all the tables right
> now is as followed. (data is a bytearray of a serialized
> google buffer object)
> 
>  	{NAME => 'entries', IS_ROOT =>
> 'false', IS_META => 'false', FAMILIES
> =>
> [{NAME => 'data', BLOOMFILTER =>
> 'false', COMPRESSION => 'NONE', VERSIONS
> =>
> '3', LENGTH => '2147483647', TTL =>
> '-1', IN_MEMORY => 'false', BLOCKCACHE
> => 'false'}]}
> 
> I reran everything from scratch with the new table scheme
> and got the same
> exception again, just on a different table this time:
> (Disabling the
> bloomfilter, compression and the blockcache doesn't
> seem to have any effect)
> 
> 2008-11-30 23:22:20,774 ERROR
> org.apache.hadoop.hbase.regionserver.CompactSplitThread:
> Compaction failed
> for region entries,,1228075277421
> java.lang.IllegalArgumentException: maxValue must be > 0
> 	at
> org.onelab.filter.HashFunction.<init>(HashFunction.java:84)
> 	at org.onelab.filter.Filter.<init>(Filter.java:97)
> 	at
> org.onelab.filter.BloomFilter.<init>(BloomFilter.java:102)
> 	at
> org.apache.hadoop.hbase.regionserver.HStoreFile$BloomFilterMapFile$Writer.<init>(HStoreFile.java:829)
> 	at
> org.apache.hadoop.hbase.regionserver.HStoreFile.getWriter(HStoreFile.java:436)
> 	at
> org.apache.hadoop.hbase.regionserver.HStore.compact(HStore.java:889)
> 	at
> org.apache.hadoop.hbase.regionserver.HRegion.compactStores(HRegion.java:902)
> 	at
> org.apache.hadoop.hbase.regionserver.HRegion.compactStores(HRegion.java:860)
> 	at
> org.apache.hadoop.hbase.regionserver.CompactSplitThread.run(CompactSplitThread.java:83)
> 
> The log file is also full of these kind of errors: (before
> and after)
> 
> 2008-11-30 23:22:44,500 INFO org.apache.hadoop.ipc.Server:
> IPC Server
> handler 16 on 60020, call next(8976385860586379110) from
> x.x.x.203:52747:
> error: org.apache.hadoop.hbase.UnknownScannerException:
> Name:
> 8976385860586379110
> org.apache.hadoop.hbase.UnknownScannerException: Name:
> 8976385860586379110
> 	at
> org.apache.hadoop.hbase.regionserver.HRegionServer.next(HRegionServer.java:1077)
> 	at sun.reflect.GeneratedMethodAccessor4.invoke(Unknown
> Source)
> 	at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
> 	at java.lang.reflect.Method.invoke(Method.java:585)
> 	at
> org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:554)
> 	at
> org.apache.hadoop.ipc.Server$Handler.run(Server.java:888)
> 
> Shortly afterwards I got the dfs error on the regionserver
> again, on a different table though (and might be completely
> unreleated and not important?):
> 
> 2008-11-30 23:26:55,885 WARN
> org.apache.hadoop.dfs.DFSClient: Exception
> while reading from blk_-9066140877711029349_706715 of
> /hbase/webrequestscache/2091560474/data/mapfiles/1510543474646532027/data
> from x.x.x.204:50010: java.io.IOException: Premeture EOF
> from inputStream
> 	at
> org.apache.hadoop.io.IOUtils.readFully(IOUtils.java:102)
> 	at
> org.apache.hadoop.dfs.DFSClient$BlockReader.readChunk(DFSClient.java:996)
> 	at
> org.apache.hadoop.fs.FSInputChecker.readChecksumChunk(FSInputChecker.java:236)
> 	at
> org.apache.hadoop.fs.FSInputChecker.read1(FSInputChecker.java:191)
> 	at
> org.apache.hadoop.fs.FSInputChecker.read(FSInputChecker.java:159)
> 	at
> org.apache.hadoop.dfs.DFSClient$BlockReader.read(DFSClient.java:858)
> 	at
> org.apache.hadoop.dfs.DFSClient$DFSInputStream.readBuffer(DFSClient.java:1384)
> 	at
> org.apache.hadoop.dfs.DFSClient$DFSInputStream.read(DFSClient.java:1420)
> 	at
> java.io.DataInputStream.readFully(DataInputStream.java:176)
> 	at
> org.apache.hadoop.io.DataOutputBuffer$Buffer.write(DataOutputBuffer.java:64)
> 	at
> org.apache.hadoop.io.DataOutputBuffer.write(DataOutputBuffer.java:102)
> 	at
> org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:1933)
> 	at
> org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:1833)
> 	at
> org.apache.hadoop.io.MapFile$Reader.seekInternal(MapFile.java:463)
> 	at
> org.apache.hadoop.io.MapFile$Reader.getClosest(MapFile.java:558)
> 	at
> org.apache.hadoop.io.MapFile$Reader.getClosest(MapFile.java:541)
> 	at
> org.apache.hadoop.hbase.regionserver.HStoreFile$BloomFilterMapFile$Reader.getClosest(HStoreFile.java:761)
> 	at
> org.apache.hadoop.hbase.regionserver.HStore.get(HStore.java:1291)
> 	at
> org.apache.hadoop.hbase.regionserver.HRegion.get(HRegion.java:1154)
> 	at
> org.apache.hadoop.hbase.regionserver.HRegionServer.get(HRegionServer.java:1020)
> 	at sun.reflect.GeneratedMethodAccessor9.invoke(Unknown
> Source)
> 	at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
> 	at java.lang.reflect.Method.invoke(Method.java:585)
> 	at
> org.apache.hadoop.hbase.ipc.HbaseRPC$Server.call(HbaseRPC.java:554)
> 	at
> org.apache.hadoop.ipc.Server$Handler.run(Server.java:888)
> 
> Datnode entries related to that block:
> 08/11/30 22:44:19 INFO dfs.DataNode: Receiving block
> blk_-9066140877711029349_706715 src: /x.x.x.204:44313 dest:
> /x.x.x.204:50010
> 08/11/30 22:44:41 INFO dfs.DataNode: Received block
> blk_-9066140877711029349_706715 of size 33554432 from
> /x.x.x.204
> 08/11/30 22:44:41 INFO dfs.DataNode: PacketResponder 3 for
> block
> blk_-9066140877711029349_706715 terminating
> 08/11/30 22:53:18 WARN dfs.DataNode:
> DatanodeRegistration(x.x.x.204:50010,
> storageID=DS-364968361-x.x.x.204-50010-1220223683238,
> infoPort=50075,
> ipcPort=50020):Got exception while serving
> blk_-9066140877711029349_706715
> to /x.x.x.204:
> java.net.SocketTimeoutException: 480000 millis timeout
> while waiting for
> channel to be ready for write. ch :
> java.nio.channels.SocketChannel[connected
> local=/x.x.x.204:50010
> remote=/x.x.x.204:46220]
>         at
> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:185)
>         at
> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>         at
> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>         at
> org.apache.hadoop.dfs.DataNode$BlockSender.sendChunks(DataNode.java:1873)
>         at
> org.apache.hadoop.dfs.DataNode$BlockSender.sendBlock(DataNode.java:1967)
>         at
> org.apache.hadoop.dfs.DataNode$DataXceiver.readBlock(DataNode.java:1109)
>         at
> org.apache.hadoop.dfs.DataNode$DataXceiver.run(DataNode.java:1037)
>         at java.lang.Thread.run(Thread.java:595)
> 
> 
> The master server doesn't have any Exceptions in its
> log, except for two
> UnknownScannerExceptions, but which might be related to
> myself killing
> mapreduce jobs.
> 
> St.Ack, could you please reupload the config file to the
> wiki, so I can try
> it out. There are no files listed, neither under
> http://wiki.apache.org/hadoop/TroubleShooting?action=AttachFile
> nor under
> http://wiki.apache.org/hadoop/Hbase/Troubleshooting?action=AttachFile
> ?
> 
> As I have written before, I use the release versions of
> hbase 0.18.1 and
> hadoop 0.18.1. I have just seen that version 0.19 of hadoop
> has been
> released, which also fixes a few bugs that could lead to
> data
> loss/corruption. Should I try it with hadoop 0.19 or 0.18.2
> and with the
> latest hbase version from the trunk?
> 
> As for the performance, this is not so important right now.
> I will
> investigate this later when hbase runs reliable :-).
> 
> Thanks a lot,
> Thibaut
> -- 
> View this message in context:
> http://www.nabble.com/CompactSplitThread-Exception-and-other-questions-tp20753458p20763567.html
> Sent from the HBase User mailing list archive at
> Nabble.com.


      

Mime
View raw message