hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Jean-Daniel Cryans (JIRA)" <j...@apache.org>
Subject [jira] Updated: (HBASE-2506) Too easy to OOME a RS
Date Fri, 30 Apr 2010 20:46:54 GMT

     [ https://issues.apache.org/jira/browse/HBASE-2506?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]

Jean-Daniel Cryans updated HBASE-2506:
--------------------------------------

    Description: 
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.

  was:
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.


> Too easy to OOME a RS
> ---------------------
>
>                 Key: HBASE-2506
>                 URL: https://issues.apache.org/jira/browse/HBASE-2506
>             Project: Hadoop HBase
>          Issue Type: Bug
>            Reporter: Jean-Daniel Cryans
>             Fix For: 0.20.5, 0.21.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.


Mime
View raw message