[
https://issues.apache.org/jira/browse/HBASE-2506?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12964485#action_12964485
]
Andrew Purtell commented on HBASE-2506:
---------------------------------------
Up on d...@hbase, Ted Yu said:
{quote}
I am wondering if HBase can utilize the following to balance load across
region servers for reduced GC pause:
http://download.oracle.com/javase/1.5.0/docs/api/java/lang/management/MemoryPoolMXBean.html
especially Usage Threshold Notifications.
{quote}
I've also been thinking the regionservers should take some action -- exactly
what surely will be subject to debate -- to protect themselves should they
enter a low memory condition, such as one or more of:
- Defer compactions
- Defer splits
- Slow down writers
- Randomly choose a region to abandon
In a nasty test case of mine I observe cascading failure where enough really
aggressive writers can cause a regionserver to OOME, (4GB or 8GB heaps, all
config settings set to defaults), then after regions are redeployed cause the
next victim to OOME, and on and on until all regionservers are dead. OOMEs
typically happen during splits or compactions. This happens up on EC2 so I
think the relatively slow I/O causes HBase, eventually, to overcommit heap.
In general, there is a lack of a back-pressure mechanism to slow down writers
that are overcommitting the system.
> Too easy to OOME a RS
> ---------------------
>
> Key: HBASE-2506
> URL: https://issues.apache.org/jira/browse/HBASE-2506
> Project: HBase
> Issue Type: Bug
> Reporter: Jean-Daniel Cryans
> Fix For: 0.92.0
>
>
> Testing a cluster with 1GB heap, I found that we are letting the region
> servers kill themselves too easily when scanning using pre-fetching. To
> reproduce, get 10-20M rows using PE and run a count in the shell using CACHE
> => 30000 or any other very high number. For good measure, here's the stack
> trace:
> {code}
> 2010-04-30 13:20:23,241 FATAL
> org.apache.hadoop.hbase.regionserver.HRegionServer: OutOfMemoryError,
> aborting.
> java.lang.OutOfMemoryError: Java heap space
> at java.util.Arrays.copyOf(Arrays.java:2786)
> at java.io.ByteArrayOutputStream.write(ByteArrayOutputStream.java:94)
> at java.io.DataOutputStream.write(DataOutputStream.java:90)
> at org.apache.hadoop.hbase.client.Result.writeArray(Result.java:478)
> at
> org.apache.hadoop.hbase.io.HbaseObjectWritable.writeObject(HbaseObjectWritable.java:312)
> at
> org.apache.hadoop.hbase.io.HbaseObjectWritable.write(HbaseObjectWritable.java:229)
> at
> org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:941)
> 2010-04-30 13:20:23,241 INFO
> org.apache.hadoop.hbase.regionserver.HRegionServer: Dump of metrics:
> request=0.0, regions=29, stores=29, storefiles=44, storefileIndexSize=6,
> memstoreSize=255,
> compactionQueueSize=0, usedHeap=926, maxHeap=987, blockCacheSize=1700064,
> blockCacheFree=205393696, blockCacheCount=0, blockCacheHitRatio=0
> {code}
> I guess the same could happen with largish write buffers. We need something
> better than OOME.
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.