leith wrote:
we've reran our tests, uncommented HEAP_SIZE=1000 in our hbase-env,
and have updated trunk to the latest.
IIRC, 1000M is default size. Perhaps bump the size up -- 1200/1400?
But things might just be better overall because of the patch in
HBASE-745 that was committed yesterday.
we have 5 regions loaded up in our regionserver
our test hammers hbase as we import a bunch of files into our hbase
tables. now, what we see is that hbase seems to be blocking such that
nothing gets imported. everything is up and running (hdfs
namenode/datanodes, hbase master/regionserver)
These log messages "Blocking updates for 'IPC Server handler 4 on 60020'
on region dmls,,1216768730386: Memcache size 64.0m is >= than blocking
64.0m size " are part of 'normal' operation when hbase is under heavy
load. To protect itself it puts up a (usually very) temporary block
until it has had a chance to relieve itself of in-memory pressure
flushing its memcache out to the filesystem. Once this is done, away it
goes again taking on updates.
You should enable DEBUG. You'll see more of whats going on. See the
FAQ for how (and make sure you have your ulimit filedescriptors set high
-- see also in FAQ for how).
Regards the below IOException in your datanode logs, do repercussions
show in hbase logs?
St.Ack
---------------------------------------
hdfs datanode
--------------------------------------
2008-07-22 17:10:41,828 WARN org.apache.hadoop.dfs.DataNode:
64.62.244.2:50010:Got exception while serving blk_-355911506373371046
to /64.62.244.2:
java.io.IOException: Connection reset by peer
at sun.nio.ch.FileDispatcher.write0(Native Method)
at sun.nio.ch.SocketDispatcher.write(SocketDispatcher.java:29)
at sun.nio.ch.IOUtil.writeFromNativeBuffer(IOUtil.java:104)
at sun.nio.ch.IOUtil.write(IOUtil.java:75)
at sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:334)
at
org.apache.hadoop.net.SocketOutputStream$Writer.performIO(SocketOutputStream.java:53)
at
org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:140)
at
org.apache.hadoop.net.SocketOutputStream.write(SocketOutputStream.java:144)
at
org.apache.hadoop.net.SocketOutputStream.write(SocketOutputStream.java:105)
at java.io.BufferedOutputStream.write(BufferedOutputStream.java:105)
at java.io.DataOutputStream.write(DataOutputStream.java:90)
at
org.apache.hadoop.dfs.DataNode$BlockSender.sendChunks(DataNode.java:1774)
at
org.apache.hadoop.dfs.DataNode$BlockSender.sendBlock(DataNode.java:1813)
at
org.apache.hadoop.dfs.DataNode$DataXceiver.readBlock(DataNode.java:1039)
at org.apache.hadoop.dfs.DataNode$DataXceiver.run(DataNode.java:968)
at java.lang.Thread.run(Thread.java:619)
2008-07-22 17:10:41,828 ERROR org.apache.hadoop.dfs.DataNode:
64.62.244.2:50010:DataXceiver: java.io.IOException: Connection reset
by peer
at sun.nio.ch.FileDispatcher.write0(Native Method)
at sun.nio.ch.SocketDispatcher.write(SocketDispatcher.java:29)
at sun.nio.ch.IOUtil.writeFromNativeBuffer(IOUtil.java:104)
at sun.nio.ch.IOUtil.write(IOUtil.java:75)
at sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:334)
at
org.apache.hadoop.net.SocketOutputStream$Writer.performIO(SocketOutputStream.java:53)
at
org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:140)
at
org.apache.hadoop.net.SocketOutputStream.write(SocketOutputStream.java:144)
at
org.apache.hadoop.net.SocketOutputStream.write(SocketOutputStream.java:105)
at java.io.BufferedOutputStream.write(BufferedOutputStream.java:105)
at java.io.DataOutputStream.write(DataOutputStream.java:90)
at
org.apache.hadoop.dfs.DataNode$BlockSender.sendChunks(DataNode.java:1774)
at
org.apache.hadoop.dfs.DataNode$BlockSender.sendBlock(DataNode.java:1813)
at
org.apache.hadoop.dfs.DataNode$DataXceiver.readBlock(DataNode.java:1039)
at org.apache.hadoop.dfs.DataNode$DataXceiver.run(DataNode.java:968)
at java.lang.Thread.run(Thread.java:619)
2008-07-22 17:10:41,834 INFO org.apache.hadoop.dfs.DataNode:
64.62.244.2:50010 Served block blk_8154627018748859939 to /64.62.244.2
2008-07-22 17:10:41,843 INFO org.apache.hadoop.dfs.DataNode:
64.62.244.2:50010 Served block blk_6062812905615890844 to /64.62.244.2
2008-07-22 17:10:41,987 INFO org.apache.hadoop.dfs.DataNode:
64.62.244.2:50010 Served block blk_-3883168343557890264 to /64.62.244.2
St.Ack
leith wrote:
looking at our region logs, we've noticed that the compaction thread
constantly runs into exceptions. the entire log is filled with
something like this:
----------------------------------
2008-07-22 12:29:52,759 WARN
org.apache.hadoop.hbase.regionserver.HStore: Exception closing
reader for 242866774/new
java.io.IOException: Stream closed
at
org.apache.hadoop.dfs.DFSClient$DFSInputStream.close(DFSClient.java:1319)
at java.io.FilterInputStream.close(FilterInputStream.java:155)
at
org.apache.hadoop.io.SequenceFile$Reader.close(SequenceFile.java:1581)
at org.apache.hadoop.io.MapFile$Reader.close(MapFile.java:577)
at
org.apache.hadoop.hbase.regionserver.HStore.closeCompactionReaders(HStore.java:917)
at
org.apache.hadoop.hbase.regionserver.HStore.compactHStoreFiles(HStore.java:910)
at
org.apache.hadoop.hbase.regionserver.HStore.compact(HStore.java:787)
at
org.apache.hadoop.hbase.regionserver.HRegion.compactStores(HRegion.java:887)
at
org.apache.hadoop.hbase.regionserver.HRegion.compactStores(HRegion.java:847)
at
org.apache.hadoop.hbase.regionserver.CompactSplitThread.run(CompactSplitThread.java:84)
-------------------------------------
the regionserver is taking about up a good amount of memory on our
system, and nothing is happening except for i assume
compaction/split processes.
these only seem to be warnings, but there is so many of them, it
would be nice to get a second opinion on this.
we've also gotten an 'out of memory' exception a few times from the
compaction thread, and those actually ended up killing the thread,
resulting in the region server shutting itself down.
thanks,
/leith